From 3e754532d5200a183d6f5aae9c91447b3b6faa77 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 5 Jun 2024 16:15:25 +0200 Subject: [PATCH 001/816] Azure Blob Storage added option too check object after write --- src/Core/Settings.h | 5 +-- .../IO/WriteBufferFromAzureBlobStorage.cpp | 33 +++++++++++++++++++ .../IO/WriteBufferFromAzureBlobStorage.h | 1 + .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 1 + .../AzureBlobStorage/AzureObjectStorage.h | 5 ++- 5 files changed, 42 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index dc61a049de8..9188033a1ec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -112,8 +112,9 @@ class IColumn; M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ - M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ - M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ + M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ + M(Bool, azure_check_objects_after_upload, false, "Check each uploaded object to azure blob storage with head request to be sure that upload was successful", 0) \ + M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..8d42928abda 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -19,6 +19,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int AZURE_BLOB_STORAGE_ERROR; +} + struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; @@ -59,6 +64,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::move(schedule_), settings_->max_inflight_parts_for_one_file, limitedLog)) + , check_objects_after_upload(settings_->check_objects_after_upload) { allocateBuffer(); } @@ -151,6 +157,33 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); } + + if (check_objects_after_upload) + { + Azure::Storage::Blobs::ListBlobsOptions options; + options.Prefix = blob_path; + options.PageSizeHint = 1; + + auto blobs_list_response = blob_container_client->ListBlobs(options); + auto blobs_list = blobs_list_response.Blobs; + + bool found = false; + + for (const auto & blob : blobs_list) + { + if (blob_path == blob.Name) + { + found = true; + break; + } + } + + if (!found) + throw Exception( + ErrorCodes::AZURE_BLOB_STORAGE_ERROR, + "Object {} not uploaded to azure blob storage, it's a bug in Azure Blob Storage or its API.", + blob_path); + } } void WriteBufferFromAzureBlobStorage::nextImpl() diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 3da6d843991..6d16f17de46 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -80,6 +80,7 @@ private: bool first_buffer=true; std::unique_ptr task_tracker; + bool check_objects_after_upload; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index bae58f0b9c6..20cfb9567b5 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -263,6 +263,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); + settings->check_objects_after_upload = config.getUInt64(config_prefix + ".check_objects_after_upload", false); return settings; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 8ead696cf78..b7c9907c3b2 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -32,7 +32,8 @@ struct AzureObjectStorageSettings size_t max_inflight_parts_for_one_file_, size_t strict_upload_part_size_, size_t upload_part_size_multiply_factor_, - size_t upload_part_size_multiply_parts_count_threshold_) + size_t upload_part_size_multiply_parts_count_threshold_, + bool check_objects_after_upload_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) @@ -47,6 +48,7 @@ struct AzureObjectStorageSettings , strict_upload_part_size(strict_upload_part_size_) , upload_part_size_multiply_factor(upload_part_size_multiply_factor_) , upload_part_size_multiply_parts_count_threshold(upload_part_size_multiply_parts_count_threshold_) + , check_objects_after_upload(check_objects_after_upload_) { } @@ -67,6 +69,7 @@ struct AzureObjectStorageSettings size_t strict_upload_part_size = 0; size_t upload_part_size_multiply_factor = 2; size_t upload_part_size_multiply_parts_count_threshold = 500; + bool check_objects_after_upload = false; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; From 5fea4166f1a06fb049c8fae0d5c18857347fc52d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 20 Jun 2024 16:35:59 +0200 Subject: [PATCH 002/816] Updated SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cdc955b38bc..76229e03160 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -114,6 +114,7 @@ static const std::map Date: Fri, 28 Jun 2024 13:35:05 +0200 Subject: [PATCH 003/816] Addressed review comments, added test --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 34 ++--- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- .../test_check_after_upload.py | 129 ++++++++++++++++++ 4 files changed, 144 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index f1b5050eb64..8149b66eb92 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -172,29 +172,21 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() if (check_objects_after_upload) { - Azure::Storage::Blobs::ListBlobsOptions options; - options.Prefix = blob_path; - options.PageSizeHint = 1; - - auto blobs_list_response = blob_container_client->ListBlobs(options); - auto blobs_list = blobs_list_response.Blobs; - - bool found = false; - - for (const auto & blob : blobs_list) + try { - if (blob_path == blob.Name) - { - found = true; - break; - } + auto blob_client = blob_container_client->GetBlobClient(blob_path); + blob_client.GetProperties(); + return; + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) + throw Exception( + ErrorCodes::AZURE_BLOB_STORAGE_ERROR, + "Object {} not uploaded to azure blob storage, it's a bug in Azure Blob Storage or its API.", + blob_path); + throw; } - - if (!found) - throw Exception( - ErrorCodes::AZURE_BLOB_STORAGE_ERROR, - "Object {} not uploaded to azure blob storage, it's a bug in Azure Blob Storage or its API.", - blob_path); } } diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 6ef84c40632..6f021d8c57a 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -90,7 +90,7 @@ private: size_t hidden_size = 0; std::unique_ptr task_tracker; - bool check_objects_after_upload; + bool check_objects_after_upload = false; std::deque detached_part_data; }; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 20cfb9567b5..b25d7fcbf8d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -263,7 +263,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); - settings->check_objects_after_upload = config.getUInt64(config_prefix + ".check_objects_after_upload", false); + settings->check_objects_after_upload = config.getBool(config_prefix + ".check_objects_after_upload", false); return settings; } diff --git a/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py b/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py new file mode 100644 index 00000000000..e55c961273e --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py @@ -0,0 +1,129 @@ +import logging +import os + +import pytest + +from helpers.cluster import ClickHouseCluster +from test_storage_azure_blob_storage.test import azure_query + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +NODE_NAME = "node" +TABLE_NAME = "blob_storage_table" +AZURE_BLOB_STORAGE_DISK = "blob_storage_disk" +LOCAL_DISK = "hdd" +CONTAINER_NAME = "cont" + + +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/disk_storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + + azure_blob_storage + http://azurite1:{port}/devstoreaccount1 + cont + false + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + 100000 + 100000 + 10 + 10 + true + + + local + / + + + + + +
+ blob_storage_disk +
+ + hdd + +
+
+
+
+
+""" + ) + return path + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) + cluster.add_instance( + NODE_NAME, + main_configs=[ + path, + ], + with_azurite=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +# Note: use azure_query for selects and inserts and create table queries. +# For inserts there is no guarantee that retries will not result in duplicates. +# But it is better to retry anyway because connection related errors +# happens in fact only for inserts because reads already have build-in retries in code. + + +def create_table(node, table_name, **additional_settings): + settings = { + "storage_policy": "blob_storage_policy", + "old_parts_lifetime": 1, + "index_granularity": 512, + "temporary_directories_lifetime": 1, + } + settings.update(additional_settings) + + create_table_statement = f""" + CREATE TABLE {table_name} ( + dt Date, + id Int64, + data String, + INDEX min_max (id) TYPE minmax GRANULARITY 3 + ) ENGINE=MergeTree() + PARTITION BY dt + ORDER BY (dt, id) + SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" + + azure_query(node, f"DROP TABLE IF EXISTS {table_name}") + azure_query(node, create_table_statement) + assert ( + azure_query(node, f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" + ) + +def test_simple(cluster): + node = cluster.instances[NODE_NAME] + create_table(node, TABLE_NAME) + + values = "('2021-11-13',3,'hello')" + azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values}") + assert ( + azure_query(node, f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") + == values + ) From dacff6e2ebc0bfcc66f953b899395f788eab682d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Jun 2024 11:56:20 +0000 Subject: [PATCH 004/816] Automatic style fix --- .../test_check_after_upload.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py b/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py index e55c961273e..8d6cf01ee0e 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py +++ b/tests/integration/test_storage_azure_blob_storage/test_check_after_upload.py @@ -114,9 +114,10 @@ def create_table(node, table_name, **additional_settings): azure_query(node, f"DROP TABLE IF EXISTS {table_name}") azure_query(node, create_table_statement) assert ( - azure_query(node, f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" + azure_query(node, f"SELECT COUNT(*) FROM {table_name} FORMAT Values") == "(0)" ) + def test_simple(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) @@ -124,6 +125,6 @@ def test_simple(cluster): values = "('2021-11-13',3,'hello')" azure_query(node, f"INSERT INTO {TABLE_NAME} VALUES {values}") assert ( - azure_query(node, f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") - == values + azure_query(node, f"SELECT dt, id, data FROM {TABLE_NAME} FORMAT Values") + == values ) From 8bc4f29ef0aa2b28011f9647ecf3d8f3c2bf3ca5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 28 Jun 2024 14:01:23 +0200 Subject: [PATCH 005/816] Fixed settings --- src/Core/Settings.h | 4 ++-- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 08794e6529e..155f5cc6766 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -116,9 +116,9 @@ class IColumn; M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ - M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ + M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, azure_check_objects_after_upload, false, "Check each uploaded object to azure blob storage with head request to be sure that upload was successful", 0) \ - M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ + M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 27b78e9c489..0f45adb4738 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -263,7 +263,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); - settings->check_objects_after_upload = config.getBool(config_prefix + ".check_objects_after_upload", false); + settings->check_objects_after_upload = config.getBool(config_prefix + ".check_objects_after_upload", context.getSettings().azure_check_objects_after_upload); return settings; } From 47aef9b81e6950f84e39ce282709fa85caef2b04 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 28 Jun 2024 17:20:33 +0200 Subject: [PATCH 006/816] Fixed build --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 0f45adb4738..22d4f64b4f2 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -263,7 +263,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); - settings->check_objects_after_upload = config.getBool(config_prefix + ".check_objects_after_upload", context.getSettings().azure_check_objects_after_upload); + settings->check_objects_after_upload = config.getBool(config_prefix + ".check_objects_after_upload", context->getSettings().azure_check_objects_after_upload); return settings; } From a131e0c317b2a100e0e4f1fca542cca0a5585955 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 1 Jul 2024 17:49:54 +0200 Subject: [PATCH 007/816] Remove unwanted return --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8149b66eb92..082b7ce2080 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -176,7 +176,6 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() { auto blob_client = blob_container_client->GetBlobClient(blob_path); blob_client.GetProperties(); - return; } catch (const Azure::Storage::StorageException & e) { From ae18dc3fb63de396d2ba6da8acfd3be1afd289a3 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 17 Jul 2024 19:47:05 +0200 Subject: [PATCH 008/816] fix for DateTime64 cast in set --- src/Interpreters/Set.cpp | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index f33418f45ac..410e34d6758 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -278,6 +279,16 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } +ColumnPtr returnIfEquals(const ColumnPtr & lhs, const ColumnPtr & rhs) +{ + if (rhs->size() != lhs->size()) + return nullptr; + for (size_t i = 0; i < lhs->size(); i++) + if (lhs->getDataAt(i) != rhs->getDataAt(i)) + return nullptr; + return lhs; +} + ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const { size_t num_key_columns = columns.size(); @@ -331,7 +342,13 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); } - materialized_columns.emplace_back() = result; + ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen + if (returnIfEquals(column_before_cast.column, result) == nullptr && isDateTime64(column_before_cast.column->getDataType())) + col_to_emplace = column_before_cast.column; + else + col_to_emplace = result; + + materialized_columns.emplace_back() = col_to_emplace; key_columns.emplace_back() = materialized_columns.back().get(); } From a5ee6789901a389e6a0309cae957bcaf6b32aea8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 26 Jul 2024 19:44:49 +0200 Subject: [PATCH 009/816] fixes + test --- src/Interpreters/Set.cpp | 20 +++++++++++-------- ...8_datetime_cast_losing_precision.reference | 1 + .../03208_datetime_cast_losing_precision.sql | 1 + 3 files changed, 14 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference create mode 100644 tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 410e34d6758..3170d1a54d2 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -279,14 +280,17 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } -ColumnPtr returnIfEquals(const ColumnPtr & lhs, const ColumnPtr & rhs) +ColumnPtr returnFilteredColumn(const ColumnPtr & first, const ColumnPtr & second) { - if (rhs->size() != lhs->size()) + ConstantFilterDescription second_const_descr(*second); + if (second_const_descr.always_true) + return second; + + if (second_const_descr.always_false) return nullptr; - for (size_t i = 0; i < lhs->size(); i++) - if (lhs->getDataAt(i) != rhs->getDataAt(i)) - return nullptr; - return lhs; + + FilterDescription filter_descr(*second); + return first->filter(*filter_descr.data, 0); } ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const @@ -343,8 +347,8 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen - if (returnIfEquals(column_before_cast.column, result) == nullptr && isDateTime64(column_before_cast.column->getDataType())) - col_to_emplace = column_before_cast.column; + if (isDateTime64(column_before_cast.column->getDataType())) + col_to_emplace = returnFilteredColumn(column_before_cast.column, res->getPtr()); else col_to_emplace = result; diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference new file mode 100644 index 00000000000..c227083464f --- /dev/null +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql new file mode 100644 index 00000000000..f58c6a2d6a1 --- /dev/null +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql @@ -0,0 +1 @@ +SELECT now64() IN (SELECT now()); From a401a0e3f578f2847d83ab789388b6c9b22313f3 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 26 Jul 2024 19:51:14 +0200 Subject: [PATCH 010/816] empty commit From 0846bd037e3115f8c11547a03238526324816fea Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 26 Jul 2024 21:26:51 +0200 Subject: [PATCH 011/816] Update 03208_datetime_cast_losing_precision.reference --- .../0_stateless/03208_datetime_cast_losing_precision.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference index c227083464f..573541ac970 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -1 +1 @@ -0 \ No newline at end of file +0 From 2665ef59f1a80b609890e586cabb588920758865 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sat, 27 Jul 2024 11:39:15 +0200 Subject: [PATCH 012/816] Fix msan --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 3170d1a54d2..99b6983506c 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -287,7 +287,7 @@ ColumnPtr returnFilteredColumn(const ColumnPtr & first, const ColumnPtr & second return second; if (second_const_descr.always_false) - return nullptr; + return first; FilterDescription filter_descr(*second); return first->filter(*filter_descr.data, 0); From d84a1dad1ae3e6e7649bfdadf6ed80fea5856144 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sun, 28 Jul 2024 11:54:07 +0200 Subject: [PATCH 013/816] Update Set.cpp --- src/Interpreters/Set.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 99b6983506c..ae673fba86c 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -346,6 +346,9 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); } + if (!col_to_emplace) + col_to_emplace = column_before_cast.column; + ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen if (isDateTime64(column_before_cast.column->getDataType())) col_to_emplace = returnFilteredColumn(column_before_cast.column, res->getPtr()); From f192d8008a08cbb1320a85596e329f9d7db27881 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Sun, 28 Jul 2024 12:07:44 +0200 Subject: [PATCH 014/816] Update Set.cpp --- src/Interpreters/Set.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index ae673fba86c..c2e66d007cd 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -346,15 +346,15 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); } - if (!col_to_emplace) - col_to_emplace = column_before_cast.column; - ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen if (isDateTime64(column_before_cast.column->getDataType())) col_to_emplace = returnFilteredColumn(column_before_cast.column, res->getPtr()); else col_to_emplace = result; + if (!col_to_emplace) + col_to_emplace = column_before_cast.column; + materialized_columns.emplace_back() = col_to_emplace; key_columns.emplace_back() = materialized_columns.back().get(); } From 2a0fe4172d14cd7248ff1a3c32b2509e8058e877 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 00:12:59 +0200 Subject: [PATCH 015/816] Update Set.cpp --- src/Interpreters/Set.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index c2e66d007cd..97497c80a14 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -290,7 +290,9 @@ ColumnPtr returnFilteredColumn(const ColumnPtr & first, const ColumnPtr & second return first; FilterDescription filter_descr(*second); - return first->filter(*filter_descr.data, 0); + if (filter_descr.data) + return first->filter(*filter_descr.data, 0); + return nullptr; } ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const From 558d697eb803d06921dc2b749eccb4caa73a4701 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 15:12:30 +0200 Subject: [PATCH 016/816] fixes --- src/Interpreters/Set.cpp | 726 --------------------------------------- 1 file changed, 726 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 97497c80a14..e69de29bb2d 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -1,726 +0,0 @@ -#include - -#include - -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int SET_SIZE_LIMIT_EXCEEDED; - extern const int TYPE_MISMATCH; - extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; -} - - -template -void NO_INLINE Set::insertFromBlockImpl( - Method & method, - const ColumnRawPtrs & key_columns, - size_t rows, - SetVariants & variants, - ConstNullMapPtr null_map, - ColumnUInt8::Container * out_filter) -{ - if (null_map) - { - if (out_filter) - insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); - else - insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); - } - else - { - if (out_filter) - insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); - else - insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); - } -} - - -template -void NO_INLINE Set::insertFromBlockImplCase( - Method & method, - const ColumnRawPtrs & key_columns, - size_t rows, - SetVariants & variants, - [[maybe_unused]] ConstNullMapPtr null_map, - [[maybe_unused]] ColumnUInt8::Container * out_filter) -{ - typename Method::State state(key_columns, key_sizes, nullptr); - - /// For all rows - for (size_t i = 0; i < rows; ++i) - { - if constexpr (has_null_map) - { - if ((*null_map)[i]) - { - if constexpr (build_filter) - { - (*out_filter)[i] = false; - } - continue; - } - } - - [[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool); - - if constexpr (build_filter) - (*out_filter)[i] = emplace_result.isInserted(); - } -} - - -DataTypes Set::getElementTypes(DataTypes types, bool transform_null_in) -{ - for (auto & type : types) - { - if (const auto * low_cardinality_type = typeid_cast(type.get())) - type = low_cardinality_type->getDictionaryType(); - - if (!transform_null_in) - type = removeNullable(type); - } - - return types; -} - - -void Set::setHeader(const ColumnsWithTypeAndName & header) -{ - std::lock_guard lock(rwlock); - - if (!data.empty()) - return; - - keys_size = header.size(); - ColumnRawPtrs key_columns; - key_columns.reserve(keys_size); - data_types.reserve(keys_size); - set_elements_types.reserve(keys_size); - - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns materialized_columns; - - /// Remember the columns we will work with - for (size_t i = 0; i < keys_size; ++i) - { - materialized_columns.emplace_back(header.at(i).column->convertToFullColumnIfConst()); - key_columns.emplace_back(materialized_columns.back().get()); - data_types.emplace_back(header.at(i).type); - set_elements_types.emplace_back(header.at(i).type); - - /// Convert low cardinality column to full. - if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) - { - data_types.back() = low_cardinality_type->getDictionaryType(); - set_elements_types.back() = low_cardinality_type->getDictionaryType(); - materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality()); - key_columns.back() = materialized_columns.back().get(); - } - } - - /// We will insert to the Set only keys, where all components are not NULL. - ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder; - if (!transform_null_in) - { - /// We convert nullable columns to non nullable we also need to update nullable types - for (size_t i = 0; i < set_elements_types.size(); ++i) - { - data_types[i] = removeNullable(data_types[i]); - set_elements_types[i] = removeNullable(set_elements_types[i]); - } - - extractNestedColumnsAndNullMap(key_columns, null_map); - } - - /// Choose data structure to use for the set. - data.init(SetVariants::chooseMethod(key_columns, key_sizes)); -} - -void Set::fillSetElements() -{ - fill_set_elements = true; - set_elements.reserve(keys_size); - for (const auto & type : set_elements_types) - set_elements.emplace_back(type->createColumn()); -} - -bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) -{ - Columns cols; - cols.reserve(columns.size()); - for (const auto & column : columns) - cols.emplace_back(column.column); - return insertFromColumns(cols); -} - -bool Set::insertFromColumns(const Columns & columns) -{ - size_t rows = columns.at(0)->size(); - - SetKeyColumns holder; - /// Filter to extract distinct values from the block. - if (fill_set_elements) - holder.filter = ColumnUInt8::create(rows); - - bool inserted = insertFromColumns(columns, holder); - if (inserted && fill_set_elements) - { - if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount()) - { - /// Drop filled elementes - fill_set_elements = false; - set_elements.clear(); - } - else - appendSetElements(holder); - } - - return inserted; -} - -bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) -{ - std::lock_guard lock(rwlock); - - if (data.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); - - holder.key_columns.reserve(keys_size); - holder.materialized_columns.reserve(keys_size); - - /// Remember the columns we will work with - for (size_t i = 0; i < keys_size; ++i) - { - holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - holder.key_columns.emplace_back(holder.materialized_columns.back().get()); - } - - size_t rows = columns.at(0)->size(); - - /// We will insert to the Set only keys, where all components are not NULL. - ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder; - if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map); - - switch (data.type) - { - case SetVariants::Type::EMPTY: - break; -#define M(NAME) \ - case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ - break; - APPLY_FOR_SET_VARIANTS(M) -#undef M - } - - return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); -} - -void Set::appendSetElements(SetKeyColumns & holder) -{ - if (holder.key_columns.size() != keys_size || set_elements.size() != keys_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of key columns for set. Expected {} got {} and {}", - keys_size, holder.key_columns.size(), set_elements.size()); - - size_t rows = holder.key_columns.at(0)->size(); - for (size_t i = 0; i < keys_size; ++i) - { - auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows); - if (set_elements[i]->empty()) - set_elements[i] = filtered_column; - else - set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); - if (transform_null_in && holder.null_map_holder) - set_elements[i]->insert(Null{}); - } -} - -void Set::checkIsCreated() const -{ - if (!is_created.load()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); -} - -ColumnPtr returnFilteredColumn(const ColumnPtr & first, const ColumnPtr & second) -{ - ConstantFilterDescription second_const_descr(*second); - if (second_const_descr.always_true) - return second; - - if (second_const_descr.always_false) - return first; - - FilterDescription filter_descr(*second); - if (filter_descr.data) - return first->filter(*filter_descr.data, 0); - return nullptr; -} - -ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const -{ - size_t num_key_columns = columns.size(); - - if (0 == num_key_columns) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns passed to Set::execute method."); - - auto res = ColumnUInt8::create(); - ColumnUInt8::Container & vec_res = res->getData(); - vec_res.resize(columns.at(0).column->size()); - - if (vec_res.empty()) - return res; - - std::shared_lock lock(rwlock); - - /// If the set is empty. - if (data_types.empty()) - { - if (negative) - memset(vec_res.data(), 1, vec_res.size()); - else - memset(vec_res.data(), 0, vec_res.size()); - return res; - } - - checkColumnsNumber(num_key_columns); - - /// Remember the columns we will work with. Also check that the data types are correct. - ColumnRawPtrs key_columns; - key_columns.reserve(num_key_columns); - - /// The constant columns to the left of IN are not supported directly. For this, they first materialize. - Columns materialized_columns; - materialized_columns.reserve(num_key_columns); - - for (size_t i = 0; i < num_key_columns; ++i) - { - ColumnPtr result; - - const auto & column_before_cast = columns.at(i); - ColumnWithTypeAndName column_to_cast - = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name}; - - if (!transform_null_in && data_types[i]->canBeInsideNullable()) - { - result = castColumnAccurateOrNull(column_to_cast, data_types[i], cast_cache.get()); - } - else - { - result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); - } - - ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen - if (isDateTime64(column_before_cast.column->getDataType())) - col_to_emplace = returnFilteredColumn(column_before_cast.column, res->getPtr()); - else - col_to_emplace = result; - - if (!col_to_emplace) - col_to_emplace = column_before_cast.column; - - materialized_columns.emplace_back() = col_to_emplace; - key_columns.emplace_back() = materialized_columns.back().get(); - } - - /// We will check existence in Set only for keys whose components do not contain any NULL value. - ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder; - if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - - executeOrdinary(key_columns, vec_res, negative, null_map); - - return res; -} - -bool Set::hasNull() const -{ - checkIsCreated(); - - if (!transform_null_in) - return false; - - if (data_types.size() != 1) - return false; - - if (!data_types[0]->isNullable()) - return false; - - auto col = data_types[0]->createColumn(); - col->insert(Field()); - auto res = execute({ColumnWithTypeAndName(std::move(col), data_types[0], std::string())}, false); - return res->getBool(0); -} - -bool Set::empty() const -{ - std::shared_lock lock(rwlock); - return data.empty(); -} - -size_t Set::getTotalRowCount() const -{ - std::shared_lock lock(rwlock); - return data.getTotalRowCount(); -} - -size_t Set::getTotalByteCount() const -{ - std::shared_lock lock(rwlock); - return data.getTotalByteCount(); -} - - -template -void NO_INLINE Set::executeImpl( - Method & method, - const ColumnRawPtrs & key_columns, - ColumnUInt8::Container & vec_res, - bool negative, - size_t rows, - ConstNullMapPtr null_map) const -{ - if (null_map) - executeImplCase(method, key_columns, vec_res, negative, rows, null_map); - else - executeImplCase(method, key_columns, vec_res, negative, rows, null_map); -} - - -template -void NO_INLINE Set::executeImplCase( - Method & method, - const ColumnRawPtrs & key_columns, - ColumnUInt8::Container & vec_res, - bool negative, - size_t rows, - ConstNullMapPtr null_map) const -{ - Arena pool; - typename Method::State state(key_columns, key_sizes, nullptr); - - /// NOTE Optimization is not used for consecutive identical strings. - - /// For all rows - for (size_t i = 0; i < rows; ++i) - { - if (has_null_map && (*null_map)[i]) - { - vec_res[i] = negative; - } - else - { - auto find_result = state.findKey(method.data, i, pool); - vec_res[i] = negative ^ find_result.isFound(); - } - } -} - - -void Set::executeOrdinary( - const ColumnRawPtrs & key_columns, - ColumnUInt8::Container & vec_res, - bool negative, - ConstNullMapPtr null_map) const -{ - size_t rows = key_columns[0]->size(); - - switch (data.type) - { - case SetVariants::Type::EMPTY: - break; -#define M(NAME) \ - case SetVariants::Type::NAME: \ - executeImpl(*data.NAME, key_columns, vec_res, negative, rows, null_map); \ - break; - APPLY_FOR_SET_VARIANTS(M) -#undef M - } -} - -void Set::checkColumnsNumber(size_t num_key_columns) const -{ - if (data_types.size() != num_key_columns) - { - throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, - "Number of columns in section IN doesn't match. {} at left, {} at right.", - num_key_columns, data_types.size()); - } -} - -bool Set::areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const -{ - /// Out-of-bound access can happen when same set expression built with different columns. - /// Caller may call this method to make sure that the set is indeed the one they want - /// without awaring data_types.size(). - if (set_type_idx >= data_types.size()) - return false; - return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx])) - ->equals(*removeNullable(recursiveRemoveLowCardinality(other_type))); -} - -void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const -{ - if (!this->areTypesEqual(set_type_idx, other_type)) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Types of column {} in section IN don't match: " - "{} on the left, {} on the right", toString(set_type_idx + 1), - other_type->getName(), data_types[set_type_idx]->getName()); -} - -MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector && indexes_mapping_) - : has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_)) -{ - // std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex " - // << set_elements.size() << ' ' << indexes_mapping.size() << std::endl; - // for (const auto & vv : indexes_mapping) - // std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl; - - ::sort(indexes_mapping.begin(), indexes_mapping.end(), - [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) - { - return std::tie(l.key_index, l.tuple_index) < std::tie(r.key_index, r.tuple_index); - }); - - indexes_mapping.erase(std::unique( - indexes_mapping.begin(), indexes_mapping.end(), - [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) - { - return l.key_index == r.key_index; - }), indexes_mapping.end()); - - size_t tuple_size = indexes_mapping.size(); - ordered_set.resize(tuple_size); - - for (size_t i = 0; i < tuple_size; ++i) - ordered_set[i] = set_elements[indexes_mapping[i].tuple_index]; - - Block block_to_sort; - SortDescription sort_description; - for (size_t i = 0; i < tuple_size; ++i) - { - String column_name = "_" + toString(i); - block_to_sort.insert({ordered_set[i], nullptr, column_name}); - sort_description.emplace_back(column_name, 1, 1); - } - - sortBlock(block_to_sort, sort_description); - - for (size_t i = 0; i < tuple_size; ++i) - ordered_set[i] = block_to_sort.getByPosition(i).column; -} - - -/** Return the BoolMask where: - * 1: the intersection of the set and the range is non-empty - * 2: the range contains elements not in the set - */ -BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, const DataTypes & data_types, bool single_point) const -{ - size_t tuple_size = indexes_mapping.size(); - // std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl; - - FieldValues left_point; - FieldValues right_point; - left_point.reserve(tuple_size); - right_point.reserve(tuple_size); - - for (size_t i = 0; i < tuple_size; ++i) - { - left_point.emplace_back(ordered_set[i]->cloneEmpty()); - right_point.emplace_back(ordered_set[i]->cloneEmpty()); - } - - bool left_included = true; - bool right_included = true; - - for (size_t i = 0; i < tuple_size; ++i) - { - std::optional new_range = KeyCondition::applyMonotonicFunctionsChainToRange( - key_ranges[indexes_mapping[i].key_index], - indexes_mapping[i].functions, - data_types[indexes_mapping[i].key_index], - single_point); - - if (!new_range) - return {true, true}; - - left_point[i].update(new_range->left); - left_included &= new_range->left_included; - right_point[i].update(new_range->right); - right_included &= new_range->right_included; - } - - /// lhs < rhs return -1 - /// lhs == rhs return 0 - /// lhs > rhs return 1 - auto compare = [](const IColumn & lhs, const FieldValue & rhs, size_t row) - { - if (rhs.isNegativeInfinity()) - return 1; - if (rhs.isPositiveInfinity()) - { - Field f; - lhs.get(row, f); - if (f.isNull()) - return 0; // +Inf == +Inf - else - return -1; - } - return lhs.compareAt(row, 0, *rhs.column, 1); - }; - - auto less = [this, &compare, tuple_size](size_t row, const auto & point) - { - for (size_t i = 0; i < tuple_size; ++i) - { - int res = compare(*ordered_set[i], point[i], row); - if (res) - return res < 0; - } - return false; - }; - - auto equals = [this, &compare, tuple_size](size_t row, const auto & point) - { - for (size_t i = 0; i < tuple_size; ++i) - if (compare(*ordered_set[i], point[i], row) != 0) - return false; - return true; - }; - - /** Because each hyperrectangle maps to a contiguous sequence of elements - * laid out in the lexicographically increasing order, the set intersects the range - * if and only if either bound coincides with an element or at least one element - * is between the lower bounds - */ - auto indices = collections::range(0, size()); - auto left_lower = std::lower_bound(indices.begin(), indices.end(), left_point, less); - auto right_lower = std::lower_bound(indices.begin(), indices.end(), right_point, less); - - /// A special case of 1-element KeyRange. It's useful for partition pruning. - bool one_element_range = true; - for (size_t i = 0; i < tuple_size; ++i) - { - auto & left = left_point[i]; - auto & right = right_point[i]; - if (left.isNormal() && right.isNormal()) - { - if (0 != left.column->compareAt(0, 0, *right.column, 1)) - { - one_element_range = false; - break; - } - } - else if ((left.isPositiveInfinity() && right.isPositiveInfinity()) || (left.isNegativeInfinity() && right.isNegativeInfinity())) - { - /// Special value equality. - } - else - { - one_element_range = false; - break; - } - } - if (one_element_range && has_all_keys) - { - /// Here we know that there is one element in range. - /// The main difference with the normal case is that we can definitely say that - /// condition in this range is always TRUE (can_be_false = 0) or always FALSE (can_be_true = 0). - - /// Check if it's an empty range - if (!left_included || !right_included) - return {false, true}; - else if (left_lower != indices.end() && equals(*left_lower, left_point)) - return {true, false}; - else - return {false, true}; - } - - /// If there are more than one element in the range, it can always be false. Thus we only need to check if it may be true or not. - /// Given left_lower >= left_point, right_lower >= right_point, find if there may be a match in between left_lower and right_lower. - if (left_lower + 1 < right_lower) - { - /// There is a point in between: left_lower + 1 - return {true, true}; - } - else if (left_lower + 1 == right_lower) - { - /// Need to check if left_lower is a valid match, as left_point <= left_lower < right_point <= right_lower. - /// Note: left_lower is valid. - if (left_included || !equals(*left_lower, left_point)) - return {true, true}; - - /// We are unlucky that left_point fails to cover a point. Now we need to check if right_point can cover right_lower. - /// Check if there is a match at the right boundary. - return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), true}; - } - else // left_lower == right_lower - { - /// Need to check if right_point is a valid match, as left_point < right_point <= left_lower = right_lower. - /// Check if there is a match at the left boundary. - return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), true}; - } -} - -bool MergeTreeSetIndex::hasMonotonicFunctionsChain() const -{ - for (const auto & mapping : indexes_mapping) - if (!mapping.functions.empty()) - return true; - return false; -} - -void FieldValue::update(const Field & x) -{ - if (x.isNegativeInfinity() || x.isPositiveInfinity()) - value = x; - else - { - /// Keep at most one element in column. - if (!column->empty()) - column->popBack(1); - column->insert(x); - value = Field(); // Set back to normal value. - } -} - -} From b354fc23ce90ca3399d3e4e3dd072390dab90362 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Jul 2024 15:19:12 +0200 Subject: [PATCH 017/816] fixes --- src/Interpreters/Set.cpp | 724 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 724 insertions(+) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index e69de29bb2d..0d498c164c7 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -0,0 +1,724 @@ +#include + +#include + +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int SET_SIZE_LIMIT_EXCEEDED; + extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; +} + + +template +void NO_INLINE Set::insertFromBlockImpl( + Method & method, + const ColumnRawPtrs & key_columns, + size_t rows, + SetVariants & variants, + ConstNullMapPtr null_map, + ColumnUInt8::Container * out_filter) +{ + if (null_map) + { + if (out_filter) + insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); + else + insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); + } + else + { + if (out_filter) + insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); + else + insertFromBlockImplCase(method, key_columns, rows, variants, null_map, out_filter); + } +} + + +template +void NO_INLINE Set::insertFromBlockImplCase( + Method & method, + const ColumnRawPtrs & key_columns, + size_t rows, + SetVariants & variants, + [[maybe_unused]] ConstNullMapPtr null_map, + [[maybe_unused]] ColumnUInt8::Container * out_filter) +{ + typename Method::State state(key_columns, key_sizes, nullptr); + + /// For all rows + for (size_t i = 0; i < rows; ++i) + { + if constexpr (has_null_map) + { + if ((*null_map)[i]) + { + if constexpr (build_filter) + { + (*out_filter)[i] = false; + } + continue; + } + } + + [[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool); + + if constexpr (build_filter) + (*out_filter)[i] = emplace_result.isInserted(); + } +} + + +DataTypes Set::getElementTypes(DataTypes types, bool transform_null_in) +{ + for (auto & type : types) + { + if (const auto * low_cardinality_type = typeid_cast(type.get())) + type = low_cardinality_type->getDictionaryType(); + + if (!transform_null_in) + type = removeNullable(type); + } + + return types; +} + + +void Set::setHeader(const ColumnsWithTypeAndName & header) +{ + std::lock_guard lock(rwlock); + + if (!data.empty()) + return; + + keys_size = header.size(); + ColumnRawPtrs key_columns; + key_columns.reserve(keys_size); + data_types.reserve(keys_size); + set_elements_types.reserve(keys_size); + + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + Columns materialized_columns; + + /// Remember the columns we will work with + for (size_t i = 0; i < keys_size; ++i) + { + materialized_columns.emplace_back(header.at(i).column->convertToFullColumnIfConst()); + key_columns.emplace_back(materialized_columns.back().get()); + data_types.emplace_back(header.at(i).type); + set_elements_types.emplace_back(header.at(i).type); + + /// Convert low cardinality column to full. + if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) + { + data_types.back() = low_cardinality_type->getDictionaryType(); + set_elements_types.back() = low_cardinality_type->getDictionaryType(); + materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality()); + key_columns.back() = materialized_columns.back().get(); + } + } + + /// We will insert to the Set only keys, where all components are not NULL. + ConstNullMapPtr null_map{}; + ColumnPtr null_map_holder; + if (!transform_null_in) + { + /// We convert nullable columns to non nullable we also need to update nullable types + for (size_t i = 0; i < set_elements_types.size(); ++i) + { + data_types[i] = removeNullable(data_types[i]); + set_elements_types[i] = removeNullable(set_elements_types[i]); + } + + extractNestedColumnsAndNullMap(key_columns, null_map); + } + + /// Choose data structure to use for the set. + data.init(SetVariants::chooseMethod(key_columns, key_sizes)); +} + +void Set::fillSetElements() +{ + fill_set_elements = true; + set_elements.reserve(keys_size); + for (const auto & type : set_elements_types) + set_elements.emplace_back(type->createColumn()); +} + +bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) +{ + Columns cols; + cols.reserve(columns.size()); + for (const auto & column : columns) + cols.emplace_back(column.column); + return insertFromColumns(cols); +} + +bool Set::insertFromColumns(const Columns & columns) +{ + size_t rows = columns.at(0)->size(); + + SetKeyColumns holder; + /// Filter to extract distinct values from the block. + if (fill_set_elements) + holder.filter = ColumnUInt8::create(rows); + + bool inserted = insertFromColumns(columns, holder); + if (inserted && fill_set_elements) + { + if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount()) + { + /// Drop filled elementes + fill_set_elements = false; + set_elements.clear(); + } + else + appendSetElements(holder); + } + + return inserted; +} + +bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) +{ + std::lock_guard lock(rwlock); + + if (data.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); + + holder.key_columns.reserve(keys_size); + holder.materialized_columns.reserve(keys_size); + + /// Remember the columns we will work with + for (size_t i = 0; i < keys_size; ++i) + { + holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + holder.key_columns.emplace_back(holder.materialized_columns.back().get()); + } + + size_t rows = columns.at(0)->size(); + + /// We will insert to the Set only keys, where all components are not NULL. + ConstNullMapPtr null_map{}; + ColumnPtr null_map_holder; + if (!transform_null_in) + null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map); + + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } + + return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); +} + +void Set::appendSetElements(SetKeyColumns & holder) +{ + if (holder.key_columns.size() != keys_size || set_elements.size() != keys_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of key columns for set. Expected {} got {} and {}", + keys_size, holder.key_columns.size(), set_elements.size()); + + size_t rows = holder.key_columns.at(0)->size(); + for (size_t i = 0; i < keys_size; ++i) + { + auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows); + if (set_elements[i]->empty()) + set_elements[i] = filtered_column; + else + set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + if (transform_null_in && holder.null_map_holder) + set_elements[i]->insert(Null{}); + } +} + +void Set::checkIsCreated() const +{ + if (!is_created.load()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); +} + +ColumnPtr returnColumnOrFilter(const ColumnPtr & first, const ColumnPtr & second) +{ + ConstantFilterDescription second_const_descr(*second); + if (second_const_descr.always_true) + return nullptr; + + if (second_const_descr.always_false) + return first; + + FilterDescription filter_descr(*second); + return first->filter(*filter_descr.data, 0); +} + +ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const +{ + size_t num_key_columns = columns.size(); + + if (0 == num_key_columns) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns passed to Set::execute method."); + + auto res = ColumnUInt8::create(); + ColumnUInt8::Container & vec_res = res->getData(); + vec_res.resize(columns.at(0).column->size()); + + if (vec_res.empty()) + return res; + + std::shared_lock lock(rwlock); + + /// If the set is empty. + if (data_types.empty()) + { + if (negative) + memset(vec_res.data(), 1, vec_res.size()); + else + memset(vec_res.data(), 0, vec_res.size()); + return res; + } + + checkColumnsNumber(num_key_columns); + + /// Remember the columns we will work with. Also check that the data types are correct. + ColumnRawPtrs key_columns; + key_columns.reserve(num_key_columns); + + /// The constant columns to the left of IN are not supported directly. For this, they first materialize. + Columns materialized_columns; + materialized_columns.reserve(num_key_columns); + + for (size_t i = 0; i < num_key_columns; ++i) + { + ColumnPtr result; + + const auto & column_before_cast = columns.at(i); + ColumnWithTypeAndName column_to_cast + = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name}; + + if (!transform_null_in && data_types[i]->canBeInsideNullable()) + { + result = castColumnAccurateOrNull(column_to_cast, data_types[i], cast_cache.get()); + } + else + { + result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); + } + + ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen + if (isDateTime64(column_before_cast.column->getDataType())) + col_to_emplace = returnColumnOrFilter(column_before_cast.column, res->getPtr()); + else + col_to_emplace = result; + + if (!col_to_emplace) + col_to_emplace = column_before_cast.column; + + materialized_columns.emplace_back() = col_to_emplace; + key_columns.emplace_back() = materialized_columns.back().get(); + } + + /// We will check existence in Set only for keys whose components do not contain any NULL value. + ConstNullMapPtr null_map{}; + ColumnPtr null_map_holder; + if (!transform_null_in) + null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + + executeOrdinary(key_columns, vec_res, negative, null_map); + + return res; +} + +bool Set::hasNull() const +{ + checkIsCreated(); + + if (!transform_null_in) + return false; + + if (data_types.size() != 1) + return false; + + if (!data_types[0]->isNullable()) + return false; + + auto col = data_types[0]->createColumn(); + col->insert(Field()); + auto res = execute({ColumnWithTypeAndName(std::move(col), data_types[0], std::string())}, false); + return res->getBool(0); +} + +bool Set::empty() const +{ + std::shared_lock lock(rwlock); + return data.empty(); +} + +size_t Set::getTotalRowCount() const +{ + std::shared_lock lock(rwlock); + return data.getTotalRowCount(); +} + +size_t Set::getTotalByteCount() const +{ + std::shared_lock lock(rwlock); + return data.getTotalByteCount(); +} + + +template +void NO_INLINE Set::executeImpl( + Method & method, + const ColumnRawPtrs & key_columns, + ColumnUInt8::Container & vec_res, + bool negative, + size_t rows, + ConstNullMapPtr null_map) const +{ + if (null_map) + executeImplCase(method, key_columns, vec_res, negative, rows, null_map); + else + executeImplCase(method, key_columns, vec_res, negative, rows, null_map); +} + + +template +void NO_INLINE Set::executeImplCase( + Method & method, + const ColumnRawPtrs & key_columns, + ColumnUInt8::Container & vec_res, + bool negative, + size_t rows, + ConstNullMapPtr null_map) const +{ + Arena pool; + typename Method::State state(key_columns, key_sizes, nullptr); + + /// NOTE Optimization is not used for consecutive identical strings. + + /// For all rows + for (size_t i = 0; i < rows; ++i) + { + if (has_null_map && (*null_map)[i]) + { + vec_res[i] = negative; + } + else + { + auto find_result = state.findKey(method.data, i, pool); + vec_res[i] = negative ^ find_result.isFound(); + } + } +} + + +void Set::executeOrdinary( + const ColumnRawPtrs & key_columns, + ColumnUInt8::Container & vec_res, + bool negative, + ConstNullMapPtr null_map) const +{ + size_t rows = key_columns[0]->size(); + + switch (data.type) + { + case SetVariants::Type::EMPTY: + break; +#define M(NAME) \ + case SetVariants::Type::NAME: \ + executeImpl(*data.NAME, key_columns, vec_res, negative, rows, null_map); \ + break; + APPLY_FOR_SET_VARIANTS(M) +#undef M + } +} + +void Set::checkColumnsNumber(size_t num_key_columns) const +{ + if (data_types.size() != num_key_columns) + { + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, + "Number of columns in section IN doesn't match. {} at left, {} at right.", + num_key_columns, data_types.size()); + } +} + +bool Set::areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const +{ + /// Out-of-bound access can happen when same set expression built with different columns. + /// Caller may call this method to make sure that the set is indeed the one they want + /// without awaring data_types.size(). + if (set_type_idx >= data_types.size()) + return false; + return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx])) + ->equals(*removeNullable(recursiveRemoveLowCardinality(other_type))); +} + +void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const +{ + if (!this->areTypesEqual(set_type_idx, other_type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Types of column {} in section IN don't match: " + "{} on the left, {} on the right", toString(set_type_idx + 1), + other_type->getName(), data_types[set_type_idx]->getName()); +} + +MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector && indexes_mapping_) + : has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_)) +{ + // std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex " + // << set_elements.size() << ' ' << indexes_mapping.size() << std::endl; + // for (const auto & vv : indexes_mapping) + // std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl; + + ::sort(indexes_mapping.begin(), indexes_mapping.end(), + [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) + { + return std::tie(l.key_index, l.tuple_index) < std::tie(r.key_index, r.tuple_index); + }); + + indexes_mapping.erase(std::unique( + indexes_mapping.begin(), indexes_mapping.end(), + [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) + { + return l.key_index == r.key_index; + }), indexes_mapping.end()); + + size_t tuple_size = indexes_mapping.size(); + ordered_set.resize(tuple_size); + + for (size_t i = 0; i < tuple_size; ++i) + ordered_set[i] = set_elements[indexes_mapping[i].tuple_index]; + + Block block_to_sort; + SortDescription sort_description; + for (size_t i = 0; i < tuple_size; ++i) + { + String column_name = "_" + toString(i); + block_to_sort.insert({ordered_set[i], nullptr, column_name}); + sort_description.emplace_back(column_name, 1, 1); + } + + sortBlock(block_to_sort, sort_description); + + for (size_t i = 0; i < tuple_size; ++i) + ordered_set[i] = block_to_sort.getByPosition(i).column; +} + + +/** Return the BoolMask where: + * 1: the intersection of the set and the range is non-empty + * 2: the range contains elements not in the set + */ +BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, const DataTypes & data_types, bool single_point) const +{ + size_t tuple_size = indexes_mapping.size(); + // std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl; + + FieldValues left_point; + FieldValues right_point; + left_point.reserve(tuple_size); + right_point.reserve(tuple_size); + + for (size_t i = 0; i < tuple_size; ++i) + { + left_point.emplace_back(ordered_set[i]->cloneEmpty()); + right_point.emplace_back(ordered_set[i]->cloneEmpty()); + } + + bool left_included = true; + bool right_included = true; + + for (size_t i = 0; i < tuple_size; ++i) + { + std::optional new_range = KeyCondition::applyMonotonicFunctionsChainToRange( + key_ranges[indexes_mapping[i].key_index], + indexes_mapping[i].functions, + data_types[indexes_mapping[i].key_index], + single_point); + + if (!new_range) + return {true, true}; + + left_point[i].update(new_range->left); + left_included &= new_range->left_included; + right_point[i].update(new_range->right); + right_included &= new_range->right_included; + } + + /// lhs < rhs return -1 + /// lhs == rhs return 0 + /// lhs > rhs return 1 + auto compare = [](const IColumn & lhs, const FieldValue & rhs, size_t row) + { + if (rhs.isNegativeInfinity()) + return 1; + if (rhs.isPositiveInfinity()) + { + Field f; + lhs.get(row, f); + if (f.isNull()) + return 0; // +Inf == +Inf + else + return -1; + } + return lhs.compareAt(row, 0, *rhs.column, 1); + }; + + auto less = [this, &compare, tuple_size](size_t row, const auto & point) + { + for (size_t i = 0; i < tuple_size; ++i) + { + int res = compare(*ordered_set[i], point[i], row); + if (res) + return res < 0; + } + return false; + }; + + auto equals = [this, &compare, tuple_size](size_t row, const auto & point) + { + for (size_t i = 0; i < tuple_size; ++i) + if (compare(*ordered_set[i], point[i], row) != 0) + return false; + return true; + }; + + /** Because each hyperrectangle maps to a contiguous sequence of elements + * laid out in the lexicographically increasing order, the set intersects the range + * if and only if either bound coincides with an element or at least one element + * is between the lower bounds + */ + auto indices = collections::range(0, size()); + auto left_lower = std::lower_bound(indices.begin(), indices.end(), left_point, less); + auto right_lower = std::lower_bound(indices.begin(), indices.end(), right_point, less); + + /// A special case of 1-element KeyRange. It's useful for partition pruning. + bool one_element_range = true; + for (size_t i = 0; i < tuple_size; ++i) + { + auto & left = left_point[i]; + auto & right = right_point[i]; + if (left.isNormal() && right.isNormal()) + { + if (0 != left.column->compareAt(0, 0, *right.column, 1)) + { + one_element_range = false; + break; + } + } + else if ((left.isPositiveInfinity() && right.isPositiveInfinity()) || (left.isNegativeInfinity() && right.isNegativeInfinity())) + { + /// Special value equality. + } + else + { + one_element_range = false; + break; + } + } + if (one_element_range && has_all_keys) + { + /// Here we know that there is one element in range. + /// The main difference with the normal case is that we can definitely say that + /// condition in this range is always TRUE (can_be_false = 0) or always FALSE (can_be_true = 0). + + /// Check if it's an empty range + if (!left_included || !right_included) + return {false, true}; + else if (left_lower != indices.end() && equals(*left_lower, left_point)) + return {true, false}; + else + return {false, true}; + } + + /// If there are more than one element in the range, it can always be false. Thus we only need to check if it may be true or not. + /// Given left_lower >= left_point, right_lower >= right_point, find if there may be a match in between left_lower and right_lower. + if (left_lower + 1 < right_lower) + { + /// There is a point in between: left_lower + 1 + return {true, true}; + } + else if (left_lower + 1 == right_lower) + { + /// Need to check if left_lower is a valid match, as left_point <= left_lower < right_point <= right_lower. + /// Note: left_lower is valid. + if (left_included || !equals(*left_lower, left_point)) + return {true, true}; + + /// We are unlucky that left_point fails to cover a point. Now we need to check if right_point can cover right_lower. + /// Check if there is a match at the right boundary. + return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), true}; + } + else // left_lower == right_lower + { + /// Need to check if right_point is a valid match, as left_point < right_point <= left_lower = right_lower. + /// Check if there is a match at the left boundary. + return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), true}; + } +} + +bool MergeTreeSetIndex::hasMonotonicFunctionsChain() const +{ + for (const auto & mapping : indexes_mapping) + if (!mapping.functions.empty()) + return true; + return false; +} + +void FieldValue::update(const Field & x) +{ + if (x.isNegativeInfinity() || x.isPositiveInfinity()) + value = x; + else + { + /// Keep at most one element in column. + if (!column->empty()) + column->popBack(1); + column->insert(x); + value = Field(); // Set back to normal value. + } +} + +} From e852d0b463348f87411c5a9e241773b6f14003ca Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:01:42 +0200 Subject: [PATCH 018/816] fix fuzzers --- src/Interpreters/Set.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 0d498c164c7..c0f14f3a4c7 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -290,6 +290,8 @@ ColumnPtr returnColumnOrFilter(const ColumnPtr & first, const ColumnPtr & second return first; FilterDescription filter_descr(*second); + if (!filter_descr.data) + return nullptr; return first->filter(*filter_descr.data, 0); } From c39d7092d00056aebfae55fae3b11039b53905f1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Aug 2024 15:57:42 +0000 Subject: [PATCH 019/816] better rollbacks of columns --- src/Columns/ColumnArray.cpp | 11 ++++ src/Columns/ColumnArray.h | 3 ++ src/Columns/ColumnDynamic.h | 10 ++++ src/Columns/ColumnMap.cpp | 10 ++++ src/Columns/ColumnMap.h | 2 + src/Columns/ColumnNullable.cpp | 11 ++++ src/Columns/ColumnNullable.h | 3 ++ src/Columns/ColumnObject.cpp | 52 +++++++++++++++++++ src/Columns/ColumnObject.h | 2 + src/Columns/ColumnSparse.cpp | 16 ++++++ src/Columns/ColumnSparse.h | 3 ++ src/Columns/ColumnTuple.cpp | 21 ++++++++ src/Columns/ColumnTuple.h | 2 + src/Columns/ColumnVariant.cpp | 23 ++++++++ src/Columns/ColumnVariant.h | 2 + src/Columns/IColumn.h | 41 +++++++++++++++ src/Interpreters/AsynchronousInsertQueue.cpp | 7 ++- .../Executors/StreamingFormatExecutor.cpp | 18 +++++-- .../Executors/StreamingFormatExecutor.h | 7 ++- src/Storages/FileLog/FileLogSource.cpp | 13 ++--- src/Storages/Kafka/KafkaSource.cpp | 13 ++--- src/Storages/Kafka/StorageKafka2.cpp | 13 ++--- src/Storages/NATS/NATSSource.cpp | 13 ++--- src/Storages/RabbitMQ/RabbitMQSource.cpp | 13 ++--- .../03230_async_insert_native.reference | 0 .../0_stateless/03230_async_insert_native.sh | 23 ++++++++ 26 files changed, 282 insertions(+), 50 deletions(-) create mode 100644 tests/queries/0_stateless/03230_async_insert_native.reference create mode 100755 tests/queries/0_stateless/03230_async_insert_native.sh diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 83d4c24c769..4f3d0f0ec4b 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -369,6 +369,17 @@ void ColumnArray::popBack(size_t n) offsets_data.resize_assume_reserved(offsets_data.size() - n); } +ColumnCheckpointPtr ColumnArray::getCheckpoint() const +{ + return std::make_shared(size(), getData().getCheckpoint()); +} + +void ColumnArray::rollback(const ColumnCheckpoint & checkpoint) +{ + getOffsets().resize_assume_reserved(checkpoint.size); + getData().rollback(*assert_cast(checkpoint).nested); +} + int ColumnArray::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator) const { const ColumnArray & rhs = assert_cast(rhs_); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index f77268a8be6..fd42d502c16 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -161,6 +161,9 @@ public: ColumnPtr compress() const override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; + void forEachSubcolumn(MutableColumnCallback callback) override { callback(offsets); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..115a3bc20c5 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -304,6 +304,16 @@ public: variant_column_ptr->protect(); } + ColumnCheckpointPtr getCheckpoint() const override + { + return variant_column_ptr->getCheckpoint(); + } + + void rollback(const ColumnCheckpoint & checkpoint) override + { + variant_column_ptr->rollback(checkpoint); + } + void forEachSubcolumn(MutableColumnCallback callback) override { callback(variant_column); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 536da4d06d0..0561e8f398f 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -312,6 +312,16 @@ void ColumnMap::getExtremes(Field & min, Field & max) const max = std::move(map_max_value); } +ColumnCheckpointPtr ColumnMap::getCheckpoint() const +{ + return nested->getCheckpoint(); +} + +void ColumnMap::rollback(const ColumnCheckpoint & checkpoint) +{ + nested->rollback(checkpoint); +} + void ColumnMap::forEachSubcolumn(MutableColumnCallback callback) { callback(nested); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 39d15a586b9..d534cfa7295 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -102,6 +102,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index ec375ea5a8d..560d37721ad 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -305,6 +305,17 @@ void ColumnNullable::popBack(size_t n) getNullMapColumn().popBack(n); } +ColumnCheckpointPtr ColumnNullable::getCheckpoint() const +{ + return std::make_shared(size(), nested_column->getCheckpoint()); +} + +void ColumnNullable::rollback(const ColumnCheckpoint & checkpoint) +{ + getNullMapData().resize_assume_reserved(checkpoint.size); + nested_column->rollback(*assert_cast(checkpoint).nested); +} + ColumnPtr ColumnNullable::filter(const Filter & filt, ssize_t result_size_hint) const { ColumnPtr filtered_data = getNestedColumn().filter(filt, result_size_hint); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 78274baca51..39b326a1c44 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -143,6 +143,9 @@ public: ColumnPtr compress() const override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; + void forEachSubcolumn(MutableColumnCallback callback) override { callback(nested_column); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e397b03b69e..6be6e9d833e 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -30,6 +30,23 @@ const std::shared_ptr & getDynamicSerialization() return dynamic_serialization; } +struct ColumnObjectCheckpoint : public ColumnCheckpoint +{ + using CheckpointsMap = std::unordered_map; + + ColumnObjectCheckpoint(size_t size_, CheckpointsMap typed_paths_, CheckpointsMap dynamic_paths_, ColumnCheckpointPtr shared_data_) + : ColumnCheckpoint(size_) + , typed_paths(std::move(typed_paths_)) + , dynamic_paths(std::move(dynamic_paths_)) + , shared_data(std::move(shared_data_)) + { + } + + CheckpointsMap typed_paths; + CheckpointsMap dynamic_paths; + ColumnCheckpointPtr shared_data; +}; + } ColumnObject::ColumnObject( @@ -655,6 +672,41 @@ void ColumnObject::popBack(size_t n) shared_data->popBack(n); } +ColumnCheckpointPtr ColumnObject::getCheckpoint() const +{ + auto get_checkpoints = [](const auto & columns) + { + std::unordered_map checkpoints; + for (const auto & [name, column] : columns) + checkpoints[name] = column->getCheckpoint(); + + return checkpoints; + }; + + return std::make_shared(size(), get_checkpoints(typed_paths), get_checkpoints(dynamic_paths_ptrs), shared_data->getCheckpoint()); +} + +void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) +{ + const auto & object_checkpoint = assert_cast(checkpoint); + + for (auto & [name, column] : typed_paths) + { + const auto & nested_checkpoint = object_checkpoint.typed_paths.at(name); + chassert(nested_checkpoint); + column->rollback(*nested_checkpoint); + } + + for (auto & [name, column] : dynamic_paths_ptrs) + { + const auto & nested_checkpoint = object_checkpoint.dynamic_paths.at(name); + chassert(nested_checkpoint); + column->rollback(*nested_checkpoint); + } + + shared_data->rollback(*object_checkpoint.shared_data); +} + StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const { StringRef res(begin, 0); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index f530ed29ef3..84fe2dcafad 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -159,6 +159,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index a908d970a15..0c91174007c 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -308,6 +308,22 @@ void ColumnSparse::popBack(size_t n) _size = new_size; } +ColumnCheckpointPtr ColumnSparse::getCheckpoint() const +{ + return std::make_shared(size(), values->getCheckpoint()); +} + +void ColumnSparse::rollback(const ColumnCheckpoint & checkpoint) +{ + _size = checkpoint.size; + + const auto & nested = *assert_cast(checkpoint).nested; + chassert(nested.size > 0); + + values->rollback(nested); + getOffsetsData().resize_assume_reserved(nested.size - 1); +} + ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const { if (_size != filt.size()) diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 7a4d914e62a..dabf38a252d 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -149,6 +149,9 @@ public: ColumnPtr compress() const override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; + void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index e741eb51c68..3819ba3352b 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -254,6 +254,27 @@ void ColumnTuple::popBack(size_t n) column->popBack(n); } +ColumnCheckpointPtr ColumnTuple::getCheckpoint() const +{ + ColumnCheckpoints checkpoints; + checkpoints.reserve(columns.size()); + + for (const auto & column : columns) + checkpoints.push_back(column->getCheckpoint()); + + return std::make_shared(size(), std::move(checkpoints)); +} + +void ColumnTuple::rollback(const ColumnCheckpoint & checkpoint) +{ + column_length = checkpoint.size; + const auto & checkpoints = assert_cast(checkpoint).nested; + + chassert(columns.size() == checkpoints.size()); + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->rollback(*checkpoints[i]); +} + StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { if (columns.empty()) diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 6968294aef9..74c4dd1ffd3 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -118,6 +118,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index c6511695f5c..f73d074e726 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -739,6 +739,29 @@ void ColumnVariant::popBack(size_t n) offsets->popBack(n); } +ColumnCheckpointPtr ColumnVariant::getCheckpoint() const +{ + ColumnCheckpoints checkpoints; + checkpoints.reserve(variants.size()); + + for (const auto & column : variants) + checkpoints.push_back(column->getCheckpoint()); + + return std::make_shared(size(), std::move(checkpoints)); +} + +void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) +{ + getOffsets().resize_assume_reserved(checkpoint.size); + getLocalDiscriminators().resize_assume_reserved(checkpoint.size); + + const auto & checkpoints = assert_cast(checkpoint).nested; + chassert(variants.size() == checkpoints.size()); + + for (size_t i = 0; i < variants.size(); ++i) + variants[i]->rollback(*checkpoints[i]); +} + StringRef ColumnVariant::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { /// During any serialization/deserialization we should always use global discriminators. diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 925eab74af8..c7182467611 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -248,6 +248,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; + ColumnCheckpointPtr getCheckpoint() const override; + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index e4fe233ffdf..61c1891a7a7 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -49,6 +49,40 @@ struct EqualRange using EqualRanges = std::vector; +/// A checkpoint that contains size of column and all its subcolumns. +/// It can be used to rollback column to the previous state, for example +/// after failed parsing when column may be in inconsistent state. +struct ColumnCheckpoint +{ + explicit ColumnCheckpoint(size_t size_) : size(size_) {} + size_t size = 0; +}; + +using ColumnCheckpointPtr = std::shared_ptr; +using ColumnCheckpoints = std::vector; + +struct ColumnCheckpointWithNested : public ColumnCheckpoint +{ + ColumnCheckpointWithNested(size_t size_, ColumnCheckpointPtr nested_) + : ColumnCheckpoint(size_) + , nested(std::move(nested_)) + { + } + + ColumnCheckpointPtr nested; +}; + +struct ColumnCheckpointWithNestedTuple : public ColumnCheckpoint +{ + ColumnCheckpointWithNestedTuple(size_t size_, ColumnCheckpoints nested_) + : ColumnCheckpoint(size_) + , nested(std::move(nested_)) + { + } + + ColumnCheckpoints nested; +}; + /// Declares interface to store columns in memory. class IColumn : public COW { @@ -509,6 +543,13 @@ public: /// The operation is slow and performed only for debug builds. virtual void protect() {} + /// Returns checkpoint of current state of column. + virtual ColumnCheckpointPtr getCheckpoint() const { return std::make_shared(size()); } + + /// Rollbacks column to the checkpoint. + /// Unlike 'popBack' this method should work correctly even if column has invalid state. + virtual void rollback(const ColumnCheckpoint & checkpoint) { popBack(size() - checkpoint.size); } + /// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them. /// Shallow: doesn't do recursive calls; don't do call for itself. diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..3a2732ae837 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -971,15 +971,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( adding_defaults_transform = std::make_shared(header, columns, *format, insert_context); } - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { current_exception = e.displayText(); LOG_ERROR(logger, "Failed parsing for query '{}' with query id {}. {}", key.query_str, current_entry->query_id, current_exception); - for (const auto & column : result_columns) - if (column->size() > total_rows) - column->popBack(column->size() - total_rows); + for (size_t i = 0; i < result_columns.size(); ++i) + result_columns[i]->rollback(*checkpoints[i]); current_entry->finish(std::current_exception()); return 0; diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 12dd685a735..c1a92317df6 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -21,6 +21,7 @@ StreamingFormatExecutor::StreamingFormatExecutor( , adding_defaults_transform(std::move(adding_defaults_transform_)) , port(format->getPort().getHeader(), format.get()) , result_columns(header.cloneEmptyColumns()) + , checkpoints(result_columns.size()) { connect(format->getPort(), port); } @@ -45,6 +46,8 @@ size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) size_t StreamingFormatExecutor::execute() { + setCheckpoints(); + try { size_t new_rows = 0; @@ -77,19 +80,19 @@ size_t StreamingFormatExecutor::execute() catch (Exception & e) { format->resetParser(); - return on_error(result_columns, e); + return on_error(result_columns, checkpoints, e); } catch (std::exception & e) { format->resetParser(); auto exception = Exception(Exception::CreateFromSTDTag{}, e); - return on_error(result_columns, exception); + return on_error(result_columns, checkpoints, exception); } catch (...) { format->resetParser(); - auto exception = Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknowk exception while executing StreamingFormatExecutor with format {}", format->getName()); - return on_error(result_columns, exception); + auto exception = Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception while executing StreamingFormatExecutor with format {}", format->getName()); + return on_error(result_columns, checkpoints, exception); } } @@ -106,4 +109,11 @@ size_t StreamingFormatExecutor::insertChunk(Chunk chunk) return chunk_rows; } +void StreamingFormatExecutor::setCheckpoints() +{ + for (size_t i = 0; i < result_columns.size(); ++i) + checkpoints[i] = result_columns[i]->getCheckpoint(); +} + + } diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index 3aa90ab0360..2c7e6f9a0c6 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -19,12 +19,12 @@ public: /// and exception to rethrow it or add context to it. /// Should return number of new rows, which are added in callback /// to result columns in comparison to previous call of `execute`. - using ErrorCallback = std::function; + using ErrorCallback = std::function; StreamingFormatExecutor( const Block & header_, InputFormatPtr format_, - ErrorCallback on_error_ = [](const MutableColumns &, Exception & e) -> size_t { throw std::move(e); }, + ErrorCallback on_error_ = [](const MutableColumns &, const ColumnCheckpoints, Exception & e) -> size_t { throw std::move(e); }, SimpleTransformPtr adding_defaults_transform_ = nullptr); /// Returns numbers of new read rows. @@ -40,6 +40,8 @@ public: MutableColumns getResultColumns(); private: + void setCheckpoints(); + const Block header; const InputFormatPtr format; const ErrorCallback on_error; @@ -47,6 +49,7 @@ private: InputPort port; MutableColumns result_columns; + ColumnCheckpoints checkpoints; }; } diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index eb3ff0436a5..36faa28ac6a 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -86,21 +86,18 @@ Chunk FileLogSource::generate() std::optional exception_message; size_t total_rows = 0; - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { if (handle_error_mode == StreamingHandleErrorMode::STREAM) { exception_message = e.message(); - for (const auto & column : result_columns) + for (size_t i = 0; i < result_columns.size(); ++i) { - // We could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); + // We could already push some rows to result_columns before exception, we need to fix it. + result_columns[i]->rollback(*checkpoints[i]); // All data columns will get default value in case of error. - column->insertDefault(); + result_columns[i]->insertDefault(); } return 1; diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 3ddd0d1be8c..f03d13a2837 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -108,23 +108,20 @@ Chunk KafkaSource::generateImpl() size_t total_rows = 0; size_t failed_poll_attempts = 0; - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); if (put_error_to_stream) { exception_message = e.message(); - for (const auto & column : result_columns) + for (size_t i = 0; i < result_columns.size(); ++i) { - // read_kafka_message could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); + // We could already push some rows to result_columns before exception, we need to fix it. + result_columns[i]->rollback(*checkpoints[i]); // all data columns will get default value in case of error - column->insertDefault(); + result_columns[i]->insertDefault(); } return 1; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index 3574b46e3b0..3275a38b55a 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -817,23 +817,20 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( size_t total_rows = 0; size_t failed_poll_attempts = 0; - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { ProfileEvents::increment(ProfileEvents::KafkaMessagesFailed); if (put_error_to_stream) { exception_message = e.message(); - for (const auto & column : result_columns) + for (size_t i = 0; i < result_columns.size(); ++i) { - // read_kafka_message could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); + // We could already push some rows to result_columns before exception, we need to fix it. + result_columns[i]->rollback(*checkpoints[i]); // all data columns will get default value in case of error - column->insertDefault(); + result_columns[i]->insertDefault(); } return 1; diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index 54f479faacc..bc15e9794cd 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -102,21 +102,18 @@ Chunk NATSSource::generate() storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); std::optional exception_message; size_t total_rows = 0; - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { if (handle_error_mode == StreamingHandleErrorMode::STREAM) { exception_message = e.message(); - for (const auto & column : result_columns) + for (size_t i = 0; i < result_columns.size(); ++i) { - // We could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); + // We could already push some rows to result_columns before exception, we need to fix it. + result_columns[i]->rollback(*checkpoints[i]); // All data columns will get default value in case of error. - column->insertDefault(); + result_columns[i]->insertDefault(); } return 1; diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 15d013245d3..40e85cb06ed 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -161,21 +161,18 @@ Chunk RabbitMQSource::generateImpl() std::optional exception_message; size_t total_rows = 0; - auto on_error = [&](const MutableColumns & result_columns, Exception & e) + auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) { if (handle_error_mode == StreamingHandleErrorMode::STREAM) { exception_message = e.message(); - for (const auto & column : result_columns) + for (size_t i = 0; i < result_columns.size(); ++i) { - // We could already push some rows to result_columns - // before exception, we need to fix it. - auto cur_rows = column->size(); - if (cur_rows > total_rows) - column->popBack(cur_rows - total_rows); + // We could already push some rows to result_columns before exception, we need to fix it. + result_columns[i]->rollback(*checkpoints[i]); // All data columns will get default value in case of error. - column->insertDefault(); + result_columns[i]->insertDefault(); } return 1; diff --git a/tests/queries/0_stateless/03230_async_insert_native.reference b/tests/queries/0_stateless/03230_async_insert_native.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03230_async_insert_native.sh b/tests/queries/0_stateless/03230_async_insert_native.sh new file mode 100755 index 00000000000..5ac3e40fa31 --- /dev/null +++ b/tests/queries/0_stateless/03230_async_insert_native.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q " + DROP TABLE IF EXISTS async_inserts_native; + CREATE TABLE async_inserts_native (m Map(UInt64, UInt64), v UInt64 MATERIALIZED m[4]) ENGINE = Memory; +" + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=1000&async_insert_busy_timeout_min_ms=1000&wait_for_async_insert=1" + +# This test runs inserts with memory_tracker_fault_probability > 0 to trigger memory limit during insertion. +# If rollback of columns is wrong in that case it may produce LOGICAL_ERROR and it will caught by termintation of server in debug mode. +for _ in {1..10}; do + ${CLICKHOUSE_CLIENT} -q "SELECT (range(number), range(number))::Map(UInt64, UInt64) AS m FROM numbers(1000) FORMAT Native" | \ + ${CLICKHOUSE_CURL} -sS -X POST "${url}&max_block_size=100&memory_tracker_fault_probability=0.01&query=INSERT+INTO+async_inserts_native+FORMAT+Native" --data-binary @- >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts_native;" From eb0ae55e0297081a4e713ebdde6ddde232e71acc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 28 Aug 2024 21:36:46 +0000 Subject: [PATCH 020/816] better rollbacks of columns --- src/Columns/ColumnTuple.cpp | 4 ++-- src/Columns/ColumnVariant.cpp | 4 ++-- src/Columns/IColumn.h | 15 ++++++++------- src/Processors/Formats/IRowInputFormat.cpp | 13 ++++++------- .../03231_bson_tuple_array_map.reference | 0 .../0_stateless/03231_bson_tuple_array_map.sh | 18 ++++++++++++++++++ 6 files changed, 36 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/03231_bson_tuple_array_map.reference create mode 100755 tests/queries/0_stateless/03231_bson_tuple_array_map.sh diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 3819ba3352b..65f3285bcfc 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -262,13 +262,13 @@ ColumnCheckpointPtr ColumnTuple::getCheckpoint() const for (const auto & column : columns) checkpoints.push_back(column->getCheckpoint()); - return std::make_shared(size(), std::move(checkpoints)); + return std::make_shared(size(), std::move(checkpoints)); } void ColumnTuple::rollback(const ColumnCheckpoint & checkpoint) { column_length = checkpoint.size; - const auto & checkpoints = assert_cast(checkpoint).nested; + const auto & checkpoints = assert_cast(checkpoint).nested; chassert(columns.size() == checkpoints.size()); for (size_t i = 0; i < columns.size(); ++i) diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index f73d074e726..a8cb779ad16 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -747,7 +747,7 @@ ColumnCheckpointPtr ColumnVariant::getCheckpoint() const for (const auto & column : variants) checkpoints.push_back(column->getCheckpoint()); - return std::make_shared(size(), std::move(checkpoints)); + return std::make_shared(size(), std::move(checkpoints)); } void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) @@ -755,7 +755,7 @@ void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) getOffsets().resize_assume_reserved(checkpoint.size); getLocalDiscriminators().resize_assume_reserved(checkpoint.size); - const auto & checkpoints = assert_cast(checkpoint).nested; + const auto & checkpoints = assert_cast(checkpoint).nested; chassert(variants.size() == checkpoints.size()); for (size_t i = 0; i < variants.size(); ++i) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 61c1891a7a7..53f31d2b96d 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -54,8 +54,10 @@ using EqualRanges = std::vector; /// after failed parsing when column may be in inconsistent state. struct ColumnCheckpoint { + size_t size; + explicit ColumnCheckpoint(size_t size_) : size(size_) {} - size_t size = 0; + virtual ~ColumnCheckpoint() = default; }; using ColumnCheckpointPtr = std::shared_ptr; @@ -64,19 +66,17 @@ using ColumnCheckpoints = std::vector; struct ColumnCheckpointWithNested : public ColumnCheckpoint { ColumnCheckpointWithNested(size_t size_, ColumnCheckpointPtr nested_) - : ColumnCheckpoint(size_) - , nested(std::move(nested_)) + : ColumnCheckpoint(size_), nested(std::move(nested_)) { } ColumnCheckpointPtr nested; }; -struct ColumnCheckpointWithNestedTuple : public ColumnCheckpoint +struct ColumnCheckpointWithMultipleNested : public ColumnCheckpoint { - ColumnCheckpointWithNestedTuple(size_t size_, ColumnCheckpoints nested_) - : ColumnCheckpoint(size_) - , nested(std::move(nested_)) + ColumnCheckpointWithMultipleNested(size_t size_, ColumnCheckpoints nested_) + : ColumnCheckpoint(size_), nested(std::move(nested_)) { } @@ -548,6 +548,7 @@ public: /// Rollbacks column to the checkpoint. /// Unlike 'popBack' this method should work correctly even if column has invalid state. + /// Sizes of columns in checkpoint must be less or equal than current. virtual void rollback(const ColumnCheckpoint & checkpoint) { popBack(size() - checkpoint.size); } /// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them. diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 0b6c81923db..2a0695764b2 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -104,6 +104,7 @@ Chunk IRowInputFormat::read() size_t num_columns = header.columns(); MutableColumns columns = header.cloneEmptyColumns(); + ColumnCheckpoints checkpoints(columns.size()); block_missing_values.clear(); @@ -130,6 +131,9 @@ Chunk IRowInputFormat::read() { try { + for (size_t column_idx = 0; column_idx < columns.size(); ++column_idx) + checkpoints[column_idx] = columns[column_idx]->getCheckpoint(); + info.read_columns.clear(); continue_reading = readRow(columns, info); @@ -193,14 +197,9 @@ Chunk IRowInputFormat::read() syncAfterError(); - /// Truncate all columns in block to initial size (remove values, that was appended to only part of columns). - + /// Rollback all columns in block to initial size (remove values, that was appended to only part of columns). for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) - { - auto & column = columns[column_idx]; - if (column->size() > num_rows) - column->popBack(column->size() - num_rows); - } + columns[column_idx]->rollback(*checkpoints[column_idx]); } } } diff --git a/tests/queries/0_stateless/03231_bson_tuple_array_map.reference b/tests/queries/0_stateless/03231_bson_tuple_array_map.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03231_bson_tuple_array_map.sh b/tests/queries/0_stateless/03231_bson_tuple_array_map.sh new file mode 100755 index 00000000000..600b15fb70a --- /dev/null +++ b/tests/queries/0_stateless/03231_bson_tuple_array_map.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.data + +$CLICKHOUSE_LOCAL -q "select tuple(1, x'00000000000000000000FFFF0000000000') as x format BSONEachRow" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', BSONEachRow, 'x Tuple(UInt32, IPv6)') settings input_format_allow_errors_num=1" + +$CLICKHOUSE_LOCAL -q "select [x'00000000000000000000FFFF00000000', x'00000000000000000000FFFF0000000000'] as x format BSONEachRow" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', BSONEachRow, 'x Array(IPv6)') settings input_format_allow_errors_num=1" + +$CLICKHOUSE_LOCAL -q "select map('key1', x'00000000000000000000FFFF00000000', 'key2', x'00000000000000000000FFFF0000000000') as x format BSONEachRow" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', BSONEachRow, 'x Map(String, IPv6)') settings input_format_allow_errors_num=1" + +rm $DATA_FILE From bb2716251b46ebacaf5b3767f5d12d1b2870a68c Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 31 Aug 2024 19:44:31 +0000 Subject: [PATCH 021/816] wip: WORKLOADs and RESOURCEs added --- src/CMakeLists.txt | 1 + src/Common/ErrorCodes.cpp | 2 + .../Workload/IWorkloadEntityStorage.h | 77 +++++ .../Workload/WorkloadEntityDiskStorage.cpp | 296 ++++++++++++++++++ .../Workload/WorkloadEntityDiskStorage.h | 48 +++ .../Workload/WorkloadEntityKeeperStorage.cpp | 0 .../Workload/WorkloadEntityKeeperStorage.h | 0 .../Workload/WorkloadEntityStorageBase.cpp | 195 ++++++++++++ .../Workload/WorkloadEntityStorageBase.h | 73 +++++ src/Parsers/ASTCreateResourceQuery.cpp | 47 +++ src/Parsers/ASTCreateResourceQuery.h | 32 ++ src/Parsers/ASTCreateWorkloadQuery.cpp | 67 ++++ src/Parsers/ASTCreateWorkloadQuery.h | 35 +++ src/Parsers/CommonParsers.h | 2 + src/Parsers/ParserCreateResourceQuery.cpp | 62 ++++ src/Parsers/ParserCreateResourceQuery.h | 16 + src/Parsers/ParserCreateWorkloadQuery.cpp | 76 +++++ src/Parsers/ParserCreateWorkloadQuery.h | 16 + 18 files changed, 1045 insertions(+) create mode 100644 src/Common/Scheduler/Workload/IWorkloadEntityStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h create mode 100644 src/Parsers/ASTCreateResourceQuery.cpp create mode 100644 src/Parsers/ASTCreateResourceQuery.h create mode 100644 src/Parsers/ASTCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ASTCreateWorkloadQuery.h create mode 100644 src/Parsers/ParserCreateResourceQuery.cpp create mode 100644 src/Parsers/ParserCreateResourceQuery.h create mode 100644 src/Parsers/ParserCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ParserCreateWorkloadQuery.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1889bba3b39..5b20bf6c27d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -114,6 +114,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) +add_headers_and_sources(dbms Common/Scheduler/Workload) if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1055b3d34db..492854cba53 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -609,6 +609,8 @@ M(728, UNEXPECTED_DATA_TYPE) \ M(729, ILLEGAL_TIME_SERIES_TAGS) \ M(730, REFRESH_FAILED) \ + M(731, WORKLOAD_ENTITY_ALREADY_EXISTS) \ + M(732, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h new file mode 100644 index 00000000000..65978a71be0 --- /dev/null +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -0,0 +1,77 @@ +#pragma once + +#include + +#include + +#include + + +namespace DB +{ + +class IAST; +struct Settings; + +enum class WorkloadEntityType : uint8_t +{ + Workload, + Resource, +}; + +/// Interface for a storage of workload entities (WORKLOAD and RESOURCE). +class IWorkloadEntityStorage +{ +public: + virtual ~IWorkloadEntityStorage() = default; + + /// Whether this storage can replicate entities to another node. + virtual bool isReplicated() const { return false; } + virtual String getReplicationID() const { return ""; } + + /// Loads all entities. Can be called once - if entities are already loaded the function does nothing. + virtual void loadEntities() = 0; + + /// Get entity by name. If no entity stored with entity_name throws exception. + virtual ASTPtr get(const String & entity_name) const = 0; + + /// Get entity by name. If no entity stored with entity_name return nullptr. + virtual ASTPtr tryGet(const String & entity_name) const = 0; + + /// Check if entity with entity_name is stored. + virtual bool has(const String & entity_name) const = 0; + + /// Get all entity names. + virtual std::vector getAllEntityNames() const = 0; + + /// Get all entities. + virtual std::vector> getAllEntities() const = 0; + + /// Check whether any entity have been stored. + virtual bool empty() const = 0; + + /// Stops watching. + virtual void stopWatching() {} + + /// Immediately reloads all entities, throws an exception if failed. + virtual void reloadEntities() = 0; + + /// Stores an entity. + virtual bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + /// Removes an entity. + virtual bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp new file mode 100644 index 00000000000..d9ca8bca0a0 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -0,0 +1,296 @@ +#include + +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include +#include + +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DIRECTORY_DOESNT_EXIST; + extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; + extern const int UNKNOWN_WORKLOAD_ENTITY; +} + + +namespace +{ + /// Converts a path to an absolute path and append it with a separator. + String makeDirectoryPathCanonical(const String & directory_path) + { + auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path); + if (canonical_directory_path.has_filename()) + canonical_directory_path += std::filesystem::path::preferred_separator; + return canonical_directory_path; + } +} + +WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) + : WorkloadEntityStorageBase(global_context_) + , dir_path{makeDirectoryPathCanonical(dir_path_)} + , log{getLogger("WorkloadEntityDiskStorage")} +{ +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name) +{ + return tryLoadEntity(entity_type, entity_name, getFilePath(entity_type, entity_name), /* check_file_exists= */ true); +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & path, bool check_file_exists) +{ + LOG_DEBUG(log, "Loading workload entity {} from file {}", backQuote(entity_name), path); + + try + { + if (check_file_exists && !fs::exists(path)) + return nullptr; + + /// There is .sql file with workload entity creation statement. + ReadBufferFromFile in(path); + + String entity_create_query; + readStringUntilEOF(entity_create_query, in); + + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + ParserCreateWorkloadQuery parser; + ASTPtr ast = parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef().max_parser_depth, + global_context->getSettingsRef().max_parser_backtracks); + return ast; + } + case WorkloadEntityType::Resource: + { + ParserCreateResourceQuery parser; + ASTPtr ast = parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef().max_parser_depth, + global_context->getSettingsRef().max_parser_backtracks); + return ast; + } + } + } + catch (...) + { + tryLogCurrentException(log, fmt::format("while loading workload entity {} from path {}", backQuote(entity_name), path)); + return nullptr; /// Failed to load this entity, will ignore it + } +} + + +void WorkloadEntityDiskStorage::loadEntities() +{ + if (!entities_loaded) + loadEntitiesImpl(); +} + + +void WorkloadEntityDiskStorage::reloadEntities() +{ + loadEntitiesImpl(); +} + + +void WorkloadEntityDiskStorage::loadEntitiesImpl() +{ + LOG_INFO(log, "Loading workload entities from {}", dir_path); + + if (!std::filesystem::exists(dir_path)) + { + LOG_DEBUG(log, "The directory for workload entities ({}) does not exist: nothing to load", dir_path); + return; + } + + std::vector> entities_name_and_queries; + + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + if (it->isDirectory()) + continue; + + const String & file_name = it.name(); + + if (startsWith(file_name, "workload_") && endsWith(file_name, ".sql")) + { + size_t prefix_length = strlen("workload_"); + size_t suffix_length = strlen(".sql"); + String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Workload, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + + if (startsWith(file_name, "resource_") && endsWith(file_name, ".sql")) + { + size_t prefix_length = strlen("resource_"); + size_t suffix_length = strlen(".sql"); + String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Resource, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + } + + setAllEntities(entities_name_and_queries); + entities_loaded = true; + + LOG_DEBUG(log, "Workload entities loaded"); +} + + +void WorkloadEntityDiskStorage::createDirectory() +{ + std::error_code create_dir_error_code; + fs::create_directories(dir_path, create_dir_error_code); + if (!fs::exists(dir_path) || !fs::is_directory(dir_path) || create_dir_error_code) + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "Couldn't create directory {} reason: '{}'", + dir_path, create_dir_error_code.message()); +} + + +bool WorkloadEntityDiskStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + createDirectory(); + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Storing workload entity {} to file {}", backQuote(entity_name), file_path); + + if (fs::exists(file_path)) + { + if (throw_if_exists) + throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + } + + WriteBufferFromOwnString create_statement_buf; + formatAST(*create_entity_query, create_statement_buf, false); + writeChar('\n', create_statement_buf); + String create_statement = create_statement_buf.str(); + + String temp_file_path = file_path + ".tmp"; + + try + { + WriteBufferFromFile out(temp_file_path, create_statement.size()); + writeString(create_statement, out); + out.next(); + if (settings.fsync_metadata) + out.sync(); + out.close(); + + if (replace_if_exists) + fs::rename(temp_file_path, file_path); + else + renameNoReplace(temp_file_path, file_path); + } + catch (...) + { + fs::remove(temp_file_path); + throw; + } + + LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); + return true; +} + + +bool WorkloadEntityDiskStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Removing workload entity {} stored in file {}", backQuote(entity_name), file_path); + + bool existed = fs::remove(file_path); + + if (!existed) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); + return true; +} + + +String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, const String & entity_name) const +{ + String file_path; + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + file_path = dir_path + "workload_" + escapeForFileName(entity_name) + ".sql"; + break; + } + case WorkloadEntityType::Resource: + { + file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; + break; + } + } + return file_path; +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h new file mode 100644 index 00000000000..22c0ea4b83d --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Loads workload entities from a specified folder. +class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); + + void loadEntities() override; + + void reloadEntities() override; + +private: + bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void createDirectory(); + void loadEntitiesImpl(); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & file_path, bool check_file_exists); + String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; + + String dir_path; + LoggerPtr log; + std::atomic entities_loaded = false; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp new file mode 100644 index 00000000000..a0b6ebc9267 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -0,0 +1,195 @@ +#include + +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; + extern const int UNKNOWN_WORKLOAD_ENTITY; +} + +namespace +{ + +ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const ContextPtr & context) +{ + UNUSED(context); + auto ptr = create_query.clone(); + auto & res = typeid_cast(*ptr); // TODO(serxa): we should also check for ASTCreateResourceQuery + res.if_not_exists = false; + res.or_replace = false; + return ptr; +} + +} + +WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) + : global_context(std::move(global_context_)) +{} + +ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const +{ + std::lock_guard lock(mutex); + + auto it = entities.find(entity_name); + if (it == entities.end()) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, + "The workload entity name '{}' is not saved", + entity_name); + + return it->second; +} + +ASTPtr WorkloadEntityStorageBase::tryGet(const std::string & entity_name) const +{ + std::lock_guard lock(mutex); + + auto it = entities.find(entity_name); + if (it == entities.end()) + return nullptr; + + return it->second; +} + +bool WorkloadEntityStorageBase::has(const String & entity_name) const +{ + return tryGet(entity_name) != nullptr; +} + +std::vector WorkloadEntityStorageBase::getAllEntityNames() const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + entity_names.reserve(entities.size()); + + for (const auto & [name, _] : entities) + entity_names.emplace_back(name); + + return entity_names; +} + +bool WorkloadEntityStorageBase::empty() const +{ + std::lock_guard lock(mutex); + return entities.empty(); +} + +bool WorkloadEntityStorageBase::storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + std::lock_guard lock{mutex}; + auto it = entities.find(entity_name); + if (it != entities.end()) + { + if (throw_if_exists) + throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + } + + bool stored = storeEntityImpl( + current_context, + entity_type, + entity_name, + create_entity_query, + throw_if_exists, + replace_if_exists, + settings); + + if (stored) + entities[entity_name] = create_entity_query; + + return stored; +} + +bool WorkloadEntityStorageBase::removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + std::lock_guard lock(mutex); + auto it = entities.find(entity_name); + if (it == entities.end()) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + bool removed = removeEntityImpl( + current_context, + entity_type, + entity_name, + throw_if_not_exists); + + if (removed) + entities.erase(entity_name); + + return removed; +} + +std::unique_lock WorkloadEntityStorageBase::getLock() const +{ + return std::unique_lock{mutex}; +} + +void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) +{ + std::unordered_map normalized_entities; + for (const auto & [entity_name, create_query] : new_entities) + normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); + + std::lock_guard lock(mutex); + entities = std::move(normalized_entities); +} + +std::vector> WorkloadEntityStorageBase::getAllEntities() const +{ + std::lock_guard lock{mutex}; + std::vector> all_entities; + all_entities.reserve(entities.size()); + std::copy(entities.begin(), entities.end(), std::back_inserter(all_entities)); + return all_entities; +} + +void WorkloadEntityStorageBase::setEntity(const String & entity_name, const IAST & create_entity_query) +{ + std::lock_guard lock(mutex); + entities[entity_name] = normalizeCreateWorkloadEntityQuery(create_entity_query, global_context); +} + +void WorkloadEntityStorageBase::removeEntity(const String & entity_name) +{ + std::lock_guard lock(mutex); + entities.erase(entity_name); +} + +void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) +{ + boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; + std::lock_guard lock(mutex); + for (auto it = entities.begin(); it != entities.end();) + { + auto current = it++; + if (!names_set_to_keep.contains(current->first)) + entities.erase(current); + } +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h new file mode 100644 index 00000000000..f6dafc033c2 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -0,0 +1,73 @@ +#pragma once + +#include +#include + +#include +#include + +#include + +namespace DB +{ + +class WorkloadEntityStorageBase : public IWorkloadEntityStorage +{ +public: + explicit WorkloadEntityStorageBase(ContextPtr global_context_); + ASTPtr get(const String & entity_name) const override; + + ASTPtr tryGet(const String & entity_name) const override; + + bool has(const String & entity_name) const override; + + std::vector getAllEntityNames() const override; + + std::vector> getAllEntities() const override; + + bool empty() const override; + + bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + +protected: + virtual bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + virtual bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; + + std::unique_lock getLock() const; + void setAllEntities(const std::vector> & new_entities); + void setEntity(const String & entity_name, const IAST & create_entity_query); + void removeEntity(const String & entity_name); + void removeAllEntitiesExcept(const Strings & entity_names_to_keep); + + std::unordered_map entities; // Maps entity name into CREATE entity query + mutable std::recursive_mutex mutex; + + ContextPtr global_context; +}; + +} diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp new file mode 100644 index 00000000000..adb3e0b6e45 --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateResourceQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->resource_name = resource_name->clone(); + res->children.push_back(res->resource_name); + + return res; +} + +void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + settings.ostr << "OR REPLACE "; + + settings.ostr << "RESOURCE "; + + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (settings.hilite ? hilite_none : ""); + + formatOnCluster(settings); +} + +String ASTCreateResourceQuery::getResourceName() const +{ + String name; + tryGetIdentifierNameInto(resource_name, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h new file mode 100644 index 00000000000..3d571807ec4 --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + ASTPtr resource_name; + // TODO(serxa): add resource definition + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateResourceQuery" + (delim + getResourceName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getResourceName() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..a6906dbcf65 --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateWorkloadQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->workload_name = workload_name->clone(); + res->children.push_back(res->workload_name); + + // TODO(serxa): clone settings + + return res; +} + +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + settings.ostr << "OR REPLACE "; + + settings.ostr << "WORKLOAD "; + + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (settings.hilite ? hilite_none : ""); + + formatOnCluster(settings); + + if (hasParent()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (settings.hilite ? hilite_none : ""); + } +} + +String ASTCreateWorkloadQuery::getWorkloadName() const +{ + String name; + tryGetIdentifierNameInto(workload_name, name); + return name; +} + +bool ASTCreateWorkloadQuery::hasParent() const +{ + return workload_parent != nullptr; +} + +String ASTCreateWorkloadQuery::getWorkloadParent() const +{ + String name; + tryGetIdentifierNameInto(workload_parent, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h new file mode 100644 index 00000000000..bdd3a831aeb --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + ASTPtr workload_name; + ASTPtr workload_parent; + // TODO(serxa): add workload settings (weight and priority should also go inside settings, because they can differ for different resources) + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateWorkloadQuery" + (delim + getWorkloadName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getWorkloadName() const; + bool hasParent() const; + String getWorkloadParent() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index ab0e70eb0e5..aef505668d6 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -407,6 +407,7 @@ namespace DB MR_MACROS(REPLACE_PARTITION, "REPLACE PARTITION") \ MR_MACROS(REPLACE, "REPLACE") \ MR_MACROS(RESET_SETTING, "RESET SETTING") \ + MR_MACROS(RESOURCE, "RESOURCE") \ MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \ MR_MACROS(RESTORE, "RESTORE") \ MR_MACROS(RESTRICT, "RESTRICT") \ @@ -519,6 +520,7 @@ namespace DB MR_MACROS(WHEN, "WHEN") \ MR_MACROS(WHERE, "WHERE") \ MR_MACROS(WINDOW, "WINDOW") \ + MR_MACROS(WORKLOAD, "WORKLOAD") \ MR_MACROS(QUALIFY, "QUALIFY") \ MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \ MR_MACROS(WITH_CHECK, "WITH CHECK") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp new file mode 100644 index 00000000000..4921debdf52 --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.cpp @@ -0,0 +1,62 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + // TODO(serxa): parse resource definition + + ASTPtr resource_name; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto create_resource_query = std::make_shared(); + node = create_resource_query; + + create_resource_query->resource_name = resource_name; + create_resource_query->children.push_back(resource_name); + + create_resource_query->or_replace = or_replace; + create_resource_query->if_not_exists = if_not_exists; + create_resource_query->cluster = std::move(cluster_str); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateResourceQuery.h b/src/Parsers/ParserCreateResourceQuery.h new file mode 100644 index 00000000000..1b7c9fc4a7f --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE RESOURCE cache_io (WRITE DISK s3diskWithCache, READ DISK s3diskWithCache) +class ParserCreateResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..ab0b0e3eb36 --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -0,0 +1,76 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserIdentifier workload_name_p; + ParserKeyword s_on(Keyword::ON); + ParserKeyword s_in(Keyword::IN); + // TODO(serxa): parse workload settings + + ASTPtr workload_name; + ASTPtr workload_parent; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (s_in.ignore(pos, expected)) + { + if (!workload_name_p.parse(pos, workload_parent, expected)) + return false; + } + + auto create_workload_query = std::make_shared(); + node = create_workload_query; + + create_workload_query->workload_name = workload_name; + create_workload_query->children.push_back(workload_name); + + if (workload_parent) + { + create_workload_query->workload_parent = workload_parent; + create_workload_query->children.push_back(workload_parent); + } + + create_workload_query->or_replace = or_replace; + create_workload_query->if_not_exists = if_not_exists; + create_workload_query->cluster = std::move(cluster_str); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.h b/src/Parsers/ParserCreateWorkloadQuery.h new file mode 100644 index 00000000000..62c89affeda --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE WORKLOAD production IN all SETTINGS weight = 3, max_speed = '1G' FOR network_read, max_speed = '2G' FOR network_write +class ParserCreateWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} From 6b6cfd4e1677f23b989449f49a89d15093e543d8 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:08:24 +0000 Subject: [PATCH 022/816] Integrate workload entity storage into server --- programs/server/Server.cpp | 2 + programs/server/config.xml | 4 ++ .../Workload/createWorkloadEntityStorage.cpp | 48 +++++++++++++++++++ .../Workload/createWorkloadEntityStorage.h | 11 +++++ src/Interpreters/Context.cpp | 36 +++++++++++++- src/Interpreters/Context.h | 5 ++ 6 files changed, 105 insertions(+), 1 deletion(-) create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fb5717ba33f..996542741f9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2088,6 +2088,8 @@ try database_catalog.assertDatabaseExists(default_database); /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); + /// Load WORKLOADs and RESOURCEs. + global_context->getWorkloadEntityStorage().loadObjects(); } catch (...) { diff --git a/programs/server/config.xml b/programs/server/config.xml index 10ad831465a..b41f0344bb2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1386,6 +1386,10 @@ If not specified they will be stored locally. --> + + + diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp new file mode 100644 index 00000000000..dde995db6e1 --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int INVALID_CONFIG_PARAMETER; +} + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context) +{ + const String zookeeper_path_key = "workload_zookeeper_path"; + const String disk_path_key = "workload_path"; + + const auto & config = global_context->getConfigRef(); + if (config.has(zookeeper_path_key)) + { + if (config.has(disk_path_key)) + { + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "'{}' and '{}' must not be both specified in the config", + zookeeper_path_key, + disk_path_key); + } + abort(); // TODO(serxa): crate WorkloadEntityKeeperStorage object + //return std::make_unique(global_context, config.getString(zookeeper_path_key)); + } + else + { + String default_path = fs::path{global_context->getPath()} / "workload" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); + } +} + +} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h new file mode 100644 index 00000000000..936e1275010 --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context); + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 373cc91ebcb..9f0ad40f446 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -64,7 +64,6 @@ #include #include #include -#include #include #include #include @@ -89,6 +88,8 @@ #include #include #include +#include +#include #include #include #include @@ -270,6 +271,9 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag user_defined_sql_objects_storage_initialized; mutable std::unique_ptr user_defined_sql_objects_storage; + mutable OnceFlag workload_entity_storage_initialized; + mutable std::unique_ptr workload_entity_storage; + #if USE_NLP mutable OnceFlag synonyms_extensions_initialized; mutable std::optional synonyms_extensions; @@ -609,6 +613,7 @@ struct ContextSharedPart : boost::noncopyable SHUTDOWN(log, "dictionaries loader", external_dictionaries_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "UDFs loader", external_user_defined_executable_functions_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "another UDFs storage", user_defined_sql_objects_storage, stopWatching()); + SHUTDOWN(log, "workload entity storage", workload_entity_storage, stopWatching()); LOG_TRACE(log, "Shutting down named sessions"); Session::shutdownNamedSessions(); @@ -640,6 +645,7 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_external_dictionaries_loader; std::unique_ptr delete_external_user_defined_executable_functions_loader; std::unique_ptr delete_user_defined_sql_objects_storage; + std::unique_ptr delete_workload_entity_storage; std::unique_ptr delete_buffer_flush_schedule_pool; std::unique_ptr delete_schedule_pool; std::unique_ptr delete_distributed_schedule_pool; @@ -724,6 +730,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader = std::move(external_dictionaries_loader); delete_external_user_defined_executable_functions_loader = std::move(external_user_defined_executable_functions_loader); delete_user_defined_sql_objects_storage = std::move(user_defined_sql_objects_storage); + delete_workload_entity_storage = std::move(workload_entity_storage); delete_buffer_flush_schedule_pool = std::move(buffer_flush_schedule_pool); delete_schedule_pool = std::move(schedule_pool); delete_distributed_schedule_pool = std::move(distributed_schedule_pool); @@ -742,6 +749,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader.reset(); delete_external_user_defined_executable_functions_loader.reset(); delete_user_defined_sql_objects_storage.reset(); + delete_workload_entity_storage.reset(); delete_ddl_worker.reset(); delete_buffer_flush_schedule_pool.reset(); delete_schedule_pool.reset(); @@ -2903,6 +2911,32 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } +const IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const +{ + callOnce(shared->workload_entity_storage_initialized, [&] { + shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); + }); + + SharedLockGuard lock(shared->mutex); + return *shared->workload_entity_storage; +} + +IWorkloadEntityStorage & Context::getWorkloadEntityStorage() +{ + callOnce(shared->workload_entity_storage_initialized, [&] { + shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); + }); + + std::lock_guard lock(shared->mutex); + return *shared->workload_entity_storage; +} + +void Context::setWorkloadEntityStorage(std::unique_ptr storage) +{ + std::lock_guard lock(shared->mutex); + shared->workload_entity_storage = std::move(storage); +} + #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fb5337158ba..a6fd119f152 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -70,6 +70,7 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; class IUserDefinedSQLObjectsStorage; +class IWorkloadEntityStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; @@ -879,6 +880,10 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); + const IWorkloadEntityStorage & getWorkloadEntityStorage() const; + IWorkloadEntityStorage & getWorkloadEntityStorage(); + void setWorkloadEntityStorage(std::unique_ptr storage); + #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; Lemmatizers & getLemmatizers() const; From 93bcf2d8bcccba54ec86e187e5f9990a20062f22 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:28:16 +0000 Subject: [PATCH 023/816] add ASTs for DROP WORKLOAD and DROP RESOURCE --- src/Parsers/ASTDropResourceQuery.cpp | 25 +++++++++++++++++++++++++ src/Parsers/ASTDropResourceQuery.h | 28 ++++++++++++++++++++++++++++ src/Parsers/ASTDropWorkloadQuery.cpp | 25 +++++++++++++++++++++++++ src/Parsers/ASTDropWorkloadQuery.h | 28 ++++++++++++++++++++++++++++ 4 files changed, 106 insertions(+) create mode 100644 src/Parsers/ASTDropResourceQuery.cpp create mode 100644 src/Parsers/ASTDropResourceQuery.h create mode 100644 src/Parsers/ASTDropWorkloadQuery.cpp create mode 100644 src/Parsers/ASTDropWorkloadQuery.h diff --git a/src/Parsers/ASTDropResourceQuery.cpp b/src/Parsers/ASTDropResourceQuery.cpp new file mode 100644 index 00000000000..753ac4e30e7 --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropResourceQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropResourceQuery.h b/src/Parsers/ASTDropResourceQuery.h new file mode 100644 index 00000000000..e1534ea454a --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String resource_name; + + bool if_exists = false; + + String getID(char) const override { return "DropResourceQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.cpp b/src/Parsers/ASTDropWorkloadQuery.cpp new file mode 100644 index 00000000000..3192223c4b3 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropWorkloadQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.h b/src/Parsers/ASTDropWorkloadQuery.h new file mode 100644 index 00000000000..99c3a011447 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String workload_name; + + bool if_exists = false; + + String getID(char) const override { return "DropWorkloadQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} From 31e2205c4e5e3b4dc9d8d6ae391d83a8d9f85afe Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:28:56 +0000 Subject: [PATCH 024/816] fix workload entity storage start --- programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 996542741f9..dd6bf291354 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -86,6 +86,7 @@ #include #include #include +#include #include #include #include "MetricsTransmitter.h" @@ -2089,7 +2090,7 @@ try /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); /// Load WORKLOADs and RESOURCEs. - global_context->getWorkloadEntityStorage().loadObjects(); + global_context->getWorkloadEntityStorage().loadEntities(); } catch (...) { From 3a486d79bfb432f24d83051c8e9c53d8e39d8e8a Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:35:52 +0000 Subject: [PATCH 025/816] add parsers for DROP WORKLOAD and DROP RESOURCE queries --- src/Parsers/ParserDropResourceQuery.cpp | 52 +++++++++++++++++++++++++ src/Parsers/ParserDropResourceQuery.h | 14 +++++++ src/Parsers/ParserDropWorkloadQuery.cpp | 52 +++++++++++++++++++++++++ src/Parsers/ParserDropWorkloadQuery.h | 14 +++++++ 4 files changed, 132 insertions(+) create mode 100644 src/Parsers/ParserDropResourceQuery.cpp create mode 100644 src/Parsers/ParserDropResourceQuery.h create mode 100644 src/Parsers/ParserDropWorkloadQuery.cpp create mode 100644 src/Parsers/ParserDropWorkloadQuery.h diff --git a/src/Parsers/ParserDropResourceQuery.cpp b/src/Parsers/ParserDropResourceQuery.cpp new file mode 100644 index 00000000000..6c078281828 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr resource_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_resource_query = std::make_shared(); + drop_resource_query->if_exists = if_exists; + drop_resource_query->cluster = std::move(cluster_str); + + node = drop_resource_query; + + drop_resource_query->resource_name = resource_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropResourceQuery.h b/src/Parsers/ParserDropResourceQuery.h new file mode 100644 index 00000000000..651603d1e90 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP RESOURCE resource1 +class ParserDropResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/ParserDropWorkloadQuery.cpp b/src/Parsers/ParserDropWorkloadQuery.cpp new file mode 100644 index 00000000000..edc82c8f30a --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier workload_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr workload_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_workload_query = std::make_shared(); + drop_workload_query->if_exists = if_exists; + drop_workload_query->cluster = std::move(cluster_str); + + node = drop_workload_query; + + drop_workload_query->workload_name = workload_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropWorkloadQuery.h b/src/Parsers/ParserDropWorkloadQuery.h new file mode 100644 index 00000000000..af060caf303 --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP WORKLOAD workload1 +class ParserDropWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} From a58d27166b22da253ec1e214a48ed3f2177ed85c Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:39:42 +0000 Subject: [PATCH 026/816] register workload and resource queries parsers --- src/Parsers/ParserQuery.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 22ddc25019f..b0f4715e2a3 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,8 +1,12 @@ #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -48,6 +52,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; ParserCreateNamedCollectionQuery create_named_collection_p; ParserDropNamedCollectionQuery drop_named_collection_p; ParserAlterNamedCollectionQuery alter_named_collection_p; @@ -74,6 +82,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) + || create_workload_p.parse(pos, node, expected) + || drop_workload_p.parse(pos, node, expected) + || create_resource_p.parse(pos, node, expected) + || drop_resource_p.parse(pos, node, expected) || create_named_collection_p.parse(pos, node, expected) || drop_named_collection_p.parse(pos, node, expected) || alter_named_collection_p.parse(pos, node, expected) From 90764466172c29867aa148541f27824cbbd966db Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:11:42 +0000 Subject: [PATCH 027/816] add interpreters for CREATE WORKLOAD and RESOURCE queries --- src/Access/Common/AccessType.h | 4 ++ src/Access/ContextAccess.cpp | 6 +- .../InterpreterCreateResourceQuery.cpp | 61 +++++++++++++++++++ .../InterpreterCreateResourceQuery.h | 25 ++++++++ .../InterpreterCreateWorkloadQuery.cpp | 61 +++++++++++++++++++ .../InterpreterCreateWorkloadQuery.h | 25 ++++++++ 6 files changed, 180 insertions(+), 2 deletions(-) create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.h create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..e70229b62e8 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -99,6 +99,8 @@ enum class AccessType : uint8_t M(CREATE_ARBITRARY_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables with arbitrary table engine */\ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ + M(CREATE_WORKLOAD, "", GLOBAL, CREATE) /* allows to execute CREATE WORKLOAD */ \ + M(CREATE_RESOURCE, "", GLOBAL, CREATE) /* allows to execute CREATE RESOURCE */ \ M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ @@ -108,6 +110,8 @@ enum class AccessType : uint8_t implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ + M(DROP_WORKLOAD, "", GLOBAL, DROP) /* allows to execute DROP WORKLOAD */\ + M(DROP_RESOURCE, "", GLOBAL, DROP) /* allows to execute DROP RESOURCE */\ M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index e50521a0730..d856341fade 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -689,15 +689,17 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; + const AccessFlags workload_ddl = AccessType::CREATE_WORKLOAD | AccessType::DROP_WORKLOAD; + const AccessFlags resource_ddl = AccessType::CREATE_RESOURCE | AccessType::DROP_RESOURCE; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; const AccessFlags table_and_dictionary_and_function_ddl = table_ddl | dictionary_ddl | function_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; - const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; + const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | workload_ddl | resource_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; - const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; + const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl | workload_ddl | resource_ddl; const AccessFlags introspection_flags = AccessType::INTROSPECTION; }; static const PrecalculatedFlags precalc; diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp new file mode 100644 index 00000000000..78f5b535cb1 --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.cpp @@ -0,0 +1,61 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateResourceQuery::execute() +{ + ASTCreateResourceQuery & create_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_RESOURCE); + + if (create_resource_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!create_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto resource_name = create_resource_query.getResourceName(); + //bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; + //bool replace_if_exists = create_resource_query.or_replace; + + // TODO(serxa): validate and register entity + + return {}; +} + +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateResourceQuery.h b/src/Interpreters/InterpreterCreateResourceQuery.h new file mode 100644 index 00000000000..4bd427e5e8f --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..1057fb14604 --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp @@ -0,0 +1,61 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateWorkloadQuery::execute() +{ + ASTCreateWorkloadQuery & create_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_WORKLOAD); + + if (create_workload_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!create_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto workload_name = create_workload_query.getWorkloadName(); + //bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; + //bool replace_if_exists = create_workload_query.or_replace; + + // TODO(serxa): validate and register entity + + return {}; +} + +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.h b/src/Interpreters/InterpreterCreateWorkloadQuery.h new file mode 100644 index 00000000000..319388fb64c --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} From 2183c73077a7d1477ca4a5993f9776112be8607c Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:22:18 +0000 Subject: [PATCH 028/816] add interpreters for DROP WORKLOAD and RESOURCE queries --- .../InterpreterDropResourceQuery.cpp | 56 +++++++++++++++++++ .../InterpreterDropResourceQuery.h | 21 +++++++ .../InterpreterDropWorkloadQuery.cpp | 56 +++++++++++++++++++ .../InterpreterDropWorkloadQuery.h | 21 +++++++ 4 files changed, 154 insertions(+) create mode 100644 src/Interpreters/InterpreterDropResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterDropResourceQuery.h create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.h diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp new file mode 100644 index 00000000000..49071a0a1aa --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.cpp @@ -0,0 +1,56 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropResourceQuery::execute() +{ + ASTDropResourceQuery & drop_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!drop_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + //bool throw_if_not_exists = !drop_resource_query.if_exists; + + // TODO(serxa): validate and unregister entity + + return {}; +} + +void registerInterpreterDropResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropResourceQuery.h b/src/Interpreters/InterpreterDropResourceQuery.h new file mode 100644 index 00000000000..588f26fb88c --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp new file mode 100644 index 00000000000..da022d4d054 --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.cpp @@ -0,0 +1,56 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropWorkloadQuery::execute() +{ + ASTDropWorkloadQuery & drop_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!drop_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + //bool throw_if_not_exists = !drop_workload_query.if_exists; + + // TODO(serxa): validate and unregister entity + + return {}; +} + +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.h b/src/Interpreters/InterpreterDropWorkloadQuery.h new file mode 100644 index 00000000000..1297c95e949 --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} From 7f6694b3705aa5dc929776bc357863b8769733da Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:25:28 +0000 Subject: [PATCH 029/816] register workload entities queries interpreters --- src/Interpreters/InterpreterFactory.cpp | 20 ++++++++++++++++++++ src/Interpreters/registerInterpreters.cpp | 8 ++++++++ 2 files changed, 28 insertions(+) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 12b3b510098..5ae29fbe913 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -3,9 +3,13 @@ #include #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -326,6 +330,22 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { interpreter_name = "InterpreterDropFunctionQuery"; } + else if (query->as()) + { + interpreter_name = "InterpreterCreateWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterCreateResourceQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropResourceQuery"; + } else if (query->as()) { interpreter_name = "InterpreterCreateIndexQuery"; diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp index 481d0597a85..838b3a669da 100644 --- a/src/Interpreters/registerInterpreters.cpp +++ b/src/Interpreters/registerInterpreters.cpp @@ -52,6 +52,10 @@ void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory); +void registerInterpreterDropResourceQuery(InterpreterFactory & factory); void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); void registerInterpreterDropIndexQuery(InterpreterFactory & factory); @@ -111,6 +115,10 @@ void registerInterpreters() registerInterpreterTransactionControlQuery(factory); registerInterpreterCreateFunctionQuery(factory); registerInterpreterDropFunctionQuery(factory); + registerInterpreterCreateWorkloadQuery(factory); + registerInterpreterDropWorkloadQuery(factory); + registerInterpreterCreateResourceQuery(factory); + registerInterpreterDropResourceQuery(factory); registerInterpreterCreateIndexQuery(factory); registerInterpreterCreateNamedCollectionQuery(factory); registerInterpreterDropIndexQuery(factory); From d932d0ae4f459a3a7ed3d1942034aa85b72f7ee5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 2 Sep 2024 14:16:00 +0000 Subject: [PATCH 030/816] fix performance of parsing row formats --- src/Columns/ColumnArray.cpp | 6 ++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnDynamic.h | 5 +++ src/Columns/ColumnMap.cpp | 5 +++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 6 ++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnObject.cpp | 34 +++++++++++++++---- src/Columns/ColumnObject.h | 1 + src/Columns/ColumnSparse.cpp | 6 ++++ src/Columns/ColumnSparse.h | 1 + src/Columns/ColumnTuple.cpp | 10 ++++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 10 ++++++ src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 7 ++-- .../Executors/StreamingFormatExecutor.cpp | 13 +++---- .../Executors/StreamingFormatExecutor.h | 2 -- src/Processors/Formats/IRowInputFormat.cpp | 5 ++- 19 files changed, 97 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 4f3d0f0ec4b..0c6d7c4e5c6 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -374,6 +374,12 @@ ColumnCheckpointPtr ColumnArray::getCheckpoint() const return std::make_shared(size(), getData().getCheckpoint()); } +void ColumnArray::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + getData().updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnArray::rollback(const ColumnCheckpoint & checkpoint) { getOffsets().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index fd42d502c16..ec14b096055 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -162,6 +162,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 1445aee4176..6f8335044a7 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -309,6 +309,11 @@ public: return variant_column_ptr->getCheckpoint(); } + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override + { + variant_column_ptr->updateCheckpoint(checkpoint); + } + void rollback(const ColumnCheckpoint & checkpoint) override { variant_column_ptr->rollback(checkpoint); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 0561e8f398f..7ebbed930d8 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -317,6 +317,11 @@ ColumnCheckpointPtr ColumnMap::getCheckpoint() const return nested->getCheckpoint(); } +void ColumnMap::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + nested->updateCheckpoint(checkpoint); +} + void ColumnMap::rollback(const ColumnCheckpoint & checkpoint) { nested->rollback(checkpoint); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index d534cfa7295..575114f8d3a 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -103,6 +103,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 560d37721ad..61feca60e42 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -310,6 +310,12 @@ ColumnCheckpointPtr ColumnNullable::getCheckpoint() const return std::make_shared(size(), nested_column->getCheckpoint()); } +void ColumnNullable::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + nested_column->updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnNullable::rollback(const ColumnCheckpoint & checkpoint) { getNullMapData().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 39b326a1c44..32ce66c5965 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -144,6 +144,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 6be6e9d833e..4c33f042954 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -686,22 +686,44 @@ ColumnCheckpointPtr ColumnObject::getCheckpoint() const return std::make_shared(size(), get_checkpoints(typed_paths), get_checkpoints(dynamic_paths_ptrs), shared_data->getCheckpoint()); } +void ColumnObject::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & object_checkpoint = assert_cast(checkpoint); + + auto update_checkpoints = [&](const auto & columns_map, auto & checkpoints_map) + { + for (const auto & [name, column] : columns_map) + { + auto & nested = checkpoints_map[name]; + if (!nested) + nested = column->getCheckpoint(); + else + column->updateCheckpoint(*nested); + } + }; + + checkpoint.size = size(); + update_checkpoints(typed_paths, object_checkpoint.typed_paths); + update_checkpoints(dynamic_paths, object_checkpoint.dynamic_paths); + shared_data->updateCheckpoint(*object_checkpoint.shared_data); +} + void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) { const auto & object_checkpoint = assert_cast(checkpoint); for (auto & [name, column] : typed_paths) { - const auto & nested_checkpoint = object_checkpoint.typed_paths.at(name); - chassert(nested_checkpoint); - column->rollback(*nested_checkpoint); + const auto & nested = object_checkpoint.typed_paths.at(name); + chassert(nested); + column->rollback(*nested); } for (auto & [name, column] : dynamic_paths_ptrs) { - const auto & nested_checkpoint = object_checkpoint.dynamic_paths.at(name); - chassert(nested_checkpoint); - column->rollback(*nested_checkpoint); + const auto & nested = object_checkpoint.dynamic_paths.at(name); + chassert(nested); + column->rollback(*nested); } shared_data->rollback(*object_checkpoint.shared_data); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 84fe2dcafad..e444db099b0 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -160,6 +160,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0c91174007c..a0e47e65fc6 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -313,6 +313,12 @@ ColumnCheckpointPtr ColumnSparse::getCheckpoint() const return std::make_shared(size(), values->getCheckpoint()); } +void ColumnSparse::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + values->updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnSparse::rollback(const ColumnCheckpoint & checkpoint) { _size = checkpoint.size; diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index dabf38a252d..619dce63c1e 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -150,6 +150,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 65f3285bcfc..51617359318 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -265,6 +265,16 @@ ColumnCheckpointPtr ColumnTuple::getCheckpoint() const return std::make_shared(size(), std::move(checkpoints)); } +void ColumnTuple::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & checkpoints = assert_cast(checkpoint).nested; + chassert(checkpoints.size() == columns.size()); + + checkpoint.size = size(); + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->updateCheckpoint(*checkpoints[i]); +} + void ColumnTuple::rollback(const ColumnCheckpoint & checkpoint) { column_length = checkpoint.size; diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 74c4dd1ffd3..c73f90f13d9 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -119,6 +119,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index a8cb779ad16..b03313fd6d0 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -750,6 +750,16 @@ ColumnCheckpointPtr ColumnVariant::getCheckpoint() const return std::make_shared(size(), std::move(checkpoints)); } +void ColumnVariant::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & checkpoints = assert_cast(checkpoint).nested; + chassert(checkpoints.size() == variants.size()); + + checkpoint.size = size(); + for (size_t i = 0; i < variants.size(); ++i) + variants[i]->updateCheckpoint(*checkpoints[i]); +} + void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) { getOffsets().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index c7182467611..332c36d1153 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -249,6 +249,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 53f31d2b96d..95becba3fdb 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -60,7 +60,7 @@ struct ColumnCheckpoint virtual ~ColumnCheckpoint() = default; }; -using ColumnCheckpointPtr = std::shared_ptr; +using ColumnCheckpointPtr = std::shared_ptr; using ColumnCheckpoints = std::vector; struct ColumnCheckpointWithNested : public ColumnCheckpoint @@ -546,9 +546,12 @@ public: /// Returns checkpoint of current state of column. virtual ColumnCheckpointPtr getCheckpoint() const { return std::make_shared(size()); } + /// Updates the checkpoint with current state. It is used to avoid extra allocations in 'getCheckpoint'. + virtual void updateCheckpoint(ColumnCheckpoint & checkpoint) const { checkpoint.size = size(); } + /// Rollbacks column to the checkpoint. /// Unlike 'popBack' this method should work correctly even if column has invalid state. - /// Sizes of columns in checkpoint must be less or equal than current. + /// Sizes of columns in checkpoint must be less or equal than current size. virtual void rollback(const ColumnCheckpoint & checkpoint) { popBack(size() - checkpoint.size); } /// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them. diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 1491198f0cb..2d4b87e9f4d 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -25,6 +25,9 @@ StreamingFormatExecutor::StreamingFormatExecutor( , checkpoints(result_columns.size()) { connect(format->getPort(), port); + + for (size_t i = 0; i < result_columns.size(); ++i) + checkpoints[i] = result_columns[i]->getCheckpoint(); } MutableColumns StreamingFormatExecutor::getResultColumns() @@ -54,7 +57,8 @@ size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) size_t StreamingFormatExecutor::execute() { - setCheckpoints(); + for (size_t i = 0; i < result_columns.size(); ++i) + result_columns[i]->updateCheckpoint(*checkpoints[i]); try { @@ -117,11 +121,4 @@ size_t StreamingFormatExecutor::insertChunk(Chunk chunk) return chunk_rows; } -void StreamingFormatExecutor::setCheckpoints() -{ - for (size_t i = 0; i < result_columns.size(); ++i) - checkpoints[i] = result_columns[i]->getCheckpoint(); -} - - } diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index f0c443edd13..3db5a92ae98 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -43,8 +43,6 @@ public: void setQueryParameters(const NameToNameMap & parameters); private: - void setCheckpoints(); - const Block header; const InputFormatPtr format; const ErrorCallback on_error; diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 2a0695764b2..0d65fc3b5fa 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -104,7 +104,10 @@ Chunk IRowInputFormat::read() size_t num_columns = header.columns(); MutableColumns columns = header.cloneEmptyColumns(); + ColumnCheckpoints checkpoints(columns.size()); + for (size_t column_idx = 0; column_idx < columns.size(); ++column_idx) + checkpoints[column_idx] = columns[column_idx]->getCheckpoint(); block_missing_values.clear(); @@ -132,7 +135,7 @@ Chunk IRowInputFormat::read() try { for (size_t column_idx = 0; column_idx < columns.size(); ++column_idx) - checkpoints[column_idx] = columns[column_idx]->getCheckpoint(); + columns[column_idx]->updateCheckpoint(*checkpoints[column_idx]); info.read_columns.clear(); continue_reading = readRow(columns, info); From 2ef36b36acb1926b70b1d4b64c7d3d83783e483c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:06:44 +0000 Subject: [PATCH 031/816] add notification system for workload entity changes --- .../Workload/IWorkloadEntityStorage.h | 13 ++ .../Workload/WorkloadEntityDiskStorage.cpp | 2 + .../Workload/WorkloadEntityStorageBase.cpp | 113 +++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 41 ++++++- 4 files changed, 150 insertions(+), 19 deletions(-) diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index 65978a71be0..113cefe3f46 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -17,6 +18,8 @@ enum class WorkloadEntityType : uint8_t { Workload, Resource, + + MAX }; /// Interface for a storage of workload entities (WORKLOAD and RESOURCE). @@ -72,6 +75,16 @@ public: WorkloadEntityType entity_type, const String & entity_name, bool throw_if_not_exists) = 0; + + using OnChangedHandler = std::function; + + /// Subscribes for all changes. + virtual scope_guard subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) = 0; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index d9ca8bca0a0..b14a96c771a 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -108,6 +108,7 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, global_context->getSettingsRef().max_parser_backtracks); return ast; } + case WorkloadEntityType::MAX: return nullptr; } } catch (...) @@ -289,6 +290,7 @@ String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, co file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; break; } + case WorkloadEntityType::MAX: break; } return file_path; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index a0b6ebc9267..dfcd5f9b7da 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -111,16 +112,19 @@ bool WorkloadEntityStorageBase::storeEntity( settings); if (stored) + { entities[entity_name] = create_entity_query; + onEntityAdded(entity_type, entity_name, create_entity_query); + } return stored; } bool WorkloadEntityStorageBase::removeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) { std::lock_guard lock(mutex); auto it = entities.find(entity_name); @@ -139,11 +143,94 @@ bool WorkloadEntityStorageBase::removeEntity( throw_if_not_exists); if (removed) + { entities.erase(entity_name); + onEntityRemoved(entity_type, entity_name); + } return removed; } +scope_guard WorkloadEntityStorageBase::subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) +{ + std::lock_guard lock{handlers->mutex}; + auto & list = handlers->by_type[static_cast(entity_type)]; + list.push_back(handler); + auto handler_it = std::prev(list.end()); + + return [my_handlers = handlers, entity_type, handler_it] + { + std::lock_guard lock2{my_handlers->mutex}; + auto & list2 = my_handlers->by_type[static_cast(entity_type)]; + list2.erase(handler_it); + }; +} + +void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + event.entity = new_entity; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + event.entity = changed_entity; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::sendNotifications() +{ + /// Only one thread can send notification at any time. + std::lock_guard sending_notifications_lock{sending_notifications}; + + std::unique_lock queue_lock{queue_mutex}; + while (!queue.empty()) + { + auto event = std::move(queue.front()); + queue.pop(); + queue_lock.unlock(); + + std::vector current_handlers; + { + std::lock_guard handlers_lock{handlers->mutex}; + boost::range::copy(handlers->by_type[static_cast(event.type)], std::back_inserter(current_handlers)); + } + + for (const auto & handler : current_handlers) + { + try + { + handler(event.type, event.name, event.entity); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + queue_lock.lock(); + } +} + std::unique_lock WorkloadEntityStorageBase::getLock() const { return std::unique_lock{mutex}; @@ -155,6 +242,11 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector> WorkloadEntityStorageBase::getAllEntities return all_entities; } -void WorkloadEntityStorageBase::setEntity(const String & entity_name, const IAST & create_entity_query) -{ - std::lock_guard lock(mutex); - entities[entity_name] = normalizeCreateWorkloadEntityQuery(create_entity_query, global_context); -} - -void WorkloadEntityStorageBase::removeEntity(const String & entity_name) -{ - std::lock_guard lock(mutex); - entities.erase(entity_name); -} - +// TODO(serxa): add notifications or remove this function void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) { boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index f6dafc033c2..9e9e8170a8e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include +#include #include #include @@ -42,6 +44,10 @@ public: const String & entity_name, bool throw_if_not_exists) override; + virtual scope_guard subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) override; + protected: virtual bool storeEntityImpl( const ContextPtr & current_context, @@ -60,12 +66,41 @@ protected: std::unique_lock getLock() const; void setAllEntities(const std::vector> & new_entities); - void setEntity(const String & entity_name, const IAST & create_entity_query); - void removeEntity(const String & entity_name); void removeAllEntitiesExcept(const Strings & entity_names_to_keep); - std::unordered_map entities; // Maps entity name into CREATE entity query + /// Called by derived class after a new workload entity has been added. + void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); + + /// Called by derived class after an workload entity has been changed. + void onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity); + + /// Called by derived class after an workload entity has been removed. + void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); + + /// Sends notifications to subscribers about changes in workload entities + /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). + void sendNotifications(); + + struct Handlers + { + std::mutex mutex; + std::list by_type[static_cast(WorkloadEntityType::MAX)]; + }; + /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. + std::shared_ptr handlers; + + struct Event + { + WorkloadEntityType type; + String name; + ASTPtr entity; + }; + std::queue queue; + std::mutex queue_mutex; + std::mutex sending_notifications; + mutable std::recursive_mutex mutex; + std::unordered_map entities; // Maps entity name into CREATE entity query ContextPtr global_context; }; From 8e61a5c0b6b198dd97e0f21feacb06ce64196b86 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:13:22 +0000 Subject: [PATCH 032/816] fix normalizeCreateWorkloadEntityQuery() --- .../Workload/WorkloadEntityStorageBase.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index dfcd5f9b7da..7e7a4e526f1 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -23,9 +24,16 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const Conte { UNUSED(context); auto ptr = create_query.clone(); - auto & res = typeid_cast(*ptr); // TODO(serxa): we should also check for ASTCreateResourceQuery - res.if_not_exists = false; - res.or_replace = false; + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } return ptr; } From 840d284e36a4717fef6a14ed9d4ee35972374f51 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:59:30 +0000 Subject: [PATCH 033/816] attach interpreters to storage --- .../Workload/WorkloadEntityStorageBase.cpp | 3 +++ src/Interpreters/InterpreterCreateResourceQuery.cpp | 13 ++++++++++--- src/Interpreters/InterpreterCreateWorkloadQuery.cpp | 13 ++++++++++--- src/Interpreters/InterpreterDropResourceQuery.cpp | 8 ++++++-- src/Interpreters/InterpreterDropWorkloadQuery.cpp | 8 ++++++-- 5 files changed, 35 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 7e7a4e526f1..33e6227b998 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -101,6 +101,9 @@ bool WorkloadEntityStorageBase::storeEntity( const Settings & settings) { std::lock_guard lock{mutex}; + + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); + auto it = entities.find(entity_name); if (it != entities.end()) { diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp index 78f5b535cb1..c6eca7a90d8 100644 --- a/src/Interpreters/InterpreterCreateResourceQuery.cpp +++ b/src/Interpreters/InterpreterCreateResourceQuery.cpp @@ -41,10 +41,17 @@ BlockIO InterpreterCreateResourceQuery::execute() current_context->checkAccess(access_rights_elements); auto resource_name = create_resource_query.getResourceName(); - //bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; - //bool replace_if_exists = create_resource_query.or_replace; + bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; + bool replace_if_exists = create_resource_query.or_replace; - // TODO(serxa): validate and register entity + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Resource, + resource_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); return {}; } diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp index 1057fb14604..41d0f52c685 100644 --- a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp @@ -41,10 +41,17 @@ BlockIO InterpreterCreateWorkloadQuery::execute() current_context->checkAccess(access_rights_elements); auto workload_name = create_workload_query.getWorkloadName(); - //bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; - //bool replace_if_exists = create_workload_query.or_replace; + bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; + bool replace_if_exists = create_workload_query.or_replace; - // TODO(serxa): validate and register entity + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Workload, + workload_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); return {}; } diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp index 49071a0a1aa..848a74fda23 100644 --- a/src/Interpreters/InterpreterDropResourceQuery.cpp +++ b/src/Interpreters/InterpreterDropResourceQuery.cpp @@ -37,9 +37,13 @@ BlockIO InterpreterDropResourceQuery::execute() current_context->checkAccess(access_rights_elements); - //bool throw_if_not_exists = !drop_resource_query.if_exists; + bool throw_if_not_exists = !drop_resource_query.if_exists; - // TODO(serxa): validate and unregister entity + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Resource, + drop_resource_query.resource_name, + throw_if_not_exists); return {}; } diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp index da022d4d054..bbaa2beb4cd 100644 --- a/src/Interpreters/InterpreterDropWorkloadQuery.cpp +++ b/src/Interpreters/InterpreterDropWorkloadQuery.cpp @@ -37,9 +37,13 @@ BlockIO InterpreterDropWorkloadQuery::execute() current_context->checkAccess(access_rights_elements); - //bool throw_if_not_exists = !drop_workload_query.if_exists; + bool throw_if_not_exists = !drop_workload_query.if_exists; - // TODO(serxa): validate and unregister entity + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Workload, + drop_workload_query.workload_name, + throw_if_not_exists); return {}; } From 85e7641299a6de2614dfc24f2cf932252b6e59c2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 11:03:15 +0000 Subject: [PATCH 034/816] add system.workloads table with test --- .../Workload/IWorkloadEntityStorage.h | 3 ++ .../Workload/WorkloadEntityStorageBase.cpp | 24 ++++++++++ .../Workload/WorkloadEntityStorageBase.h | 1 + .../System/StorageSystemWorkloads.cpp | 48 +++++++++++++++++++ src/Storages/System/StorageSystemWorkloads.h | 29 +++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + .../03232_workload_create_and_drop.reference | 5 ++ .../03232_workload_create_and_drop.sql | 11 +++++ 8 files changed, 123 insertions(+) create mode 100644 src/Storages/System/StorageSystemWorkloads.cpp create mode 100644 src/Storages/System/StorageSystemWorkloads.h create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.sql diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index 113cefe3f46..cff09a2259d 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -47,6 +47,9 @@ public: /// Get all entity names. virtual std::vector getAllEntityNames() const = 0; + /// Get all entity names of specified type. + virtual std::vector getAllEntityNames(WorkloadEntityType entity_type) const = 0; + /// Get all entities. virtual std::vector> getAllEntities() const = 0; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 33e6227b998..e3bf6d4af7f 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -37,6 +37,16 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const Conte return ptr; } +WorkloadEntityType getEntityType(const ASTPtr & ptr) +{ + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Workload; + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Resource; + chassert(false); + return WorkloadEntityType::MAX; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -85,6 +95,20 @@ std::vector WorkloadEntityStorageBase::getAllEntityNames() const return entity_names; } +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + for (const auto & [name, entity] : entities) + { + if (getEntityType(entity) == entity_type) + entity_names.emplace_back(name); + } + + return entity_names; +} + bool WorkloadEntityStorageBase::empty() const { std::lock_guard lock(mutex); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 9e9e8170a8e..8ec92675ddb 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -24,6 +24,7 @@ public: bool has(const String & entity_name) const override; std::vector getAllEntityNames() const override; + std::vector getAllEntityNames(WorkloadEntityType entity_type) const override; std::vector> getAllEntities() const override; diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp new file mode 100644 index 00000000000..dad2750d8c0 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemWorkloads::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the workload."}, + {"parent", std::make_shared(), "The name of the parent workload."}, + {"create_query", std::make_shared(), "CREATE query of the workload."}, + }; +} + +void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & workload_names = storage.getAllEntityNames(WorkloadEntityType::Workload); + for (const auto & workload_name : workload_names) + { + auto ast = storage.get(workload_name); + auto & workload = typeid_cast(*ast); + res_columns[0]->insert(workload_name); + res_columns[1]->insert(workload.getWorkloadParent()); + res_columns[2]->insert(queryToString(ast)); + } +} + +void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for workloads and resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for workloads and resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemWorkloads.h b/src/Storages/System/StorageSystemWorkloads.h new file mode 100644 index 00000000000..9d4770a02b8 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `workloads` system table, which allows you to get a list of all workloads +class StorageSystemWorkloads final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemWorkloads"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 816ba5095b1..728e83135a3 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -229,6 +230,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "s3queue", "Contains in-memory state of S3Queue metadata and currently processed rows per file."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); + attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); if (has_zookeeper) { diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference new file mode 100644 index 00000000000..4bac2ef71f2 --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -0,0 +1,5 @@ +all CREATE WORKLOAD `all` +development all CREATE WORKLOAD development IN `all` +production all CREATE WORKLOAD production IN `all` +all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql new file mode 100644 index 00000000000..38a7dad7cbc --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +CREATE OR REPLACE WORKLOAD all; +SELECT name, parent, create_query FROM system.workloads; +CREATE WORKLOAD IF NOT EXISTS production IN all; +CREATE WORKLOAD development IN all; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD IF EXISTS production; +DROP WORKLOAD development; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD all; From 4cd8272186416c94a371be79d6845f3d010b52b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 18:21:53 +0000 Subject: [PATCH 035/816] fix rollback of ColumnDynamic --- src/Columns/ColumnDynamic.cpp | 84 ++++++++++++++++++++++ src/Columns/ColumnDynamic.h | 12 ++-- src/Columns/ColumnObject.cpp | 28 +++++--- src/Columns/tests/gtest_column_dynamic.cpp | 69 ++++++++++++++++++ 4 files changed, 174 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 9b55879a4f0..8e345c1fc2f 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -979,6 +979,90 @@ ColumnPtr ColumnDynamic::compress() const }); } +void ColumnDynamic::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & nested = assert_cast(checkpoint).nested; + const auto & variants = variant_column_ptr->getVariants(); + size_t old_size = nested.size(); + + for (size_t i = 0; i < old_size; ++i) + { + variants[i]->updateCheckpoint(*nested[i]); + } + + /// If column has new variants since last checkpoint create checkpoints for them. + if (old_size < variants.size()) + { + nested.resize(variants.size()); + for (size_t i = old_size; i < variants.size(); ++i) + nested[i] = variants[i]->getCheckpoint(); + } + + checkpoint.size = size(); +} + + +DataTypePtr ColumnDynamic::popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n) +{ + const auto & type_variant = assert_cast(*info.variant_type); + + std::unordered_map discriminator_to_name; + std::unordered_map name_to_data_type; + + for (const auto & [name, discriminator] : info.variant_name_to_discriminator) + discriminator_to_name.emplace(discriminator, name); + + for (const auto & type : type_variant.getVariants()) + name_to_data_type.emplace(type->getName(), type); + + /// Remove last n variants according to global discriminators. + /// This code relies on invariant that new variants are always added to the end in ColumnVariant. + for (auto it = local_to_global_discriminators.rbegin(); it < local_to_global_discriminators.rbegin() + n; ++it) + discriminator_to_name.erase(*it); + + DataTypes new_variants; + for (const auto & [d, name] : discriminator_to_name) + new_variants.push_back(name_to_data_type.at(name)); + + return std::make_shared(std::move(new_variants)); +} + +void ColumnDynamic::rollback(const ColumnCheckpoint & checkpoint) +{ + const auto & nested = assert_cast(checkpoint).nested; + chassert(nested.size() <= variant_column_ptr->getNumVariants()); + + /// The structure hasn't changed, so we can use generic rollback of Variant column + if (nested.size() == variant_column_ptr->getNumVariants()) + { + variant_column_ptr->rollback(checkpoint); + return; + } + + auto new_subcolumns = variant_column_ptr->getVariants(); + auto new_discriminators_map = variant_column_ptr->getLocalToGlobalDiscriminatorsMapping(); + auto new_discriminators_column = variant_column_ptr->getLocalDiscriminatorsPtr(); + auto new_offses_column = variant_column_ptr->getOffsetsPtr(); + + /// Remove new variants that were added since last checkpoint. + auto new_variant_type = popBackVariants(variant_info, new_discriminators_map, variant_column_ptr->getNumVariants() - nested.size()); + createVariantInfo(new_variant_type); + variant_mappings_cache.clear(); + + new_subcolumns.resize(nested.size()); + new_discriminators_map.resize(nested.size()); + + /// Manually rollback internals of Variant column + new_discriminators_column->assumeMutable()->popBack(new_discriminators_column->size() - checkpoint.size); + new_offses_column->assumeMutable()->popBack(new_offses_column->size() - checkpoint.size); + + for (size_t i = 0; i < nested.size(); ++i) + new_subcolumns[i]->rollback(*nested[i]); + + variant_column = ColumnVariant::create(new_discriminators_column, new_offses_column, Columns(new_subcolumns.begin(), new_subcolumns.end()), new_discriminators_map); + variant_column_ptr = variant_column_ptr = assert_cast(variant_column.get()); +} + void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { if (source_columns.empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 6f8335044a7..6a050f6e5b1 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -309,15 +309,9 @@ public: return variant_column_ptr->getCheckpoint(); } - void updateCheckpoint(ColumnCheckpoint & checkpoint) const override - { - variant_column_ptr->updateCheckpoint(checkpoint); - } + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; - void rollback(const ColumnCheckpoint & checkpoint) override - { - variant_column_ptr->rollback(checkpoint); - } + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override { @@ -456,6 +450,8 @@ private: void updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type); + static DataTypePtr popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n); + WrappedPtr variant_column; /// Store and use pointer to ColumnVariant to avoid virtual calls. /// ColumnDynamic is widely used inside ColumnObject for each path and diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 4c33f042954..c1b31731147 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -712,20 +712,26 @@ void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) { const auto & object_checkpoint = assert_cast(checkpoint); - for (auto & [name, column] : typed_paths) + auto rollback_columns = [&](auto & columns_map, const auto & checkpoints_map) { - const auto & nested = object_checkpoint.typed_paths.at(name); - chassert(nested); - column->rollback(*nested); - } + NameSet names_to_remove; - for (auto & [name, column] : dynamic_paths_ptrs) - { - const auto & nested = object_checkpoint.dynamic_paths.at(name); - chassert(nested); - column->rollback(*nested); - } + /// Rollback subcolumns and remove paths that were not in checkpoint. + for (auto & [name, column] : columns_map) + { + auto it = checkpoints_map.find(name); + if (it == checkpoints_map.end()) + names_to_remove.insert(name); + else + column->rollback(*it->second); + } + for (const auto & name : names_to_remove) + columns_map.erase(name); + }; + + rollback_columns(typed_paths, object_checkpoint.typed_paths); + rollback_columns(dynamic_paths, object_checkpoint.dynamic_paths); shared_data->rollback(*object_checkpoint.shared_data); } diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index de76261229d..f956f60b378 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -920,3 +920,72 @@ TEST(ColumnDynamic, compare) ASSERT_EQ(column_from->compareAt(3, 2, *column_from, -1), -1); ASSERT_EQ(column_from->compareAt(3, 4, *column_from, -1), -1); } + +TEST(ColumnDynamic, rollback) +{ + auto check_variant = [](const ColumnVariant & column_variant, std::vector sizes) + { + ASSERT_EQ(column_variant.getNumVariants(), sizes.size()); + size_t num_rows = 0; + + for (size_t i = 0; i < sizes.size(); ++i) + { + ASSERT_EQ(column_variant.getVariants()[i]->size(), sizes[i]); + num_rows += sizes[i]; + } + + ASSERT_EQ(num_rows, column_variant.size()); + }; + + auto check_checkpoint = [&](const ColumnCheckpoint & cp, std::vector sizes) + { + const auto & nested = assert_cast(cp).nested; + ASSERT_EQ(nested.size(), sizes.size()); + size_t num_rows = 0; + + for (size_t i = 0; i < sizes.size(); ++i) + { + ASSERT_EQ(nested[i]->size, sizes[i]); + num_rows += sizes[i]; + } + + ASSERT_EQ(num_rows, cp.size); + }; + + std::vector>> checkpoints; + + auto column = ColumnDynamic::create(2); + auto checkpoint = column->getCheckpoint(); + + column->insert(Field(42)); + + column->updateCheckpoint(*checkpoint); + checkpoints.emplace_back(checkpoint, std::vector{0, 1}); + + column->insert(Field("str1")); + column->rollback(*checkpoint); + + check_checkpoint(*checkpoint, checkpoints.back().second); + check_variant(column->getVariantColumn(), checkpoints.back().second); + + column->insert("str1"); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{0, 1, 1}); + + column->insert("str2"); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{0, 1, 2}); + + column->insert(Array({1, 2})); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{1, 1, 2}); + + column->insert(Field(42.42)); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{2, 1, 2}); + + for (const auto & [cp, sizes] : checkpoints) + { + auto column_copy = column->clone(); + column_copy->rollback(*cp); + + check_checkpoint(*cp, sizes); + check_variant(assert_cast(*column_copy).getVariantColumn(), sizes); + } +} From ae1a8393b0ae0ceefc3627683d7ae6a0cb42659f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 22:16:22 +0000 Subject: [PATCH 036/816] add test for ColumnObject --- src/Columns/tests/gtest_column_object.cpp | 63 +++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/src/Columns/tests/gtest_column_object.cpp b/src/Columns/tests/gtest_column_object.cpp index f6a1da64ba3..a20bd26fabd 100644 --- a/src/Columns/tests/gtest_column_object.cpp +++ b/src/Columns/tests/gtest_column_object.cpp @@ -5,6 +5,7 @@ #include #include +#include "Core/Field.h" #include using namespace DB; @@ -349,3 +350,65 @@ TEST(ColumnObject, SkipSerializedInArena) pos = col2->skipSerializedInArena(pos); ASSERT_EQ(pos, end); } + +TEST(ColumnObject, rollback) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, a.a UInt32, a.b UInt32)"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + const auto & typed_paths = col_object.getTypedPaths(); + const auto & dynamic_paths = col_object.getDynamicPaths(); + const auto & shared_data = col_object.getSharedDataColumn(); + + auto assert_sizes = [&](size_t size) + { + for (const auto & [name, column] : typed_paths) + ASSERT_EQ(column->size(), size); + + for (const auto & [name, column] : dynamic_paths) + ASSERT_EQ(column->size(), size); + + ASSERT_EQ(shared_data.size(), size); + }; + + auto checkpoint = col_object.getCheckpoint(); + + col_object.insert(Object{{"a.a", Field{1u}}}); + col_object.updateCheckpoint(*checkpoint); + + col_object.insert(Object{{"a.b", Field{2u}}}); + col_object.insert(Object{{"a.a", Field{3u}}}); + + col_object.rollback(*checkpoint); + + assert_sizes(1); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 0); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*typed_paths.at("a.b"))[0], Field{0u}); + + col_object.insert(Object{{"a.c", Field{"ccc"}}}); + + checkpoint = col_object.getCheckpoint(); + + col_object.insert(Object{{"a.d", Field{"ddd"}}}); + col_object.insert(Object{{"a.e", Field{"eee"}}}); + + assert_sizes(4); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 2); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field{"ccc"}); + ASSERT_EQ((*dynamic_paths.at("a.d"))[2], Field{"ddd"}); + + col_object.rollback(*checkpoint); + + assert_sizes(2); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 1); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field{"ccc"}); +} From cef9eb80d9004b55887e3a88063fba8300b721f9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 22:47:59 +0000 Subject: [PATCH 037/816] better checkpoints for ColumnString --- src/Columns/ColumnString.cpp | 17 +++++++++++++++++ src/Columns/ColumnString.h | 4 ++++ 2 files changed, 21 insertions(+) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 00cf3bd9c30..269c20397b4 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -240,6 +240,23 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const return permuteImpl(*this, perm, limit); } +ColumnCheckpointPtr ColumnString::getCheckpoint() const +{ + auto nested = std::make_shared(chars.size()); + return std::make_shared(size(), std::move(nested)); +} + +void ColumnString::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + assert_cast(checkpoint).nested->size = chars.size(); +} + +void ColumnString::rollback(const ColumnCheckpoint & checkpoint) +{ + offsets.resize_assume_reserved(checkpoint.size); + chars.resize_assume_reserved(assert_cast(checkpoint).nested->size); +} void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index ec0563b3f00..c2371412437 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -194,6 +194,10 @@ public: offsets.resize_assume_reserved(offsets.size() - n); } + ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; + void rollback(const ColumnCheckpoint & checkpoint) override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; From 9edc66d458d2e9376ed52582dc25ab7934ea9085 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 6 Sep 2024 19:22:59 +0000 Subject: [PATCH 038/816] simplify scheduler constraints --- src/Common/Scheduler/ISchedulerConstraint.h | 25 +------------- .../Scheduler/Nodes/SemaphoreConstraint.h | 9 +---- .../Scheduler/Nodes/ThrottlerConstraint.h | 11 ++----- src/Common/Scheduler/ResourceRequest.cpp | 33 +++++++++++++++++-- src/Common/Scheduler/ResourceRequest.h | 18 +++++++--- 5 files changed, 49 insertions(+), 47 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index a976206de74..754f6dd404f 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -15,8 +15,7 @@ namespace DB * When constraint is again satisfied, scheduleActivation() is called from finishRequest(). * * Derived class behaviour requirements: - * - dequeueRequest() must fill `request->constraint` iff it is nullptr; - * - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`. + * - dequeueRequest() must call `request->addConstraint()`. */ class ISchedulerConstraint : public ISchedulerNode { @@ -29,30 +28,8 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; - void setParent(ISchedulerNode * parent_) override - { - ISchedulerNode::setParent(parent_); - - // Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint - for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent) - { - if (auto * constraint = dynamic_cast(node)) - { - parent_constraint = constraint; - break; - } - } - } - /// For introspection of current state (true = satisfied, false = violated) virtual bool isSatisfied() = 0; - -protected: - // Reference to nearest parent that is also derived from ISchedulerConstraint. - // Request can traverse through multiple constraints while being dequeue from hierarchy, - // while finishing request should traverse the same chain in reverse order. - // NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest() - ISchedulerConstraint * parent_constraint = nullptr; }; } diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 92c6af9db18..46b048ce34c 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -69,10 +69,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + request->addConstraint(this); // Update state on request arrival std::unique_lock lock(mutex); @@ -87,10 +84,6 @@ public: void finishRequest(ResourceRequest * request) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // Update state on request departure std::unique_lock lock(mutex); bool was_active = active(); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 56866336f50..4e2faa6b233 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -79,10 +79,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + // We don't do `request->addConstraint(this)` because `finishRequest()` is no-op updateBucket(request->cost); @@ -93,12 +90,8 @@ public: return {request, active()}; } - void finishRequest(ResourceRequest * request) override + void finishRequest(ResourceRequest *) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // NOTE: Token-bucket constraint does not require any action when consumption ends } diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 26e8084cdfa..91394108f5d 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -1,13 +1,42 @@ #include #include +#include + +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + void ResourceRequest::finish() { - if (constraint) - constraint->finishRequest(this); + // Iterate over constraints in reverse order + for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints)) + { + if (constraint) + constraint->finishRequest(this); + } +} + +void ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +{ + for (auto & constraint : constraints) + { + if (!constraint) + { + constraint = new_constraint; + return; + } + } + // TODO(serxa): is it possible to validate it during enqueue of resource request to avoid LOGICAL_ERRORs in the scheduler thread? possible but will not cover case of moving queue with requests inside to invalid position + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Max number of simultaneous workload constraints exceeded ({}). Remove extra constraints before using this workload.", + ResourceMaxConstraints); } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d64f624cec5..635353b569b 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -15,6 +16,10 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); +// TODO(serxa): validate hierarchy to avoid too many constrants +/// Max number of constraints for a request to pass though (depth of constaints chain) +constexpr size_t ResourceMaxConstraints = 8; + /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -49,9 +54,10 @@ public: /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; - /// Scheduler node to be notified on consumption finish - /// Auto-filled during request enqueue/dequeue - ISchedulerConstraint * constraint; + /// Scheduler nodes to be notified on consumption finish + /// Auto-filled during request dequeue + /// Vector is not used to avoid allocations in the scheduler thread + std::array constraints; explicit ResourceRequest(ResourceCost cost_ = 1) { @@ -62,7 +68,8 @@ public: void reset(ResourceCost cost_) { cost = cost_; - constraint = nullptr; + for (auto & constraint : constraints) + constraint = nullptr; // Note that list_base_hook should be reset independently (by intrusive list) } @@ -79,6 +86,9 @@ public: /// ResourceRequest should not be destructed or reset before calling to `finish()`. /// WARNING: this function MUST not be called if request was canceled. void finish(); + + /// Is called from the scheduler thread to fill `constraints` chain + void addConstraint(ISchedulerConstraint * new_constraint); }; } From 8e2f98a032378588e932e929fa1a46680846f367 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 7 Sep 2024 15:47:39 +0000 Subject: [PATCH 039/816] Make a clean start with v1.21.2 --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 71b06c22760..878cf51ff05 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d +Subproject commit 878cf51ff0516da8e50235e770f52c75e8dc11d8 From 35f27bf36db43d67121584bcf7bfc407c05ae2c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 7 Sep 2024 15:59:48 +0000 Subject: [PATCH 040/816] Bump krb5 to v1.21.3 --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 878cf51ff05..c5b4b994c18 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 878cf51ff0516da8e50235e770f52c75e8dc11d8 +Subproject commit c5b4b994c18db86933255907a97eee5993fd18fe From 14542d6779652c7c0b78efca3fa74fb6ae4a66f6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:26:54 +0000 Subject: [PATCH 041/816] added main building block UnifiedSchedulerNode --- src/Common/Priority.h | 5 +- src/Common/Scheduler/ISchedulerConstraint.h | 4 + src/Common/Scheduler/ISchedulerNode.h | 29 +- src/Common/Scheduler/ISchedulerQueue.h | 4 + src/Common/Scheduler/Nodes/FairPolicy.h | 4 + src/Common/Scheduler/Nodes/FifoQueue.h | 4 + src/Common/Scheduler/Nodes/PriorityPolicy.h | 4 + .../Scheduler/Nodes/SemaphoreConstraint.h | 7 + .../Scheduler/Nodes/ThrottlerConstraint.h | 8 + .../Scheduler/Nodes/UnifiedSchedulerNode.h | 346 ++++++++++++++++++ src/Common/Scheduler/SchedulingSettings.h | 38 ++ 11 files changed, 445 insertions(+), 8 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h create mode 100644 src/Common/Scheduler/SchedulingSettings.h diff --git a/src/Common/Priority.h b/src/Common/Priority.h index 8952fe4dd5a..f0e5787ae91 100644 --- a/src/Common/Priority.h +++ b/src/Common/Priority.h @@ -6,6 +6,7 @@ /// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize struct Priority { - Int64 value = 0; /// Note that lower value means higher priority. - constexpr operator Int64() const { return value; } /// NOLINT + using Value = Int64; + Value value = 0; /// Note that lower value means higher priority. + constexpr operator Value() const { return value; } /// NOLINT }; diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index 754f6dd404f..3bee9c1b424 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -24,6 +24,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + /// Resource consumption by `request` is finished. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index c051829e336..6d3132f79c1 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -57,7 +57,13 @@ struct SchedulerNodeInfo SchedulerNodeInfo() = default; - explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + explicit SchedulerNodeInfo(double weight_, Priority priority_ = {}) + { + setWeight(weight_); + setPriority(priority_); + } + + explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config, const String & config_prefix = {}) { setWeight(config.getDouble(config_prefix + ".weight", weight)); setPriority(config.getInt64(config_prefix + ".priority", priority)); @@ -78,6 +84,11 @@ struct SchedulerNodeInfo priority.value = value; } + void setPriority(Priority value) + { + priority = value; + } + // To check if configuration update required bool equals(const SchedulerNodeInfo & o) const { @@ -123,6 +134,11 @@ public: , info(config, config_prefix) {} + ISchedulerNode(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : event_queue(event_queue_) + , info(info_) + {} + virtual ~ISchedulerNode() = default; /// Checks if two nodes configuration is equal @@ -134,10 +150,11 @@ public: /// Attach new child virtual void attachChild(const std::shared_ptr & child) = 0; - /// Detach and destroy child + /// Detach child + /// NOTE: child might be destroyed if the only reference was stored in parent virtual void removeChild(ISchedulerNode * child) = 0; - /// Get attached child by name + /// Get attached child by name (for tests only) virtual ISchedulerNode * getChild(const String & child_name) = 0; /// Activation of child due to the first pending request @@ -147,7 +164,7 @@ public: /// Returns true iff node is active virtual bool isActive() = 0; - /// Returns number of active children + /// Returns number of active children (for introspection only). virtual size_t activeChildren() = 0; /// Returns the first request to be executed as the first component of resulting pair. @@ -155,10 +172,10 @@ public: virtual std::pair dequeueRequest() = 0; /// Returns full path string using names of every parent - String getPath() + String getPath() const { String result; - ISchedulerNode * ptr = this; + const ISchedulerNode * ptr = this; while (ptr->parent) { result = "/" + ptr->basename + result; diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index 532f4bf6c63..e816050a50e 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -21,6 +21,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + // Wrapper for `enqueueRequest()` that should be used to account for available resource budget void enqueueRequestUsingBudget(ResourceRequest * request) { diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index fba637e979e..b6be26bea98 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -48,6 +48,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + FairPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 9fbc6d1ae65..49f3e268bc8 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -30,6 +30,10 @@ public: : ISchedulerQueue(event_queue_, config, config_prefix) {} + FifoQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerQueue(event_queue_, info_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 91dc95600d5..17fcbfd3139 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -39,6 +39,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + explicit PriorityPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & node_info) + : ISchedulerNode(event_queue_, node_info) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 46b048ce34c..a2d8df48065 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/Scheduler/ISchedulerNode.h" #include #include @@ -24,6 +25,12 @@ public: , max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost))) {} + SemaphoreConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, Int64 max_requests_, Int64 max_cost_) + : ISchedulerConstraint(event_queue_, info_) + , max_requests(max_requests_) + , max_cost(max_cost_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 4e2faa6b233..7c64dd51ac1 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -28,6 +28,14 @@ public: , tokens(max_burst) {} + ThrottlerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, double max_speed_, double max_burst_) + : ISchedulerConstraint(event_queue_, info_) + , max_speed(max_speed_) + , max_burst(max_burst_) + , last_update(event_queue_->now()) + , tokens(max_burst) + {} + ~ThrottlerConstraint() override { // We should cancel event on destruction to avoid dangling references from event queue diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h new file mode 100644 index 00000000000..46ea5f0f340 --- /dev/null +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -0,0 +1,346 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +/* + * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. + * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes + * are also present. This approch is easiers for manipulations in runtime than using multiple types of nodes. + * + * Unified node is capable of updating its internal structure based on: + * 1. Number of children (fifo if =0 or fairness/priority if >0). + * 2. Priorities of its children (for subtree structure). + * 3. `SchedulingSettings` associated with unified node (for throttler and semaphore constraints). + * + * In general, unified node has "internal" subtree with the following structure: + * + * THIS <-- UnifiedSchedulerNode object + * | + * THROTTLER <-- [Optional] Throttling scheduling constraint + * | + * [If no children]------ SEMAPHORE <-- [Optional] Semaphore constraint + * | | + * FIFO PRIORITY <-- [Optional] Scheduling policy distinguishing priorities + * .-------' '-------. + * FAIRNESS[p1] ... FAIRNESS[pN] <-- [Optional] Policies for fairness if priorities are equal + * / \ / \ + * CHILD[p1,w1] ... CHILD[p1,wM] CHILD[pN,w1] ... CHILD[pN,wM] <-- Unified children (UnifiedSchedulerNode objects) + * + * NOTE: to distinguish different kinds of children we use the following terms: + * - immediate child: child of unified object (THROTTLER); + * - unified child: leaf of this "internal" subtree (CHILD[p,w]); + * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) + */ +class UnifiedSchedulerNode : public ISchedulerNode +{ +private: + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, const SchedulerNodePtr & new_parent) + { + reparent(node, new_parent.get()); + } + + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) + { + if (!new_parent || new_parent == node->parent) + return; + if (node->parent) + node->parent->removeChild(node.get()); + new_parent->attachChild(node); + } + + /// A branch of the tree for a specific priority value + struct FairnessBranch { + SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached + std::unordered_map children; // basename -> child + + SchedulerNodePtr getRoot() + { + chassert(!children.empty()); + if (root) + return root; + return children.begin()->second; // There should be exactly one child + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) + throw Exception( + ErrorCodes::INVALID_SCHEDULER_NODE, + "Can't add another child with the same path: {}", + it->second->getPath()); + + if (children.size() == 2) + { + // Insert fair node if we have just added the second child + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->info.setPriority(child->info.priority); + root->basename = fmt::format("p{}_fair", child->info.priority.value); + for (auto & [_, node] : children) + reparent(node, root); + return root; // New root has been created + } + else if (children.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + }; + + /// Handles all the children nodes with intermediate fair and/or priority nodes + struct ChildrenBranch + { + SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached + std::unordered_map branches; /// Branches for different priority values + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + bool existing_branch = branches.contains(child->info.priority); + auto & child_branch = branches[child->info.priority]; + auto branch_root = child_branch.attachUnifiedChild(event_queue_, child); + + if (existing_branch) + { + if (branch_root) + reparent(branch_root, root); + return {}; + } + else + { + chassert(branch_root); + if (branches.size() == 2) + { + // Insert priority node if we have just added the second branch + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->basename = "prio"; + for (auto & [_, branch] : branches) + reparent(branch.getRoot(), root); + return root; // New root has been created + } + else if (branches.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + } + }; + + /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. + struct QueueOrChildrenBranch + { + SchedulerNodePtr queue; /// FifoQueue node is used if there are no children + ChildrenBranch branch; /// Used if there is at least one child + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) + { + createQueue(event_queue_); + return queue; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (queue) + removeQueue(event_queue_); + return branch.attachUnifiedChild(event_queue_, child); + } + + private: + void createQueue(EventQueue * event_queue_) + { + queue = std::make_shared(event_queue_, SchedulerNodeInfo{}); + queue->basename = "fifo"; + } + + void removeQueue(EventQueue *) + { + // TODO(serxa): cancel all requests, this unified node is not capable of service resoruce requests now + queue.reset(); + } + }; + + /// Handles all the nodes under this unified node + /// Specifically handles constraints with `QueueOrChildrenBranch` under it + struct ConstraintsBranch + { + SchedulerNodePtr throttler; + SchedulerNodePtr semaphore; + QueueOrChildrenBranch branch; + SchedulingSettings settings; + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_, const SchedulingSettings & settings_) + { + settings = settings_; + SchedulerNodePtr node = branch.initialize(event_queue_); + if (settings.hasSemaphore()) + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_requests, settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + if (settings.hasThrottler()) + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_speed, settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + return node; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) + { + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } + }; + +public: + explicit UnifiedSchedulerNode(EventQueue * event_queue_, const SchedulingSettings & settings) + : ISchedulerNode(event_queue_, SchedulerNodeInfo(settings.weight, settings.priority)) + { + immediate_child = impl.initialize(event_queue, settings); + reparent(immediate_child, this); + } + + bool equals(ISchedulerNode *) override + { + assert(false); + return false; + } + + /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate node + /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children + void attachUnifiedChild(const SchedulerNodePtr & child) + { + if (auto new_child = impl.attachUnifiedChild(event_queue, child)) + reparent(new_child, this); + } + + /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) + /// NOTE: Changing a priority of a unified child may lead to change of its parent. + void updateUnifiedChildPriority(const SchedulerNodePtr & child, Priority old_priority, Priority new_priority) + { + UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority + } + + /// Updates scheduling settings. Set of constraints might change. + /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) + void updateSchedulingSettings(const SchedulingSettings & new_settings) + { + UNUSED(new_settings); // TODO: implement updateSchedulingSettings + } + + /// Attaches an immediate child (used through `reparent()`) + void attachChild(const SchedulerNodePtr & child_) override + { + immediate_child = child_; + immediate_child->setParent(this); + + // Activate if required + if (immediate_child->isActive()) + activateChild(immediate_child.get()); + } + + /// Removes an immediate child (used through `reparent()`) + void removeChild(ISchedulerNode * child) override + { + if (immediate_child.get() == child) + { + child_active = false; // deactivate + immediate_child->setParent(nullptr); // detach + immediate_child.reset(); + } + } + + ISchedulerNode * getChild(const String & child_name) override + { + if (immediate_child->basename == child_name) + return immediate_child.get(); + else + return nullptr; + } + + std::pair dequeueRequest() override + { + auto [request, child_now_active] = immediate_child->dequeueRequest(); + if (!request) + return {nullptr, false}; + + child_active = child_now_active; + if (!child_active) + busy_periods++; + incrementDequeued(request->cost); + return {request, child_active}; + } + + bool isActive() override + { + return child_active; + } + + /// Shows number of immediate active children (for introspection) + size_t activeChildren() override + { + return child_active; + } + + /// Activate an immediate child + void activateChild(ISchedulerNode * child) override + { + if (child == immediate_child.get()) + if (!std::exchange(child_active, true) && parent) + parent->activateChild(this); + } + +private: + ConstraintsBranch impl; + SchedulerNodePtr immediate_child; // An immediate child (actually the root of the whole subtree) + bool child_active = false; +}; + +} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h new file mode 100644 index 00000000000..4c6eff2b1e9 --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +struct SchedulingSettings +{ + /// Priority and weight among siblings + double weight = 1.0; + Priority priority; + + /// Throttling constraints. + /// Up to 2 independent throttlers: one for average speed and one for peek speed. + static constexpr double default_burst_seconds = 1.0; + double max_speed = 0; // Zero means unlimited + double max_burst = 0; // default is `default_burst_seconds * max_speed` + + /// Limits total number of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_requests = std::numeric_limits::max(); + Int64 max_requests = default_max_requests; + + /// Limits total cost of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_cost = std::numeric_limits::max(); + Int64 max_cost = default_max_cost; + + bool hasThrottler() const { return max_speed != 0; } + bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } + + // TODO(serxa): add helper functions for parsing, printing and validating +}; + +} From 7bf7b516a753dd106bfb5d56da71eb814775274e Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:27:54 +0000 Subject: [PATCH 042/816] add smoke test for UnifiedSchedulerNode --- .../Scheduler/Nodes/tests/ResourceTest.h | 19 ++++++++++++++-- .../tests/gtest_unified_scheduler_node.cpp | 22 +++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index ea3f9edf765..6583e2beb0f 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -14,10 +14,12 @@ #include #include +#include #include #include #include #include +#include namespace DB { @@ -37,10 +39,17 @@ struct ResourceTestBase Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)}; String config_prefix = "node"; + return add(event_queue, root_node, path, std::ref(*config), config_prefix); + } + + template + static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) + { + if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); - root_node.reset(new TClass(event_queue, *config, config_prefix)); + root_node.reset(new TClass(event_queue, std::forward(args)...)); return static_cast(root_node.get()); } @@ -65,7 +74,7 @@ struct ResourceTestBase } EXPECT_TRUE(!child_name.empty()); // wrong path - SchedulerNodePtr node = std::make_shared(event_queue, *config, config_prefix); + SchedulerNodePtr node = std::make_shared(event_queue, std::forward(args)...); node->basename = child_name; parent->attachChild(node); return static_cast(node.get()); @@ -126,6 +135,12 @@ public: ResourceTestBase::add(&event_queue, root_node, path, xml); } + template + void addCustom(const String & path, Args... args) + { + ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); + } + void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp new file mode 100644 index 00000000000..2acda88ef17 --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -0,0 +1,22 @@ +#include +#include + +#include + +#include +#include + +using namespace DB; + +using ResourceTest = ResourceTestClass; + +TEST(SchedulerUnifiedNode, Smoke) +{ + ResourceTest t; + + t.addCustom("/", SchedulingSettings{}); + + t.enqueue("/fifo", {10, 10}); + t.dequeue(2); + t.consumed("fifo", 20); +} From 4f70f48272444a07514c42268862a952dae29e49 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 11 Sep 2024 19:29:53 +0000 Subject: [PATCH 043/816] add more tests --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 33 ++++++++--- .../Scheduler/Nodes/tests/ResourceTest.h | 39 ++++++++++-- .../tests/gtest_unified_scheduler_node.cpp | 59 +++++++++++++++++++ 3 files changed, 117 insertions(+), 14 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 46ea5f0f340..85b22b02cfa 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -64,7 +64,8 @@ private: /// Helper function for managing a parent of a node static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) { - if (!new_parent || new_parent == node->parent) + chassert(new_parent); + if (new_parent == node->parent) return; if (node->parent) node->parent->removeChild(node.get()); @@ -130,7 +131,12 @@ private: if (existing_branch) { if (branch_root) - reparent(branch_root, root); + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } return {}; } else @@ -247,13 +253,7 @@ public: reparent(immediate_child, this); } - bool equals(ISchedulerNode *) override - { - assert(false); - return false; - } - - /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate node + /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const SchedulerNodePtr & child) { @@ -275,6 +275,19 @@ public: UNUSED(new_settings); // TODO: implement updateSchedulingSettings } + /// Returns the queue to be used for resource requests or `nullptr` if it has unified children + ISchedulerQueue * getQueue() + { + return static_cast(impl.branch.queue.get()); + } + +protected: // Hide all the ISchedulerNode interface methods as an implementation details + bool equals(ISchedulerNode *) override + { + assert(false); + return false; + } + /// Attaches an immediate child (used through `reparent()`) void attachChild(const SchedulerNodePtr & child_) override { @@ -343,4 +356,6 @@ private: bool child_active = false; }; +using UnifiedSchedulerNodePtr = std::shared_ptr; + } diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 6583e2beb0f..4adc0ae7028 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/Scheduler/SchedulingSettings.h" #include #include #include @@ -7,6 +8,7 @@ #include #include #include +#include #include #include @@ -15,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -45,7 +48,6 @@ struct ResourceTestBase template static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) { - if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); @@ -141,6 +143,32 @@ public: ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); } + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const SchedulingSettings & settings = {}) + { + return createUnifiedNode(basename, {}, settings); + } + + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const UnifiedSchedulerNodePtr & parent, const SchedulingSettings & settings = {}) + { + auto node = std::make_shared(&event_queue, settings); + node->basename = basename; + if (parent) + { + parent->attachUnifiedChild(node); + } + else + { + EXPECT_TRUE(root_node.get() == nullptr); + root_node = node; + } + return node; + } + + void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) + { + enqueueImpl(node->getQueue(), costs); + } + void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first @@ -161,13 +189,14 @@ public: pos = String::npos; } } - ISchedulerQueue * queue = dynamic_cast(node); - ASSERT_TRUE(queue != nullptr); // not a queue + enqueueImpl(dynamic_cast(node), costs); + } + void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs) + { + ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - { queue->enqueueRequest(new Request(cost, queue->basename)); - } processEvents(); // to activate queues } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 2acda88ef17..41a5c7f1036 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,6 +1,7 @@ #include #include +#include "Common/Priority.h" #include #include @@ -20,3 +21,61 @@ TEST(SchedulerUnifiedNode, Smoke) t.dequeue(2); t.consumed("fifo", 20); } + +TEST(SchedulerUnifiedNode, Fairness) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(); + t.consumed("A", 60); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, Priority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10}); + t.enqueue(b, {10, 10, 10}); + t.enqueue(c, {10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 0); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.dequeue(); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 0); +} From 4401b4dda087ee323871fa2cb5da929c3322e26c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:06:39 +0000 Subject: [PATCH 044/816] fix request naming for tests --- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 4adc0ae7028..762cb64a307 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -166,7 +166,7 @@ public: void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { - enqueueImpl(node->getQueue(), costs); + enqueueImpl(node->getQueue(), costs, node->basename); } void enqueue(const String & path, const std::vector & costs) @@ -192,11 +192,11 @@ public: enqueueImpl(dynamic_cast(node), costs); } - void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs) + void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs, const String & name = {}) { ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - queue->enqueueRequest(new Request(cost, queue->basename)); + queue->enqueueRequest(new Request(cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues } From 86515e1bce1bb1bb7c3829619fb673713d96723c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:27:32 +0000 Subject: [PATCH 045/816] add more tests for fairness and priority --- .../tests/gtest_unified_scheduler_node.cpp | 224 +++++++++++++++++- 1 file changed, 223 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 41a5c7f1036..92c616ff65c 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -22,7 +22,7 @@ TEST(SchedulerUnifiedNode, Smoke) t.consumed("fifo", 20); } -TEST(SchedulerUnifiedNode, Fairness) +TEST(SchedulerUnifiedNode, FairnessWeight) { ResourceTest t; @@ -46,6 +46,152 @@ TEST(SchedulerUnifiedNode, Fairness) t.consumed("B", 20); } +TEST(SchedulerUnifiedNode, FairnessActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + auto c = t.createUnifiedNode("C", all); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 10); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(4); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 10); + + t.enqueue(b, {10, 10}); + t.dequeue(1); + t.consumed("B", 10); + + t.enqueue(c, {10, 10}); + t.dequeue(1); + t.consumed("C", 10); + + t.dequeue(2); // A B or B A + t.consumed("A", 10); + t.consumed("B", 10); +} + +TEST(SchedulerUnifiedNode, FairnessMaxMin) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + + t.enqueue(a, {10, 10}); // make sure A is never empty + + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + + t.dequeue(2); + t.consumed("A", 20); +} + +TEST(SchedulerUnifiedNode, FairnessHierarchical) +{ + ResourceTest t; + + + auto all = t.createUnifiedNode("all"); + auto x = t.createUnifiedNode("X", all); + auto y = t.createUnifiedNode("Y", all); + auto a = t.createUnifiedNode("A", x); + auto b = t.createUnifiedNode("B", x); + auto c = t.createUnifiedNode("C", y); + auto d = t.createUnifiedNode("D", y); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("B", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("D", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("D", 40); + } +} + TEST(SchedulerUnifiedNode, Priority) { ResourceTest t; @@ -79,3 +225,79 @@ TEST(SchedulerUnifiedNode, Priority) t.consumed("B", 0); t.consumed("C", 0); } + +TEST(SchedulerUnifiedNode, PriorityActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); + + t.enqueue(b, {10, 10, 10}); + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.enqueue(c, {10, 10}); + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); +} + +TEST(SchedulerUnifiedNode, List) +{ + ResourceTest t; + + std::list list; + list.push_back(t.createUnifiedNode("all")); + + for (int length = 1; length < 5; length++) + { + String name = fmt::format("L{}", length); + list.push_back(t.createUnifiedNode(name, list.back())); + + for (int i = 0; i < 3; i++) + { + t.enqueue(list.back(), {10, 10}); + t.dequeue(1); + t.consumed(name, 10); + + for (int j = 0; j < 3; j++) + { + t.enqueue(list.back(), {10, 10, 10}); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + } + + t.dequeue(1); + t.consumed(name, 10); + } + } +} From 3ff86a4347741335e7c7b163e13eadbd7ec24107 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:43:48 +0000 Subject: [PATCH 046/816] add tests for max_speed and max_bust --- .../tests/gtest_unified_scheduler_node.cpp | 121 +++++++++++++++++- 1 file changed, 120 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 92c616ff65c..bddfeb19851 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,9 +1,9 @@ #include #include -#include "Common/Priority.h" #include +#include #include #include @@ -301,3 +301,122 @@ TEST(SchedulerUnifiedNode, List) } } } + +TEST(SchedulerUnifiedNode, ThrottlerLeakyBucket) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(3)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(4)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, ThrottlerPacing) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + // Zero burst allows you to send one request of any `size` and than throttle for `size/max_speed` seconds. + // Useful if outgoing traffic should be "paced", i.e. have the least possible burstiness. + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 1.0, .max_burst = 0.0}); + + t.enqueue(all, {1, 2, 3, 1, 2, 1}); + int output[] = {1, 2, 0, 3, 0, 0, 1, 2, 0, 1, 0}; + for (int i = 0; i < std::size(output); i++) + { + t.process(start + std::chrono::seconds(i)); + t.consumed("all", output[i]); + } +} + +TEST(SchedulerUnifiedNode, ThrottlerBucketFilling) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 60); // 5 sec * 10 tokens/sec = 50 tokens + 1 extra request to go below zero + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 40); // Consume rest + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {95, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 101); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} + +TEST(SchedulerUnifiedNode, ThrottlerAndFairness) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + auto a = t.createUnifiedNode("A", all, {.weight = 10.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 90.0, .priority = Priority{}}); + + ResourceCost req_cost = 1; + ResourceCost total_cost = 2000; + for (int i = 0; i < total_cost / req_cost; i++) + { + t.enqueue(a, {req_cost}); + t.enqueue(b, {req_cost}); + } + + double shareA = 0.1; + double shareB = 0.9; + + // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share + auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); + auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + + double consumedA = 0; + double consumedB = 0; + for (int seconds = 0; seconds < 100; seconds++) + { + t.process(start + std::chrono::seconds(seconds)); + double arrival_curve = 100.0 + 10.0 * seconds + req_cost; + t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); + t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); + consumedA = arrival_curve * shareA; + consumedB = arrival_curve * shareB; + } +} From ca1567da0311a58d7ba0217ca3fbb5d3fcf806f5 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 18:01:22 +0000 Subject: [PATCH 047/816] abort resource requests in queue dtor --- src/Common/Scheduler/Nodes/FifoQueue.h | 11 ++++++++++ .../Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- .../Scheduler/Nodes/tests/ResourceTest.h | 7 +++++++ .../Nodes/tests/gtest_resource_scheduler.cpp | 5 +++++ src/Common/Scheduler/ResourceGuard.h | 20 +++++++++++++++++++ src/Common/Scheduler/ResourceRequest.h | 6 +++++- 6 files changed, 50 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 49f3e268bc8..b3c8bbcffbf 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -34,6 +34,17 @@ public: : ISchedulerQueue(event_queue_, info_) {} + ~FifoQueue() override + { + while (!requests.empty()) + { + ResourceRequest * request = &requests.front(); + requests.pop_front(); + request->failed(std::make_exception_ptr( + Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request was destructed"))); + } + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 85b22b02cfa..4bdcaca3cb1 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -12,7 +12,6 @@ #include #include -#include namespace DB { @@ -192,7 +191,8 @@ private: void removeQueue(EventQueue *) { - // TODO(serxa): cancel all requests, this unified node is not capable of service resoruce requests now + // This unified node will not be able to process resource requests any longer + // All remaining resource requests are be aborted on queue destruction queue.reset(); } }; diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 9bf70e42e1a..0e246ed2273 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -119,6 +120,7 @@ class ResourceTestClass : public ResourceTestBase struct Request : public ResourceRequest { String name; + std::exception_ptr exception; Request(ResourceCost cost_, const String & name_) : ResourceRequest(cost_) @@ -128,6 +130,11 @@ class ResourceTestClass : public ResourceTestBase void execute() override { } + + void failed(const std::exception_ptr & ptr) override + { + exception = ptr; + } }; public: diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ddfe0cfbc6f..8eaa4ebb840 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -101,6 +101,11 @@ struct MyRequest : public ResourceRequest if (on_execute) on_execute(); } + + void failed(const std::exception_ptr &) override + { + FAIL(); + } }; TEST(SchedulerRoot, Smoke) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index cf97f7acf93..6ff22edd221 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -12,6 +12,7 @@ #include #include +#include #include @@ -34,6 +35,11 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor @@ -109,12 +115,25 @@ public: dequeued_cv.notify_one(); } + // This function is executed inside scheduler thread and wakes thread issued this `request`. + // That thread will throw an exception. + void failed(const std::exception_ptr & ptr) override + { + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + exception = ptr; + dequeued_cv.notify_one(); + } + void wait() { CurrentMetrics::Increment scheduled(metrics->scheduled_count); auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); + if (exception) + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Resource request failed: {}", getExceptionMessage(exception, /* with_stacktrace = */ false)); } void finish(ResourceCost real_cost_, ResourceLink link_) @@ -151,6 +170,7 @@ public: std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; + std::exception_ptr exception; }; /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d394459819e..24afcc98b57 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -81,10 +82,13 @@ public: /// (e.g. setting an std::promise or creating a job in a thread pool) virtual void execute() = 0; + /// Callback to trigger an error in case if resource is unavailable. + virtual void failed(const std::exception_ptr & ptr) = 0; + /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// WARNING: this function MUST not be called if request was canceled. + /// WARNING: this function MUST not be called if request was canceled or failed. void finish(); /// Is called from the scheduler thread to fill `constraints` chain From 7acc10444cee1a6d4ef5079f5a682e3102fc1535 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 18:20:53 +0000 Subject: [PATCH 048/816] add test for queue destruction --- .../tests/gtest_unified_scheduler_node.cpp | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index bddfeb19851..24a8950d9ae 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -420,3 +420,31 @@ TEST(SchedulerUnifiedNode, ThrottlerAndFairness) consumedB = arrival_curve * shareB; } } + +TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + // This will destory the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} From 6307ada396541f82b143c1a19691fd0589fda32b Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 19:58:31 +0000 Subject: [PATCH 049/816] add purgeQueue() with test --- src/Common/Scheduler/ISchedulerQueue.h | 5 +++ src/Common/Scheduler/Nodes/FifoQueue.h | 26 ++++++++--- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 14 +++++- .../Scheduler/Nodes/tests/ResourceTest.h | 26 ++++++++--- .../tests/gtest_unified_scheduler_node.cpp | 45 +++++++++++++++++++ 5 files changed, 101 insertions(+), 15 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index e07f797cb42..6c77cee6b9d 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -51,6 +51,11 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual bool cancelRequest(ResourceRequest * request) = 0; + /// Fails all the resource requests in queue and marks this queue as not usable. + /// Afterwards any new request will be failed on `enqueueRequest()`. + /// NOTE: This is done for queues that are about to be destructed. + virtual void purgeQueue() = 0; + /// For introspection ResourceCost getBudget() const { diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index b3c8bbcffbf..c95125b21bf 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -36,13 +36,7 @@ public: ~FifoQueue() override { - while (!requests.empty()) - { - ResourceRequest * request = &requests.front(); - requests.pop_front(); - request->failed(std::make_exception_ptr( - Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request was destructed"))); - } + chassert(requests.empty()); } bool equals(ISchedulerNode * other) override @@ -57,6 +51,8 @@ public: void enqueueRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue is about to be destructed"); queue_cost += request->cost; bool was_empty = requests.empty(); requests.push_back(*request); @@ -81,6 +77,8 @@ public: bool cancelRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + return false; // Any request should already be failed or executed if (request->is_linked()) { // It's impossible to check that `request` is indeed inserted to this queue and not another queue. @@ -103,6 +101,19 @@ public: return false; } + void purgeQueue() override + { + std::lock_guard lock(mutex); + is_not_usable = true; + while (!requests.empty()) + { + ResourceRequest * request = &requests.front(); + requests.pop_front(); + request->failed(std::make_exception_ptr( + Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request is about to be destructed"))); + } + } + bool isActive() override { std::lock_guard lock(mutex); @@ -146,6 +157,7 @@ private: std::mutex mutex; Int64 queue_cost = 0; boost::intrusive::list requests; + bool is_not_usable = false; }; } diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 4bdcaca3cb1..fa284ed5254 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -71,6 +72,13 @@ private: new_parent->attachChild(node); } + /// Helper function for managing a parent of a node + static void detach(const SchedulerNodePtr & node) + { + if (node->parent) + node->parent->removeChild(node.get()); + } + /// A branch of the tree for a specific priority value struct FairnessBranch { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached @@ -193,6 +201,8 @@ private: { // This unified node will not be able to process resource requests any longer // All remaining resource requests are be aborted on queue destruction + detach(queue); + std::static_pointer_cast(queue)->purgeQueue(); queue.reset(); } }; @@ -276,9 +286,9 @@ public: } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children - ISchedulerQueue * getQueue() + std::shared_ptr getQueue() { - return static_cast(impl.branch.queue.get()); + return static_pointer_cast(impl.branch.queue); } protected: // Hide all the ISchedulerNode interface methods as an implementation details diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 0e246ed2273..acb8504ce30 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -119,11 +119,12 @@ class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest { + ResourceTestClass * test; String name; - std::exception_ptr exception; - Request(ResourceCost cost_, const String & name_) + Request(ResourceTestClass * test_, ResourceCost cost_, const String & name_) : ResourceRequest(cost_) + , test(test_) , name(name_) {} @@ -131,13 +132,19 @@ class ResourceTestClass : public ResourceTestBase { } - void failed(const std::exception_ptr & ptr) override + void failed(const std::exception_ptr &) override { - exception = ptr; + test->failed_cost += cost; + delete this; } }; public: + ~ResourceTestClass() + { + dequeue(); // Just to avoid any leaks of `Request` object + } + template void add(const String & path, const String & xml = {}) { @@ -173,7 +180,7 @@ public: void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { - enqueueImpl(node->getQueue(), costs, node->basename); + enqueueImpl(node->getQueue().get(), costs, node->basename); } void enqueue(const String & path, const std::vector & costs) @@ -203,7 +210,7 @@ public: { ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - queue->enqueueRequest(new Request(cost, name.empty() ? queue->basename : name)); + queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues } @@ -259,6 +266,12 @@ public: consumed_cost[name] -= value; } + void failed(ResourceCost value) + { + EXPECT_EQ(failed_cost, value); + failed_cost -= value; + } + void processEvents() { while (event_queue.tryProcess()) {} @@ -268,6 +281,7 @@ private: EventQueue event_queue; SchedulerNodePtr root_node; std::unordered_map consumed_cost; + ResourceCost failed_cost = 0; }; template diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 24a8950d9ae..faebaa72b71 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include #include #include @@ -448,3 +450,46 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.dequeue(2); t.consumed("A", 20); } + + +TEST(SchedulerUnifiedNode, ResourceGuardException) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + std::thread consumer([queue = all->getQueue()] + { + ResourceLink link{.queue = queue.get()}; + try + { + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link); + FAIL(); + } + catch (...) + { + } + }); + + // This will destory the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + consumer.join(); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} From f8599391253d2e10679505c6d0879ea4277130eb Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 13 Sep 2024 15:49:17 +0000 Subject: [PATCH 050/816] allow only unified children for unified nodes --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 49 ++++++++++++++----- 1 file changed, 38 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index fa284ed5254..c3c8ca2134a 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -23,6 +23,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class UnifiedSchedulerNode; +using UnifiedSchedulerNodePtr = std::shared_ptr; + /* * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes @@ -82,7 +85,7 @@ private: /// A branch of the tree for a specific priority value struct FairnessBranch { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached - std::unordered_map children; // basename -> child + std::unordered_map children; // basename -> child SchedulerNodePtr getRoot() { @@ -94,7 +97,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) throw Exception( @@ -129,7 +132,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { bool existing_branch = branches.contains(child->info.priority); auto & child_branch = branches[child->info.priority]; @@ -183,10 +186,10 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (queue) - removeQueue(event_queue_); + removeQueue(); return branch.attachUnifiedChild(event_queue_, child); } @@ -197,7 +200,7 @@ private: queue->basename = "fifo"; } - void removeQueue(EventQueue *) + void removeQueue() { // This unified node will not be able to process resource requests any longer // All remaining resource requests are be aborted on queue destruction @@ -240,7 +243,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) { @@ -265,7 +268,7 @@ public: /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children - void attachUnifiedChild(const SchedulerNodePtr & child) + void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) { if (auto new_child = impl.attachUnifiedChild(event_queue, child)) reparent(new_child, this); @@ -273,7 +276,7 @@ public: /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) /// NOTE: Changing a priority of a unified child may lead to change of its parent. - void updateUnifiedChildPriority(const SchedulerNodePtr & child, Priority old_priority, Priority new_priority) + void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) { UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority } @@ -291,6 +294,32 @@ public: return static_pointer_cast(impl.branch.queue); } + /// Returns nodes that could be accessed with raw pointers by resource requests (queue and constraints) + /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints + /// for that queue might change in future versions, and `request->constraints` might reference nodes not in + /// the initial set of nodes returned by `getClassifierNodes()`. To avoid destruction of such additinal nodes + /// classifier must (indirectly) hold nodes return by `getClassifierNodes()` for all future versions of all unified nodes. + /// Such a version control is done by `IOResourceManager`. + std::vector getClassifierNodes() + { + std::vector result; + if (impl.branch.queue) + result.push_back(impl.branch.queue); + if (impl.semaphore) + result.push_back(impl.semaphore); + if (impl.throttler) + result.push_back(impl.throttler); + for (auto & [_, branch] : impl.branch.branch.branches) + { + for (auto & [_, child] : branch.children) + { + auto nodes = child->getClassifierNodes(); + result.insert(result.end(), nodes.begin(), nodes.end()); + } + } + return result; + } + protected: // Hide all the ISchedulerNode interface methods as an implementation details bool equals(ISchedulerNode *) override { @@ -366,6 +395,4 @@ private: bool child_active = false; }; -using UnifiedSchedulerNodePtr = std::shared_ptr; - } From 5473b5a051eb90040684eb4f2be4fadb94b26ed9 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 14 Sep 2024 09:45:59 +0000 Subject: [PATCH 051/816] get rid of ResourceManagersFactory --- programs/server/Server.cpp | 2 - .../Nodes/DynamicResourceManager.cpp | 6 -- .../Nodes/registerResourceManagers.cpp | 15 ----- .../Nodes/registerResourceManagers.h | 8 --- .../Scheduler/Nodes/tests/ResourceTest.h | 3 +- src/Common/Scheduler/ResourceManagerFactory.h | 55 ------------------- .../Scheduler/createResourceManager.cpp | 17 ++++++ src/Common/Scheduler/createResourceManager.h | 11 ++++ src/Interpreters/Context.cpp | 4 +- 9 files changed, 31 insertions(+), 90 deletions(-) delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.cpp delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.h delete mode 100644 src/Common/Scheduler/ResourceManagerFactory.h create mode 100644 src/Common/Scheduler/createResourceManager.cpp create mode 100644 src/Common/Scheduler/createResourceManager.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8043b10bead..66651c7ada3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -85,7 +85,6 @@ #include #include #include -#include #include #include #include @@ -781,7 +780,6 @@ try registerFormats(); registerRemoteFileMetadatas(); registerSchedulerNodes(); - registerResourceManagers(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index 6b9f6318903..29b3aefacf1 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include @@ -253,9 +252,4 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) future.get(); } -void registerDynamicResourceManager(ResourceManagerFactory & factory) -{ - factory.registerMethod("dynamic"); -} - } diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp b/src/Common/Scheduler/Nodes/registerResourceManagers.cpp deleted file mode 100644 index c5d5ba5b981..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDynamicResourceManager(ResourceManagerFactory &); - -void registerResourceManagers() -{ - auto & factory = ResourceManagerFactory::instance(); - registerDynamicResourceManager(factory); -} - -} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.h b/src/Common/Scheduler/Nodes/registerResourceManagers.h deleted file mode 100644 index 243b25a9587..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.h +++ /dev/null @@ -1,8 +0,0 @@ -#pragma once - -namespace DB -{ - -void registerResourceManagers(); - -} diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index acb8504ce30..c8cc0ed0e57 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -10,7 +10,6 @@ #include #include #include -#include #include @@ -32,7 +31,7 @@ struct ResourceTestBase { ResourceTestBase() { - [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }(); + [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); return true; }(); } template diff --git a/src/Common/Scheduler/ResourceManagerFactory.h b/src/Common/Scheduler/ResourceManagerFactory.h deleted file mode 100644 index 52f271e51b1..00000000000 --- a/src/Common/Scheduler/ResourceManagerFactory.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include -#include - -#include - -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SCHEDULER_NODE; -} - -class ResourceManagerFactory : private boost::noncopyable -{ -public: - static ResourceManagerFactory & instance() - { - static ResourceManagerFactory ret; - return ret; - } - - ResourceManagerPtr get(const String & name) - { - std::lock_guard lock{mutex}; - if (auto iter = methods.find(name); iter != methods.end()) - return iter->second(); - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name); - } - - template - void registerMethod(const String & name) - { - std::lock_guard lock{mutex}; - methods[name] = [] () - { - return std::make_shared(); - }; - } - -private: - std::mutex mutex; - using Method = std::function; - std::unordered_map methods; -}; - -} diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp new file mode 100644 index 00000000000..b0b7f731a89 --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -0,0 +1,17 @@ +#include +#include +#include +#include + +namespace DB +{ + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) +{ + UNUSED(global_context); + // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together + // const auto & config = global_context->getConfigRef(); + return std::make_shared(); +} + +} diff --git a/src/Common/Scheduler/createResourceManager.h b/src/Common/Scheduler/createResourceManager.h new file mode 100644 index 00000000000..d80a17f3bff --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context); + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6cacf7bd516..371a8d3900d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -89,7 +89,7 @@ #include #include #include -#include +#include #include #include #include @@ -1677,7 +1677,7 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { callOnce(shared->resource_manager_initialized, [&] { - shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); + shared->resource_manager = createResourceManager(getGlobalContext()); }); return shared->resource_manager; From b31268e1926737cfc15ec9ad1c9e30ec999859d2 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 17 Sep 2024 07:18:06 +0000 Subject: [PATCH 052/816] 1) Create replica_dir in DDLWorker 2) Mark replicas active in DDLWorker --- src/Databases/DatabaseReplicatedWorker.h | 2 +- src/Interpreters/DDLWorker.cpp | 155 +++++++++++++++++++---- src/Interpreters/DDLWorker.h | 24 ++-- 3 files changed, 146 insertions(+), 35 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 2309c831839..51ff0f96e6d 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -40,7 +40,7 @@ public: UInt64 getCurrentInitializationDurationMs() const; private: bool initializeMainThread() override; - void initializeReplication(); + void initializeReplication() override; void initializeLogPointer(const String & processed_entry_name); DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 697fd0f406b..408561b7606 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -1,48 +1,47 @@ -#include -#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include #include +#include +#include #include #include #include #include -#include -#include #include -#include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include + +#include +#include #include + #include #include #include -#include -#include - -#include namespace fs = std::filesystem; - namespace CurrentMetrics { extern const Metric DDLWorkerThreads; @@ -102,6 +101,12 @@ DDLWorker::DDLWorker( if (queue_dir.back() == '/') queue_dir.resize(queue_dir.size() - 1); + // replicas_dir is at the same level as queue_dir + // E.g: + // queue_dir: /clickhouse/task_queue/ddl + // replicas_dir: /clickhouse/task_queue/replicas + replicas_dir = fs::path(queue_dir).parent_path() / "replicas"; + if (config) { task_max_lifetime = config->getUInt64(prefix + ".task_max_lifetime", static_cast(task_max_lifetime)); @@ -1059,6 +1064,11 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) String query_path_prefix = fs::path(queue_dir) / "query-"; zookeeper->createAncestors(query_path_prefix); + NameSet host_ids; + for (const HostID & host : entry.hosts) + host_ids.emplace(host.toString()); + createReplicaDirs(zookeeper, host_ids); + String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); if (max_pushed_entry_metric) { @@ -1098,6 +1108,7 @@ bool DDLWorker::initializeMainThread() { auto zookeeper = getAndSetZooKeeper(); zookeeper->createAncestors(fs::path(queue_dir) / ""); + initializeReplication(); initialized = true; return true; } @@ -1159,6 +1170,7 @@ void DDLWorker::runMainThread() } cleanup_event->set(); + markReplicasActive(reinitialized); scheduleTasks(reinitialized); subsequent_errors_count = 0; @@ -1216,6 +1228,97 @@ void DDLWorker::runMainThread() } +void DDLWorker::initializeReplication() +{ + auto zookeeper = getAndSetZooKeeper(); + + zookeeper->createAncestors(replicas_dir / ""); + + NameSet host_id_set; + for (const auto & it : context->getClusters()) + { + auto cluster = it.second; + for (const auto & host_ids : cluster->getHostIDs()) + for (const auto & host_id : host_ids) + host_id_set.emplace(host_id); + } + + createReplicaDirs(zookeeper, host_id_set); +} + +void DDLWorker::createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids) +{ + for (const auto & host_id : host_ids) + zookeeper->createAncestors(replicas_dir / host_id / ""); +} + +void DDLWorker::markReplicasActive(bool reinitialized) +{ + auto zookeeper = getAndSetZooKeeper(); + + if (reinitialized) + { + // Reset all active_node_holders + for (auto & it : active_node_holders) + { + auto & active_node_holder = it.second.second; + if (active_node_holder) + active_node_holder->setAlreadyRemoved(); + active_node_holder.reset(); + } + + active_node_holders.clear(); + } + + const auto maybe_secure_port = context->getTCPPortSecure(); + const auto port = context->getTCPPort(); + + Coordination::Stat replicas_stat; + Strings host_ids = zookeeper->getChildren(replicas_dir, &replicas_stat); + NameSet local_host_ids; + for (const auto & host_id : host_ids) + { + if (active_node_holders.contains(host_id)) + continue; + + try + { + HostID host = HostID::fromString(host_id); + /// The port is considered local if it matches TCP or TCP secure port that the server is listening. + bool is_local_host = (maybe_secure_port && host.isLocalAddress(*maybe_secure_port)) || host.isLocalAddress(port); + + if (is_local_host) + local_host_ids.emplace(host_id); + } + catch (const Exception & e) + { + LOG_WARNING(log, "Unable to check if host {} is a local address, exception: {}", host_id, e.displayText()); + continue; + } + } + + for (const auto & host_id : local_host_ids) + { + auto it = active_node_holders.find(host_id); + if (it != active_node_holders.end()) + { + continue; + } + + /// Create "active" node (remove previous one if necessary) + String active_path = replicas_dir / host_id / "active"; + String active_id = toString(ServerUUID::get()); + zookeeper->deleteEphemeralNodeIfContentMatches(active_path, active_id); + + LOG_TRACE(log, "Trying to mark a replica active: active_path={}, active_id={}", active_path, active_id); + + zookeeper->create(active_path, active_id, zkutil::CreateMode::Ephemeral); + auto active_node_holder_zookeeper = zookeeper; + auto active_node_holder = zkutil::EphemeralNodeHolder::existing(active_path, *active_node_holder_zookeeper); + active_node_holders[host_id] = {active_node_holder_zookeeper, active_node_holder}; + } +} + void DDLWorker::runCleanupThread() { setThreadName("DDLWorkerClnr"); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 6d1dabda54f..fd4735b5baa 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -1,22 +1,23 @@ #pragma once -#include +#include +#include +#include #include +#include #include #include #include -#include -#include -#include +#include #include -#include -#include +#include #include #include -#include #include +namespace fs = std::filesystem; + namespace zkutil { class ZooKeeper; @@ -146,6 +147,10 @@ protected: /// Return false if the worker was stopped (stop_flag = true) virtual bool initializeMainThread(); + virtual void initializeReplication(); + + void createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids); + void markReplicasActive(bool reinitialized); void runMainThread(); void runCleanupThread(); @@ -157,7 +162,8 @@ protected: std::string host_fqdn; /// current host domain name std::string host_fqdn_id; /// host_name:port - std::string queue_dir; /// dir with queue of queries + std::string queue_dir; /// dir with queue of queries + fs::path replicas_dir; mutable std::mutex zookeeper_mutex; ZooKeeperPtr current_zookeeper TSA_GUARDED_BY(zookeeper_mutex); @@ -199,6 +205,8 @@ protected: const CurrentMetrics::Metric * max_entry_metric; const CurrentMetrics::Metric * max_pushed_entry_metric; + + std::unordered_map> active_node_holders; }; From 4be513451800d54c611f6f3211797cbc40717320 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 17 Sep 2024 07:41:13 +0000 Subject: [PATCH 053/816] 1) Refactor DDLQueryStatusSource: - Rename DDLQueryStatusSource to DistributedQueryStatusSource, and make it as a base class - Create two subclasses: DDLOnClusterQueryStatusSource, ReplicatedDatabaseQueryStatusSource derived from DDLQueryStatusSource 2) Support stop waiting offline hosts in DDLOnClusterQueryStatusSource --- src/Databases/DatabaseReplicated.cpp | 88 +-- src/Databases/DatabaseReplicated.h | 2 + .../DDLOnClusterQueryStatusSource.cpp | 152 ++++++ .../DDLOnClusterQueryStatusSource.h | 29 + .../DistributedQueryStatusSource.cpp | 262 +++++++++ .../DistributedQueryStatusSource.h | 62 +++ .../ReplicatedDatabaseQueryStatusSource.cpp | 163 ++++++ .../ReplicatedDatabaseQueryStatusSource.h | 39 ++ src/Interpreters/executeDDLQueryOnCluster.cpp | 511 +----------------- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- 10 files changed, 791 insertions(+), 519 deletions(-) create mode 100644 src/Interpreters/DDLOnClusterQueryStatusSource.cpp create mode 100644 src/Interpreters/DDLOnClusterQueryStatusSource.h create mode 100644 src/Interpreters/DistributedQueryStatusSource.cpp create mode 100644 src/Interpreters/DistributedQueryStatusSource.h create mode 100644 src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp create mode 100644 src/Interpreters/ReplicatedDatabaseQueryStatusSource.h diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3d64c82ba7d..1ae83a83b9d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -4,47 +4,49 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace DB { @@ -420,7 +422,6 @@ void DatabaseReplicated::fillClusterAuthInfo(String collection_name, const Poco: cluster_auth_info.cluster_secure_connection = config_ref.getBool(config_prefix + ".cluster_secure_connection", false); } - void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessLevel mode) { try @@ -1068,7 +1069,8 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex hosts_to_wait.push_back(unfiltered_hosts[i]); } - return getDistributedDDLStatus(node_path, entry, query_context, &hosts_to_wait); + + return getQueryStatus(node_path, query_context, hosts_to_wait); } static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) @@ -2003,4 +2005,20 @@ void registerDatabaseReplicated(DatabaseFactory & factory) }; factory.registerDatabase("Replicated", create_fn); } + +BlockIO DatabaseReplicated::getQueryStatus(const String & node_path, ContextPtr context_, const Strings & hosts_to_wait) +{ + BlockIO io; + if (context_->getSettingsRef().distributed_ddl_task_timeout == 0) + return io; + + auto source = std::make_shared(node_path, context_, hosts_to_wait); + io.pipeline = QueryPipeline(std::move(source)); + + if (context_->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE + || context_->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE) + io.pipeline.complete(std::make_shared(io.pipeline.getHeader())); + + return io; +} } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index db683be8f36..491b60c400b 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -151,6 +151,8 @@ private: void waitDatabaseStarted() const override; void stopLoading() override; + static BlockIO getQueryStatus(const String & node_path, ContextPtr context, const Strings & hosts_to_wait); + String zookeeper_path; String shard_name; String replica_name; diff --git a/src/Interpreters/DDLOnClusterQueryStatusSource.cpp b/src/Interpreters/DDLOnClusterQueryStatusSource.cpp new file mode 100644 index 00000000000..2f531caf94e --- /dev/null +++ b/src/Interpreters/DDLOnClusterQueryStatusSource.cpp @@ -0,0 +1,152 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int TIMEOUT_EXCEEDED; +} + +DDLOnClusterQueryStatusSource::DDLOnClusterQueryStatusSource( + const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait) + : DistributedQueryStatusSource(zk_node_path, getSampleBlock(context_), context_, hosts_to_wait, "DDLOnClusterQueryStatusSource") +{ +} + +ExecutionStatus DDLOnClusterQueryStatusSource::checkStatus(const String & host_id) +{ + fs::path status_path = fs::path(node_path) / "finished" / host_id; + return getExecutionStatus(status_path); +} + +Chunk DDLOnClusterQueryStatusSource::generateChunkWithUnfinishedHosts() const +{ + NameSet unfinished_hosts = waiting_hosts; + for (const auto & host_id : finished_hosts) + unfinished_hosts.erase(host_id); + + NameSet active_hosts_set = NameSet{current_active_hosts.begin(), current_active_hosts.end()}; + + /// Query is not finished on the rest hosts, so fill the corresponding rows with NULLs. + MutableColumns columns = output.getHeader().cloneEmptyColumns(); + for (const String & host_id : unfinished_hosts) + { + size_t num = 0; + auto [host, port] = parseHostAndPort(host_id); + columns[num++]->insert(host); + columns[num++]->insert(port); + columns[num++]->insert(Field{}); + columns[num++]->insert(Field{}); + columns[num++]->insert(unfinished_hosts.size()); + columns[num++]->insert(current_active_hosts.size()); + } + return Chunk(std::move(columns), unfinished_hosts.size()); +} + +Strings DDLOnClusterQueryStatusSource::getNodesToWait() +{ + return {String(fs::path(node_path) / "finished"), String(fs::path(node_path) / "active")}; +} +Chunk DDLOnClusterQueryStatusSource::handleTimeoutExceeded() +{ + timeout_exceeded = true; + + size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished; + size_t num_active_hosts = current_active_hosts.size(); + + constexpr auto msg_format = "Distributed DDL task {} is not finished on {} of {} hosts " + "({} of them are currently executing the task, {} are inactive). " + "They are going to execute the query in background. Was waiting for {} seconds{}"; + + if (throw_on_timeout || (throw_on_timeout_only_active && !stop_waiting_offline_hosts)) + { + if (!first_exception) + first_exception = std::make_unique(Exception( + ErrorCodes::TIMEOUT_EXCEEDED, + msg_format, + node_path, + num_unfinished_hosts, + waiting_hosts.size(), + num_active_hosts, + offline_hosts.size(), + watch.elapsedSeconds(), + stop_waiting_offline_hosts ? "" : ", which is longer than distributed_ddl_task_timeout")); + + return {}; + } + + LOG_INFO( + log, + msg_format, + node_path, + num_unfinished_hosts, + waiting_hosts.size(), + num_active_hosts, + offline_hosts.size(), + watch.elapsedSeconds(), + stop_waiting_offline_hosts ? "" : "which is longer than distributed_ddl_task_timeout"); + + return generateChunkWithUnfinishedHosts(); +} +Chunk DDLOnClusterQueryStatusSource::stopWaitingOfflineHosts() +{ + // Same logic as timeout exceeded + return handleTimeoutExceeded(); +} +void DDLOnClusterQueryStatusSource::handleNonZeroStatusCode(const ExecutionStatus & status, const String & host_id) +{ + assert(status.code != 0); + + if (!first_exception && context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW) + { + auto [host, port] = parseHostAndPort(host_id); + first_exception + = std::make_unique(Exception(status.code, "There was an error on [{}:{}]: {}", host, port, status.message)); + } +} +void DDLOnClusterQueryStatusSource::fillHostStatus(const String & host_id, const ExecutionStatus & status, MutableColumns & columns) +{ + size_t num = 0; + auto [host, port] = parseHostAndPort(host_id); + columns[num++]->insert(host); + columns[num++]->insert(port); + columns[num++]->insert(status.code); + columns[num++]->insert(status.message); + columns[num++]->insert(waiting_hosts.size() - num_hosts_finished); + columns[num++]->insert(current_active_hosts.size()); +} + +Block DDLOnClusterQueryStatusSource::getSampleBlock(ContextPtr context_) +{ + auto output_mode = context_->getSettingsRef().distributed_ddl_output_mode; + + auto maybe_make_nullable = [&](const DataTypePtr & type) -> DataTypePtr + { + if (output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE + || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE) + return type; + return std::make_shared(type); + }; + + + return Block{ + {std::make_shared(), "host"}, + {std::make_shared(), "port"}, + {maybe_make_nullable(std::make_shared()), "status"}, + {maybe_make_nullable(std::make_shared()), "error"}, + {std::make_shared(), "num_hosts_remaining"}, + {std::make_shared(), "num_hosts_active"}, + }; +} + +} diff --git a/src/Interpreters/DDLOnClusterQueryStatusSource.h b/src/Interpreters/DDLOnClusterQueryStatusSource.h new file mode 100644 index 00000000000..fb86aa43661 --- /dev/null +++ b/src/Interpreters/DDLOnClusterQueryStatusSource.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class DDLOnClusterQueryStatusSource final : public DistributedQueryStatusSource +{ +public: + DDLOnClusterQueryStatusSource(const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait); + + String getName() const override { return "DDLOnClusterQueryStatus"; } + +protected: + ExecutionStatus checkStatus(const String & host_id) override; + Chunk generateChunkWithUnfinishedHosts() const override; + Strings getNodesToWait() override; + Chunk handleTimeoutExceeded() override; + Chunk stopWaitingOfflineHosts() override; + void handleNonZeroStatusCode(const ExecutionStatus & status, const String & host_id) override; + void fillHostStatus(const String & host_id, const ExecutionStatus & status, MutableColumns & columns) override; + +private: + static Block getSampleBlock(ContextPtr context_); +}; +} diff --git a/src/Interpreters/DistributedQueryStatusSource.cpp b/src/Interpreters/DistributedQueryStatusSource.cpp new file mode 100644 index 00000000000..f006decc0f8 --- /dev/null +++ b/src/Interpreters/DistributedQueryStatusSource.cpp @@ -0,0 +1,262 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNFINISHED; +} + +DistributedQueryStatusSource::DistributedQueryStatusSource( + const String & zk_node_path, Block block, ContextPtr context_, const Strings & hosts_to_wait, const char * logger_name) + : ISource(block), node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(getLogger(logger_name)) +{ + auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; + throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; + throw_on_timeout_only_active + = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; + + waiting_hosts = NameSet(hosts_to_wait.begin(), hosts_to_wait.end()); + + only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE + || output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE + || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; + + addTotalRowsApprox(waiting_hosts.size()); + timeout_seconds = context->getSettingsRef().distributed_ddl_task_timeout; +} + + +IProcessor::Status DistributedQueryStatusSource::prepare() +{ + /// This method is overloaded to throw exception after all data is read. + /// Exception is pushed into pipe (instead of simply being thrown) to ensure the order of data processing and exception. + + if (finished) + { + if (first_exception) + { + if (!output.canPush()) + return Status::PortFull; + + output.pushException(std::make_exception_ptr(*first_exception)); + } + + output.finish(); + return Status::Finished; + } + else + return ISource::prepare(); +} + +NameSet DistributedQueryStatusSource::getOfflineHosts(const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper) +{ + fs::path replicas_path; + if (node_path.ends_with('/')) + replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas"; + else + replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas"; + + Strings paths; + Strings hosts_array; + for (const auto & host : hosts_to_wait) + { + hosts_array.push_back(host); + paths.push_back(replicas_path / host / "active"); + } + + NameSet offline; + auto res = zookeeper->tryGet(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error == Coordination::Error::ZNONODE) + offline.insert(hosts_array[i]); + + if (offline.size() == hosts_to_wait.size()) + { + /// Avoid reporting that all hosts are offline + LOG_WARNING(log, "Did not find active hosts, will wait for all {} hosts. This should not happen often", offline.size()); + return {}; + } + + return offline; +} + +Strings DistributedQueryStatusSource::getNewAndUpdate(const Strings & current_finished_hosts) +{ + Strings diff; + for (const String & host : current_finished_hosts) + { + if (!waiting_hosts.contains(host)) + { + if (!ignoring_hosts.contains(host)) + { + ignoring_hosts.emplace(host); + LOG_INFO(log, "Unexpected host {} appeared in task {}", host, node_path); + } + continue; + } + + if (!finished_hosts.contains(host)) + { + diff.emplace_back(host); + finished_hosts.emplace(host); + } + } + + return diff; +} + + +ExecutionStatus DistributedQueryStatusSource::getExecutionStatus(const fs::path & status_path) +{ + ExecutionStatus status(-1, "Cannot obtain error message"); + + String status_data; + bool finished_exists = false; + + auto retries_ctl = ZooKeeperRetriesControl( + "executeDDLQueryOnCluster", getLogger("DDLQueryStatusSource"), getRetriesInfo(), context->getProcessListElement()); + retries_ctl.retryLoop([&]() { finished_exists = context->getZooKeeper()->tryGet(status_path, status_data); }); + if (finished_exists) + status.tryDeserializeText(status_data); + + return status; +} + +ZooKeeperRetriesInfo DistributedQueryStatusSource::getRetriesInfo() +{ + const auto & config_ref = Context::getGlobalContextInstance()->getConfigRef(); + return ZooKeeperRetriesInfo( + config_ref.getInt("distributed_ddl_keeper_max_retries", 5), + config_ref.getInt("distributed_ddl_keeper_initial_backoff_ms", 100), + config_ref.getInt("distributed_ddl_keeper_max_backoff_ms", 5000)); +} + +std::pair DistributedQueryStatusSource::parseHostAndPort(const String & host_id) +{ + String host = host_id; + UInt16 port = 0; + auto host_and_port = Cluster::Address::fromString(host_id); + host = host_and_port.first; + port = host_and_port.second; + return {host, port}; +} + +Chunk DistributedQueryStatusSource::generate() +{ + bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size(); + + /// Seems like num_hosts_finished cannot be strictly greater than waiting_hosts.size() + assert(num_hosts_finished <= waiting_hosts.size()); + + if (all_hosts_finished || timeout_exceeded) + return {}; + + size_t try_number = 0; + while (true) + { + if (isCancelled()) + return {}; + + if (stop_waiting_offline_hosts) + { + return stopWaitingOfflineHosts(); + } + + if ((timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)) + { + return handleTimeoutExceeded(); + } + + sleepForMilliseconds(std::min(1000, 50 * try_number)); + + bool node_exists = false; + Strings tmp_hosts; + Strings tmp_active_hosts; + + { + auto retries_ctl = ZooKeeperRetriesControl( + "executeDistributedQueryOnCluster", getLogger(getName()), getRetriesInfo(), context->getProcessListElement()); + retries_ctl.retryLoop( + [&]() + { + auto zookeeper = context->getZooKeeper(); + Strings paths = getNodesToWait(); + auto res = zookeeper->tryGetChildren(paths); + for (size_t i = 0; i < res.size(); ++i) + if (res[i].error != Coordination::Error::ZOK && res[i].error != Coordination::Error::ZNONODE) + throw Coordination::Exception::fromPath(res[i].error, paths[i]); + + if (res[0].error == Coordination::Error::ZNONODE) + node_exists = zookeeper->exists(node_path); + else + node_exists = true; + tmp_hosts = res[0].names; + tmp_active_hosts = res[1].names; + + if (only_running_hosts) + offline_hosts = getOfflineHosts(waiting_hosts, zookeeper); + }); + } + + if (!node_exists) + { + /// Paradoxically, this exception will be throw even in case of "never_throw" mode. + + if (!first_exception) + first_exception = std::make_unique(Exception( + ErrorCodes::UNFINISHED, + "Cannot provide query execution status. The query's node {} has been deleted by the cleaner" + " since it was finished (or its lifetime is expired)", + node_path)); + return {}; + } + + Strings new_hosts = getNewAndUpdate(tmp_hosts); + ++try_number; + + if (only_running_hosts) + { + size_t num_finished_or_offline = 0; + for (const auto & host : waiting_hosts) + num_finished_or_offline += finished_hosts.contains(host) || offline_hosts.contains(host); + + if (num_finished_or_offline == waiting_hosts.size()) + stop_waiting_offline_hosts = true; + } + + if (new_hosts.empty()) + continue; + + current_active_hosts = std::move(tmp_active_hosts); + + MutableColumns columns = output.getHeader().cloneEmptyColumns(); + for (const String & host_id : new_hosts) + { + ExecutionStatus status = checkStatus(host_id); + + if (status.code != 0) + { + handleNonZeroStatusCode(status, host_id); + } + + ++num_hosts_finished; + fillHostStatus(host_id, status, columns); + } + + return Chunk(std::move(columns), new_hosts.size()); + } +} + +} diff --git a/src/Interpreters/DistributedQueryStatusSource.h b/src/Interpreters/DistributedQueryStatusSource.h new file mode 100644 index 00000000000..a7aad497a1e --- /dev/null +++ b/src/Interpreters/DistributedQueryStatusSource.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + +namespace DB +{ +class DistributedQueryStatusSource : public ISource +{ +public: + DistributedQueryStatusSource( + const String & zk_node_path, Block block, ContextPtr context_, const Strings & hosts_to_wait, const char * logger_name); + + Chunk generate() override; + Status prepare() override; + +protected: + virtual ExecutionStatus checkStatus(const String & host_id) = 0; + virtual Chunk generateChunkWithUnfinishedHosts() const = 0; + virtual Strings getNodesToWait() = 0; + virtual Chunk handleTimeoutExceeded() = 0; + virtual Chunk stopWaitingOfflineHosts() = 0; + virtual void handleNonZeroStatusCode(const ExecutionStatus & status, const String & host_id) = 0; + virtual void fillHostStatus(const String & host_id, const ExecutionStatus & status, MutableColumns & columns) = 0; + + virtual NameSet getOfflineHosts(const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper); + + Strings getNewAndUpdate(const Strings & current_finished_hosts); + ExecutionStatus getExecutionStatus(const fs::path & status_path); + + static ZooKeeperRetriesInfo getRetriesInfo(); + static std::pair parseHostAndPort(const String & host_id); + + String node_path; + ContextPtr context; + Stopwatch watch; + LoggerPtr log; + + NameSet waiting_hosts; /// hosts from task host list + NameSet finished_hosts; /// finished hosts from host list + NameSet ignoring_hosts; /// appeared hosts that are not in hosts list + Strings current_active_hosts; /// Hosts that are currently executing the task + NameSet offline_hosts; /// Hosts that are not currently running + size_t num_hosts_finished = 0; + + /// Save the first detected error and throw it at the end of execution + std::unique_ptr first_exception; + + Int64 timeout_seconds = 120; + bool throw_on_timeout = true; + bool throw_on_timeout_only_active = false; + bool only_running_hosts = false; + + bool timeout_exceeded = false; + bool stop_waiting_offline_hosts = false; +}; +} diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp new file mode 100644 index 00000000000..cf2e8ce8558 --- /dev/null +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp @@ -0,0 +1,163 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int TIMEOUT_EXCEEDED; +extern const int LOGICAL_ERROR; +} + +ReplicatedDatabaseQueryStatusSource::ReplicatedDatabaseQueryStatusSource( + const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait) + : DistributedQueryStatusSource(zk_node_path, getSampleBlock(), context_, hosts_to_wait, "ReplicatedDatabaseQueryStatusSource") +{ +} + +ExecutionStatus ReplicatedDatabaseQueryStatusSource::checkStatus(const String & host_id) +{ + /// Replicated database retries in case of error, it should not write error status. +#ifdef DEBUG_OR_SANITIZER_BUILD + fs::path status_path = fs::path(node_path) / "finished" / host_id; + return getExecutionStatus(status_path); +#else + return ExecutionStatus{0}; +#endif +} + +Chunk ReplicatedDatabaseQueryStatusSource::generateChunkWithUnfinishedHosts() const +{ + NameSet unfinished_hosts = waiting_hosts; + for (const auto & host_id : finished_hosts) + unfinished_hosts.erase(host_id); + + NameSet active_hosts_set = NameSet{current_active_hosts.begin(), current_active_hosts.end()}; + + /// Query is not finished on the rest hosts, so fill the corresponding rows with NULLs. + MutableColumns columns = output.getHeader().cloneEmptyColumns(); + for (const String & host_id : unfinished_hosts) + { + size_t num = 0; + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); + if (active_hosts_set.contains(host_id)) + columns[num++]->insert(IN_PROGRESS); + else + columns[num++]->insert(QUEUED); + + columns[num++]->insert(unfinished_hosts.size()); + columns[num++]->insert(current_active_hosts.size()); + } + return Chunk(std::move(columns), unfinished_hosts.size()); +} + +Strings ReplicatedDatabaseQueryStatusSource::getNodesToWait() +{ + String node_to_wait = "finished"; + if (context->getSettingsRef().database_replicated_enforce_synchronous_settings) + { + node_to_wait = "synced"; + } + + return {String(fs::path(node_path) / node_to_wait), String(fs::path(node_path) / "active")}; +} + +Chunk ReplicatedDatabaseQueryStatusSource::handleTimeoutExceeded() +{ + timeout_exceeded = true; + + size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished; + size_t num_active_hosts = current_active_hosts.size(); + + constexpr auto msg_format = "ReplicatedDatabase DDL task {} is not finished on {} of {} hosts " + "({} of them are currently executing the task, {} are inactive). " + "They are going to execute the query in background. Was waiting for {} seconds{}"; + + if (throw_on_timeout || (throw_on_timeout_only_active && !stop_waiting_offline_hosts)) + { + if (!first_exception) + first_exception = std::make_unique(Exception( + ErrorCodes::TIMEOUT_EXCEEDED, + msg_format, + node_path, + num_unfinished_hosts, + waiting_hosts.size(), + num_active_hosts, + offline_hosts.size(), + watch.elapsedSeconds(), + stop_waiting_offline_hosts ? "" : ", which is longer than distributed_ddl_task_timeout")); + + /// For Replicated database print a list of unfinished hosts as well. Will return empty block on next iteration. + return generateChunkWithUnfinishedHosts(); + } + + LOG_INFO( + log, + msg_format, + node_path, + num_unfinished_hosts, + waiting_hosts.size(), + num_active_hosts, + offline_hosts.size(), + watch.elapsedSeconds(), + stop_waiting_offline_hosts ? "" : "which is longer than distributed_ddl_task_timeout"); + + return generateChunkWithUnfinishedHosts(); +} + +Chunk ReplicatedDatabaseQueryStatusSource::stopWaitingOfflineHosts() +{ + // Same logic as timeout exceeded + return handleTimeoutExceeded(); +} + +void ReplicatedDatabaseQueryStatusSource::handleNonZeroStatusCode(const ExecutionStatus & status, const String & host_id) +{ + assert(status.code != 0); + + if (!first_exception && context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); + } +} + +void ReplicatedDatabaseQueryStatusSource::fillHostStatus(const String & host_id, const ExecutionStatus & status, MutableColumns & columns) +{ + size_t num = 0; + if (status.code != 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); + auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); + columns[num++]->insert(shard); + columns[num++]->insert(replica); + columns[num++]->insert(OK); +} + +Block ReplicatedDatabaseQueryStatusSource::getSampleBlock() +{ + auto get_status_enum = []() + { + return std::make_shared(DataTypeEnum8::Values{ + {"OK", static_cast(OK)}, + {"IN_PROGRESS", static_cast(IN_PROGRESS)}, + {"QUEUED", static_cast(QUEUED)}, + }); + }; + + return Block{ + {std::make_shared(), "shard"}, + {std::make_shared(), "replica"}, + {get_status_enum(), "status"}, + {std::make_shared(), "num_hosts_remaining"}, + {std::make_shared(), "num_hosts_active"}, + }; +} + +} diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h new file mode 100644 index 00000000000..8b00c756596 --- /dev/null +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class ReplicatedDatabaseQueryStatusSource final : public DistributedQueryStatusSource +{ +public: + ReplicatedDatabaseQueryStatusSource(const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait); + + String getName() const override { return "ReplicatedDatabaseQueryStatus"; } + +protected: + ExecutionStatus checkStatus(const String & host_id) override; + Chunk generateChunkWithUnfinishedHosts() const override; + Strings getNodesToWait() override; + Chunk handleTimeoutExceeded() override; + Chunk stopWaitingOfflineHosts() override; + void handleNonZeroStatusCode(const ExecutionStatus & status, const String & host_id) override; + void fillHostStatus(const String & host_id, const ExecutionStatus & status, MutableColumns & columns) override; + +private: + static Block getSampleBlock(); + + enum ReplicatedDatabaseQueryStatus + { + /// Query is (successfully) finished + OK = 0, + /// Query is not finished yet, but replica is currently executing it + IN_PROGRESS = 1, + /// Replica is not available or busy with previous queries. It will process query asynchronously + QUEUED = 2, + }; +}; +} diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 1b57ad2b622..b71c73ff452 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -1,52 +1,41 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include #include #include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include #include +#include +#include -namespace fs = std::filesystem; - namespace DB { namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; - extern const int TIMEOUT_EXCEEDED; - extern const int UNFINISHED; - extern const int QUERY_IS_PROHIBITED; - extern const int LOGICAL_ERROR; +extern const int NOT_IMPLEMENTED; +extern const int QUERY_IS_PROHIBITED; +extern const int LOGICAL_ERROR; } -static ZooKeeperRetriesInfo getRetriesInfo() -{ - const auto & config_ref = Context::getGlobalContextInstance()->getConfigRef(); - return ZooKeeperRetriesInfo( - config_ref.getInt("distributed_ddl_keeper_max_retries", 5), - config_ref.getInt("distributed_ddl_keeper_initial_backoff_ms", 100), - config_ref.getInt("distributed_ddl_keeper_max_backoff_ms", 5000)); -} bool isSupportedAlterTypeForOnClusterDDLQuery(int type) { @@ -187,72 +176,19 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.initial_query_id = context->getClientInfo().initial_query_id; String node_path = ddl_worker.enqueueQuery(entry); - return getDistributedDDLStatus(node_path, entry, context, /* hosts_to_wait */ nullptr); + return getDDLOnClusterStatus(node_path, entry, context); } - -class DDLQueryStatusSource final : public ISource -{ -public: - DDLQueryStatusSource( - const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const Strings * hosts_to_wait); - - String getName() const override { return "DDLQueryStatus"; } - Chunk generate() override; - Status prepare() override; - -private: - static Block getSampleBlock(ContextPtr context_, bool hosts_to_wait); - - Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts); - - std::pair parseHostAndPort(const String & host_id) const; - - Chunk generateChunkWithUnfinishedHosts() const; - - enum ReplicatedDatabaseQueryStatus - { - /// Query is (successfully) finished - OK = 0, - /// Query is not finished yet, but replica is currently executing it - IN_PROGRESS = 1, - /// Replica is not available or busy with previous queries. It will process query asynchronously - QUEUED = 2, - }; - - String node_path; - ContextPtr context; - Stopwatch watch; - LoggerPtr log; - - NameSet waiting_hosts; /// hosts from task host list - NameSet finished_hosts; /// finished hosts from host list - NameSet ignoring_hosts; /// appeared hosts that are not in hosts list - Strings current_active_hosts; /// Hosts that are currently executing the task - NameSet offline_hosts; /// Hosts that are not currently running - size_t num_hosts_finished = 0; - - /// Save the first detected error and throw it at the end of execution - std::unique_ptr first_exception; - - Int64 timeout_seconds = 120; - bool is_replicated_database = false; - bool throw_on_timeout = true; - bool throw_on_timeout_only_active = false; - bool only_running_hosts = false; - - bool timeout_exceeded = false; - bool stop_waiting_offline_hosts = false; -}; - - -BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, const Strings * hosts_to_wait) +BlockIO getDDLOnClusterStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context) { BlockIO io; if (context->getSettingsRef().distributed_ddl_task_timeout == 0) return io; + Strings hosts_to_wait; + for (const HostID & host : entry.hosts) + hosts_to_wait.push_back(host.toString()); - auto source = std::make_shared(node_path, entry, context, hosts_to_wait); + auto source = std::make_shared(node_path, context, hosts_to_wait); io.pipeline = QueryPipeline(std::move(source)); if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE || @@ -262,397 +198,6 @@ BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & en return io; } -Block DDLQueryStatusSource::getSampleBlock(ContextPtr context_, bool hosts_to_wait) -{ - auto output_mode = context_->getSettingsRef().distributed_ddl_output_mode; - - auto maybe_make_nullable = [&](const DataTypePtr & type) -> DataTypePtr - { - if (output_mode == DistributedDDLOutputMode::THROW || - output_mode == DistributedDDLOutputMode::NONE || - output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE) - return type; - return std::make_shared(type); - }; - - auto get_status_enum = []() - { - return std::make_shared( - DataTypeEnum8::Values - { - {"OK", static_cast(OK)}, - {"IN_PROGRESS", static_cast(IN_PROGRESS)}, - {"QUEUED", static_cast(QUEUED)}, - }); - }; - - if (hosts_to_wait) - { - return Block{ - {std::make_shared(), "shard"}, - {std::make_shared(), "replica"}, - {get_status_enum(), "status"}, - {std::make_shared(), "num_hosts_remaining"}, - {std::make_shared(), "num_hosts_active"}, - }; - } - else - { - return Block{ - {std::make_shared(), "host"}, - {std::make_shared(), "port"}, - {maybe_make_nullable(std::make_shared()), "status"}, - {maybe_make_nullable(std::make_shared()), "error"}, - {std::make_shared(), "num_hosts_remaining"}, - {std::make_shared(), "num_hosts_active"}, - }; - } -} - -DDLQueryStatusSource::DDLQueryStatusSource( - const String & zk_node_path, const DDLLogEntry & entry, ContextPtr context_, const Strings * hosts_to_wait) - : ISource(getSampleBlock(context_, static_cast(hosts_to_wait))) - , node_path(zk_node_path) - , context(context_) - , watch(CLOCK_MONOTONIC_COARSE) - , log(getLogger("DDLQueryStatusSource")) -{ - auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; - throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; - throw_on_timeout_only_active = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; - - if (hosts_to_wait) - { - waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end()); - is_replicated_database = true; - only_running_hosts = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || - output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT_ONLY_ACTIVE || - output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; - } - else - { - for (const HostID & host : entry.hosts) - waiting_hosts.emplace(host.toString()); - } - - addTotalRowsApprox(waiting_hosts.size()); - timeout_seconds = context->getSettingsRef().distributed_ddl_task_timeout; -} - -std::pair DDLQueryStatusSource::parseHostAndPort(const String & host_id) const -{ - String host = host_id; - UInt16 port = 0; - if (!is_replicated_database) - { - auto host_and_port = Cluster::Address::fromString(host_id); - host = host_and_port.first; - port = host_and_port.second; - } - return {host, port}; -} - -Chunk DDLQueryStatusSource::generateChunkWithUnfinishedHosts() const -{ - NameSet unfinished_hosts = waiting_hosts; - for (const auto & host_id : finished_hosts) - unfinished_hosts.erase(host_id); - - NameSet active_hosts_set = NameSet{current_active_hosts.begin(), current_active_hosts.end()}; - - /// Query is not finished on the rest hosts, so fill the corresponding rows with NULLs. - MutableColumns columns = output.getHeader().cloneEmptyColumns(); - for (const String & host_id : unfinished_hosts) - { - size_t num = 0; - if (is_replicated_database) - { - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); - if (active_hosts_set.contains(host_id)) - columns[num++]->insert(IN_PROGRESS); - else - columns[num++]->insert(QUEUED); - } - else - { - auto [host, port] = parseHostAndPort(host_id); - columns[num++]->insert(host); - columns[num++]->insert(port); - columns[num++]->insert(Field{}); - columns[num++]->insert(Field{}); - } - columns[num++]->insert(unfinished_hosts.size()); - columns[num++]->insert(current_active_hosts.size()); - } - return Chunk(std::move(columns), unfinished_hosts.size()); -} - -static NameSet getOfflineHosts(const String & node_path, const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper, LoggerPtr log) -{ - fs::path replicas_path; - if (node_path.ends_with('/')) - replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas"; - else - replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas"; - - Strings paths; - Strings hosts_array; - for (const auto & host : hosts_to_wait) - { - hosts_array.push_back(host); - paths.push_back(replicas_path / host / "active"); - } - - NameSet offline; - auto res = zookeeper->tryGet(paths); - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error == Coordination::Error::ZNONODE) - offline.insert(hosts_array[i]); - - if (offline.size() == hosts_to_wait.size()) - { - /// Avoid reporting that all hosts are offline - LOG_WARNING(log, "Did not find active hosts, will wait for all {} hosts. This should not happen often", offline.size()); - return {}; - } - - return offline; -} - -Chunk DDLQueryStatusSource::generate() -{ - bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size(); - - /// Seems like num_hosts_finished cannot be strictly greater than waiting_hosts.size() - assert(num_hosts_finished <= waiting_hosts.size()); - - if (all_hosts_finished || timeout_exceeded) - return {}; - - String node_to_wait = "finished"; - if (is_replicated_database && context->getSettingsRef().database_replicated_enforce_synchronous_settings) - node_to_wait = "synced"; - - size_t try_number = 0; - - while (true) - { - if (isCancelled()) - return {}; - - if (stop_waiting_offline_hosts || (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)) - { - timeout_exceeded = true; - - size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished; - size_t num_active_hosts = current_active_hosts.size(); - - constexpr auto msg_format = "Distributed DDL task {} is not finished on {} of {} hosts " - "({} of them are currently executing the task, {} are inactive). " - "They are going to execute the query in background. Was waiting for {} seconds{}"; - - if (throw_on_timeout || (throw_on_timeout_only_active && !stop_waiting_offline_hosts)) - { - if (!first_exception) - first_exception = std::make_unique(Exception(ErrorCodes::TIMEOUT_EXCEEDED, - msg_format, node_path, num_unfinished_hosts, waiting_hosts.size(), num_active_hosts, offline_hosts.size(), - watch.elapsedSeconds(), stop_waiting_offline_hosts ? "" : ", which is longer than distributed_ddl_task_timeout")); - - /// For Replicated database print a list of unfinished hosts as well. Will return empty block on next iteration. - if (is_replicated_database) - return generateChunkWithUnfinishedHosts(); - return {}; - } - - LOG_INFO(log, msg_format, node_path, num_unfinished_hosts, waiting_hosts.size(), num_active_hosts, offline_hosts.size(), - watch.elapsedSeconds(), stop_waiting_offline_hosts ? "" : "which is longer than distributed_ddl_task_timeout"); - - return generateChunkWithUnfinishedHosts(); - } - - sleepForMilliseconds(std::min(1000, 50 * try_number)); - - bool node_exists = false; - Strings tmp_hosts; - Strings tmp_active_hosts; - - { - auto retries_ctl = ZooKeeperRetriesControl( - "executeDDLQueryOnCluster", getLogger("DDLQueryStatusSource"), getRetriesInfo(), context->getProcessListElement()); - retries_ctl.retryLoop([&]() - { - auto zookeeper = context->getZooKeeper(); - Strings paths = {String(fs::path(node_path) / node_to_wait), String(fs::path(node_path) / "active")}; - auto res = zookeeper->tryGetChildren(paths); - for (size_t i = 0; i < res.size(); ++i) - if (res[i].error != Coordination::Error::ZOK && res[i].error != Coordination::Error::ZNONODE) - throw Coordination::Exception::fromPath(res[i].error, paths[i]); - - if (res[0].error == Coordination::Error::ZNONODE) - node_exists = zookeeper->exists(node_path); - else - node_exists = true; - tmp_hosts = res[0].names; - tmp_active_hosts = res[1].names; - - if (only_running_hosts) - offline_hosts = getOfflineHosts(node_path, waiting_hosts, zookeeper, log); - }); - } - - if (!node_exists) - { - /// Paradoxically, this exception will be throw even in case of "never_throw" mode. - - if (!first_exception) - first_exception = std::make_unique(Exception(ErrorCodes::UNFINISHED, - "Cannot provide query execution status. The query's node {} has been deleted by the cleaner" - " since it was finished (or its lifetime is expired)", - node_path)); - return {}; - } - - Strings new_hosts = getNewAndUpdate(tmp_hosts); - ++try_number; - - if (only_running_hosts) - { - size_t num_finished_or_offline = 0; - for (const auto & host : waiting_hosts) - num_finished_or_offline += finished_hosts.contains(host) || offline_hosts.contains(host); - - if (num_finished_or_offline == waiting_hosts.size()) - stop_waiting_offline_hosts = true; - } - - if (new_hosts.empty()) - continue; - - current_active_hosts = std::move(tmp_active_hosts); - - MutableColumns columns = output.getHeader().cloneEmptyColumns(); - for (const String & host_id : new_hosts) - { - ExecutionStatus status(-1, "Cannot obtain error message"); - - /// Replicated database retries in case of error, it should not write error status. -#ifdef DEBUG_OR_SANITIZER_BUILD - bool need_check_status = true; -#else - bool need_check_status = !is_replicated_database; -#endif - if (need_check_status) - { - String status_data; - bool finished_exists = false; - - auto retries_ctl = ZooKeeperRetriesControl( - "executeDDLQueryOnCluster", - getLogger("DDLQueryStatusSource"), - getRetriesInfo(), - context->getProcessListElement()); - retries_ctl.retryLoop([&]() - { - finished_exists = context->getZooKeeper()->tryGet(fs::path(node_path) / "finished" / host_id, status_data); - }); - if (finished_exists) - status.tryDeserializeText(status_data); - } - else - { - status = ExecutionStatus{0}; - } - - - if (status.code != 0 && !first_exception - && context->getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW) - { - if (is_replicated_database) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - - auto [host, port] = parseHostAndPort(host_id); - first_exception = std::make_unique(Exception(status.code, - "There was an error on [{}:{}]: {}", host, port, status.message)); - } - - ++num_hosts_finished; - - size_t num = 0; - if (is_replicated_database) - { - if (status.code != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There was an error on {}: {} (probably it's a bug)", host_id, status.message); - auto [shard, replica] = DatabaseReplicated::parseFullReplicaName(host_id); - columns[num++]->insert(shard); - columns[num++]->insert(replica); - columns[num++]->insert(OK); - } - else - { - auto [host, port] = parseHostAndPort(host_id); - columns[num++]->insert(host); - columns[num++]->insert(port); - columns[num++]->insert(status.code); - columns[num++]->insert(status.message); - } - columns[num++]->insert(waiting_hosts.size() - num_hosts_finished); - columns[num++]->insert(current_active_hosts.size()); - } - - return Chunk(std::move(columns), new_hosts.size()); - } -} - -IProcessor::Status DDLQueryStatusSource::prepare() -{ - /// This method is overloaded to throw exception after all data is read. - /// Exception is pushed into pipe (instead of simply being thrown) to ensure the order of data processing and exception. - - if (finished) - { - if (first_exception) - { - if (!output.canPush()) - return Status::PortFull; - - output.pushException(std::make_exception_ptr(*first_exception)); - } - - output.finish(); - return Status::Finished; - } - else - return ISource::prepare(); -} - -Strings DDLQueryStatusSource::getNewAndUpdate(const Strings & current_list_of_finished_hosts) -{ - Strings diff; - for (const String & host : current_list_of_finished_hosts) - { - if (!waiting_hosts.contains(host)) - { - if (!ignoring_hosts.contains(host)) - { - ignoring_hosts.emplace(host); - LOG_INFO(log, "Unexpected host {} appeared in task {}", host, node_path); - } - continue; - } - - if (!finished_hosts.contains(host)) - { - diff.emplace_back(host); - finished_hosts.emplace(host); - } - } - - return diff; -} - - bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context) { const auto * query = dynamic_cast(query_ptr.get()); diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index d3365553875..61d6ba75cf0 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -43,7 +43,7 @@ struct DDLQueryOnClusterParams /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}); -BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context, const Strings * hosts_to_wait); +BlockIO getDDLOnClusterStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context); bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context); From ec4e0ed1b2c2b355dddd07d18736d8e993a9d620 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 17 Sep 2024 11:18:19 +0000 Subject: [PATCH 054/816] add notification sending --- src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp | 1 + src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index b14a96c771a..c794d2717e4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -128,6 +128,7 @@ void WorkloadEntityDiskStorage::loadEntities() void WorkloadEntityDiskStorage::reloadEntities() { + // TODO(serxa): it does not send notifications, maybe better to remove this method completely loadEntitiesImpl(); } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index e3bf6d4af7f..ad5a3166cf6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -152,6 +152,8 @@ bool WorkloadEntityStorageBase::storeEntity( onEntityAdded(entity_type, entity_name, create_entity_query); } + sendNotifications(); + return stored; } @@ -183,6 +185,8 @@ bool WorkloadEntityStorageBase::removeEntity( onEntityRemoved(entity_type, entity_name); } + sendNotifications(); + return removed; } From ab6bb3b2a60d060ca0ac2a2dfe423721bec765b7 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 17 Sep 2024 11:19:38 +0000 Subject: [PATCH 055/816] initial implementation of IOResourceManager --- src/Common/Scheduler/IResourceManager.h | 2 +- src/Common/Scheduler/ISchedulerNode.h | 2 + .../Nodes/DynamicResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/FairPolicy.h | 6 + src/Common/Scheduler/Nodes/FifoQueue.h | 6 + .../Scheduler/Nodes/IOResourceManager.cpp | 502 ++++++++++++++++++ .../Scheduler/Nodes/IOResourceManager.h | 272 ++++++++++ src/Common/Scheduler/Nodes/PriorityPolicy.h | 6 + .../Scheduler/Nodes/SemaphoreConstraint.h | 6 + .../Scheduler/Nodes/ThrottlerConstraint.h | 8 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 75 ++- .../Nodes/tests/gtest_event_queue.cpp | 6 + src/Common/Scheduler/SchedulerRoot.h | 6 + .../Scheduler/createResourceManager.cpp | 6 +- .../System/StorageSystemScheduler.cpp | 18 +- 15 files changed, 887 insertions(+), 36 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.cpp create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.h diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index 8a7077ac3d5..c4a5c590ba7 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -51,7 +51,7 @@ public: virtual ClassifierPtr acquire(const String & classifier_name) = 0; /// For introspection, see `system.scheduler` table - using VisitorFunc = std::function; + using VisitorFunc = std::function; virtual void forEachNode(VisitorFunc visitor) = 0; }; diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 6d3132f79c1..d68a32e8290 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -141,6 +141,8 @@ public: virtual ~ISchedulerNode() = default; + virtual const String & getTypeName() const = 0; + /// Checks if two nodes configuration is equal virtual bool equals(ISchedulerNode * other) { diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index 29b3aefacf1..88b4eec063d 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -244,7 +244,7 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { for (auto & [name, resource] : state_ref->resources) for (auto & [path, node] : resource->nodes) - visitor(name, path, node.type, node.ptr); + visitor(name, path, node.ptr.get()); promise.set_value(); }); diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index b6be26bea98..81bfaaadf19 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -52,6 +52,12 @@ public: : ISchedulerNode(event_queue_, info_) {} + const String & getTypeName() const override + { + static String type_name("fair"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index c95125b21bf..79963a45b3b 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -39,6 +39,12 @@ public: chassert(requests.empty()); } + const String & getTypeName() const override + { + static String type_name("fifo"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp new file mode 100644 index 00000000000..9e6b4ebb254 --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -0,0 +1,502 @@ +#include "Common/Scheduler/IResourceManager.h" +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; + extern const int RESOURCE_NOT_FOUND; + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +namespace +{ + String getEntityName(const ASTPtr & ast) + { + if (auto * create = typeid_cast(ast.get())) + return create->getWorkloadName(); + if (auto * create = typeid_cast(ast.get())) + return create->getResourceName(); + return "unknown-workload-entity"; + } +} + +IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resource_name) +{ + auto * create = typeid_cast(ast.get()); + name = create->getWorkloadName(); + parent = create->getWorkloadParent(); + // TODO(serxa): parse workload settings specifically for `resource_name` + UNUSED(resource_name); +} + +IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) + : resource_entity(resource_entity_) + , resource_name(getEntityName(resource_entity)) +{ + scheduler.start(); +} + +IOResourceManager::Resource::~Resource() +{ + // TODO(serxa): destroy all workloads, purge all queue, abort all resource requests + scheduler.stop(); +} + +void IOResourceManager::Resource::createNode(const NodeInfo & info) +{ + // TODO(serxa): make sure all possible callers validate empty workload name! + if (info.name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", + resource_name); + + // TODO(serxa): make sure all possible callers validate self-reference! + if (info.name == info.parent) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", + info.name, resource_name); + + if (node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", + info.name, resource_name); + + // TODO(serxa): make sure all possible callers validate parent existence, add tests for creating workload with invalid parent + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + // TODO(serxa): make sure all possible callers validate second root, add tests for creating the second root + if (info.parent.empty() && root_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", + info.name, root_node->basename, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = std::make_shared(scheduler.event_queue, info.settings); + node->basename = info.name; + if (!info.parent.empty()) + node_for_workload[info.parent]->attachUnifiedChild(node); + else + { + root_node = node; + scheduler.attachChild(root_node); + } + node_for_workload[info.name] = node; + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::deleteNode(const NodeInfo & info) +{ + if (!node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for removing workload '{}' does not exist in resource '{}'", + info.name, resource_name); + + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for removing workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + auto node = node_for_workload[info.name]; + + // TODO(serxa): make sure all possible callers validate that removing workload has no children workloads + if (node->hasUnifiedChildren()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", + info.name, resource_name); + + executeInSchedulerThread([&, this] + { + if (!info.parent.empty()) + node_for_workload[info.parent]->detachUnifiedChild(node); + else + { + chassert(node == root_node); + scheduler.removeChild(root_node.get()); + root_node.reset(); + } + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const NodeInfo & new_info) +{ + if (old_info.name != new_info.name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", + old_info.name, new_info.name, resource_name); + + if (old_info.parent != new_info.parent && (old_info.parent.empty() || old_info.parent.empty())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", + old_info.name, old_info.parent, new_info.parent, resource_name); + + if (!node_for_workload.contains(old_info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for updating workload '{}' does not exist in resource '{}'", + old_info.name, resource_name); + + if (!old_info.parent.empty() && !node_for_workload.contains(old_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Old parent node '{}' for updating workload '{}' does not exist in resource '{}'", + old_info.parent, old_info.name, resource_name); + + if (!new_info.parent.empty() && !node_for_workload.contains(new_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "New parent node '{}' for updating workload '{}' does not exist in resource '{}'", + new_info.parent, new_info.name, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = node_for_workload[old_info.name]; + bool detached = false; + if (old_info.parent != new_info.parent) + { + node_for_workload[old_info.parent]->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_info.settings); + if (!detached && !old_info.parent.empty() && old_info.settings.priority != new_info.settings.priority) + node_for_workload[old_info.parent]->updateUnifiedChildPriority( + node, + old_info.settings.priority, + new_info.settings.priority); + + if (detached) + node_for_workload[new_info.parent]->attachUnifiedChild(node); + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateCurrentVersion() +{ + auto previous_version = current_version; + + // Create a full list of constraints and queues in the current hierarchy + current_version = std::make_shared(); + if (root_node) + root_node->addRawPointerNodes(current_version->nodes); + + // See details in version control section of description in IOResourceManager.h + if (previous_version) + { + previous_version->newer_version = current_version; + // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. + // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it + } +} + +IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_) + : resource_manager(resource_manager_) + , workload_entity(workload_entity_) +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->createNode(NodeInfo(workload_entity, resource_name)); +} + +IOResourceManager::Workload::~Workload() +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->deleteNode(NodeInfo(workload_entity, resource_name)); +} + +void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); + workload_entity = new_entity; +} + +String IOResourceManager::Workload::getParent() const +{ + return typeid_cast(workload_entity.get())->getWorkloadParent(); +} + +IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) + : storage(storage_) +{ + workload_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Workload, [this] ( + WorkloadEntityType, + const String & entity_name, + const ASTPtr & entity) + { + try + { + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + } + catch (...) + { + // TODO(serxa): handle CRUD errors + } + }); + resource_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Resource, [this] ( + WorkloadEntityType, + const String & entity_name, + const ASTPtr & entity /* new or changed entity, null if removed */) + { + try + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + } + catch (...) + { + // TODO(serxa): handle CRUD errors + } + }); +} + +void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) +{ + // No-op +} + +void IOResourceManager::createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + workload_iter->second->updateWorkload(ast); + else + workloads.emplace(workload_name, std::make_shared(this, ast)); +} + +void IOResourceManager::deleteWorkload(const String & workload_name) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + workloads.erase(workload_iter); + else + { + // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } +} + +void IOResourceManager::createResource(const String & resource_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + { + // Resource to be created already exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } + else + { + // Add all workloads into the new resource + auto resource = std::make_shared(ast); + for (Workload * workload : topologicallySortedWorkloads()) + resource->createNode(NodeInfo(workload->workload_entity, resource_name)); + + // Attach the resource + resources.emplace(resource_name, resource); + } +} + +void IOResourceManager::deleteResource(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + { + resources.erase(resource_iter); + } + else + { + // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } +} + +IOResourceManager::Classifier::~Classifier() +{ + // Detach classifier from all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(attachments.size()); + for (auto & [resource_name, attachment] : attachments) + { + futures.emplace_back(attachment.resource->detachClassifier(std::move(attachment.version))); + attachment.link.reset(); // Just in case because it is not valid any longer + } + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // There should not be any exceptions because it just destruct few objects, but let's rethrow just in case + for (auto & future : futures) + future.get(); + + // This unreferences and probably destroys `Resource` objects. + // NOTE: We cannot do it in the scheduler threads (because thread cannot join itself). + attachments.clear(); +} + +std::future IOResourceManager::Resource::detachClassifier(VersionPtr && version) +{ + auto detach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semanticss + auto future = detach_promise->get_future(); + scheduler.event_queue->enqueue([detached_version = std::move(version), promise = std::move(detach_promise)] mutable + { + try + { + // Unreferences and probably destroys the version and scheduler nodes it owns. + // The main reason from moving destruction into the scheduler thread is to + // free memory in the same thread it was allocated to avoid memtrackers drift. + detached_version.reset(); + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +ResourceLink IOResourceManager::Classifier::get(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto iter = attachments.find(resource_name); iter != attachments.end()) + return iter->second.link; + else + throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Access denied to resource '{}'", resource_name); +} + +void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link) +{ + std::unique_lock lock{mutex}; + chassert(!attachments.contains(resource->getName())); + attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; +} + +std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) +{ + auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics + auto future = attach_promise->get_future(); + scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] mutable + { + try + { + if (auto iter = node_for_workload.find(workload_name); iter != node_for_workload.end()) + { + auto queue = iter->second->getQueue(); + if (!queue) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to use workload '{}' that have children for resource '{}'", + workload_name, resource_name); + classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); + } + else + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to find workload '{}' for resource '{}'", workload_name, resource_name); + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +ClassifierPtr IOResourceManager::acquire(const String & workload_name) +{ + auto classifier = std::make_shared(); + + // Attach classifier to all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(resources.size()); + for (auto & [resource_name, resource] : resources) + futures.emplace_back(resource->attachClassifier(*classifier, workload_name)); + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // Rethrow exceptions if any + for (auto & future : futures) + future.get(); + + return classifier; +} + +void IOResourceManager::Resource::forEachResourceNode(IResourceManager::VisitorFunc & visitor) +{ + executeInSchedulerThread([&, this] + { + for (auto & [path, node] : node_for_workload) + { + node->forEachSchedulerNode([&] (ISchedulerNode * scheduler_node) + { + visitor(resource_name, scheduler_node->getPath(), scheduler_node); + }); + } + }); +} + +void IOResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +{ + // Gather resource upfront to avoid holding mutex for a long time + std::map sorted_resources; + { + std::unique_lock lock{mutex}; + for (auto & [resource_name, resource] : resources) + sorted_resources[resource_name] = resource; + } + + /// Run tasks one by one to avoid concurrent calls to visitor + for (auto & [resource_name, resource] : sorted_resources) + resource->forEachResourceNode(visitor); +} + +void IOResourceManager::topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads) +{ + if (visited.contains(workload)) + return; + visited.insert(workload); + + // Recurse into parent (if any) + String parent = workload->getParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + chassert(parent_iter != workloads.end()); // validations check that all parents exist + topologicallySortedWorkloadsImpl(parent_iter->second.get(), visited, sorted_workloads); + } + + sorted_workloads.push_back(workload); +} + +std::vector IOResourceManager::topologicallySortedWorkloads() +{ + std::vector sorted_workloads; + std::unordered_set visited; + for (auto & [workload_name, workload] : workloads) + topologicallySortedWorkloadsImpl(workload.get(), visited, sorted_workloads); + return sorted_workloads; +} + +} diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h new file mode 100644 index 00000000000..157507ed56b --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -0,0 +1,272 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +/* + * Implementation of `IResourceManager` that creates hierarchy of scheduler nodes according to + * workload entities (WORKLOADs and RESOURCEs). It subscribes for updates in IWorkloadEntityStorage and + * creates hierarchy of UnifiedSchedulerNode identical to the hierarchy of WORKLOADs. + * For every RESOURCE an independent hierarchy of scheduler nodes is created. + * + * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. + * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). + * After DROP RESOURCE parts of hierarchy might be keept alive while at least one query uses it. + * + * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having + * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. + * + * Classifiers are used (1) to access IO resources and (2) to keep shared ownership of scheduling nodes. + * This allows `ResourceRequest` and `ResourceLink` to hold raw pointers as long as + * `ClassifierPtr` is acquired and held. + * + * === RESOURCE ARCHITECTURE === + * Let's consider how a single resource is implemented. Every workload is represented by corresponding UnifiedSchedulerNode. + * Every UnifiedSchedulerNode manages its own subtree of ISchedulerNode objects (see details in UnifiedSchedulerNode.h) + * UnifiedSchedulerNode for workload w/o children has a queue, which provide a ResourceLink for consumption. + * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. + * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). + * + * Here is an example of SQL and corresponding heirarchy of scheduler nodes: + * CREATE RESOURCE my_io_resource (...) + * CREATE WORKLOAD all + * CREATE WORKLOAD production PARENT all + * CREATE WORKLOAD development PARENT all + * + * root - SchedulerRoot (with scheduler thread and EventQueue) + * | + * all - UnifiedSchedulerNode + * | + * p0_fair - FairPolicy (part of parent UnifiedSchedulerNode internal structure) + * / \ + * production development - UnifiedSchedulerNode + * | | + * queue queue - FifoQueue (part of parent UnifiedSchedulerNode internal structure) + * + * === UPDATING WORKLOADS === + * Workload may be created, updated or deleted. + * Updating a child of a workload might lead to updating other workloads: + * 1. Workload itself: it's structure depend on settings of children workloads + * (e.g. fifo node of a leaf workload is remove when the first child is added; + * and a fair node is inserted after the first two children are added). + * 2. Other children: for them path to root might be changed (e.g. intermediate priority node is inserted) + * + * === VERSION CONTROL === + * Versions are created on hierarchy updates and hold ownership of nodes that are used through raw pointers. + * Classifier reference version of every resource it use. Older version reference newer version. + * Here is a diagram explaining version control based on Version objects (for 1 resource): + * + * [nodes] [nodes] [nodes] + * ^ ^ ^ + * | | | + * version1 --> version2 -...-> versionN + * ^ ^ ^ + * | | | + * old_classifier new_classifier current_version + * + * Previous version should hold reference to a newer version. It is required for proper handling of updates. + * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). + * It may move a queue to a new position in the hierarchy or create/destry constraints, thus resource requests + * created by old classifier may reference constraints of newer versions through `request->constraints` which + * is filled during dequeueRequst(). + * + * === THREADS === + * scheduler thread: + * - one thread per resource + * - uses event_queue (per resource) for processing w/o holding mutex for every scheduler node + * - handle resource requests + * - node activations + * - scheduler hierarchy updates + * query thread: + * - multiple independent threads + * - send resource requests + * - acquire and release classifiers (via scheduler event queues) + * control thread: + * - modify workload and resources through subscription + * + * === SYNCHRONIZATION === + * List of related sync primitives and their roles: + * IOResourceManager::mutex + * - protects resource manager data structures - resource and workloads + * - serialize control thread actions + * IOResourceManager::Resource::scheduler->event_queue + * - serializes scheduler hierarchy events + * - events are created in control and query threads + * - all events are processed by specific scheduler thread + * - hierarchy-wide actions: requests dequeueing, activations propagation and nodes updates. + * - resource version control management + * FifoQueue::mutex and SemaphoreContraint::mutex + * - serializes query and scheduler threads on specific node accesses + * - resource request processing: enqueueRequest(), dequeueRequest() and finishRequest() + */ +class IOResourceManager : public IResourceManager +{ +public: + explicit IOResourceManager(IWorkloadEntityStorage & storage_); + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + ClassifierPtr acquire(const String & workload_name) override; + void forEachNode(VisitorFunc visitor) override; + +private: + // Forward declarations + struct NodeInfo; + struct Version; + class Resource; + struct Workload; + class Classifier; + + friend struct Workload; + + using VersionPtr = std::shared_ptr; + using ResourcePtr = std::shared_ptr; + using WorkloadPtr = std::shared_ptr; + + /// Helper for parsing workload AST for a specific resource + struct NodeInfo + { + String name; // Workload name + String parent; // Name of parent workload + SchedulingSettings settings; // Settings specific for a given resource + + NodeInfo(const ASTPtr & ast, const String & resource_name); + }; + + /// Ownership control for scheduler nodes, which could be referenced by raw pointers + struct Version + { + std::vector nodes; + VersionPtr newer_version; + }; + + /// Holds a thread and hierarchy of unified scheduler nodes for specific RESOURCE + class Resource : public std::enable_shared_from_this, boost::noncopyable + { + public: + explicit Resource(const ASTPtr & resource_entity_); + ~Resource(); + + const String & getName() const { return resource_name; } + + /// Hierarchy management + void createNode(const NodeInfo & info); + void deleteNode(const NodeInfo & info); + void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); + + /// Updates a classifier to contain a reference for specified workload + std::future attachClassifier(Classifier & classifier, const String & workload_name); + + /// Remove classifier reference. This destroys scheduler nodes in proper scheduler thread + std::future detachClassifier(VersionPtr && version); + + /// Introspection + void forEachResourceNode(IOResourceManager::VisitorFunc & visitor); + + private: + void updateCurrentVersion(); + + template + void executeInSchedulerThread(Task && task) + { + std::promise promise; + auto future = promise.get_future(); + scheduler.event_queue->enqueue([&] + { + try + { + task(); + promise.set_value(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } + }); + future.get(); // Blocks until execution is done in the scheduler thread + } + + const ASTPtr resource_entity; + const String resource_name; + SchedulerRoot scheduler; + + // TODO(serxa): consider using resource_manager->mutex + scheduler thread for updates and mutex only for reading to avoid slow acquire/release of classifier + /// These field should be accessed only by the scheduler thread + std::unordered_map node_for_workload; + UnifiedSchedulerNodePtr root_node; + VersionPtr current_version; + }; + + struct Workload : boost::noncopyable + { + IOResourceManager * resource_manager; + ASTPtr workload_entity; + + Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_); + ~Workload(); + + void updateWorkload(const ASTPtr & new_entity); + String getParent() const; + }; + + class Classifier : public IClassifier + { + public: + ~Classifier() override; + + /// Implements IClassifier interface + /// NOTE: It is called from query threads (possibly multiple) + ResourceLink get(const String & resource_name) override; + + /// Attaches/detaches a specific resource + /// NOTE: It is called from scheduler threads (possibly multiple) + void attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link); + void detach(const ResourcePtr & resource); + + private: + IOResourceManager * resource_manager; + std::mutex mutex; + struct Attachment { + ResourcePtr resource; + VersionPtr version; + ResourceLink link; + }; + std::unordered_map attachments; // TSA_GUARDED_BY(mutex); + }; + + void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); + void deleteWorkload(const String & workload_name); + void createResource(const String & resource_name, const ASTPtr & ast); + void deleteResource(const String & resource_name); + + // Topological sorting of worklaods + void topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads); + std::vector topologicallySortedWorkloads(); + + IWorkloadEntityStorage & storage; + scope_guard workload_change_subscription; + scope_guard resource_change_subscription; + + std::mutex mutex; + std::unordered_map workloads; // TSA_GUARDED_BY(mutex); + std::unordered_map resources; // TSA_GUARDED_BY(mutex); +}; + +} diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 17fcbfd3139..ea8bde718a2 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -43,6 +43,12 @@ public: : ISchedulerNode(event_queue_, node_info) {} + const String & getTypeName() const override + { + static String type_name("priority"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index a2d8df48065..eab093f6b00 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -31,6 +31,12 @@ public: , max_cost(max_cost_) {} + const String & getTypeName() const override + { + static String type_name("inflight_limit"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 7c64dd51ac1..40b51f24b98 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -3,8 +3,6 @@ #include #include -#include -#include #include @@ -42,6 +40,12 @@ public: event_queue->cancelPostponed(postponed); } + const String & getTypeName() const override + { + static String type_name("bandwidth_limit"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index c3c8ca2134a..76685319c34 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -266,7 +266,7 @@ public: reparent(immediate_child, this); } - /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes + /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) { @@ -274,18 +274,28 @@ public: reparent(new_child, this); } + /// Detaches unified child and update all the intermediate nodes. + /// Detached child could be safely attached to another parent. + /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children + void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) + { + UNUSED(child); // TODO(serxa): implement detachUnifiedChild() + } + /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) /// NOTE: Changing a priority of a unified child may lead to change of its parent. void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) { - UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority + UNUSED(child, old_priority, new_priority); // TODO(serxa): implement updateUnifiedChildPriority() } /// Updates scheduling settings. Set of constraints might change. /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) void updateSchedulingSettings(const SchedulingSettings & new_settings) { - UNUSED(new_settings); // TODO: implement updateSchedulingSettings + UNUSED(new_settings); // TODO(serxa): implement updateSchedulingSettings() + info.setPriority(new_settings.priority); + info.setWeight(new_settings.weight); } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children @@ -294,33 +304,58 @@ public: return static_pointer_cast(impl.branch.queue); } - /// Returns nodes that could be accessed with raw pointers by resource requests (queue and constraints) + /// Collects nodes that could be accessed with raw pointers by resource requests (queue and constraints) /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints - /// for that queue might change in future versions, and `request->constraints` might reference nodes not in - /// the initial set of nodes returned by `getClassifierNodes()`. To avoid destruction of such additinal nodes - /// classifier must (indirectly) hold nodes return by `getClassifierNodes()` for all future versions of all unified nodes. - /// Such a version control is done by `IOResourceManager`. - std::vector getClassifierNodes() + /// for that queue might change in future, and `request->constraints` might reference nodes not in + /// the initial set of nodes returned by `addRawPointerNodes()`. To avoid destruction of such additional nodes + /// classifier must (indirectly) hold nodes return by `addRawPointerNodes()` for all future versions of + /// all unified nodes. Such a version control is done by `IOResourceManager`. + void addRawPointerNodes(std::vector & nodes) { - std::vector result; - if (impl.branch.queue) - result.push_back(impl.branch.queue); - if (impl.semaphore) - result.push_back(impl.semaphore); if (impl.throttler) - result.push_back(impl.throttler); + nodes.push_back(impl.throttler); + if (impl.semaphore) + nodes.push_back(impl.semaphore); + if (impl.branch.queue) + nodes.push_back(impl.branch.queue); for (auto & [_, branch] : impl.branch.branch.branches) { for (auto & [_, child] : branch.children) - { - auto nodes = child->getClassifierNodes(); - result.insert(result.end(), nodes.begin(), nodes.end()); - } + child->addRawPointerNodes(nodes); + } + } + + bool hasUnifiedChildren() const + { + return impl.branch.queue == nullptr; + } + + /// Introspection. Calls a visitor for self and every internal node. Do not recurse into unified children. + void forEachSchedulerNode(std::function visitor) + { + visitor(this); + if (impl.throttler) + visitor(impl.throttler.get()); + if (impl.semaphore) + visitor(impl.semaphore.get()); + if (impl.branch.queue) + visitor(impl.branch.queue.get()); + if (impl.branch.branch.root) // priority + visitor(impl.branch.branch.root.get()); + for (auto & [_, branch] : impl.branch.branch.branches) + { + if (branch.root) // fairness + visitor(branch.root.get()); } - return result; } protected: // Hide all the ISchedulerNode interface methods as an implementation details + const String & getTypeName() const override + { + static String type_name("unified"); + return type_name; + } + bool equals(ISchedulerNode *) override { assert(false); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp index 07798f78080..9989215ba7b 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -13,6 +13,12 @@ public: , log(log_) {} + const String & getTypeName() const override + { + static String type_name("fake"); + return type_name; + } + void attachChild(const SchedulerNodePtr & child) override { log += " +" + child->basename; diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 5307aadc3cc..e2ed133f662 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -95,6 +95,12 @@ public: } } + const String & getTypeName() const override + { + static String type_name("scheduler"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b0b7f731a89..b71b450979f 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -8,10 +9,9 @@ namespace DB ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - UNUSED(global_context); - // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together + // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled // const auto & config = global_context->getConfigRef(); - return std::make_shared(); + return std::make_shared(global_context->getWorkloadEntityStorage()); } } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index b42c807d6fc..8784ba084ce 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -84,12 +84,12 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { - context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, ISchedulerNode * node) { size_t i = 0; res_columns[i++]->insert(resource); res_columns[i++]->insert(path); - res_columns[i++]->insert(type); + res_columns[i++]->insert(node->getTypeName()); res_columns[i++]->insert(node->info.weight); res_columns[i++]->insert(node->info.priority.value); res_columns[i++]->insert(node->isActive()); @@ -118,23 +118,23 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c if (auto * parent = dynamic_cast(node->parent)) { - if (auto value = parent->getChildVRuntime(node.get())) + if (auto value = parent->getChildVRuntime(node)) vruntime = *value; } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) system_vruntime = ptr->getSystemVRuntime(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) budget = ptr->getBudget(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) is_satisfied = ptr->isSatisfied(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); std::tie(max_requests, max_cost) = ptr->getLimits(); } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(max_speed, max_burst) = ptr->getParams(); throttling_us = ptr->getThrottlingDuration().count() / 1000; From 3bb616d2cf7084b9a0b996b3fdfcca99d64ec190 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 17 Sep 2024 15:06:19 +0000 Subject: [PATCH 056/816] Add test_ddl_worker_replicas --- .../test_ddl_worker_replicas/__init__.py | 0 .../configs/remote_servers.xml | 30 +++++++++ .../test_ddl_worker_replicas/test.py | 67 +++++++++++++++++++ 3 files changed, 97 insertions(+) create mode 100644 tests/integration/test_ddl_worker_replicas/__init__.py create mode 100644 tests/integration/test_ddl_worker_replicas/configs/remote_servers.xml create mode 100644 tests/integration/test_ddl_worker_replicas/test.py diff --git a/tests/integration/test_ddl_worker_replicas/__init__.py b/tests/integration/test_ddl_worker_replicas/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_ddl_worker_replicas/configs/remote_servers.xml b/tests/integration/test_ddl_worker_replicas/configs/remote_servers.xml new file mode 100644 index 00000000000..c505345cf7f --- /dev/null +++ b/tests/integration/test_ddl_worker_replicas/configs/remote_servers.xml @@ -0,0 +1,30 @@ + + + + + true + + node1 + 9000 + + + node2 + 9000 + + + + true + + node3 + 9000 + + + node4 + 9000 + + + + + + 1 + diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py new file mode 100644 index 00000000000..f9ce2575e00 --- /dev/null +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -0,0 +1,67 @@ +import pytest +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node4 = cluster.add_instance( + "node4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_ddl_worker_replicas(started_cluster): + replica_list = node1.query( + "SELECT name FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas'" + ).strip() + + replica_list = list(replica_list.split("\n")) + expected_replicas = ["node1:9000", "node2:9000", "node3:9000", "node4:9000"] + assert expected_replicas.sort() == replica_list.sort() + + for replica in replica_list: + result = node1.query( + f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/{replica}'" + ).strip() + + lines = list(result.split("\n")) + assert len(lines) == 1 + + parts = list(lines[0].split("\t")) + assert len(parts) == 3 + assert parts[0] == "active" + assert len(parts[1]) != 0 + assert len(parts[2]) != 0 + + node4.stop() + time.sleep(1) + + result = node1.query( + f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" + ).strip() + + lines = list(result.split("\n")) + assert len(lines) == 1 + assert len(lines[0]) == 0 From 37c92f0fc9292950c20791b77d8986d982163f90 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 18 Sep 2024 00:41:56 +0000 Subject: [PATCH 057/816] Add test_ddl_on_cluster_stop_waiting_for_offline_hosts test --- .../__init__.py | 0 .../configs/remote_servers.xml | 30 +++++++ .../test.py | 90 +++++++++++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/__init__.py create mode 100644 tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/configs/remote_servers.xml create mode 100644 tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py diff --git a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/__init__.py b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/configs/remote_servers.xml b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/configs/remote_servers.xml new file mode 100644 index 00000000000..c505345cf7f --- /dev/null +++ b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/configs/remote_servers.xml @@ -0,0 +1,30 @@ + + + + + true + + node1 + 9000 + + + node2 + 9000 + + + + true + + node3 + 9000 + + + node4 + 9000 + + + + + + 1 + diff --git a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py new file mode 100644 index 00000000000..d7dc1618802 --- /dev/null +++ b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py @@ -0,0 +1,90 @@ +import pytest +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +node4 = cluster.add_instance( + "node4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_stop_waiting_for_offline_hosts(started_cluster): + timeout = 10 + settings = {"distributed_ddl_task_timeout": timeout} + + start = time.time() + node1.query( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) + assert time.time() - start < timeout + + start = time.time() + node1.query( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) + assert time.time() - start < timeout + + node4.stop() + + start = time.time() + with pytest.raises(Exception) as err: + node1.query( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) + assert "Return code: 159" in str(err.value) + assert time.time() - start >= timeout + + start = time.time() + with pytest.raises(Exception) as err: + node1.query( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) + assert "Return code: 159" in str(err.value) + assert time.time() - start >= timeout + + settings = { + "distributed_ddl_task_timeout": timeout, + "distributed_ddl_output_mode": "throw_only_active", + } + + start = time.time() + node1.query( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) + assert time.time() - start < timeout + + start = time.time() + node1.query( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) + assert time.time() - start < timeout From 0940ab0ca5c60a854b66a2881429d5ec043e9e0a Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 18 Sep 2024 07:21:38 +0000 Subject: [PATCH 058/816] Fix compilation error --- src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp index cf2e8ce8558..15a007aa069 100644 --- a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp @@ -21,7 +21,7 @@ ReplicatedDatabaseQueryStatusSource::ReplicatedDatabaseQueryStatusSource( { } -ExecutionStatus ReplicatedDatabaseQueryStatusSource::checkStatus(const String & host_id) +ExecutionStatus ReplicatedDatabaseQueryStatusSource::checkStatus([[maybe_unused]] const String & host_id) { /// Replicated database retries in case of error, it should not write error status. #ifdef DEBUG_OR_SANITIZER_BUILD From 4092830e8bd57c7138643ca31bb52a5bc9e07984 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 18 Sep 2024 10:50:02 +0000 Subject: [PATCH 059/816] Fix fillHostStatus in ReplicatedDatabaseQueryStatusSource --- src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp index 15a007aa069..72d3a17a308 100644 --- a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp @@ -138,6 +138,8 @@ void ReplicatedDatabaseQueryStatusSource::fillHostStatus(const String & host_id, columns[num++]->insert(shard); columns[num++]->insert(replica); columns[num++]->insert(OK); + columns[num++]->insert(waiting_hosts.size() - num_hosts_finished); + columns[num++]->insert(current_active_hosts.size()); } Block ReplicatedDatabaseQueryStatusSource::getSampleBlock() From b2f357b457a89a53b8a0b31db686bb913fc29dee Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 19 Sep 2024 06:41:17 +0000 Subject: [PATCH 060/816] No need to and in DatabaseReplicated --- src/Databases/DatabaseReplicatedWorker.h | 6 ++++++ src/Interpreters/DDLWorker.h | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 51ff0f96e6d..586c49f6d6e 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -38,6 +38,12 @@ public: UInt32 getLogPointer() const; UInt64 getCurrentInitializationDurationMs() const; + +protected: + // No need to `createReplicaDirs` and `markReplicasActive` + void createReplicaDirs(const ZooKeeperPtr &, const NameSet &) override { } + void markReplicasActive(bool) override { } + private: bool initializeMainThread() override; void initializeReplication() override; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index fd4735b5baa..01d0b505108 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -149,8 +149,8 @@ protected: virtual bool initializeMainThread(); virtual void initializeReplication(); - void createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids); - void markReplicasActive(bool reinitialized); + virtual void createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids); + virtual void markReplicasActive(bool reinitialized); void runMainThread(); void runCleanupThread(); From b83fd18c9cebd4a7af4287f9c1e11b4f5410f21d Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 18:12:43 +0000 Subject: [PATCH 061/816] resolve conflict --- src/Common/ErrorCodes.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3144fb757f3..4c1593e2f2d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -609,12 +609,9 @@ M(728, UNEXPECTED_DATA_TYPE) \ M(729, ILLEGAL_TIME_SERIES_TAGS) \ M(730, REFRESH_FAILED) \ -<<<<<<< HEAD - M(731, WORKLOAD_ENTITY_ALREADY_EXISTS) \ - M(732, UNKNOWN_WORKLOAD_ENTITY) \ -======= M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \ ->>>>>>> master + M(732, WORKLOAD_ENTITY_ALREADY_EXISTS) \ + M(733, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ From 50168629b0838d560abca5d5f07b0277d9eb0385 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 19:07:44 +0000 Subject: [PATCH 062/816] fix subscription handlers --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index ad5a3166cf6..4ba16ade9d5 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -50,7 +50,8 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) - : global_context(std::move(global_context_)) + : handlers(std::make_shared()) + , global_context(std::move(global_context_)) {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const From 32f6699c8f6f0428f5f2b9aee2f2284a8b979222 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 19:09:41 +0000 Subject: [PATCH 063/816] .gitignore /programs/server/workload --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 4bc162c1b0f..8a745655cbf 100644 --- a/.gitignore +++ b/.gitignore @@ -159,6 +159,7 @@ website/package-lock.json /programs/server/store /programs/server/uuid /programs/server/coordination +/programs/server/workload # temporary test files tests/queries/0_stateless/test_* From b60d1427a92cf4ac920e162ea35feb0f440b8bc4 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 22:17:08 +0000 Subject: [PATCH 064/816] fix destruction order --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 9 +++++++++ src/Common/Scheduler/Nodes/IOResourceManager.h | 1 + 2 files changed, 10 insertions(+) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 9e6b4ebb254..e684cb9a16f 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -197,6 +197,7 @@ void IOResourceManager::Resource::updateCurrentVersion() previous_version->newer_version = current_version; // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } } @@ -266,6 +267,14 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) }); } +IOResourceManager::~IOResourceManager() +{ + resource_change_subscription.reset(); + workload_change_subscription.reset(); + resources.clear(); + workloads.clear(); +} + void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) { // No-op diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 157507ed56b..02a5e420be9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -122,6 +122,7 @@ class IOResourceManager : public IResourceManager { public: explicit IOResourceManager(IWorkloadEntityStorage & storage_); + ~IOResourceManager() override; void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & workload_name) override; void forEachNode(VisitorFunc visitor) override; From 36b8481793903aaa03d89c2e5f1bbb1a1a6dfb35 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 00:13:55 +0000 Subject: [PATCH 065/816] improve workload entities subscription model --- .../Scheduler/Nodes/IOResourceManager.cpp | 54 +++--- .../Scheduler/Nodes/IOResourceManager.h | 3 +- .../Workload/IWorkloadEntityStorage.h | 20 +-- .../Workload/WorkloadEntityDiskStorage.cpp | 7 - .../Workload/WorkloadEntityDiskStorage.h | 3 - .../Workload/WorkloadEntityStorageBase.cpp | 157 ++++++++++++------ .../Workload/WorkloadEntityStorageBase.h | 20 +-- 7 files changed, 149 insertions(+), 115 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index e684cb9a16f..0c204afa97f 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -1,4 +1,3 @@ -#include "Common/Scheduler/IResourceManager.h" #include #include @@ -231,34 +230,34 @@ String IOResourceManager::Workload::getParent() const IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) : storage(storage_) { - workload_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Workload, [this] ( - WorkloadEntityType, - const String & entity_name, - const ASTPtr & entity) + subscription = storage.getAllEntitiesAndSubscribe( + [this] (const std::vector & events) { try { - if (entity) - createOrUpdateWorkload(entity_name, entity); - else - deleteWorkload(entity_name); - } - catch (...) - { - // TODO(serxa): handle CRUD errors - } - }); - resource_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Resource, [this] ( - WorkloadEntityType, - const String & entity_name, - const ASTPtr & entity /* new or changed entity, null if removed */) - { - try - { - if (entity) - createResource(entity_name, entity); - else - deleteResource(entity_name); + for (auto [entity_type, entity_name, entity] : events) + { + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + break; + } + case WorkloadEntityType::Resource: + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + break; + } + case WorkloadEntityType::MAX: break; + } + } } catch (...) { @@ -269,8 +268,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) IOResourceManager::~IOResourceManager() { - resource_change_subscription.reset(); - workload_change_subscription.reset(); + subscription.reset(); resources.clear(); workloads.clear(); } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 02a5e420be9..0cb1887d1cd 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -262,8 +262,7 @@ private: std::vector topologicallySortedWorkloads(); IWorkloadEntityStorage & storage; - scope_guard workload_change_subscription; - scope_guard resource_change_subscription; + scope_guard subscription; std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index cff09a2259d..adb3a808eea 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -59,9 +59,6 @@ public: /// Stops watching. virtual void stopWatching() {} - /// Immediately reloads all entities, throws an exception if failed. - virtual void reloadEntities() = 0; - /// Stores an entity. virtual bool storeEntity( const ContextPtr & current_context, @@ -79,15 +76,16 @@ public: const String & entity_name, bool throw_if_not_exists) = 0; - using OnChangedHandler = std::function; + struct Event + { + WorkloadEntityType type; + String name; + ASTPtr entity; /// new or changed entity, null if removed + }; + using OnChangedHandler = std::function &)>; - /// Subscribes for all changes. - virtual scope_guard subscribeForChanges( - WorkloadEntityType entity_type, - const OnChangedHandler & handler) = 0; + /// Gets all current entries, pass them through `handler` and subscribes for all later changes. + virtual scope_guard getAllEntitiesAndSubscribe(const OnChangedHandler & handler) = 0; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index c794d2717e4..51016fac4fb 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -126,13 +126,6 @@ void WorkloadEntityDiskStorage::loadEntities() } -void WorkloadEntityDiskStorage::reloadEntities() -{ - // TODO(serxa): it does not send notifications, maybe better to remove this method completely - loadEntitiesImpl(); -} - - void WorkloadEntityDiskStorage::loadEntitiesImpl() { LOG_INFO(log, "Loading workload entities from {}", dir_path); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index 22c0ea4b83d..ceb736372ae 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -13,11 +13,8 @@ class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase { public: WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); - void loadEntities() override; - void reloadEntities() override; - private: bool storeEntityImpl( const ContextPtr & current_context, diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4ba16ade9d5..8e7f630365d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -8,6 +8,10 @@ #include #include +#include +#include + + namespace DB { @@ -15,6 +19,7 @@ namespace ErrorCodes { extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int LOGICAL_ERROR; } namespace @@ -47,6 +52,34 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + // Recurse into parent (if any) + String parent = typeid_cast(ast.get())->getWorkloadParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + if (parent_iter == workloads.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload metadata inconsistency: Workload '{}' parent '{}' does not exist. This must be fixed manually.", name, parent); + topologicallySortedWorkloadsImpl(parent, parent_iter->second, workloads, visited, sorted_workloads); + } + + sorted_workloads.emplace_back(name, ast); +} + +std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) +{ + std::vector> sorted_workloads; + std::unordered_set visited; + for (const auto & [name, ast] : workloads) + topologicallySortedWorkloadsImpl(name, ast, workloads, visited, sorted_workloads); + return sorted_workloads; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -125,7 +158,7 @@ bool WorkloadEntityStorageBase::storeEntity( bool replace_if_exists, const Settings & settings) { - std::lock_guard lock{mutex}; + std::unique_lock lock{mutex}; create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); @@ -153,7 +186,7 @@ bool WorkloadEntityStorageBase::storeEntity( onEntityAdded(entity_type, entity_name, create_entity_query); } - sendNotifications(); + unlockAndNotify(lock); return stored; } @@ -164,7 +197,7 @@ bool WorkloadEntityStorageBase::removeEntity( const String & entity_name, bool throw_if_not_exists) { - std::lock_guard lock(mutex); + std::unique_lock lock(mutex); auto it = entities.find(entity_name); if (it == entities.end()) { @@ -186,88 +219,79 @@ bool WorkloadEntityStorageBase::removeEntity( onEntityRemoved(entity_type, entity_name); } - sendNotifications(); + unlockAndNotify(lock); return removed; } -scope_guard WorkloadEntityStorageBase::subscribeForChanges( - WorkloadEntityType entity_type, - const OnChangedHandler & handler) +scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) { - std::lock_guard lock{handlers->mutex}; - auto & list = handlers->by_type[static_cast(entity_type)]; - list.push_back(handler); - auto handler_it = std::prev(list.end()); + scope_guard result; - return [my_handlers = handlers, entity_type, handler_it] + std::vector current_state; { - std::lock_guard lock2{my_handlers->mutex}; - auto & list2 = my_handlers->by_type[static_cast(entity_type)]; - list2.erase(handler_it); - }; + std::unique_lock lock{mutex}; + chassert(queue.empty()); + makeEventsForAllEntities(lock); + current_state = std::move(queue); + + std::lock_guard lock2{handlers->mutex}; + handlers->list.push_back(handler); + auto handler_it = std::prev(handlers->list.end()); + result = [my_handlers = handlers, handler_it] + { + std::lock_guard lock3{my_handlers->mutex}; + my_handlers->list.erase(handler_it); + }; + } + + // When you subscribe you get all the entities back to your handler immediately if already loaded, or later when loaded + handler(current_state); + + return result; } void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - event.entity = new_entity; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); } void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - event.entity = changed_entity; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = changed_entity}); } void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); } -void WorkloadEntityStorageBase::sendNotifications() +void WorkloadEntityStorageBase::unlockAndNotify(std::unique_lock & mutex_lock) { - /// Only one thread can send notification at any time. - std::lock_guard sending_notifications_lock{sending_notifications}; - - std::unique_lock queue_lock{queue_mutex}; - while (!queue.empty()) + /// Only one thread can send notification at any time, that is why we need `mutex_lock` + if (!queue.empty()) { - auto event = std::move(queue.front()); - queue.pop(); - queue_lock.unlock(); + auto events = std::move(queue); std::vector current_handlers; { std::lock_guard handlers_lock{handlers->mutex}; - boost::range::copy(handlers->by_type[static_cast(event.type)], std::back_inserter(current_handlers)); + boost::range::copy(handlers->list, std::back_inserter(current_handlers)); } + mutex_lock.unlock(); + for (const auto & handler : current_handlers) { try { - handler(event.type, event.name, event.entity); + handler(events); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } } - - queue_lock.lock(); } } @@ -276,21 +300,54 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } + void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) { + std::unordered_map normalized_entities; for (const auto & [entity_name, create_query] : new_entities) normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); // TODO(serxa): do validation and throw LOGICAL_ERROR if failed - // Note that notifications are not sent, because it is hard to send notifications in right order to maintain invariants. - // Another code path using getAllEntities() should be used for initialization - - std::lock_guard lock(mutex); + std::unique_lock lock(mutex); + chassert(entities.empty()); entities = std::move(normalized_entities); + + // Quick check to avoid extra work + { + std::lock_guard lock2(handlers->mutex); + if (handlers->list.empty()) + return; + } + + makeEventsForAllEntities(lock); + unlockAndNotify(lock); } + +void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock &) +{ + std::unordered_map workloads; + std::unordered_map resources; + for (auto & [entity_name, ast] : entities) + { + if (typeid_cast(ast.get())) + workloads.emplace(entity_name, ast); + else if (typeid_cast(ast.get())) + resources.emplace(entity_name, ast); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + } + + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); + + for (auto & [entity_name, ast] : resources) + onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); +} + + std::vector> WorkloadEntityStorageBase::getAllEntities() const { std::lock_guard lock{mutex}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 8ec92675ddb..bf8a89a67c4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -45,8 +44,7 @@ public: const String & entity_name, bool throw_if_not_exists) override; - virtual scope_guard subscribeForChanges( - WorkloadEntityType entity_type, + virtual scope_guard getAllEntitiesAndSubscribe( const OnChangedHandler & handler) override; protected: @@ -66,7 +64,9 @@ protected: bool throw_if_not_exists) = 0; std::unique_lock getLock() const; + void setAllEntities(const std::vector> & new_entities); + void makeEventsForAllEntities(std::unique_lock & lock); void removeAllEntitiesExcept(const Strings & entity_names_to_keep); /// Called by derived class after a new workload entity has been added. @@ -80,25 +80,17 @@ protected: /// Sends notifications to subscribers about changes in workload entities /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). - void sendNotifications(); + void unlockAndNotify(std::unique_lock & lock); struct Handlers { std::mutex mutex; - std::list by_type[static_cast(WorkloadEntityType::MAX)]; + std::list list; }; /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. std::shared_ptr handlers; - struct Event - { - WorkloadEntityType type; - String name; - ASTPtr entity; - }; - std::queue queue; - std::mutex queue_mutex; - std::mutex sending_notifications; + std::vector queue; mutable std::recursive_mutex mutex; std::unordered_map entities; // Maps entity name into CREATE entity query From d01655625fd7efec73b798bcb535e0bc8af690f8 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 12:06:59 +0000 Subject: [PATCH 066/816] Add `async_load_system_database` setting --- programs/local/LocalServer.cpp | 4 +-- programs/server/Server.cpp | 13 ++++---- src/Core/ServerSettings.h | 1 + src/Interpreters/loadMetadata.cpp | 49 ++++++++++++++++++------------- src/Interpreters/loadMetadata.h | 4 +-- 5 files changed, 39 insertions(+), 32 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 00d4ee1ca65..53465916e33 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -778,11 +778,11 @@ void LocalServer::processConfig() status.emplace(fs::path(path) / "status", StatusFile::write_full_info); LOG_DEBUG(log, "Loading metadata from {}", path); - auto startup_system_tasks = loadMetadataSystem(global_context); + auto load_system_metadata_tasks = loadMetadataSystem(global_context); attachSystemTablesServer(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE), false); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); - waitLoad(TablesLoaderForegroundPoolId, startup_system_tasks); + waitLoad(TablesLoaderForegroundPoolId, load_system_metadata_tasks); if (!getClientConfiguration().has("only-system-tables")) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9cf0e08e0ef..d1948a499a2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2055,6 +2055,7 @@ try LOG_INFO(log, "Loading metadata from {}", path_str); + LoadTaskPtrs load_system_metadata_tasks; LoadTaskPtrs load_metadata_tasks; // Make sure that if exception is thrown during startup async, new async loading jobs are not going to be called. @@ -2072,12 +2073,8 @@ try auto & database_catalog = DatabaseCatalog::instance(); /// We load temporary database first, because projections need it. database_catalog.initializeAndLoadTemporaryDatabase(); - auto system_startup_tasks = loadMetadataSystem(global_context); - maybeConvertSystemDatabase(global_context, system_startup_tasks); - /// This has to be done before the initialization of system logs, - /// otherwise there is a race condition between the system database initialization - /// and creation of new tables in the database. - waitLoad(TablesLoaderForegroundPoolId, system_startup_tasks); + load_system_metadata_tasks = loadMetadataSystem(global_context, server_settings.async_load_system_database); + maybeConvertSystemDatabase(global_context, load_system_metadata_tasks); /// Startup scripts can depend on the system log tables. if (config().has("startup_scripts") && !server_settings.prepare_system_log_tables_on_startup.changed) @@ -2224,10 +2221,12 @@ try global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, global_context, &config(), "distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID), - load_metadata_tasks); + joinTasks(load_system_metadata_tasks, load_metadata_tasks)); } /// Do not keep tasks in server, they should be kept inside databases. Used here to make dependent tasks only. + load_system_metadata_tasks.clear(); + load_system_metadata_tasks.shrink_to_fit(); load_metadata_tasks.clear(); load_metadata_tasks.shrink_to_fit(); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f3059c5370b..932225e65c6 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -136,6 +136,7 @@ namespace DB M(UInt64, tables_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \ M(UInt64, tables_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \ M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ + M(Bool, async_load_system_database, false, "Enable asynchronous loading of system tables that are not required on server startup. Queries to not yet loaded tables will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ M(Seconds, keep_alive_timeout, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, "The number of seconds that ClickHouse waits for incoming requests before closing the connection.", 0) \ M(UInt64, max_keep_alive_requests, 10000, "The maximum number of requests handled via a single http keepalive connection before the server closes this connection.", 0) \ diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 8eaf26672e2..6bb697440a7 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -382,7 +382,7 @@ static void convertOrdinaryDatabaseToAtomic(LoggerPtr log, ContextMutablePtr con /// Converts database with Ordinary engine to Atomic. Does nothing if database is not Ordinary. /// Can be called only during server startup when there are no queries from users. -static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const String & database_name, LoadTaskPtrs * startup_tasks = nullptr) +static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const String & database_name, const LoadTaskPtrs & load_system_metadata_tasks = {}) { LoggerPtr log = getLogger("loadMetadata"); @@ -409,12 +409,8 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons try { - if (startup_tasks) // NOTE: only for system database - { - /// It's not quite correct to run DDL queries while database is not started up. - waitLoad(TablesLoaderForegroundPoolId, *startup_tasks); - startup_tasks->clear(); - } + /// It's not quite correct to run DDL queries while database is not started up. + waitLoad(TablesLoaderForegroundPoolId, load_system_metadata_tasks); auto local_context = Context::createCopy(context); @@ -464,13 +460,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons }; TablesLoader loader{context, databases, LoadingStrictnessLevel::FORCE_RESTORE}; waitLoad(TablesLoaderForegroundPoolId, loader.loadTablesAsync()); - - /// Startup tables if they were started before conversion and detach/attach - if (startup_tasks) // NOTE: only for system database - *startup_tasks = loader.startupTablesAsync(); // We have loaded old database(s), replace tasks to startup new database - else - // An old database was already loaded, so we should load new one as well - waitLoad(TablesLoaderForegroundPoolId, loader.startupTablesAsync()); + waitLoad(TablesLoaderForegroundPoolId, loader.startupTablesAsync()); } catch (Exception & e) { @@ -482,13 +472,13 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons } } -void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system_startup_tasks) +void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & load_system_metadata_tasks) { /// TODO remove this check, convert system database unconditionally if (context->getSettingsRef()[Setting::allow_deprecated_database_ordinary]) return; - maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, &system_startup_tasks); + maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, load_system_metadata_tasks); } void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context) @@ -511,7 +501,7 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu fs::remove(convert_flag_path); } -LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context) +LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context, bool async_load_system_database) { loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic"); loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA, "Memory"); @@ -524,11 +514,28 @@ LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context) {DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, DatabaseCatalog::instance().getDatabase(DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)}, }; TablesLoader loader{context, databases, LoadingStrictnessLevel::FORCE_RESTORE}; - auto tasks = loader.loadTablesAsync(); - waitLoad(TablesLoaderForegroundPoolId, tasks); - /// Will startup tables in system database after all databases are loaded. - return loader.startupTablesAsync(); + auto load_tasks = loader.loadTablesAsync(); + auto startup_tasks = loader.startupTablesAsync(); + + if (async_load_system_database) + { + scheduleLoad(load_tasks); + scheduleLoad(startup_tasks); + + // Do NOT wait, just return tasks for continuation or later wait. + return joinTasks(load_tasks, startup_tasks); + } + else + { + waitLoad(TablesLoaderForegroundPoolId, load_tasks); + + /// This has to be done before the initialization of system logs `initializeSystemLogs()`, + /// otherwise there is a race condition between the system database initialization + /// and creation of new tables in the database. + waitLoad(TablesLoaderForegroundPoolId, startup_tasks); + return {}; + } } } diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index b0d97d53de3..84ca829462e 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -8,10 +8,10 @@ namespace DB /// Load tables from system database. Only real tables like query_log, part_log. /// You should first load system database, then attach system tables that you need into it, then load other databases. -/// It returns tasks to startup system tables. +/// It returns tasks that are still in progress if `async_load_system_database = true` otherwise it wait for all jobs to be done. /// Background operations in system tables may slowdown loading of the rest tables, /// so we startup system tables after all databases are loaded. -[[nodiscard]] LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context); +[[nodiscard]] LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context, bool async_load_system_database = false); /// Load tables from databases and add them to context. Databases 'system' and 'information_schema' are ignored. /// Use separate function to load system tables. From 64359a54fd82491a41dba78cdf6259569c6c5e6b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 14:35:48 +0000 Subject: [PATCH 067/816] rename: DynamicResourceManager -> CustomResourceManager --- ...eManager.cpp => CustomResourceManager.cpp} | 26 +++++++++---------- ...ourceManager.h => CustomResourceManager.h} | 8 +++--- ....cpp => gtest_custom_resource_manager.cpp} | 8 +++--- .../Scheduler/createResourceManager.cpp | 4 +-- 4 files changed, 24 insertions(+), 22 deletions(-) rename src/Common/Scheduler/Nodes/{DynamicResourceManager.cpp => CustomResourceManager.cpp} (87%) rename src/Common/Scheduler/Nodes/{DynamicResourceManager.h => CustomResourceManager.h} (90%) rename src/Common/Scheduler/Nodes/tests/{gtest_dynamic_resource_manager.cpp => gtest_custom_resource_manager.cpp} (94%) diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp similarity index 87% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.cpp rename to src/Common/Scheduler/Nodes/CustomResourceManager.cpp index 88b4eec063d..caaae11cdc7 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int INVALID_SCHEDULER_NODE; } -DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) +CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) : classifiers(config) { Poco::Util::AbstractConfiguration::Keys keys; @@ -34,7 +34,7 @@ DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util: } } -DynamicResourceManager::State::Resource::Resource( +CustomResourceManager::State::Resource::Resource( const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, @@ -91,7 +91,7 @@ DynamicResourceManager::State::Resource::Resource( throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name); } -DynamicResourceManager::State::Resource::~Resource() +CustomResourceManager::State::Resource::~Resource() { // NOTE: we should rely on `attached_to` and cannot use `parent`, // NOTE: because `parent` can be `nullptr` in case attachment is still in event queue @@ -105,14 +105,14 @@ DynamicResourceManager::State::Resource::~Resource() } } -DynamicResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +CustomResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) : type(config.getString(config_prefix + ".type", "fifo")) , ptr(SchedulerNodeFactory::instance().get(type, event_queue, config, config_prefix)) { ptr->basename = name; } -bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManager::State::Resource & o) const +bool CustomResourceManager::State::Resource::equals(const CustomResourceManager::State::Resource & o) const { if (nodes.size() != o.nodes.size()) return false; @@ -129,14 +129,14 @@ bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManage return true; } -bool DynamicResourceManager::State::Node::equals(const DynamicResourceManager::State::Node & o) const +bool CustomResourceManager::State::Node::equals(const CustomResourceManager::State::Node & o) const { if (type != o.type) return false; return ptr->equals(o.ptr.get()); } -DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::StatePtr & state_, const String & classifier_name) +CustomResourceManager::Classifier::Classifier(const CustomResourceManager::StatePtr & state_, const String & classifier_name) : state(state_) { // State is immutable, but nodes are mutable and thread-safe @@ -161,7 +161,7 @@ DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::Sta } } -ResourceLink DynamicResourceManager::Classifier::get(const String & resource_name) +ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) return iter->second; @@ -169,13 +169,13 @@ ResourceLink DynamicResourceManager::Classifier::get(const String & resource_nam throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); } -DynamicResourceManager::DynamicResourceManager() +CustomResourceManager::CustomResourceManager() : state(new State()) { scheduler.start(); } -void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { StatePtr new_state = std::make_shared(scheduler.event_queue, config); @@ -217,7 +217,7 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } -ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) +ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state StatePtr state_ref; @@ -229,7 +229,7 @@ ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) return std::make_shared(state_ref, classifier_name); } -void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { // Acquire a reference to the current state StatePtr state_ref; diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.h b/src/Common/Scheduler/Nodes/CustomResourceManager.h similarity index 90% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.h rename to src/Common/Scheduler/Nodes/CustomResourceManager.h index 4b0a3a48b61..c78fe672b33 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.h +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.h @@ -10,7 +10,9 @@ namespace DB { /* - * Implementation of `IResourceManager` supporting arbitrary dynamic hierarchy of scheduler nodes. + * Implementation of `IResourceManager` supporting arbitrary hierarchy of scheduler nodes. + * Scheduling hierarchies for every resource is described through server xml or yaml configuration. + * Configuration could be changed dynamically without server restart. * All resources are controlled by single root `SchedulerRoot`. * * State of manager is set of resources attached to the scheduler. States are referenced by classifiers. @@ -24,10 +26,10 @@ namespace DB * violation will apply to fairness. Old version exists as long as there is at least one classifier * instance referencing it. Classifiers are typically attached to queries and will be destructed with them. */ -class DynamicResourceManager : public IResourceManager +class CustomResourceManager : public IResourceManager { public: - DynamicResourceManager(); + CustomResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp similarity index 94% rename from src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp rename to src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp index 3328196cced..495654d45ce 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp @@ -2,15 +2,15 @@ #include -#include +#include #include using namespace DB; -using ResourceTest = ResourceTestManager; +using ResourceTest = ResourceTestManager; using TestGuard = ResourceTest::Guard; -TEST(SchedulerDynamicResourceManager, Smoke) +TEST(SchedulerCustomResourceManager, Smoke) { ResourceTest t; @@ -49,7 +49,7 @@ TEST(SchedulerDynamicResourceManager, Smoke) } } -TEST(SchedulerDynamicResourceManager, Fairness) +TEST(SchedulerCustomResourceManager, Fairness) { // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). // Requests from A use `value = 1` and from B `value = -1` is used. diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b71b450979f..b6fc0b4f01c 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -9,7 +9,7 @@ namespace DB ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled + // TODO(serxa): combine CustomResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled // const auto & config = global_context->getConfigRef(); return std::make_shared(global_context->getWorkloadEntityStorage()); } From 1053530a86336ce62fcef81a20bd4bd2a8c47798 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 15:39:01 +0000 Subject: [PATCH 068/816] add ResourceManagerDispatcher to combine io and custom managers --- src/Common/Scheduler/IResourceManager.h | 6 ++ .../Scheduler/Nodes/ClassifiersConfig.cpp | 3 +- .../Scheduler/Nodes/ClassifiersConfig.h | 1 + .../Scheduler/Nodes/CustomResourceManager.cpp | 11 +++ .../Scheduler/Nodes/CustomResourceManager.h | 4 +- .../Scheduler/Nodes/IOResourceManager.cpp | 19 +++- .../Scheduler/Nodes/IOResourceManager.h | 4 +- .../Scheduler/createResourceManager.cpp | 93 ++++++++++++++++++- 8 files changed, 133 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index c4a5c590ba7..b6199c91db7 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -26,6 +26,9 @@ class IClassifier : private boost::noncopyable public: virtual ~IClassifier() = default; + /// Returns true iff resource access is allowed by this classifier + virtual bool has(const String & resource_name) = 0; + /// Returns ResourceLink that should be used to access resource. /// Returned link is valid until classifier destruction. virtual ResourceLink get(const String & resource_name) = 0; @@ -46,6 +49,9 @@ public: /// Initialize or reconfigure manager. virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0; + /// Returns true iff given resource is controlled though this manager. + virtual bool hasResource(const String & resource_name) const = 0; + /// Obtain a classifier instance required to get access to resources. /// Note that it holds resource configuration, so should be destructed when query is done. virtual ClassifierPtr acquire(const String & classifier_name) = 0; diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 192f97645a0..4b0b0eaccfa 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -31,10 +31,11 @@ ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & c const ClassifierDescription & ClassifiersConfig::get(const String & classifier_name) { + static ClassifierDescription empty; if (auto it = classifiers.find(classifier_name); it != classifiers.end()) return it->second; else - throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name); + return empty; } } diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.h b/src/Common/Scheduler/Nodes/ClassifiersConfig.h index 186c49943ad..62db719568b 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.h +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.h @@ -10,6 +10,7 @@ namespace DB /// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class") struct ClassifierDescription : std::unordered_map { + ClassifierDescription() = default; ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); }; diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp index caaae11cdc7..0559b3cae0a 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp @@ -161,6 +161,11 @@ CustomResourceManager::Classifier::Classifier(const CustomResourceManager::State } } +bool CustomResourceManager::Classifier::has(const String & resource_name) +{ + return resources.find(resource_name) != resources.end(); +} + ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) @@ -217,6 +222,12 @@ void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfig // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } +bool CustomResourceManager::hasResource(const String & resource_name) const +{ + std::lock_guard lock{mutex}; + return state->resources.find(resource_name) != state->resources.end(); +} + ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.h b/src/Common/Scheduler/Nodes/CustomResourceManager.h index c78fe672b33..900a9c4e50b 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.h +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.h @@ -31,6 +31,7 @@ class CustomResourceManager : public IResourceManager public: CustomResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; @@ -81,6 +82,7 @@ private: { public: Classifier(const StatePtr & state_, const String & classifier_name); + bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; private: std::unordered_map resources; // accessible resources by names @@ -88,7 +90,7 @@ private: }; SchedulerRoot scheduler; - std::mutex mutex; + mutable std::mutex mutex; StatePtr state; }; diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 0c204afa97f..e956cca1862 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -382,6 +382,12 @@ std::future IOResourceManager::Resource::detachClassifier(VersionPtr && ve return future; } +bool IOResourceManager::Classifier::has(const String & resource_name) +{ + std::unique_lock lock{mutex}; + return attachments.find(resource_name) != attachments.end(); +} + ResourceLink IOResourceManager::Classifier::get(const String & resource_name) { std::unique_lock lock{mutex}; @@ -402,7 +408,7 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla { auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics auto future = attach_promise->get_future(); - scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] mutable + scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] { try { @@ -415,7 +421,10 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); } else - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to find workload '{}' for resource '{}'", workload_name, resource_name); + { + // This resource does not have specified workload. It is either unknown or managed by another resource manager. + // We leave this resource not attached to the classifier. Access denied will be thrown later on `classifier->get(resource_name)` + } promise->set_value(); } catch (...) @@ -426,6 +435,12 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla return future; } +bool IOResourceManager::hasResource(const String & resource_name) const +{ + std::unique_lock lock{mutex}; + return resources.find(resource_name) != resources.end(); +} + ClassifierPtr IOResourceManager::acquire(const String & workload_name) { auto classifier = std::make_shared(); diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 0cb1887d1cd..f4871379456 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -124,6 +124,7 @@ public: explicit IOResourceManager(IWorkloadEntityStorage & storage_); ~IOResourceManager() override; void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & workload_name) override; void forEachNode(VisitorFunc visitor) override; @@ -234,6 +235,7 @@ private: /// Implements IClassifier interface /// NOTE: It is called from query threads (possibly multiple) + bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; /// Attaches/detaches a specific resource @@ -264,7 +266,7 @@ private: IWorkloadEntityStorage & storage; scope_guard subscription; - std::mutex mutex; + mutable std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); std::unordered_map resources; // TSA_GUARDED_BY(mutex); }; diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b6fc0b4f01c..fd9743dbf72 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -4,14 +4,101 @@ #include #include +#include +#include + + namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + +class ResourceManagerDispatcher : public IResourceManager +{ +private: + class Classifier : public IClassifier + { + public: + void addClassifier(const ClassifierPtr & classifier) + { + classifiers.push_back(classifier); + } + + bool has(const String & resource_name) override + { + for (const auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return true; + } + return false; + } + + ResourceLink get(const String & resource_name) override + { + for (auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return classifier->get(resource_name); + } + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); + } + private: + std::vector classifiers; // should be constant after initialization to avoid races + }; + +public: + void addManager(const ResourceManagerPtr & manager) + { + managers.push_back(manager); + } + + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override + { + for (auto & manager : managers) + manager->updateConfiguration(config); + } + + bool hasResource(const String & resource_name) const override + { + for (const auto & manager : managers) + { + if (manager->hasResource(resource_name)) + return true; + } + return false; + } + + ClassifierPtr acquire(const String & workload_name) override + { + auto classifier = std::make_shared(); + for (const auto & manager : managers) + classifier->addClassifier(manager->acquire(workload_name)); + return classifier; + } + + void forEachNode(VisitorFunc visitor) override + { + for (const auto & manager : managers) + manager->forEachNode(visitor); + } + +private: + std::vector managers; // Should be constant after initialization to avoid races +}; + ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - // TODO(serxa): combine CustomResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled - // const auto & config = global_context->getConfigRef(); - return std::make_shared(global_context->getWorkloadEntityStorage()); + auto dispatcher = std::make_shared(); + + // NOTE: if the same resource is described by both managers, then manager added earlier will be used. + dispatcher->addManager(std::make_shared()); + dispatcher->addManager(std::make_shared(global_context->getWorkloadEntityStorage())); + + return dispatcher; } } From 823271ddd666da379623fea99ad299e2afade42d Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 09:34:07 +0000 Subject: [PATCH 069/816] fix build: use new setting access style --- .../Workload/WorkloadEntityDiskStorage.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 51016fac4fb..5ffec270610 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -31,6 +31,13 @@ namespace fs = std::filesystem; namespace DB { +namespace Setting +{ + extern const SettingsUInt64 max_parser_backtracks; + extern const SettingsUInt64 max_parser_depth; + extern const SettingsBool fsync_metadata; +} + namespace ErrorCodes { extern const int DIRECTORY_DOESNT_EXIST; @@ -91,8 +98,8 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, entity_create_query.data() + entity_create_query.size(), "", 0, - global_context->getSettingsRef().max_parser_depth, - global_context->getSettingsRef().max_parser_backtracks); + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); return ast; } case WorkloadEntityType::Resource: @@ -104,8 +111,8 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, entity_create_query.data() + entity_create_query.size(), "", 0, - global_context->getSettingsRef().max_parser_depth, - global_context->getSettingsRef().max_parser_backtracks); + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); return ast; } case WorkloadEntityType::MAX: return nullptr; @@ -225,7 +232,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( WriteBufferFromFile out(temp_file_path, create_statement.size()); writeString(create_statement, out); out.next(); - if (settings.fsync_metadata) + if (settings[Setting::fsync_metadata]) out.sync(); out.close(); From 956b40ec24c5b143a8d7b70d0bc1618326d4e328 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 09:51:16 +0000 Subject: [PATCH 070/816] add SETTINGS clause in CREATE WORKLOAD query --- .../Scheduler/Nodes/IOResourceManager.cpp | 82 ++++++++++-------- src/Common/Scheduler/SchedulingSettings.cpp | 83 +++++++++++++++++++ src/Common/Scheduler/SchedulingSettings.h | 11 +-- src/Parsers/ASTCreateWorkloadQuery.cpp | 38 ++++++--- src/Parsers/ASTCreateWorkloadQuery.h | 2 +- src/Parsers/ParserCreateWorkloadQuery.cpp | 54 +++++++++++- 6 files changed, 219 insertions(+), 51 deletions(-) create mode 100644 src/Common/Scheduler/SchedulingSettings.cpp diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index e956cca1862..34cdaf55ee6 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -43,8 +43,7 @@ IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resourc auto * create = typeid_cast(ast.get()); name = create->getWorkloadName(); parent = create->getWorkloadParent(); - // TODO(serxa): parse workload settings specifically for `resource_name` - UNUSED(resource_name); + settings.updateFromAST(create->settings, resource_name); } IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) @@ -205,21 +204,45 @@ IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, con : resource_manager(resource_manager_) , workload_entity(workload_entity_) { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->createNode(NodeInfo(workload_entity, resource_name)); + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->createNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } IOResourceManager::Workload::~Workload() { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->deleteNode(NodeInfo(workload_entity, resource_name)); + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->deleteNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); - workload_entity = new_entity; + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); + workload_entity = new_entity; + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } String IOResourceManager::Workload::getParent() const @@ -233,36 +256,29 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) { - try + for (auto [entity_type, entity_name, entity] : events) { - for (auto [entity_type, entity_name, entity] : events) + switch (entity_type) { - switch (entity_type) + case WorkloadEntityType::Workload: { - case WorkloadEntityType::Workload: - { - if (entity) - createOrUpdateWorkload(entity_name, entity); - else - deleteWorkload(entity_name); - break; - } - case WorkloadEntityType::Resource: - { - if (entity) - createResource(entity_name, entity); - else - deleteResource(entity_name); - break; - } - case WorkloadEntityType::MAX: break; + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + break; } + case WorkloadEntityType::Resource: + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + break; + } + case WorkloadEntityType::MAX: break; } } - catch (...) - { - // TODO(serxa): handle CRUD errors - } }); } diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp new file mode 100644 index 00000000000..c7d8a19ce41 --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -0,0 +1,83 @@ +#include +#include + + +namespace DB +{ + +void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) +{ + UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST + if (auto * set = typeid_cast(settings.get())) + { + std::optional new_weight; + std::optional new_priority; + std::optional new_max_speed; + std::optional new_max_burst; + std::optional new_max_requests; + std::optional new_max_cost; + + // Read changed setting values + for (const auto & [name, value] : set->changes) + { + // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage + // TODO(serxa): and probably we should add and persist version in filename for future changes + if (name == "weight") + new_weight = value.safeGet(); + else if (name == "priority") + new_priority = Priority{value.safeGet()}; + else if (name == "max_speed") + new_max_speed = value.safeGet(); + else if (name == "max_burst") + new_max_burst = value.safeGet(); + else if (name == "max_requests") + new_max_requests = value.safeGet(); + else if (name == "max_cost") + new_max_cost = value.safeGet(); + } + + // Read setting to be reset to default values + static SchedulingSettings default_settings; + bool reset_max_burst = false; + for (const String & name : set->default_settings) + { + if (name == "weight") + new_weight = default_settings.weight; + else if (name == "priority") + new_priority = default_settings.priority; + else if (name == "max_speed") + new_max_speed = default_settings.max_speed; + else if (name == "max_burst") + reset_max_burst = true; + else if (name == "max_requests") + new_max_requests = default_settings.max_requests; + else if (name == "max_cost") + new_max_cost = default_settings.max_cost; + } + if (reset_max_burst) + new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); + + // Save new values into the `this` object + // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value + if (new_weight) + weight = *new_weight; + if (new_priority) + priority = *new_priority; + if (new_max_speed) + { + max_speed = *new_max_speed; + // We always set max_burst if max_speed is changed. + // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed + if (!new_max_burst) + max_burst = default_burst_seconds * max_speed; + } + if (new_max_burst) + max_burst = *new_max_burst; + if (new_max_requests) + max_requests = *new_max_requests; + if (new_max_cost) + max_cost = *new_max_cost; + } +} + +} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index 4c6eff2b1e9..6d790b34164 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -3,6 +3,7 @@ #include #include +#include #include @@ -12,14 +13,14 @@ namespace DB struct SchedulingSettings { /// Priority and weight among siblings - double weight = 1.0; + Float64 weight = 1.0; Priority priority; /// Throttling constraints. /// Up to 2 independent throttlers: one for average speed and one for peek speed. - static constexpr double default_burst_seconds = 1.0; - double max_speed = 0; // Zero means unlimited - double max_burst = 0; // default is `default_burst_seconds * max_speed` + static constexpr Float64 default_burst_seconds = 1.0; + Float64 max_speed = 0; // Zero means unlimited + Float64 max_burst = 0; // default is `default_burst_seconds * max_speed` /// Limits total number of concurrent resource requests that are allowed to consume static constexpr Int64 default_max_requests = std::numeric_limits::max(); @@ -32,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - // TODO(serxa): add helper functions for parsing, printing and validating + void updateFromAST(const ASTPtr & settings, const String & resource_name); }; } diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index a6906dbcf65..2d32f499438 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -15,33 +15,49 @@ ASTPtr ASTCreateWorkloadQuery::clone() const res->workload_name = workload_name->clone(); res->children.push_back(res->workload_name); - // TODO(serxa): clone settings + if (workload_parent) + { + res->workload_parent = workload_parent->clone(); + res->children.push_back(res->workload_parent); + } + + if (settings) + { + res->settings = settings->clone(); + res->children.push_back(res->settings); + } return res; } -void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format_settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - settings.ostr << "OR REPLACE "; + format_settings.ostr << "OR REPLACE "; - settings.ostr << "WORKLOAD "; + format_settings.ostr << "WORKLOAD "; if (if_not_exists) - settings.ostr << "IF NOT EXISTS "; + format_settings.ostr << "IF NOT EXISTS "; - settings.ostr << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format_settings.hilite ? hilite_none : ""); - formatOnCluster(settings); + formatOnCluster(format_settings); if (hasParent()) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN " << (settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << " IN " << (format_settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format_settings.hilite ? hilite_none : ""); + } + + if (settings) + { + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' '; + settings->format(format_settings); } } diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h index bdd3a831aeb..a17bc4a11cd 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -12,7 +12,7 @@ class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster public: ASTPtr workload_name; ASTPtr workload_parent; - // TODO(serxa): add workload settings (weight and priority should also go inside settings, because they can differ for different resources) + ASTPtr settings; bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index ab0b0e3eb36..df7342093fd 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -2,13 +2,61 @@ #include #include +#include #include #include +#include +#include +#include namespace DB { +namespace +{ + +bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) + return false; + + SettingsChanges settings_changes; + + auto parse_setting = [&] + { + SettingChange setting; + if (ParserSetQuery::parseNameValuePair(setting, pos, expected)) + { + settings_changes.push_back(std::move(setting)); + // TODO(serxa): parse optional clause: [FOR resource_name] + return true; + } + + return false; + }; + + if (!ParserList::parseUtil(pos, expected, parse_setting, false)) + return false; + + ASTPtr res_settings; + if (!settings_changes.empty()) + { + auto settings_changes_ast = std::make_shared(); + settings_changes_ast->changes = std::move(settings_changes); + settings_changes_ast->is_standalone = false; + res_settings = settings_changes_ast; + } + + settings = std::move(res_settings); + return true; + }); +} + +} + bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create(Keyword::CREATE); @@ -18,7 +66,6 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp ParserIdentifier workload_name_p; ParserKeyword s_on(Keyword::ON); ParserKeyword s_in(Keyword::IN); - // TODO(serxa): parse workload settings ASTPtr workload_name; ASTPtr workload_parent; @@ -54,6 +101,9 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } + ASTPtr settings; + parseSettings(pos, expected, settings); + auto create_workload_query = std::make_shared(); node = create_workload_query; @@ -70,6 +120,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_workload_query->if_not_exists = if_not_exists; create_workload_query->cluster = std::move(cluster_str); + create_workload_query->settings = std::move(settings); + return true; } From 32a1766d15603b1b2f59b1a25214d8d0d3cefac4 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 11:37:46 +0000 Subject: [PATCH 071/816] add resource operations AST and parser --- src/Parsers/ASTCreateResourceQuery.cpp | 47 +++++++++++--- src/Parsers/ASTCreateResourceQuery.h | 13 +++- src/Parsers/ASTCreateWorkloadQuery.cpp | 24 ++++---- src/Parsers/CommonParsers.h | 2 + src/Parsers/ParserCreateResourceQuery.cpp | 75 ++++++++++++++++++++++- src/Parsers/ParserCreateWorkloadQuery.cpp | 19 +++++- 6 files changed, 155 insertions(+), 25 deletions(-) diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp index adb3e0b6e45..73d9514bdd0 100644 --- a/src/Parsers/ASTCreateResourceQuery.cpp +++ b/src/Parsers/ASTCreateResourceQuery.cpp @@ -15,26 +15,57 @@ ASTPtr ASTCreateResourceQuery::clone() const res->resource_name = resource_name->clone(); res->children.push_back(res->resource_name); + res->operations = operations; + return res; } -void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - settings.ostr << "OR REPLACE "; + format.ostr << "OR REPLACE "; - settings.ostr << "RESOURCE "; + format.ostr << "RESOURCE "; if (if_not_exists) - settings.ostr << "IF NOT EXISTS "; + format.ostr << "IF NOT EXISTS "; - settings.ostr << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : ""); - formatOnCluster(settings); + formatOnCluster(format); + + format.ostr << " ("; + + bool first = true; + for (const auto & operation : operations) + { + if (!first) + format.ostr << ", "; + else + first = false; + + switch (operation.mode) + { + case AccessMode::Read: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "READ DISK "; + break; + } + case AccessMode::Write: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE DISK "; + break; + } + } + format.ostr << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(operation.disk) << (format.hilite ? hilite_none : ""); + } + + format.ostr << ")"; } String ASTCreateResourceQuery::getResourceName() const diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index 3d571807ec4..e1713e6b063 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -10,8 +10,19 @@ namespace DB class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster { public: + enum class AccessMode { + Read, + Write + }; + struct Operation { + AccessMode mode; + String disk; + }; + + using Operations = std::vector; + ASTPtr resource_name; - // TODO(serxa): add resource definition + Operations operations; /// List of operations that require this resource bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index 2d32f499438..869dc64daf7 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -30,34 +30,34 @@ ASTPtr ASTCreateWorkloadQuery::clone() const return res; } -void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format_settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const { - format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << "CREATE "; + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - format_settings.ostr << "OR REPLACE "; + format.ostr << "OR REPLACE "; - format_settings.ostr << "WORKLOAD "; + format.ostr << "WORKLOAD "; if (if_not_exists) - format_settings.ostr << "IF NOT EXISTS "; + format.ostr << "IF NOT EXISTS "; - format_settings.ostr << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_none : ""); - format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : ""); - formatOnCluster(format_settings); + formatOnCluster(format); if (hasParent()) { - format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << " IN " << (format_settings.hilite ? hilite_none : ""); - format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); } if (settings) { - format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' '; - settings->format(format_settings); + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; + settings->format(format); } } diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 823c78c6f19..49708d8a40b 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -389,6 +389,7 @@ namespace DB MR_MACROS(RANDOMIZE_FOR, "RANDOMIZE FOR") \ MR_MACROS(RANDOMIZED, "RANDOMIZED") \ MR_MACROS(RANGE, "RANGE") \ + MR_MACROS(READ, "READ") \ MR_MACROS(READONLY, "READONLY") \ MR_MACROS(REALM, "REALM") \ MR_MACROS(RECOMPRESS, "RECOMPRESS") \ @@ -533,6 +534,7 @@ namespace DB MR_MACROS(WITH, "WITH") \ MR_MACROS(RECURSIVE, "RECURSIVE") \ MR_MACROS(WK, "WK") \ + MR_MACROS(WRITE, "WRITE") \ MR_MACROS(WRITABLE, "WRITABLE") \ MR_MACROS(WW, "WW") \ MR_MACROS(YEAR, "YEAR") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp index 4921debdf52..1abacaee617 100644 --- a/src/Parsers/ParserCreateResourceQuery.cpp +++ b/src/Parsers/ParserCreateResourceQuery.cpp @@ -4,11 +4,79 @@ #include #include #include +#include namespace DB { +namespace +{ + +bool parseOneOperation(ASTCreateResourceQuery::Operation & operation, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier disk_name_p; + + ASTCreateResourceQuery::AccessMode mode; + ASTPtr node; + String disk; + + if (ParserKeyword(Keyword::WRITE).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Write; + else if (ParserKeyword(Keyword::READ).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Read; + else + return false; + + if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) + return false; + + if (!disk_name_p.parse(pos, node, expected)) + return false; + + if (!tryGetIdentifierNameInto(node, disk)) + return false; + + operation.mode = mode; + operation.disk = std::move(disk); + + return true; +} + +bool parseOperations(IParser::Pos & pos, Expected & expected, ASTCreateResourceQuery::Operations & operations) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + ParserToken s_open(TokenType::OpeningRoundBracket); + ParserToken s_close(TokenType::ClosingRoundBracket); + + if (!s_open.ignore(pos, expected)) + return false; + + ASTCreateResourceQuery::Operations res_operations; + + auto parse_operation = [&] + { + ASTCreateResourceQuery::Operation operation; + if (!parseOneOperation(operation, pos, expected)) + return false; + res_operations.push_back(std::move(operation)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_operation, false)) + return false; + + if (!s_close.ignore(pos, expected)) + return false; + + operations = std::move(res_operations); + return true; + }); +} + +} + bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create(Keyword::CREATE); @@ -17,7 +85,6 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); ParserKeyword s_on(Keyword::ON); ParserIdentifier resource_name_p; - // TODO(serxa): parse resource definition ASTPtr resource_name; @@ -46,6 +113,10 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } + ASTCreateResourceQuery::Operations operations; + if (!parseOperations(pos, expected, operations)) + return false; + auto create_resource_query = std::make_shared(); node = create_resource_query; @@ -56,6 +127,8 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_resource_query->if_not_exists = if_not_exists; create_resource_query->cluster = std::move(cluster_str); + create_resource_query->operations = std::move(operations); + return true; } diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index df7342093fd..427d9aa40be 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -24,15 +24,28 @@ bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) return false; SettingsChanges settings_changes; + Strings default_settings; auto parse_setting = [&] { SettingChange setting; - if (ParserSetQuery::parseNameValuePair(setting, pos, expected)) + String default_setting; + std::pair parameter; + + if (ParserSetQuery::parseNameValuePairWithParameterOrDefault(setting, default_setting, parameter, pos, expected)) { - settings_changes.push_back(std::move(setting)); + if (!default_setting.empty()) + { + default_settings.push_back(std::move(default_setting)); + return true; + } + if (!setting.name.empty()) + { + settings_changes.push_back(std::move(setting)); + return true; + } // TODO(serxa): parse optional clause: [FOR resource_name] - return true; + return false; // We do not support parameters } return false; From e713cd938159673ed0198f693ab49370e7620b6b Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 12:21:43 +0000 Subject: [PATCH 072/816] better parsing and validation of WORKLOAD settings --- src/Common/Scheduler/SchedulingSettings.cpp | 67 +++++++++++++++++++-- 1 file changed, 62 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index c7d8a19ce41..18a465fe930 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -1,10 +1,17 @@ +#include #include +#include #include namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) { UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST @@ -17,23 +24,68 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r std::optional new_max_requests; std::optional new_max_cost; + auto get_not_negative_float64 = [] (const String & name, const Field & field) { + { + UInt64 val; + if (field.tryGet(val)) + return static_cast(val); // We dont mind slight loss of precision + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return static_cast(val); // We dont mind slight loss of precision + } + } + + return field.safeGet(); + }; + + auto get_not_negative_int64 = [] (const String & name, const Field & field) { + { + UInt64 val; + if (field.tryGet(val)) + { + // Saturate on overflow + if (val > static_cast(std::numeric_limits::max())) + val = std::numeric_limits::max(); + return static_cast(val); + } + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return val; + } + } + + return field.safeGet(); + }; + // Read changed setting values for (const auto & [name, value] : set->changes) { // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage // TODO(serxa): and probably we should add and persist version in filename for future changes if (name == "weight") - new_weight = value.safeGet(); + new_weight = get_not_negative_float64(name, value); else if (name == "priority") new_priority = Priority{value.safeGet()}; else if (name == "max_speed") - new_max_speed = value.safeGet(); + new_max_speed = get_not_negative_float64(name, value); else if (name == "max_burst") - new_max_burst = value.safeGet(); + new_max_burst = get_not_negative_float64(name, value); else if (name == "max_requests") - new_max_requests = value.safeGet(); + new_max_requests = get_not_negative_int64(name, value); else if (name == "max_cost") - new_max_cost = value.safeGet(); + new_max_cost = get_not_negative_int64(name, value); } // Read setting to be reset to default values @@ -57,6 +109,11 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r if (reset_max_burst) new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); + // Validate we could use values we read in a scheduler node + { + SchedulerNodeInfo validating_node(new_weight ? *new_weight : weight, new_priority ? *new_priority : priority); + } + // Save new values into the `this` object // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value if (new_weight) From 709c8489f7548aee4730079b0322a19600079e38 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 12:28:07 +0000 Subject: [PATCH 073/816] add simple integration tests --- tests/integration/test_scheduler/test.py | 160 +++++++++++++++++++++++ 1 file changed, 160 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 31cc106a95d..02b1b4c2169 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -54,6 +54,20 @@ def set_default_configs(): yield +@pytest.fixture(scope="function", autouse=True) +def clear_workloads_and_resources(): + node.query( + f""" + -- drop resource if exist io_write; TODO(serxa): uncomment it + -- drop resource if exist io_read; + -- drop workload if exist production; + -- drop workload if exist development; + -- drop workload if exist all; + """ + ) + yield + + def update_workloads_config(**settings): xml = "" for name in settings: @@ -569,3 +583,149 @@ def test_mutation_workload_change(): assert reads_before < reads_after assert writes_before < writes_after + + +def test_create_workload(): + node.query( + f""" + create resource io_write (write disk s3); + create resource io_read (read disk s3); + create workload all settings max_cost = 1000000; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def do_checks(): + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin' and type='unified' and priority=0" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production' and type='unified' and weight=9" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/development/%' and type='fifo'" + ) + == "2\n" + ) + + do_checks() + node.restart_clickhouse() # Check that workloads persist + do_checks() + + + +def test_resource_read_and_write(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + node.query( + f""" + create resource io_write (write disk s3); + create resource io_read (read disk s3); + create workload all settings max_cost = 1000000; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def write_query(workload): + try: + node.query( + f"insert into data select * from numbers(1e5) settings workload='{workload}'" + ) + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=write_query, args=["development"]) + thread2 = threading.Thread(target=write_query, args=["production"]) + thread3 = threading.Thread(target=write_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) + + def read_query(workload): + try: + node.query(f"select sum(key*key) from data settings workload='{workload}'") + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=read_query, args=["development"]) + thread2 = threading.Thread(target=read_query, args=["production"]) + thread3 = threading.Thread(target=read_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) From 9db958dcdc494a2c08d4cb08f741d9569a4f1dfa Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 15:52:29 +0000 Subject: [PATCH 074/816] integrate DiskObjectStorage and RESOURCEs --- .../ObjectStorages/DiskObjectStorage.cpp | 65 ++++++++++++++++--- src/Disks/ObjectStorages/DiskObjectStorage.h | 9 ++- src/Interpreters/Context.cpp | 18 +---- src/Interpreters/Context.h | 4 +- 4 files changed, 65 insertions(+), 31 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 07e2edac129..a20ee53ff75 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -15,7 +15,8 @@ #include #include #include - +#include +#include namespace DB { @@ -68,8 +69,8 @@ DiskObjectStorage::DiskObjectStorage( , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) - , read_resource_name(config.getString(config_prefix + ".read_resource", "")) - , write_resource_name(config.getString(config_prefix + ".write_resource", "")) + , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) + , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) { data_source_description = DataSourceDescription{ @@ -80,6 +81,52 @@ DiskObjectStorage::DiskObjectStorage( .is_encrypted = false, .is_cached = object_storage->supportsCache(), }; + resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe( + [this] (const std::vector & events) + { + std::unique_lock lock{resource_mutex}; + for (auto [entity_type, resource_name, resource] : events) + { + if (entity_type == WorkloadEntityType::Resource) + { + if (resource) // CREATE RESOURCE + { + // We rely on the fact that every disk is allowed to be mentioned at most + // in one RESOURCE for READ and in one RESOURCE for WRITE + // TODO(serxa): add disk operations validation in workload entity storage + auto * create = typeid_cast(resource.get()); + chassert(create); + for (const auto & [mode, disk] : create->operations) + { + if (disk == name) + { + switch (mode) + { + case ASTCreateResourceQuery::AccessMode::Read: + { + read_resource_name_from_sql = resource_name; + break; + } + case ASTCreateResourceQuery::AccessMode::Write: + { + write_resource_name_from_sql = resource_name; + break; + } + } + } + } + } + else // DROP RESOURCE + { + if (read_resource_name_from_sql == resource_name) + read_resource_name_from_sql.clear(); + if (write_resource_name_from_sql == resource_name) + write_resource_name_from_sql.clear(); + } + break; + } + } + }); } StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const @@ -480,13 +527,13 @@ static inline Settings updateIOSchedulingSettings(const Settings & settings, con String DiskObjectStorage::getReadResourceName() const { std::unique_lock lock(resource_mutex); - return read_resource_name; + return read_resource_name_from_config.empty() ? read_resource_name_from_sql : read_resource_name_from_config; } String DiskObjectStorage::getWriteResourceName() const { std::unique_lock lock(resource_mutex); - return write_resource_name; + return write_resource_name_from_config.empty() ? write_resource_name_from_sql : write_resource_name_from_config; } std::unique_ptr DiskObjectStorage::readFile( @@ -551,10 +598,10 @@ void DiskObjectStorage::applyNewSettings( { std::unique_lock lock(resource_mutex); - if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name) - read_resource_name = new_read_resource_name; - if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name) - write_resource_name = new_write_resource_name; + if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name_from_config) + read_resource_name_from_config = new_read_resource_name; + if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name_from_config) + write_resource_name_from_config = new_write_resource_name; } IDisk::applyNewSettings(config, context_, config_prefix, disk_map); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 5c45a258806..d4d4dc11ffa 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -6,6 +6,8 @@ #include #include +#include + #include "config.h" @@ -242,8 +244,11 @@ private: const bool send_metadata; mutable std::mutex resource_mutex; - String read_resource_name; - String write_resource_name; + String read_resource_name_from_config; // specified in disk config.xml + String write_resource_name_from_config; // specified in disk config.xml + String read_resource_name_from_sql; // described by CREATE RESOURCE queries + String write_resource_name_from_sql; // described by CREATE RESOURCE queries + scope_guard resource_changes_subscription; std::unique_ptr metadata_helper; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d3cbbf76156..5de1dece884 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2979,17 +2979,7 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } -const IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const -{ - callOnce(shared->workload_entity_storage_initialized, [&] { - shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); - }); - - SharedLockGuard lock(shared->mutex); - return *shared->workload_entity_storage; -} - -IWorkloadEntityStorage & Context::getWorkloadEntityStorage() +IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const { callOnce(shared->workload_entity_storage_initialized, [&] { shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); @@ -2999,12 +2989,6 @@ IWorkloadEntityStorage & Context::getWorkloadEntityStorage() return *shared->workload_entity_storage; } -void Context::setWorkloadEntityStorage(std::unique_ptr storage) -{ - std::lock_guard lock(shared->mutex); - shared->workload_entity_storage = std::move(storage); -} - #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index be963d85757..114e2c96570 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -882,9 +882,7 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); - const IWorkloadEntityStorage & getWorkloadEntityStorage() const; - IWorkloadEntityStorage & getWorkloadEntityStorage(); - void setWorkloadEntityStorage(std::unique_ptr storage); + IWorkloadEntityStorage & getWorkloadEntityStorage() const; #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; From d57c28aa52c193a95928aeb6ed6c4c6c635afc6f Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 16:47:54 +0000 Subject: [PATCH 075/816] add logs for resource to disk mapping --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index a20ee53ff75..166435064da 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -104,11 +104,21 @@ DiskObjectStorage::DiskObjectStorage( { case ASTCreateResourceQuery::AccessMode::Read: { + if (read_resource_name_from_config.empty()) + LOG_INFO(log, "Using resource '{}' for READ", resource_name); + else + LOG_INFO(log, "Resource '{}' should be used for READ, but it is overriden by config to resource '{}'", + resource_name, read_resource_name_from_config); read_resource_name_from_sql = resource_name; break; } case ASTCreateResourceQuery::AccessMode::Write: { + if (write_resource_name_from_config.empty()) + LOG_INFO(log, "Using resource '{}' for WRITE", resource_name); + else + LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overriden by config to resource '{}'", + resource_name, write_resource_name_from_config); write_resource_name_from_sql = resource_name; break; } @@ -119,9 +129,15 @@ DiskObjectStorage::DiskObjectStorage( else // DROP RESOURCE { if (read_resource_name_from_sql == resource_name) + { + LOG_INFO(log, "Stop using resource '{}' for READ", resource_name); read_resource_name_from_sql.clear(); + } if (write_resource_name_from_sql == resource_name) + { + LOG_INFO(log, "Stop using resource '{}' for WRITE", resource_name); write_resource_name_from_sql.clear(); + } } break; } From 0a7bd6010bb1cee2fe415901c272b006d401cfd0 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 16:48:29 +0000 Subject: [PATCH 076/816] fix test_resource_read_and_write --- .../configs/storage_configuration.xml | 16 ++++++++++++++++ tests/integration/test_scheduler/test.py | 10 +++++----- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 823a00a05de..16cdf4a5b15 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -12,6 +12,15 @@ network_read network_write + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + @@ -21,6 +30,13 @@ + + +
+ s3_no_resource +
+
+
diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 02b1b4c2169..311bd6d7401 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -588,8 +588,8 @@ def test_mutation_workload_change(): def test_create_workload(): node.query( f""" - create resource io_write (write disk s3); - create resource io_read (read disk s3); + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); create workload all settings max_cost = 1000000; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; @@ -639,14 +639,14 @@ def test_resource_read_and_write(): node.query( f""" drop table if exists data; - create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; """ ) node.query( f""" - create resource io_write (write disk s3); - create resource io_read (read disk s3); + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); create workload all settings max_cost = 1000000; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; From 407e56adb6936b5c8a407edd23caac870b0045c0 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 18:17:27 +0000 Subject: [PATCH 077/816] fix all scheduler integration tests --- tests/integration/test_scheduler/test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 311bd6d7401..401444b2d5a 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -58,11 +58,12 @@ def set_default_configs(): def clear_workloads_and_resources(): node.query( f""" - -- drop resource if exist io_write; TODO(serxa): uncomment it - -- drop resource if exist io_read; - -- drop workload if exist production; - -- drop workload if exist development; - -- drop workload if exist all; + drop resource if exists io_write; + drop resource if exists io_read; + drop workload if exists production; + drop workload if exists development; + drop workload if exists admin; + drop workload if exists all; """ ) yield From 52204768b03c5674601cc8ac7ccccfedf5af9fdf Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 18:19:23 +0000 Subject: [PATCH 078/816] add reference validation in workload entity storage --- .../Scheduler/Nodes/IOResourceManager.cpp | 3 + .../Workload/WorkloadEntityStorageBase.cpp | 67 ++++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 2 + 3 files changed, 61 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 34cdaf55ee6..460693e1935 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -307,7 +307,10 @@ void IOResourceManager::deleteWorkload(const String & workload_name) { std::unique_lock lock{mutex}; if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + { + // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent workloads.erase(workload_iter); + } else { // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 8e7f630365d..f0c76b92870 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -52,6 +52,21 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +void forEachReference(const ASTPtr & source_entity, std::function func) +{ + if (auto * res = typeid_cast(source_entity.get())) + { + String parent = res->getWorkloadParent(); + if (!parent.empty()) + func(parent, res->getWorkloadName()); + // TODO(serxa): add references to RESOURCEs mentioned in SETTINGS clause after FOR keyword + } + if (auto * res = typeid_cast(source_entity.get())) + { + // RESOURCE has no references to be validated + } +} + void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) { if (visited.contains(name)) @@ -162,8 +177,7 @@ bool WorkloadEntityStorageBase::storeEntity( create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); - auto it = entities.find(entity_name); - if (it != entities.end()) + if (auto it = entities.find(entity_name); it != entities.end()) { if (throw_if_exists) throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); @@ -171,6 +185,13 @@ bool WorkloadEntityStorageBase::storeEntity( return false; } + forEachReference(create_entity_query, + [this] (const String & target, const String & source) + { + if (auto it = entities.find(target); it == entities.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + }); + bool stored = storeEntityImpl( current_context, entity_type, @@ -182,12 +203,16 @@ bool WorkloadEntityStorageBase::storeEntity( if (stored) { + forEachReference(create_entity_query, + [this] (const String & target, const String & source) + { + references[target].insert(source); + }); entities[entity_name] = create_entity_query; onEntityAdded(entity_type, entity_name, create_entity_query); + unlockAndNotify(lock); } - unlockAndNotify(lock); - return stored; } @@ -207,6 +232,14 @@ bool WorkloadEntityStorageBase::removeEntity( return false; } + if (auto reference_it = references.find(entity_name); reference_it != references.end()) + { + String names; + for (const String & name : reference_it->second) + names += " " + name; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); + } + bool removed = removeEntityImpl( current_context, entity_type, @@ -215,11 +248,18 @@ bool WorkloadEntityStorageBase::removeEntity( if (removed) { - entities.erase(entity_name); + forEachReference(it->second, + [this] (const String & target, const String & source) + { + references[target].erase(source); + if (references[target].empty()) + references.erase(target); + }); + entities.erase(it); onEntityRemoved(entity_type, entity_name); - } - unlockAndNotify(lock); + unlockAndNotify(lock); + } return removed; } @@ -300,10 +340,8 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } - void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) { - std::unordered_map normalized_entities; for (const auto & [entity_name, create_query] : new_entities) normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); @@ -313,6 +351,15 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector &) { std::unordered_map workloads; @@ -347,7 +393,6 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock> WorkloadEntityStorageBase::getAllEntities() const { std::lock_guard lock{mutex}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index bf8a89a67c4..a51e2392ea4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -94,6 +95,7 @@ protected: mutable std::recursive_mutex mutex; std::unordered_map entities; // Maps entity name into CREATE entity query + std::unordered_map> references; // Keep track of references between entities for validation ContextPtr global_context; }; From 9ed14e047a617101e224853cb28fd8608044a0c9 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 25 Sep 2024 02:21:19 +0000 Subject: [PATCH 079/816] Add distributed_ddl.replicas_path setting --- programs/server/Server.cpp | 19 +++++++++++++----- programs/server/config.xml | 2 ++ src/Databases/DatabaseReplicatedWorker.cpp | 9 ++++++++- src/Interpreters/DDLWorker.cpp | 20 +++++++++---------- src/Interpreters/DDLWorker.h | 16 ++++++++++----- .../configs/config.xml | 2 ++ .../test_config_xml_full/configs/config.xml | 2 ++ .../test_https_replication/configs/config.xml | 2 ++ 8 files changed, 51 insertions(+), 21 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f0c9719051f..66937e4d3a5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2217,14 +2217,23 @@ try if (has_zookeeper && config().has("distributed_ddl")) { /// DDL worker should be started after all tables were loaded - String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); + String ddl_queue_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/"); + String ddl_replicas_path = config().getString("distributed_ddl.replicas_path", "/clickhouse/task_queue/replicas/"); int pool_size = config().getInt("distributed_ddl.pool_size", 1); if (pool_size < 1) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "distributed_ddl.pool_size should be greater then 0"); - global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, global_context, &config(), - "distributed_ddl", "DDLWorker", - &CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID), - load_metadata_tasks); + global_context->setDDLWorker( + std::make_unique( + pool_size, + ddl_queue_path, + ddl_replicas_path, + global_context, + &config(), + "distributed_ddl", + "DDLWorker", + &CurrentMetrics::MaxDDLEntryID, + &CurrentMetrics::MaxPushedDDLEntryID), + load_metadata_tasks); } /// Do not keep tasks in server, they should be kept inside databases. Used here to make dependent tasks only. diff --git a/programs/server/config.xml b/programs/server/config.xml index 10ad831465a..f53cc550df2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1437,6 +1437,8 @@ /clickhouse/task_queue/ddl + + /clickhouse/task_queue/replicas diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index f5a9ccb187b..7e4e7992d7c 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -27,7 +27,14 @@ namespace ErrorCodes static constexpr const char * FORCE_AUTO_RECOVERY_DIGEST = "42"; DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_) - : DDLWorker(/* pool_size */ 1, db->zookeeper_path + "/log", context_, nullptr, {}, fmt::format("DDLWorker({})", db->getDatabaseName())) + : DDLWorker( + /* pool_size */ 1, + db->zookeeper_path + "/log", + db->zookeeper_path + "/replicas", + context_, + nullptr, + {}, + fmt::format("DDLWorker({})", db->getDatabaseName())) , database(db) { /// Pool size must be 1 to avoid reordering of log entries. diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 66ea5f9a891..989c6d87fd9 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -77,7 +77,8 @@ constexpr const char * TASK_PROCESSED_OUT_REASON = "Task has been already proces DDLWorker::DDLWorker( int pool_size_, - const std::string & zk_root_dir, + const std::string & zk_queue_dir, + const std::string & zk_replicas_dir, ContextPtr context_, const Poco::Util::AbstractConfiguration * config, const String & prefix, @@ -103,15 +104,14 @@ DDLWorker::DDLWorker( worker_pool = std::make_unique(CurrentMetrics::DDLWorkerThreads, CurrentMetrics::DDLWorkerThreadsActive, CurrentMetrics::DDLWorkerThreadsScheduled, pool_size); } - queue_dir = zk_root_dir; + queue_dir = zk_queue_dir; if (queue_dir.back() == '/') queue_dir.resize(queue_dir.size() - 1); - // replicas_dir is at the same level as queue_dir - // E.g: - // queue_dir: /clickhouse/task_queue/ddl - // replicas_dir: /clickhouse/task_queue/replicas - replicas_dir = fs::path(queue_dir).parent_path() / "replicas"; + replicas_dir = zk_replicas_dir; + if (replicas_dir.back() == '/') + replicas_dir.resize(replicas_dir.size() - 1); + if (config) { @@ -1238,7 +1238,7 @@ void DDLWorker::initializeReplication() { auto zookeeper = getAndSetZooKeeper(); - zookeeper->createAncestors(replicas_dir / ""); + zookeeper->createAncestors(fs::path(replicas_dir) / ""); NameSet host_id_set; for (const auto & it : context->getClusters()) @@ -1255,7 +1255,7 @@ void DDLWorker::initializeReplication() void DDLWorker::createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids) { for (const auto & host_id : host_ids) - zookeeper->createAncestors(replicas_dir / host_id / ""); + zookeeper->createAncestors(fs::path(replicas_dir) / host_id / ""); } void DDLWorker::markReplicasActive(bool reinitialized) @@ -1312,7 +1312,7 @@ void DDLWorker::markReplicasActive(bool reinitialized) } /// Create "active" node (remove previous one if necessary) - String active_path = replicas_dir / host_id / "active"; + String active_path = fs::path(replicas_dir) / host_id / "active"; String active_id = toString(ServerUUID::get()); zookeeper->deleteEphemeralNodeIfContentMatches(active_path, active_id); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index a90ca4a8c53..53434d18861 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -13,13 +13,11 @@ #include #include -#include #include #include #include #include -namespace fs = std::filesystem; namespace zkutil { @@ -54,8 +52,16 @@ class AccessRightsElements; class DDLWorker { public: - DDLWorker(int pool_size_, const std::string & zk_root_dir, ContextPtr context_, const Poco::Util::AbstractConfiguration * config, const String & prefix, - const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr, const CurrentMetrics::Metric * max_pushed_entry_metric_ = nullptr); + DDLWorker( + int pool_size_, + const std::string & zk_queue_dir, + const std::string & zk_replicas_dir, + ContextPtr context_, + const Poco::Util::AbstractConfiguration * config, + const String & prefix, + const String & logger_name = "DDLWorker", + const CurrentMetrics::Metric * max_entry_metric_ = nullptr, + const CurrentMetrics::Metric * max_pushed_entry_metric_ = nullptr); virtual ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node @@ -166,7 +172,7 @@ protected: std::string host_fqdn; /// current host domain name std::string host_fqdn_id; /// host_name:port std::string queue_dir; /// dir with queue of queries - fs::path replicas_dir; + std::string replicas_dir; mutable std::mutex zookeeper_mutex; ZooKeeperPtr current_zookeeper TSA_GUARDED_BY(zookeeper_mutex); diff --git a/tests/integration/test_config_corresponding_root/configs/config.xml b/tests/integration/test_config_corresponding_root/configs/config.xml index 9a38d02a036..001a98837c4 100644 --- a/tests/integration/test_config_corresponding_root/configs/config.xml +++ b/tests/integration/test_config_corresponding_root/configs/config.xml @@ -291,6 +291,8 @@ /clickhouse/task_queue/ddl + + /clickhouse/task_queue/replicas diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index 61aa0a5c724..a233bd52214 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -865,6 +865,8 @@ /clickhouse/task_queue/ddl + + /clickhouse/task_queue/replicas diff --git a/tests/integration/test_https_replication/configs/config.xml b/tests/integration/test_https_replication/configs/config.xml index 9a7a542b16e..8c1cd9beeb2 100644 --- a/tests/integration/test_https_replication/configs/config.xml +++ b/tests/integration/test_https_replication/configs/config.xml @@ -256,6 +256,8 @@ /clickhouse/task_queue/ddl + + /clickhouse/task_queue/replicas From 588d5532e524b7e1a29caac7aeed60e8f873d203 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 25 Sep 2024 02:35:57 +0000 Subject: [PATCH 080/816] Pass repliaces_path to DistributedQueryStatusSource --- src/Databases/DatabaseReplicated.cpp | 7 +++--- src/Databases/DatabaseReplicated.h | 3 ++- .../DDLOnClusterQueryStatusSource.cpp | 5 +++-- .../DDLOnClusterQueryStatusSource.h | 3 ++- src/Interpreters/DDLWorker.h | 2 ++ .../DistributedQueryStatusSource.cpp | 22 +++++++++++-------- .../DistributedQueryStatusSource.h | 8 ++++++- .../ReplicatedDatabaseQueryStatusSource.cpp | 5 +++-- .../ReplicatedDatabaseQueryStatusSource.h | 3 ++- src/Interpreters/executeDDLQueryOnCluster.cpp | 6 ++--- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- 11 files changed, 42 insertions(+), 24 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index cb22030166b..a4e7effd80d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1084,7 +1084,7 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex } - return getQueryStatus(node_path, query_context, hosts_to_wait); + return getQueryStatus(node_path, fs::path(zookeeper_path) / "replicas", query_context, hosts_to_wait); } static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) @@ -2027,13 +2027,14 @@ void registerDatabaseReplicated(DatabaseFactory & factory) factory.registerDatabase("Replicated", create_fn, {.supports_arguments = true, .supports_settings = true}); } -BlockIO DatabaseReplicated::getQueryStatus(const String & node_path, ContextPtr context_, const Strings & hosts_to_wait) +BlockIO DatabaseReplicated::getQueryStatus( + const String & node_path, const String & replicas_path, ContextPtr context_, const Strings & hosts_to_wait) { BlockIO io; if (context_->getSettingsRef()[Setting::distributed_ddl_task_timeout] == 0) return io; - auto source = std::make_shared(node_path, context_, hosts_to_wait); + auto source = std::make_shared(node_path, replicas_path, context_, hosts_to_wait); io.pipeline = QueryPipeline(std::move(source)); if (context_->getSettingsRef()[Setting::distributed_ddl_output_mode] == DistributedDDLOutputMode::NONE diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 491b60c400b..5646a6c8544 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -151,7 +151,8 @@ private: void waitDatabaseStarted() const override; void stopLoading() override; - static BlockIO getQueryStatus(const String & node_path, ContextPtr context, const Strings & hosts_to_wait); + static BlockIO + getQueryStatus(const String & node_path, const String & replicas_path, ContextPtr context, const Strings & hosts_to_wait); String zookeeper_path; String shard_name; diff --git a/src/Interpreters/DDLOnClusterQueryStatusSource.cpp b/src/Interpreters/DDLOnClusterQueryStatusSource.cpp index d60f0973921..9b5215eb41a 100644 --- a/src/Interpreters/DDLOnClusterQueryStatusSource.cpp +++ b/src/Interpreters/DDLOnClusterQueryStatusSource.cpp @@ -22,8 +22,9 @@ extern const int TIMEOUT_EXCEEDED; } DDLOnClusterQueryStatusSource::DDLOnClusterQueryStatusSource( - const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait) - : DistributedQueryStatusSource(zk_node_path, getSampleBlock(context_), context_, hosts_to_wait, "DDLOnClusterQueryStatusSource") + const String & zk_node_path, const String & zk_replicas_path, ContextPtr context_, const Strings & hosts_to_wait) + : DistributedQueryStatusSource( + zk_node_path, zk_replicas_path, getSampleBlock(context_), context_, hosts_to_wait, "DDLOnClusterQueryStatusSource") { } diff --git a/src/Interpreters/DDLOnClusterQueryStatusSource.h b/src/Interpreters/DDLOnClusterQueryStatusSource.h index fb86aa43661..cb50bde40f3 100644 --- a/src/Interpreters/DDLOnClusterQueryStatusSource.h +++ b/src/Interpreters/DDLOnClusterQueryStatusSource.h @@ -10,7 +10,8 @@ namespace DB class DDLOnClusterQueryStatusSource final : public DistributedQueryStatusSource { public: - DDLOnClusterQueryStatusSource(const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait); + DDLOnClusterQueryStatusSource( + const String & zk_node_path, const String & zk_replicas_path, ContextPtr context_, const Strings & hosts_to_wait); String getName() const override { return "DDLOnClusterQueryStatus"; } diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 649b56def4b..4b808191ec2 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -79,6 +79,8 @@ public: return queue_dir; } + std::string getReplicasDir() const { return replicas_dir; } + void startup(); virtual void shutdown(); diff --git a/src/Interpreters/DistributedQueryStatusSource.cpp b/src/Interpreters/DistributedQueryStatusSource.cpp index b6f04e75647..83701d41c57 100644 --- a/src/Interpreters/DistributedQueryStatusSource.cpp +++ b/src/Interpreters/DistributedQueryStatusSource.cpp @@ -23,8 +23,18 @@ extern const int UNFINISHED; } DistributedQueryStatusSource::DistributedQueryStatusSource( - const String & zk_node_path, Block block, ContextPtr context_, const Strings & hosts_to_wait, const char * logger_name) - : ISource(block), node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(getLogger(logger_name)) + const String & zk_node_path, + const String & zk_replicas_path, + Block block, + ContextPtr context_, + const Strings & hosts_to_wait, + const char * logger_name) + : ISource(block) + , node_path(zk_node_path) + , replicas_path(zk_replicas_path) + , context(context_) + , watch(CLOCK_MONOTONIC_COARSE) + , log(getLogger(logger_name)) { auto output_mode = context->getSettingsRef()[Setting::distributed_ddl_output_mode]; throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; @@ -66,18 +76,12 @@ IProcessor::Status DistributedQueryStatusSource::prepare() NameSet DistributedQueryStatusSource::getOfflineHosts(const NameSet & hosts_to_wait, const ZooKeeperPtr & zookeeper) { - fs::path replicas_path; - if (node_path.ends_with('/')) - replicas_path = fs::path(node_path).parent_path().parent_path().parent_path() / "replicas"; - else - replicas_path = fs::path(node_path).parent_path().parent_path() / "replicas"; - Strings paths; Strings hosts_array; for (const auto & host : hosts_to_wait) { hosts_array.push_back(host); - paths.push_back(replicas_path / host / "active"); + paths.push_back(fs::path(replicas_path) / host / "active"); } NameSet offline; diff --git a/src/Interpreters/DistributedQueryStatusSource.h b/src/Interpreters/DistributedQueryStatusSource.h index a7aad497a1e..4f58085a1f0 100644 --- a/src/Interpreters/DistributedQueryStatusSource.h +++ b/src/Interpreters/DistributedQueryStatusSource.h @@ -14,7 +14,12 @@ class DistributedQueryStatusSource : public ISource { public: DistributedQueryStatusSource( - const String & zk_node_path, Block block, ContextPtr context_, const Strings & hosts_to_wait, const char * logger_name); + const String & zk_node_path, + const String & zk_replicas_path, + Block block, + ContextPtr context_, + const Strings & hosts_to_wait, + const char * logger_name); Chunk generate() override; Status prepare() override; @@ -37,6 +42,7 @@ protected: static std::pair parseHostAndPort(const String & host_id); String node_path; + String replicas_path; ContextPtr context; Stopwatch watch; LoggerPtr log; diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp index b3bbc4a5381..09941b09238 100644 --- a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.cpp @@ -20,8 +20,9 @@ extern const int LOGICAL_ERROR; } ReplicatedDatabaseQueryStatusSource::ReplicatedDatabaseQueryStatusSource( - const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait) - : DistributedQueryStatusSource(zk_node_path, getSampleBlock(), context_, hosts_to_wait, "ReplicatedDatabaseQueryStatusSource") + const String & zk_node_path, const String & zk_replicas_path, ContextPtr context_, const Strings & hosts_to_wait) + : DistributedQueryStatusSource( + zk_node_path, zk_replicas_path, getSampleBlock(), context_, hosts_to_wait, "ReplicatedDatabaseQueryStatusSource") { } diff --git a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h index 8b00c756596..76a2d5f3f14 100644 --- a/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h +++ b/src/Interpreters/ReplicatedDatabaseQueryStatusSource.h @@ -10,7 +10,8 @@ namespace DB class ReplicatedDatabaseQueryStatusSource final : public DistributedQueryStatusSource { public: - ReplicatedDatabaseQueryStatusSource(const String & zk_node_path, ContextPtr context_, const Strings & hosts_to_wait); + ReplicatedDatabaseQueryStatusSource( + const String & zk_node_path, const String & zk_replicas_path, ContextPtr context_, const Strings & hosts_to_wait); String getName() const override { return "ReplicatedDatabaseQueryStatus"; } diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 2c620e06e1e..b4b44f56a1a 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -184,10 +184,10 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.initial_query_id = context->getClientInfo().initial_query_id; String node_path = ddl_worker.enqueueQuery(entry); - return getDDLOnClusterStatus(node_path, entry, context); + return getDDLOnClusterStatus(node_path, ddl_worker.getReplicasDir(), entry, context); } -BlockIO getDDLOnClusterStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context) +BlockIO getDDLOnClusterStatus(const String & node_path, const String & replicas_path, const DDLLogEntry & entry, ContextPtr context) { BlockIO io; if (context->getSettingsRef()[Setting::distributed_ddl_task_timeout] == 0) @@ -196,7 +196,7 @@ BlockIO getDDLOnClusterStatus(const String & node_path, const DDLLogEntry & entr for (const HostID & host : entry.hosts) hosts_to_wait.push_back(host.toString()); - auto source = std::make_shared(node_path, context, hosts_to_wait); + auto source = std::make_shared(node_path, replicas_path, context, hosts_to_wait); io.pipeline = QueryPipeline(std::move(source)); if (context->getSettingsRef()[Setting::distributed_ddl_output_mode] == DistributedDDLOutputMode::NONE diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 61d6ba75cf0..d015e8d8694 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -43,7 +43,7 @@ struct DDLQueryOnClusterParams /// Returns DDLQueryStatusSource, which reads results of query execution on each host in the cluster. BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, ContextPtr context, const DDLQueryOnClusterParams & params = {}); -BlockIO getDDLOnClusterStatus(const String & node_path, const DDLLogEntry & entry, ContextPtr context); +BlockIO getDDLOnClusterStatus(const String & node_path, const String & replicas_path, const DDLLogEntry & entry, ContextPtr context); bool maybeRemoveOnCluster(const ASTPtr & query_ptr, ContextPtr context); From 53896e1a1f9366fd9c2e38b0c3045c8606a4aaaf Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 25 Sep 2024 13:21:10 +0000 Subject: [PATCH 081/816] add `system.resources` table --- .../System/StorageSystemResources.cpp | 72 +++++++++++++++++++ src/Storages/System/StorageSystemResources.h | 29 ++++++++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 103 insertions(+) create mode 100644 src/Storages/System/StorageSystemResources.cpp create mode 100644 src/Storages/System/StorageSystemResources.h diff --git a/src/Storages/System/StorageSystemResources.cpp b/src/Storages/System/StorageSystemResources.cpp new file mode 100644 index 00000000000..692f89358e7 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.cpp @@ -0,0 +1,72 @@ +#include +#include +#include +#include +#include +#include +#include "Parsers/ASTCreateQuery.h" +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemResources::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the resource."}, + {"read_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for read operations."}, + {"write_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for write operations."}, + {"create_query", std::make_shared(), "CREATE query of the resource."}, + }; +} + +void StorageSystemResources::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & resource_names = storage.getAllEntityNames(WorkloadEntityType::Resource); + for (const auto & resource_name : resource_names) + { + auto ast = storage.get(resource_name); + auto & resource = typeid_cast(*ast); + res_columns[0]->insert(resource_name); + { + Array read_disks; + Array write_disks; + for (const auto & [mode, disk] : resource.operations) + { + switch (mode) + { + case DB::ASTCreateResourceQuery::AccessMode::Read: + { + read_disks.emplace_back(disk); + break; + } + case DB::ASTCreateResourceQuery::AccessMode::Write: + { + write_disks.emplace_back(disk); + break; + } + } + } + res_columns[1]->insert(read_disks); + res_columns[2]->insert(write_disks); + } + res_columns[3]->insert(queryToString(ast)); + } +} + +void StorageSystemResources::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemResources::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemResources.h b/src/Storages/System/StorageSystemResources.h new file mode 100644 index 00000000000..42bbcd09aa4 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `resources` system table, which allows you to get a list of all RESOURCEs +class StorageSystemResources final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemResources"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 728e83135a3..93d846a1d40 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -231,6 +232,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); + attach(context, system_database, "resources", "Contains a list of all currently existing resources."); if (has_zookeeper) { From fc49a4a146dcc7d447be476dc93eb4d8f29567a4 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 25 Sep 2024 13:21:48 +0000 Subject: [PATCH 082/816] add stateless test for CREATE and DROP RESOURCE queries --- .../0_stateless/03232_resource_create_and_drop.reference | 5 +++++ .../0_stateless/03232_resource_create_and_drop.sql | 9 +++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.sql diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.reference b/tests/queries/0_stateless/03232_resource_create_and_drop.reference new file mode 100644 index 00000000000..2a1045d314c --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.reference @@ -0,0 +1,5 @@ +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_2 ['03232_disk_2'] [] CREATE RESOURCE `03232_resource_2` (READ DISK `03232_disk_2`) +03232_resource_3 [] ['03232_disk_2'] CREATE RESOURCE `03232_resource_3` (WRITE DISK `03232_disk_2`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql new file mode 100644 index 00000000000..eb356e12448 --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.sql @@ -0,0 +1,9 @@ +CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); +CREATE RESOURCE 03232_resource_3 (WRITE DISK 03232_disk_2); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE IF EXISTS 03232_resource_2; +DROP RESOURCE 03232_resource_3; +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE 03232_resource_1; From b4d7174ccc8b39458b5b9bc6984178437ddb345f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 21 Aug 2024 19:21:21 +0000 Subject: [PATCH 083/816] [wip] select inner table for hash join --- docs/en/operations/settings/settings.md | 4 + src/Core/Joins.h | 11 ++ src/Core/Settings.cpp | 1 + src/Core/SettingsChangesHistory.cpp | 2 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 3 + .../HashJoin/HashJoinMethodsImpl.h | 18 +++- src/Interpreters/TableJoin.cpp | 55 +++++++++- src/Interpreters/TableJoin.h | 19 +++- src/Interpreters/TreeRewriter.cpp | 5 +- src/Parsers/CreateQueryUUIDs.cpp | 2 +- src/Planner/CollectColumnIdentifiers.cpp | 22 ++++ src/Planner/PlannerJoinTree.cpp | 25 +++-- src/Processors/QueryPlan/JoinStep.cpp | 13 ++- src/Processors/QueryPlan/JoinStep.h | 4 + .../QueryPlan/Optimizations/Optimizations.h | 1 + .../QueryPlan/Optimizations/optimizeJoin.cpp | 100 ++++++++++++++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 4 + .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + tests/clickhouse-test | 4 + .../02962_join_using_bug_57894.reference | 1 + .../02962_join_using_bug_57894.sql | 2 + 23 files changed, 285 insertions(+), 19 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 392b1831ce3..fcb6f610894 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5630,6 +5630,10 @@ Minimal size of block to compress in CROSS JOIN. Zero value means - disable this Default value: `1GiB`. +## query_plan_join_inner_table_selection + +Select the side of the join to be the inner table in the query plan. Possible values: 'auto', 'left', 'right'. In `auto` mode, ClickHouse will try to choose the table with the smallest number of rows. + ## use_json_alias_for_old_object_type When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type. diff --git a/src/Core/Joins.h b/src/Core/Joins.h index 96d2b51325c..41e1de43702 100644 --- a/src/Core/Joins.h +++ b/src/Core/Joins.h @@ -119,4 +119,15 @@ enum class JoinTableSide : uint8_t const char * toString(JoinTableSide join_table_side); +/// Setting to choose which table to use as the inner table in hash join +enum class JoinInnerTableSelectionMode : uint8_t +{ + /// Use left table + Left, + /// Use right table + Right, + /// Use the table with the smallest number of rows + Auto, +}; + } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 07b4ecd7a24..57dc297432a 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -345,6 +345,7 @@ namespace ErrorCodes M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ M(Bool, single_join_prefer_left_table, true, "For single JOIN in case of identifier ambiguity prefer left table", IMPORTANT) \ \ + M(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, "auto", "Select the side of the join to be the inner table in the query plan. Possible values: 'auto', 'left', 'right'.", 0) \ M(UInt64, preferred_block_size_bytes, 1000000, "This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality.", 0) \ \ M(UInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c9723deaad8..8a79853c091 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -90,6 +90,8 @@ static std::initializer_listenableEnalyzer()) + left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); if (!flag_per_row) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 320c8851ce4..5753e37ff88 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -56,7 +56,6 @@ Block HashJoinMethods::joinBlockImpl( const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - size_t existing_columns = block.columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -99,6 +98,23 @@ Block HashJoinMethods::joinBlockImpl( added_columns.buildJoinGetOutput(); else added_columns.buildOutput(); + + const auto & table_join = join.table_join; + if (table_join->enableEnalyzer()) + { + std::unordered_set left_output_columns; + for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) + left_output_columns.insert(out_column.name); + std::set to_erase; + for (size_t i = 0; i < block.columns(); ++i) + { + if (!left_output_columns.contains(block.getByPosition(i).name)) + to_erase.insert(i); + } + block.erase(to_erase); + } + size_t existing_columns = block.columns(); + for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 2532dddba3c..d17300c229e 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -143,6 +143,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_memory_usage(settings[Setting::max_memory_usage]) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) + , enable_analyzer(settings.allow_experimental_analyzer) { } @@ -161,6 +162,8 @@ void TableJoin::resetCollected() clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); + columns_from_left_table.clear(); + result_columns_from_left_table.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); @@ -203,6 +206,20 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return count; } +void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) +{ + columns_from_left_table = left_output_columns; + columns_from_joined_table = right_output_columns; +} + + +const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side) +{ + if (side == JoinTableSide::Left) + return result_columns_from_left_table; + return columns_added_by_join; +} + void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix) { NameSet joined_columns; @@ -351,9 +368,18 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const return forceNullableRight() && JoinCommon::canBecomeNullable(column_type); } +void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side) +{ + if (side == JoinTableSide::Left) + result_columns_from_left_table.push_back(joined_column); + else + columns_added_by_join.push_back(joined_column); + +} + void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - columns_added_by_join.emplace_back(joined_column); + setUsedColumn(joined_column, JoinTableSide::Right); } NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const @@ -995,5 +1021,32 @@ size_t TableJoin::getMaxMemoryUsage() const return max_memory_usage; } +void TableJoin::swapSides() +{ + assertEnableEnalyzer(); + + std::swap(key_asts_left, key_asts_right); + std::swap(left_type_map, right_type_map); + for (auto & clause : clauses) + { + std::swap(clause.key_names_left, clause.key_names_right); + std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right); + std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name); + } + + std::swap(columns_from_left_table, columns_from_joined_table); + std::swap(result_columns_from_left_table, columns_added_by_join); + + if (table_join.kind == JoinKind::Left) + table_join.kind = JoinKind::Right; + else if (table_join.kind == JoinKind::Right) + table_join.kind = JoinKind::Left; +} + +void TableJoin::assertEnableEnalyzer() const +{ + if (!enable_analyzer) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled"); +} } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index e1bae55a4ed..e0e1926fb12 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -167,6 +167,9 @@ private: ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals; + NamesAndTypesList columns_from_left_table; + NamesAndTypesList result_columns_from_left_table; + /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; /// Columns will be added to block by JOIN. @@ -202,6 +205,8 @@ private: bool is_join_with_constant = false; + bool enable_analyzer = false; + Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -266,6 +271,8 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } + bool enableEnalyzer() const { return enable_analyzer; } + void assertEnableEnalyzer() const; ActionsDAG createJoinedBlockActions(ContextPtr context) const; @@ -282,6 +289,7 @@ public: } bool allowParallelHashJoin() const; + void swapSides(); bool joinUseNulls() const { return join_use_nulls; } @@ -372,6 +380,9 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); + + void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) { columns_added_by_join = columns_added_by_join_value; @@ -397,11 +408,17 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_) { columns_from_joined_table = std::move(columns_from_joined_table_value); deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); + result_columns_from_left_table = columns_from_left_table_; + columns_from_left_table = columns_from_left_table_; } + + void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns); + const NamesAndTypesList & getOutputColumns(JoinTableSide side); + const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ea08fd92339..28e11166762 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { + auto columns_from_left_table = tables_with_columns[0].columns; const auto & right_table = tables_with_columns[1]; auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); + columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end()); + result.analyzed_join->setColumnsFromJoinedTable( + std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index fbdc6161408..1609ad43c69 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// Thus it's not safe for example to replace /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with - /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b" + /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b" /// This replacement is safe only for CREATE queries when inner target tables don't exist yet. if (!query.attach) { diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index 95f1c7d53d8..ca468a353b2 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -33,6 +34,27 @@ public: void visitImpl(const QueryTreeNodePtr & node) { + // if (node->getNodeType() == QueryTreeNodeType::QUERY) + // { + // const auto * join_node = node->as().getJoinTree()->as(); + // if (!join_node || !join_node->isUsingJoinExpression()) + // return; + + // const auto & using_list = join_node->getJoinExpression()->as(); + + // for (const auto & join_using_node : using_list.getNodes()) + // { + // const auto & join_using_expression = join_using_node->as().getExpression(); + // if (!join_using_expression) + // return; + // const auto & using_join_columns_list = join_using_expression->as().getNodes(); + // if (const auto * left_identifier = planner_context->getColumnNodeIdentifierOrNull(using_join_columns_list.at(0))) + // used_identifiers.insert(*left_identifier); + // if (const auto * right_identifier = planner_context->getColumnNodeIdentifierOrNull(using_join_columns_list.at(1))) + // used_identifiers.insert(*right_identifier); + // } + // } + if (node->getNodeType() != QueryTreeNodeType::COLUMN) return; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 28789387d27..5a57d4e572d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1512,21 +1512,29 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } const Block & left_header = left_plan.getCurrentDataStream().header; - auto left_table_names = left_header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + const Block & right_header = right_plan.getCurrentDataStream().header; - auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + auto columns_from_left_table = left_header.getNamesAndTypesList(); + auto columns_from_right_table = right_header.getNamesAndTypesList(); - for (auto & column_from_joined_table : columns_from_joined_table) + table_join->setInputColumns(columns_from_left_table, columns_from_right_table); + + for (auto & column_from_joined_table : columns_from_left_table) { - /// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && outer_scope_columns.contains(column_from_joined_table.name)) - table_join->addJoinedColumn(column_from_joined_table); + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + } + + for (auto & column_from_joined_table : columns_from_right_table) + { + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + outer_scope_columns.contains(column_from_joined_table.name)) + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); } - const Block & right_header = right_plan.getCurrentDataStream().header; auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); auto result_plan = QueryPlan(); @@ -1625,6 +1633,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ settings[Setting::max_block_size], settings[Setting::max_threads], false /*optimize_read_in_order*/); + join_step->inner_table_selection_mode = settings.query_plan_join_inner_table_selection; join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 8fe2515e323..3f79a90149f 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -55,6 +55,9 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); + if (swap_streams) + std::swap(pipelines[0], pipelines[1]); + if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( @@ -63,7 +66,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines return joined_pipeline; } - return QueryPipelineBuilder::joinPipelinesRightLeft( + auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), std::move(pipelines[1]), join, @@ -72,6 +75,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines max_streams, keep_left_read_in_order, &processors); + return pipeline; } bool JoinStep::allowPushDownToRight() const @@ -100,10 +104,9 @@ void JoinStep::describeActions(JSONBuilder::JSONMap & map) const void JoinStep::updateOutputStream() { - output_stream = DataStream - { - .header = JoiningTransform::transformHeader(input_streams[0].header, join), - }; + const auto & header = swap_streams ? input_streams[1].header : input_streams[0].header; + const auto & result_header = JoiningTransform::transformHeader(header, join); + output_stream = DataStream { .header = result_header }; } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 51ea337b7c6..46fb49947ba 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -36,6 +37,9 @@ public: bool canUpdateInputStream() const override { return true; } + JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right; + bool swap_streams = false; + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 43f07ced696..b81346e0fa1 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -116,6 +116,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &); /// Returns the name of used projection or nullopt if no projection is used. std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp new file mode 100644 index 00000000000..11e1c8d191c --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -0,0 +1,100 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +static std::optional estimateReadRowsCount(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (const auto * reading = typeid_cast(step)) + { + if (auto analyzed_result = reading->getAnalyzedResult()) + return analyzed_result->selected_rows; + if (auto analyzed_result = reading->selectRangesToRead()) + return analyzed_result->selected_rows; + return {}; + } + + if (const auto * reading = typeid_cast(step)) + return reading->getStorage()->totalRows(Settings{}); + + if (node.children.size() != 1) + return {}; + + if (typeid_cast(step) || typeid_cast(step)) + return estimateReadRowsCount(*node.children.front()); + + return {}; +} + +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) +{ + auto * join_step = typeid_cast(node.step.get()); + if (!join_step || node.children.size() != 2) + return; + + const auto & join = join_step->getJoin(); + if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported() || typeid_cast(join.get())) + return; + + const auto & table_join = join->getTableJoin(); + auto kind = table_join.kind(); + if (table_join.hasUsing() + || table_join.strictness() != JoinStrictness::All + || (kind != JoinKind::Inner && kind != JoinKind::Left + && kind != JoinKind::Right && kind != JoinKind::Full)) + return; + + bool need_swap = false; + if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto) + { + auto lhs_extimation = estimateReadRowsCount(*node.children[0]); + auto rhs_extimation = estimateReadRowsCount(*node.children[1]); + LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}", + lhs_extimation.transform(toString).value_or("unknown"), + rhs_extimation.transform(toString).value_or("unknown")); + + if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) + need_swap = true; + } + else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left) + { + need_swap = true; + } + + if (!need_swap) + return; + + const auto & streams = join_step->getInputStreams(); + if (streams.size() != 2) + return; + + const auto & left_stream_input_header = streams.front().header; + const auto & right_stream_input_header = streams.back().header; + join_step->swap_streams = true; + + auto updated_table_join = std::make_shared(table_join); + updated_table_join->swapSides(); + auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); + join_step->setJoin(std::move(updated_join)); +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index f8504d84d12..a93f891eda2 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -226,6 +227,9 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); + if (frame.next_child == 0) + optimizeJoin(*frame.node, nodes); + /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index 238c1a3aad0..a9c2d2df2c4 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -35,6 +35,8 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + const StoragePtr & getStorage() const { return storage; } + private: static constexpr auto name = "ReadFromMemoryStorage"; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 810bae86cb0..be6a9a433a5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -919,6 +919,10 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), + "query_plan_join_inner_table_selection": lambda: random.choice( + ["left", "auto"] + # ["left", "auto", "right"] + ), } @staticmethod diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.reference b/tests/queries/0_stateless/02962_join_using_bug_57894.reference index 454655081df..fc6fe462205 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.reference +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.reference @@ -31,6 +31,7 @@ 8 9 \N +--- analyzer --- 0 1 2 diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index 96190241da5..e29347beb5e 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -21,6 +21,8 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; +SELECT '--- analyzer ---'; + SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL From 12e0b14d0ddbe58f4519c2cdcc877e7ca2818298 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Aug 2024 10:37:41 +0000 Subject: [PATCH 084/816] fix column not found --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 63edd837675..c6944313ee8 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1230,7 +1230,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, return {}; size_t left_columns_count = left_sample_block.columns(); - if (table_join->enableEnalyzer()) + if (table_join->enableEnalyzer() && !table_join->hasUsing()) left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 5753e37ff88..2a7e029ab00 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -100,18 +100,17 @@ Block HashJoinMethods::joinBlockImpl( added_columns.buildOutput(); const auto & table_join = join.table_join; - if (table_join->enableEnalyzer()) + std::set block_columns_to_erase; + if (table_join->enableEnalyzer() && !table_join->hasUsing()) { std::unordered_set left_output_columns; for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) left_output_columns.insert(out_column.name); - std::set to_erase; for (size_t i = 0; i < block.columns(); ++i) { if (!left_output_columns.contains(block.getByPosition(i).name)) - to_erase.insert(i); + block_columns_to_erase.insert(i); } - block.erase(to_erase); } size_t existing_columns = block.columns(); @@ -176,6 +175,7 @@ Block HashJoinMethods::joinBlockImpl( block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } } + block.erase(block_columns_to_erase); return remaining_block; } From 7605a76a06c68dd0780af697c52531bb850cae06 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Aug 2024 11:45:54 +0000 Subject: [PATCH 085/816] fix count() with query_plan_join_inner_table_selection --- src/Planner/PlannerJoinTree.cpp | 8 ++++++++ .../02514_analyzer_drop_join_on.reference | 10 ++-------- .../0_stateless/02514_analyzer_drop_join_on.sql | 1 + .../02835_join_step_explain.reference | 16 +++++++--------- .../0_stateless/02835_join_step_explain.sql | 2 ++ 5 files changed, 20 insertions(+), 17 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5a57d4e572d..1ffdf4e8c60 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1535,6 +1535,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); } + if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) + { + if (!columns_from_left_table.empty()) + table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left); + else if (!columns_from_right_table.empty()) + table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right); + } + auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); auto result_plan = QueryPlan(); diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 2c62e278050..59983fff778 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -12,20 +12,17 @@ Header: count() UInt64 Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String - __table2.b1 UInt64 __table3.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String + Header: __table1.a2 String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 @@ -106,7 +103,6 @@ Header: bx String Header: __table1.a2 String __table2.bx String __table4.c2 String - __table4.c1 UInt64 Expression Header: __table1.a2 String __table2.bx String @@ -115,7 +111,6 @@ Header: bx String Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table2.b1 UInt64 __table4.c2 String __table4.c1 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) @@ -123,8 +118,7 @@ Header: bx String __table2.bx String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String + Header: __table1.a2 String __table2.bx String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index df84e2f50b2..b10bf38e495 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,6 +16,7 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; +SET query_plan_join_inner_table_selection = 'right'; -- { echoOn } diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 06f4a9cfc99..31205956662 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -57,19 +57,17 @@ Header: id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT :: 2 -> __table1.value_2 UInt64 : 2 - INPUT : 3 -> __table2.value_1 String : 3 - INPUT :: 4 -> __table2.value_2 UInt64 : 4 - INPUT : 5 -> __table2.id UInt64 : 5 - ALIAS __table1.id :: 0 -> id UInt64 : 6 + INPUT : 2 -> __table2.value_1 String : 2 + INPUT :: 3 -> __table2.value_2 UInt64 : 3 + INPUT : 4 -> __table2.id UInt64 : 4 + ALIAS __table1.id :: 0 -> id UInt64 : 5 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 -Positions: 6 0 3 1 + ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 4 -> rhs.id UInt64 : 2 +Positions: 5 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table1.value_2 UInt64 __table2.value_1 String __table2.value_2 UInt64 __table2.id UInt64 diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index 1cdd3684a0b..b803ddbd911 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,6 +19,8 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); +SET query_plan_join_inner_table_selection = 'right'; + EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; From 0598419d5930054ac29030d62bb9c06823d53ae8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 28 Aug 2024 11:40:12 +0000 Subject: [PATCH 086/816] Fix 'auto' join with inner table selection --- src/Interpreters/HashJoin/HashJoin.cpp | 15 ++++++++++++--- src/Interpreters/HashJoin/HashJoin.h | 3 +++ src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 2 +- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index c6944313ee8..dad8a487745 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -383,6 +383,16 @@ size_t HashJoin::getTotalByteCount() const return res; } +bool HashJoin::isUsedByAnotherAlgorithm() const +{ + return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH); +} + +bool HashJoin::canRemoveColumnsFromLeftBlock() const +{ + return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm(); +} + void HashJoin::initRightBlockStructure(Block & saved_block_sample) { if (isCrossOrComma(kind)) @@ -394,8 +404,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || - table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || + bool save_key_columns = isUsedByAnotherAlgorithm() || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression(); @@ -1230,7 +1239,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, return {}; size_t left_columns_count = left_sample_block.columns(); - if (table_join->enableEnalyzer() && !table_join->hasUsing()) + if (canRemoveColumnsFromLeftBlock()) left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 4c1ebbcdc66..d5abdc2ddb8 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -464,6 +464,9 @@ private: bool empty() const; + bool isUsedByAnotherAlgorithm() const; + bool canRemoveColumnsFromLeftBlock() const; + void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 2a7e029ab00..ab522d94e37 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -101,7 +101,7 @@ Block HashJoinMethods::joinBlockImpl( const auto & table_join = join.table_join; std::set block_columns_to_erase; - if (table_join->enableEnalyzer() && !table_join->hasUsing()) + if (join.canRemoveColumnsFromLeftBlock()) { std::unordered_set left_output_columns; for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) From 2b82db289386181f2e73c63eee7e98002e9e49fa Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 Sep 2024 09:25:38 +0000 Subject: [PATCH 087/816] setting --- src/Core/Settings.cpp | 2 +- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/TableJoin.cpp | 3 ++- src/Planner/PlannerJoinTree.cpp | 4 +++- tests/clickhouse-test | 2 +- tests/integration/helpers/random_settings.py | 2 ++ 7 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 57dc297432a..4e63c3ae957 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -345,7 +345,7 @@ namespace ErrorCodes M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ M(Bool, single_join_prefer_left_table, true, "For single JOIN in case of identifier ambiguity prefer left table", IMPORTANT) \ \ - M(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, "auto", "Select the side of the join to be the inner table in the query plan. Possible values: 'auto', 'left', 'right'.", 0) \ + M(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, "Select the side of the join to be the inner table in the query plan. Possible values: 'auto', 'left', 'right'.", 0) \ M(UInt64, preferred_block_size_bytes, 1000000, "This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality.", 0) \ \ M(UInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6bb66039afb..c413d285ba1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -65,6 +65,7 @@ class WriteBuffer; M(CLASS_NAME, IntervalOutputFormat) \ M(CLASS_NAME, JoinAlgorithm) \ M(CLASS_NAME, JoinStrictness) \ + M(CLASS_NAME, JoinInnerTableSelectionMode) \ M(CLASS_NAME, LightweightMutationProjectionMode) \ M(CLASS_NAME, LoadBalancing) \ M(CLASS_NAME, LocalFSReadMethod) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8a79853c091..25954dc544c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -70,6 +70,7 @@ static std::initializer_listinner_table_selection_mode = settings.query_plan_join_inner_table_selection; + if (settings[Setting::query_plan_join_inner_table_selection]) + join_step->inner_table_selection_mode = JoinInnerTableSelectionMode::Auto; join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index be6a9a433a5..1c606bea228 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -788,7 +788,7 @@ def threshold_generator(always_on_prob, always_off_prob, min_val, max_val): def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) - +# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { "max_insert_threads": lambda: ( diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index b2319561fd7..49498b9f778 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -5,6 +5,8 @@ def randomize_settings(): yield "max_joined_block_size_rows", random.randint(8000, 100000) if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) + if random.random() < 0.5: + yield "query_plan_join_inner_table_selection", random.choice(["auto", "left", "right"]) def write_random_settings_config(destination): From da2e6aeb32822416136195eb5d98f831fcbdb921 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 Sep 2024 11:50:27 +0000 Subject: [PATCH 088/816] join step swap header --- src/Interpreters/ConcurrentHashJoin.h | 7 ++ src/Interpreters/TableJoin.cpp | 4 +- src/Planner/CollectColumnIdentifiers.cpp | 21 ------ src/Processors/QueryPlan/JoinStep.cpp | 69 ++++++++++++++++++- .../QueryPlan/Optimizations/optimizeJoin.cpp | 2 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 1 - 6 files changed, 78 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index a911edaccc3..355218554ce 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -60,6 +60,13 @@ public: IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + + bool isCloneSupported() const override { return true; } + std::shared_ptr clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override + { + return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); + } + private: struct InternalHashJoin { diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index d4304df313c..555aaff2e06 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -209,8 +209,8 @@ size_t TableJoin::rightKeyInclusion(const String & name) const void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) { - columns_from_left_table = left_output_columns; - columns_from_joined_table = right_output_columns; + columns_from_left_table = std::move(left_output_columns); + columns_from_joined_table = std::move(right_output_columns); } diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index ca468a353b2..dd5bdd4d141 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -34,27 +34,6 @@ public: void visitImpl(const QueryTreeNodePtr & node) { - // if (node->getNodeType() == QueryTreeNodeType::QUERY) - // { - // const auto * join_node = node->as().getJoinTree()->as(); - // if (!join_node || !join_node->isUsingJoinExpression()) - // return; - - // const auto & using_list = join_node->getJoinExpression()->as(); - - // for (const auto & join_using_node : using_list.getNodes()) - // { - // const auto & join_using_expression = join_using_node->as().getExpression(); - // if (!join_using_expression) - // return; - // const auto & using_join_columns_list = join_using_expression->as().getNodes(); - // if (const auto * left_identifier = planner_context->getColumnNodeIdentifierOrNull(using_join_columns_list.at(0))) - // used_identifiers.insert(*left_identifier); - // if (const auto * right_identifier = planner_context->getColumnNodeIdentifierOrNull(using_join_columns_list.at(1))) - // used_identifiers.insert(*right_identifier); - // } - // } - if (node->getNodeType() != QueryTreeNodeType::COLUMN) return; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 3f79a90149f..0e9332c186e 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -36,6 +37,53 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } +size_t getPrefixLength(const NameSet & prefix, const Names & names) +{ + size_t i = 0; + for (; i < names.size(); ++i) + { + if (!prefix.contains(names[i])) + break; + } + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}: [{}] [{}] -> {}", __FILE__, __LINE__, fmt::join(names, ", "), fmt::join(prefix, ", "), i); + return i; +} + +std::vector getPermutationToRotate(size_t prefix_size, size_t total_size) +{ + std::vector permutation(total_size); + size_t i = prefix_size; + for (auto & elem : permutation) + { + elem = i; + i = (i + 1) % total_size; + } + return permutation; +} + +Block rotateBlock(const Block & block, size_t prefix_size) +{ + auto columns = block.getColumnsWithTypeAndName(); + std::rotate(columns.begin(), columns.begin() + prefix_size, columns.end()); + auto res = Block(std::move(columns)); + return res; +} + +NameSet getNameSetFromBlock(const Block & block) +{ + NameSet names; + for (const auto & column : block) + names.insert(column.name); + return names; +} + +Block rotateBlock(const Block & block, const Block & prefix_block) +{ + NameSet prefix_names_set = getNameSetFromBlock(prefix_block); + size_t prefix_size = getPrefixLength(prefix_names_set, block.getNames()); + return rotateBlock(block, prefix_size); +} + } JoinStep::JoinStep( @@ -55,6 +103,8 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); + NameSet rhs_names = getNameSetFromBlock(pipelines[1]->getHeader()); + if (swap_streams) std::swap(pipelines[0], pipelines[1]); @@ -75,6 +125,18 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines max_streams, keep_left_read_in_order, &processors); + + const auto & result_names = pipeline->getHeader().getNames(); + size_t prefix_size = getPrefixLength(rhs_names, result_names); + if (0 < prefix_size && prefix_size < result_names.size()) + { + auto column_permutation = getPermutationToRotate(prefix_size, result_names.size()); + pipeline->addSimpleTransform([column_perm = std::move(column_permutation)](const Block & header) + { + return std::make_shared(header, std::move(column_perm)); + }); + } + return pipeline; } @@ -105,7 +167,12 @@ void JoinStep::describeActions(JSONBuilder::JSONMap & map) const void JoinStep::updateOutputStream() { const auto & header = swap_streams ? input_streams[1].header : input_streams[0].header; - const auto & result_header = JoiningTransform::transformHeader(header, join); + + Block result_header = JoiningTransform::transformHeader(header, join); + + if (swap_streams) + result_header = rotateBlock(result_header, input_streams[1].header); + output_stream = DataStream { .header = result_header }; } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index 11e1c8d191c..8074304de52 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -52,7 +52,7 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) return; const auto & join = join_step->getJoin(); - if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported() || typeid_cast(join.get())) + if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported()) return; const auto & table_join = join->getTableJoin(); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index a93f891eda2..d58720268a6 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include From 35cf3e8b91ce20bcfd6218d8d34a5f0a96fdd03e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Sep 2024 13:34:25 +0000 Subject: [PATCH 089/816] fix stylecheck Signed-off-by: vdimir --- src/Planner/PlannerJoinTree.cpp | 5 ++--- tests/integration/helpers/random_settings.py | 4 +++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1ee0024f053..543dc1a88f6 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,7 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; - extern const SettingsBool query_plan_join_inner_table_selection; + extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection; } namespace ErrorCodes @@ -1642,8 +1642,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ settings[Setting::max_block_size], settings[Setting::max_threads], false /*optimize_read_in_order*/); - if (settings[Setting::query_plan_join_inner_table_selection]) - join_step->inner_table_selection_mode = JoinInnerTableSelectionMode::Auto; + join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index 49498b9f778..a34d8e93c47 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -6,7 +6,9 @@ def randomize_settings(): if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) if random.random() < 0.5: - yield "query_plan_join_inner_table_selection", random.choice(["auto", "left", "right"]) + yield "query_plan_join_inner_table_selection", random.choice( + ["auto", "left", "right"] + ) def write_random_settings_config(destination): From 3ee6fd9b059d4afc67bc154c90746a1e0ec51bd9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Sep 2024 14:19:20 +0000 Subject: [PATCH 090/816] Fix header --- src/Processors/QueryPlan/JoinStep.cpp | 5 +++-- src/Processors/QueryPlan/JoinStep.h | 2 ++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 0e9332c186e..fefb193827f 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -111,7 +111,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); joined_pipeline->resize(max_streams); return joined_pipeline; } @@ -120,7 +120,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines std::move(pipelines[0]), std::move(pipelines[1]), join, - output_stream->header, + join_algorithm_header, max_block_size, max_streams, keep_left_read_in_order, @@ -170,6 +170,7 @@ void JoinStep::updateOutputStream() Block result_header = JoiningTransform::transformHeader(header, join); + join_algorithm_header = result_header; if (swap_streams) result_header = rotateBlock(result_header, input_streams[1].header); diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 46fb49947ba..96c02f9fd19 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -42,6 +42,8 @@ public: private: void updateOutputStream() override; + /// Header that expected to be returned from IJoin + Block join_algorithm_header; JoinPtr join; size_t max_block_size; From fca592a31fc0c233fb971deff211e1ce7c040cfa Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Sep 2024 14:20:45 +0000 Subject: [PATCH 091/816] fix stylecheck --- tests/clickhouse-test | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1c606bea228..06a044eef32 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -788,6 +788,7 @@ def threshold_generator(always_on_prob, always_off_prob, min_val, max_val): def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) + # Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { @@ -2154,9 +2155,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( - all_tags_and_random_settings_limits[1] - ) + self.all_random_settings_limits: Dict[ + str, Dict[str, (int, int)] + ] = all_tags_and_random_settings_limits[1] self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From d39d9a876537c603626a6fbe478d32e0a208275b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 26 Sep 2024 14:30:06 +0000 Subject: [PATCH 092/816] Automatic style fix --- tests/clickhouse-test | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 06a044eef32..c1615d039cb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2155,9 +2155,9 @@ class TestSuite: ) ) self.all_tags: Dict[str, Set[str]] = all_tags_and_random_settings_limits[0] - self.all_random_settings_limits: Dict[ - str, Dict[str, (int, int)] - ] = all_tags_and_random_settings_limits[1] + self.all_random_settings_limits: Dict[str, Dict[str, (int, int)]] = ( + all_tags_and_random_settings_limits[1] + ) self.sequential_tests = [] self.parallel_tests = [] for test_name in self.all_tests: From 9642e6cdcc4d2e700efa43ebbf523ed25f728cd8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Sep 2024 15:01:59 +0000 Subject: [PATCH 093/816] add ColumnPermuteTransform --- .../Transforms/ColumnPermuteTransform.cpp | 49 +++++++++++++++++++ .../Transforms/ColumnPermuteTransform.h | 28 +++++++++++ 2 files changed, 77 insertions(+) create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.cpp create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.h diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp new file mode 100644 index 00000000000..ac7793bd136 --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +namespace +{ + +template +void applyPermutation(std::vector & data, const std::vector & permutation) +{ + std::vector res; + res.reserve(data.size()); + for (size_t i = 0; i < data.size(); ++i) + res.emplace_back(std::move(data[permutation[i]])); + data = std::move(res); +} + +Block permuteBlock(const Block & block, const std::vector & permutation) +{ + auto columns = block.getColumnsWithTypeAndName(); + applyPermutation(columns, permutation); + return Block(columns); +} + +void permuteChunk(Chunk & chunk, const std::vector & permutation) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + applyPermutation(columns, permutation); + chunk.setColumns(std::move(columns), num_rows); +} + +} + +ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, std::vector permutation_) + : ISimpleTransform(header_, permuteBlock(header_, permutation_), false) + , permutation(std::move(permutation_)) +{ +} + + +void ColumnPermuteTransform::transform(Chunk & chunk) +{ + permuteChunk(chunk, permutation); +} + + +} diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h new file mode 100644 index 00000000000..b2e3c469833 --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ColumnPermuteTransform : public ISimpleTransform +{ +public: + ColumnPermuteTransform(const Block & header_, std::vector permutation_); + + String getName() const override { return "ColumnPermuteTransform"; } + + void transform(Chunk & chunk) override; + +private: + Names column_names; + std::vector permutation; +}; + + +} From 0e9847d1ce138975ca18490494007519223ddcd1 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Fri, 27 Sep 2024 01:46:09 +0000 Subject: [PATCH 094/816] Remove sleep in test_ddl_worker_replicas test --- tests/integration/test_ddl_worker_replicas/test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py index f9ce2575e00..28ec558df31 100644 --- a/tests/integration/test_ddl_worker_replicas/test.py +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -56,9 +56,14 @@ def test_ddl_worker_replicas(started_cluster): assert len(parts[2]) != 0 node4.stop() - time.sleep(1) - result = node1.query( + # wait for node4 active path is removed + node1.query_with_retry( + sql = f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'", + check_callback= lambda result: result == 0, + ) + + result = node1.query_with_retry( f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" ).strip() From d43cb48f4ad37e42bd85072dd43503c77131a2b3 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Fri, 27 Sep 2024 02:06:27 +0000 Subject: [PATCH 095/816] Reformat the test --- tests/integration/test_ddl_worker_replicas/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py index 28ec558df31..fecb7d97d25 100644 --- a/tests/integration/test_ddl_worker_replicas/test.py +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -59,10 +59,10 @@ def test_ddl_worker_replicas(started_cluster): # wait for node4 active path is removed node1.query_with_retry( - sql = f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'", - check_callback= lambda result: result == 0, + sql=f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'", + check_callback=lambda result: result == 0, ) - + result = node1.query_with_retry( f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" ).strip() From fe5276d898256c9d38f60f3483a25f9556015751 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Fri, 27 Sep 2024 11:28:52 +0000 Subject: [PATCH 096/816] Not re-create an ephemeral node if it exists --- src/Interpreters/DDLWorker.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 7e56e0facfb..e29826c6c54 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -1311,11 +1311,11 @@ void DDLWorker::markReplicasActive(bool reinitialized) continue; } - /// Create "active" node (remove previous one if necessary) String active_path = fs::path(replicas_dir) / host_id / "active"; - String active_id = toString(ServerUUID::get()); - zookeeper->deleteEphemeralNodeIfContentMatches(active_path, active_id); + if (zookeeper->exists(active_path)) + continue; + String active_id = toString(ServerUUID::get()); LOG_TRACE(log, "Trying to mark a replica active: active_path={}, active_id={}", active_path, active_id); zookeeper->create(active_path, active_id, zkutil::CreateMode::Ephemeral); From 3c8594d401d7c625a810a61776e689083d79912a Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 27 Sep 2024 14:30:07 +0000 Subject: [PATCH 097/816] Remove unnecessary changes --- .../DataLakes/DataLakeConfiguration.h | 86 +++++++++ .../DataLakes/DeltaLakeMetadata.cpp | 40 ++-- .../DataLakes/DeltaLakeMetadata.h | 12 +- .../ObjectStorage/DataLakes/HudiMetadata.cpp | 12 +- .../ObjectStorage/DataLakes/HudiMetadata.h | 8 +- .../DataLakes/IStorageDataLake.h | 172 ------------------ .../DataLakes/IcebergMetadata.cpp | 24 +-- .../ObjectStorage/DataLakes/IcebergMetadata.h | 8 +- .../DataLakes/registerDataLakeStorages.cpp | 132 -------------- .../ObjectStorage/StorageObjectStorage.cpp | 18 +- .../ObjectStorage/StorageObjectStorage.h | 21 ++- .../registerStorageObjectStorage.cpp | 105 +++++++++++ src/TableFunctions/ITableFunctionDataLake.h | 120 ------------ .../TableFunctionObjectStorage.cpp | 90 +++++++++ .../TableFunctionObjectStorage.h | 55 ++++++ .../registerDataLakeTableFunctions.cpp | 88 --------- 16 files changed, 407 insertions(+), 584 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h delete mode 100644 src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h delete mode 100644 src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp delete mode 100644 src/TableFunctions/ITableFunctionDataLake.h delete mode 100644 src/TableFunctions/registerDataLakeTableFunctions.cpp diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h new file mode 100644 index 00000000000..6d8e64aa3b7 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -0,0 +1,86 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + +# include + + +namespace DB +{ + +template +concept StorageConfiguration = std::derived_from; + +template +class DataLakeConfiguration : public BaseStorageConfiguration, public std::enable_shared_from_this +{ +public: + using Configuration = StorageObjectStorage::Configuration; + + bool isDataLakeConfiguration() const override { return true; } + + std::string getEngineName() const override { return DataLakeMetadata::name; } + + void update(ObjectStoragePtr object_storage, ContextPtr local_context) override + { + auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); + if (current_metadata && *current_metadata == *new_metadata) + return; + + current_metadata = std::move(new_metadata); + BaseStorageConfiguration::setPaths(current_metadata->getDataFiles()); + BaseStorageConfiguration::setPartitionColumns(current_metadata->getPartitionColumns()); + } + +private: + DataLakeMetadataPtr current_metadata; + + ReadFromFormatInfo prepareReadingFromFormat( + ObjectStoragePtr object_storage, + const Strings & requested_columns, + const StorageSnapshotPtr & storage_snapshot, + bool supports_subset_of_columns, + ContextPtr local_context) override + { + auto info = DB::prepareReadingFromFormat(requested_columns, storage_snapshot, supports_subset_of_columns); + if (!current_metadata) + { + current_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); + } + auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); + if (!column_mapping.empty()) + { + for (const auto & [column_name, physical_name] : column_mapping) + { + auto & column = info.format_header.getByName(column_name); + column.name = physical_name; + } + } + return info; + } +}; + +using StorageS3IcebergConfiguration = DataLakeConfiguration; +using StorageAzureIcebergConfiguration = DataLakeConfiguration; +using StorageLocalIcebergConfiguration = DataLakeConfiguration; +using StorageS3DeltaLakeConfiguration = DataLakeConfiguration; +using StorageS3HudiConfiguration = DataLakeConfiguration; + + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index f04e868ee5a..f437faa2e90 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -55,22 +55,18 @@ namespace ErrorCodes struct DeltaLakeMetadataImpl { - using ConfigurationPtr = DeltaLakeMetadata::ConfigurationPtr; + using ConfigurationObservePtr = DeltaLakeMetadata::ConfigurationObservePtr; ObjectStoragePtr object_storage; - ConfigurationPtr configuration; + ConfigurationObservePtr configuration; ContextPtr context; /** * Useful links: * - https://github.com/delta-io/delta/blob/master/PROTOCOL.md#data-files */ - DeltaLakeMetadataImpl(ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - ContextPtr context_) - : object_storage(object_storage_) - , configuration(configuration_) - , context(context_) + DeltaLakeMetadataImpl(ObjectStoragePtr object_storage_, ConfigurationObservePtr configuration_, ContextPtr context_) + : object_storage(object_storage_), configuration(configuration_), context(context_) { } @@ -110,6 +106,7 @@ struct DeltaLakeMetadataImpl }; DeltaLakeMetadata processMetadataFiles() { + auto configuration_ptr = configuration.lock(); std::set result_files; NamesAndTypesList current_schema; DataLakePartitionColumns current_partition_columns; @@ -121,7 +118,7 @@ struct DeltaLakeMetadataImpl while (true) { const auto filename = withPadding(++current_version) + metadata_file_suffix; - const auto file_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / filename; + const auto file_path = std::filesystem::path(configuration_ptr->getPath()) / deltalake_metadata_directory / filename; if (!object_storage->exists(StoredObject(file_path))) break; @@ -135,7 +132,7 @@ struct DeltaLakeMetadataImpl } else { - const auto keys = listFiles(*object_storage, *configuration, deltalake_metadata_directory, metadata_file_suffix); + const auto keys = listFiles(*object_storage, *configuration_ptr, deltalake_metadata_directory, metadata_file_suffix); for (const String & key : keys) processMetadataFile(key, current_schema, current_partition_columns, result_files); } @@ -244,6 +241,8 @@ struct DeltaLakeMetadataImpl } } + auto configuration_ptr = configuration.lock(); + if (object->has("add")) { auto add_object = object->get("add").extract(); @@ -251,7 +250,7 @@ struct DeltaLakeMetadataImpl throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to extract `add` field"); auto path = add_object->getValue("path"); - result.insert(fs::path(configuration->getPath()) / path); + result.insert(fs::path(configuration_ptr->getPath()) / path); auto filename = fs::path(path).filename().string(); auto it = file_partition_columns.find(filename); @@ -295,7 +294,7 @@ struct DeltaLakeMetadataImpl throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to extract `remove` field"); auto path = remove_object->getValue("path"); - result.erase(fs::path(configuration->getPath()) / path); + result.erase(fs::path(configuration_ptr->getPath()) / path); } } } @@ -486,7 +485,9 @@ struct DeltaLakeMetadataImpl */ size_t readLastCheckpointIfExists() const { - const auto last_checkpoint_file = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; + auto configuration_ptr = configuration.lock(); + const auto last_checkpoint_file + = std::filesystem::path(configuration_ptr->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; if (!object_storage->exists(StoredObject(last_checkpoint_file))) return 0; @@ -552,7 +553,11 @@ struct DeltaLakeMetadataImpl return 0; const auto checkpoint_filename = withPadding(version) + ".checkpoint.parquet"; - const auto checkpoint_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / checkpoint_filename; + + auto configuration_ptr = configuration.lock(); + + const auto checkpoint_path + = std::filesystem::path(configuration_ptr->getPath()) / deltalake_metadata_directory / checkpoint_filename; LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); @@ -667,7 +672,7 @@ struct DeltaLakeMetadataImpl } LOG_TEST(log, "Adding {}", path); - const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / path); + const auto [_, inserted] = result.insert(std::filesystem::path(configuration_ptr->getPath()) / path); if (!inserted) throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", path); } @@ -678,10 +683,7 @@ struct DeltaLakeMetadataImpl LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; -DeltaLakeMetadata::DeltaLakeMetadata( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - ContextPtr context_) +DeltaLakeMetadata::DeltaLakeMetadata(ObjectStoragePtr object_storage_, ConfigurationObservePtr configuration_, ContextPtr context_) { auto impl = DeltaLakeMetadataImpl(object_storage_, configuration_, context_); auto result = impl.processMetadataFiles(); diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index a479a3dd293..549443f115e 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -12,13 +12,10 @@ namespace DB class DeltaLakeMetadata final : public IDataLakeMetadata { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ConfigurationObservePtr = StorageObjectStorage::ConfigurationObservePtr; static constexpr auto name = "DeltaLake"; - DeltaLakeMetadata( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - ContextPtr context_); + DeltaLakeMetadata(ObjectStoragePtr object_storage_, ConfigurationObservePtr configuration_, ContextPtr context_); Strings getDataFiles() const override { return data_files; } @@ -36,10 +33,7 @@ public: && data_files == deltalake_metadata->data_files; } - static DataLakeMetadataPtr create( - ObjectStoragePtr object_storage, - ConfigurationPtr configuration, - ContextPtr local_context) + static DataLakeMetadataPtr create(ObjectStoragePtr object_storage, ConfigurationObservePtr configuration, ContextPtr local_context) { return std::make_unique(object_storage, configuration, local_context); } diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index 91a586ccbf9..8a93a0ea6d3 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -43,8 +43,9 @@ namespace ErrorCodes */ Strings HudiMetadata::getDataFilesImpl() const { + auto configuration_ptr = configuration.lock(); auto log = getLogger("HudiMetadata"); - const auto keys = listFiles(*object_storage, *configuration, "", Poco::toLower(configuration->format)); + const auto keys = listFiles(*object_storage, *configuration_ptr, "", Poco::toLower(configuration_ptr->format)); using Partition = std::string; using FileID = std::string; @@ -86,13 +87,8 @@ Strings HudiMetadata::getDataFilesImpl() const return result; } -HudiMetadata::HudiMetadata( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - ContextPtr context_) - : WithContext(context_) - , object_storage(object_storage_) - , configuration(configuration_) +HudiMetadata::HudiMetadata(ObjectStoragePtr object_storage_, ConfigurationObservePtr configuration_, ContextPtr context_) + : WithContext(context_), object_storage(object_storage_), configuration(configuration_) { } diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h index b060b1b0d39..b22dfacb0ad 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -13,13 +13,13 @@ namespace DB class HudiMetadata final : public IDataLakeMetadata, private WithContext { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ConfigurationObservePtr = StorageObjectStorage::ConfigurationObservePtr; static constexpr auto name = "Hudi"; HudiMetadata( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + ConfigurationObservePtr configuration_, ContextPtr context_); Strings getDataFiles() const override; @@ -40,7 +40,7 @@ public: static DataLakeMetadataPtr create( ObjectStoragePtr object_storage, - ConfigurationPtr configuration, + ConfigurationObservePtr configuration, ContextPtr local_context) { return std::make_unique(object_storage, configuration, local_context); @@ -48,7 +48,7 @@ public: private: const ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const ConfigurationObservePtr configuration; mutable Strings data_files; std::unordered_map column_name_to_physical_name; DataLakePartitionColumns partition_columns; diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h deleted file mode 100644 index a17fd163253..00000000000 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ /dev/null @@ -1,172 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AVRO - -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -/// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) -/// Right now it's implemented on top of StorageS3 and right now it doesn't support -/// many Iceberg features like schema evolution, partitioning, positional and equality deletes. -template -class IStorageDataLake final : public StorageObjectStorage -{ -public: - using Storage = StorageObjectStorage; - using ConfigurationPtr = Storage::ConfigurationPtr; - - static StoragePtr create( - ConfigurationPtr base_configuration, - ContextPtr context, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment_, - std::optional format_settings_, - LoadingStrictnessLevel mode) - { - auto object_storage = base_configuration->createObjectStorage(context, /* is_readonly */true); - DataLakeMetadataPtr metadata; - NamesAndTypesList schema_from_metadata; - const bool use_schema_from_metadata = columns_.empty(); - - if (base_configuration->format == "auto") - base_configuration->format = "Parquet"; - - ConfigurationPtr configuration = base_configuration->clone(); - - try - { - metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - configuration->setPaths(metadata->getDataFiles()); - if (use_schema_from_metadata) - schema_from_metadata = metadata->getTableSchema(); - } - catch (...) - { - if (mode <= LoadingStrictnessLevel::CREATE) - throw; - - metadata.reset(); - configuration->setPaths({}); - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - return std::make_shared>( - base_configuration, std::move(metadata), configuration, object_storage, - context, table_id_, - use_schema_from_metadata ? ColumnsDescription(schema_from_metadata) : columns_, - constraints_, comment_, format_settings_); - } - - String getName() const override { return DataLakeMetadata::name; } - - static ColumnsDescription getTableStructureFromData( - ObjectStoragePtr object_storage_, - ConfigurationPtr base_configuration, - const std::optional & format_settings_, - ContextPtr local_context) - { - auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); - - auto schema_from_metadata = metadata->getTableSchema(); - if (!schema_from_metadata.empty()) - { - return ColumnsDescription(std::move(schema_from_metadata)); - } - else - { - ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(metadata->getDataFiles()); - std::string sample_path; - return Storage::resolveSchemaFromData( - object_storage_, configuration, format_settings_, sample_path, local_context); - } - } - - void updateConfiguration(ContextPtr local_context) override - { - Storage::updateConfiguration(local_context); - - auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - if (current_metadata && *current_metadata == *new_metadata) - return; - - current_metadata = std::move(new_metadata); - auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(current_metadata->getDataFiles()); - updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); - - Storage::configuration = updated_configuration; - } - - template - IStorageDataLake( - ConfigurationPtr base_configuration_, - DataLakeMetadataPtr metadata_, - Args &&... args) - : Storage(std::forward(args)...) - , base_configuration(base_configuration_) - , current_metadata(std::move(metadata_)) - { - if (base_configuration->format == "auto") - { - base_configuration->format = Storage::configuration->format; - } - - if (current_metadata) - { - const auto & columns = current_metadata->getPartitionColumns(); - base_configuration->setPartitionColumns(columns); - Storage::configuration->setPartitionColumns(columns); - } - } - -private: - ConfigurationPtr base_configuration; - DataLakeMetadataPtr current_metadata; - - ReadFromFormatInfo prepareReadingFromFormat( - const Strings & requested_columns, - const StorageSnapshotPtr & storage_snapshot, - bool supports_subset_of_columns, - ContextPtr local_context) override - { - auto info = DB::prepareReadingFromFormat(requested_columns, storage_snapshot, supports_subset_of_columns); - if (!current_metadata) - { - Storage::updateConfiguration(local_context); - current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - } - auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); - if (!column_mapping.empty()) - { - for (const auto & [column_name, physical_name] : column_mapping) - { - auto & column = info.format_header.getByName(column_name); - column.name = physical_name; - } - } - return info; - } -}; - -using StorageIceberg = IStorageDataLake; -using StorageDeltaLake = IStorageDataLake; -using StorageHudi = IStorageDataLake; - -} - -#endif diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index ffc4dd09a3a..11ff749fd9d 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -50,7 +50,7 @@ extern const int UNSUPPORTED_METHOD; IcebergMetadata::IcebergMetadata( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + ConfigurationObservePtr configuration_, DB::ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -381,12 +381,12 @@ std::pair getMetadataFileAndVersion( } -DataLakeMetadataPtr IcebergMetadata::create( - ObjectStoragePtr object_storage, - ConfigurationPtr configuration, - ContextPtr local_context) +DataLakeMetadataPtr +IcebergMetadata::create(ObjectStoragePtr object_storage, ConfigurationObservePtr configuration, ContextPtr local_context) { - const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); + auto configuration_ptr = configuration.lock(); + + const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration_ptr); LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); auto read_settings = local_context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); @@ -411,12 +411,13 @@ DataLakeMetadataPtr IcebergMetadata::create( if (snapshot->getValue("snapshot-id") == current_snapshot_id) { const auto path = snapshot->getValue("manifest-list"); - manifest_list_file = std::filesystem::path(configuration->getPath()) / "metadata" / std::filesystem::path(path).filename(); + manifest_list_file = std::filesystem::path(configuration_ptr->getPath()) / "metadata" / std::filesystem::path(path).filename(); break; } } - return std::make_unique(object_storage, configuration, local_context, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique( + object_storage, configuration_ptr, local_context, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -446,6 +447,7 @@ DataLakeMetadataPtr IcebergMetadata::create( */ Strings IcebergMetadata::getDataFiles() const { + auto configuration_ptr = configuration.lock(); if (!data_files.empty()) return data_files; @@ -478,7 +480,7 @@ Strings IcebergMetadata::getDataFiles() const { const auto file_path = col_str->getDataAt(i).toView(); const auto filename = std::filesystem::path(file_path).filename(); - manifest_files.emplace_back(std::filesystem::path(configuration->getPath()) / "metadata" / filename); + manifest_files.emplace_back(std::filesystem::path(configuration_ptr->getPath()) / "metadata" / filename); } NameSet files; @@ -612,9 +614,9 @@ Strings IcebergMetadata::getDataFiles() const const auto status = status_int_column->getInt(i); const auto data_path = std::string(file_path_string_column->getDataAt(i).toView()); - const auto pos = data_path.find(configuration->getPath()); + const auto pos = data_path.find(configuration_ptr->getPath()); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration->getPath(), data_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration_ptr->getPath(), data_path); const auto file_path = data_path.substr(pos); diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index 7b0deab91c3..7811bcd8b4b 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -61,13 +61,13 @@ namespace DB class IcebergMetadata : public IDataLakeMetadata, private WithContext { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ConfigurationObservePtr = StorageObjectStorage::ConfigurationObservePtr; static constexpr auto name = "Iceberg"; IcebergMetadata( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + ConfigurationObservePtr configuration_, ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -94,14 +94,14 @@ public: static DataLakeMetadataPtr create( ObjectStoragePtr object_storage, - ConfigurationPtr configuration, + ConfigurationObservePtr configuration, ContextPtr local_context); private: size_t getVersion() const { return metadata_version; } const ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const ConfigurationObservePtr configuration; Int32 metadata_version; Int32 format_version; String manifest_list_file; diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp deleted file mode 100644 index f0bd51de375..00000000000 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ /dev/null @@ -1,132 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -# include -# include -# include -# include -# include -# include - - -namespace DB -{ - -#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. - -void registerStorageIceberg(StorageFactory & factory) -{ - factory.registerStorage( - "Iceberg", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return StorageIceberg::create( - configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); - - factory.registerStorage( - "IcebergS3", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return StorageIceberg::create( - configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); - - factory.registerStorage( - "IcebergAzure", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true); - - return StorageIceberg::create( - configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::AZURE, - }); - - factory.registerStorage( - "IcebergLocal", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return StorageIceberg::create( - configuration, args.getContext(), args.table_id, args.columns, - args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::FILE, - }); -} - -#endif - -#if USE_PARQUET -void registerStorageDeltaLake(StorageFactory & factory) -{ - factory.registerStorage( - "DeltaLake", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return StorageDeltaLake::create( - configuration, args.getContext(), args.table_id, args.columns, - args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); -} -#endif - -void registerStorageHudi(StorageFactory & factory) -{ - factory.registerStorage( - "Hudi", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return StorageHudi::create( - configuration, args.getContext(), args.table_id, args.columns, - args.constraints, args.comment, std::nullopt, args.mode); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); -} - -} - -#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index bc27820707c..f62e0fe20dc 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -124,12 +124,11 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); } -void StorageObjectStorage::updateConfiguration(ContextPtr context) +void StorageObjectStorage::Configuration::update(ObjectStoragePtr object_storage_ptr, ContextPtr context) { - IObjectStorage::ApplyNewSettingsOptions options{ .allow_client_change = !configuration->isStaticConfiguration() }; - object_storage->applyNewSettings(context->getConfigRef(), configuration->getTypeName() + ".", context, options); + IObjectStorage::ApplyNewSettingsOptions options{.allow_client_change = !isStaticConfiguration()}; + object_storage_ptr->applyNewSettings(context->getConfigRef(), getTypeName() + ".", context, options); } - namespace { class ReadFromObjectStorageStep : public SourceStepWithFilter @@ -243,7 +242,8 @@ private: }; } -ReadFromFormatInfo StorageObjectStorage::prepareReadingFromFormat( +ReadFromFormatInfo StorageObjectStorage::Configuration::prepareReadingFromFormat( + ObjectStoragePtr, const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns, @@ -262,7 +262,7 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - updateConfiguration(local_context); + configuration->update(object_storage, local_context); if (partition_by && configuration->withPartitionWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, @@ -270,8 +270,8 @@ void StorageObjectStorage::read( getName()); } - const auto read_from_format_info = prepareReadingFromFormat( - column_names, storage_snapshot, supportsSubsetOfColumns(local_context), local_context); + const auto read_from_format_info = configuration->prepareReadingFromFormat( + object_storage, column_names, storage_snapshot, supportsSubsetOfColumns(local_context), local_context); const bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef()[Setting::optimize_count_from_files]; @@ -300,7 +300,7 @@ SinkToStoragePtr StorageObjectStorage::write( ContextPtr local_context, bool /* async_insert */) { - updateConfiguration(local_context); + configuration->update(object_storage, local_context); const auto sample_block = metadata_snapshot->getSampleBlock(); const auto & settings = configuration->getQuerySettings(local_context); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index f39586c23b4..9781d5dbe6e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -25,6 +25,7 @@ class StorageObjectStorage : public IStorage public: class Configuration; using ConfigurationPtr = std::shared_ptr; + using ConfigurationObservePtr = std::weak_ptr; using ObjectInfo = RelativePathWithMetadata; using ObjectInfoPtr = std::shared_ptr; using ObjectInfos = std::vector; @@ -120,16 +121,8 @@ public: const ContextPtr & context); protected: - virtual void updateConfiguration(ContextPtr local_context); - String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context); - virtual ReadFromFormatInfo prepareReadingFromFormat( - const Strings & requested_columns, - const StorageSnapshotPtr & storage_snapshot, - bool supports_subset_of_columns, - ContextPtr local_context); - static std::unique_ptr createReadBufferIterator( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, @@ -206,14 +199,26 @@ public: void setPartitionColumns(const DataLakePartitionColumns & columns) { partition_columns = columns; } const DataLakePartitionColumns & getPartitionColumns() const { return partition_columns; } + virtual bool isDataLakeConfiguration() const { return false; } + + virtual ReadFromFormatInfo prepareReadingFromFormat( + ObjectStoragePtr object_storage, + const Strings & requested_columns, + const StorageSnapshotPtr & storage_snapshot, + bool supports_subset_of_columns, + ContextPtr local_context); + String format = "auto"; String compression_method = "auto"; String structure = "auto"; + virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context); + protected: virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + void assertInitialized() const; bool initialized = false; diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index d0cacc29adf..570e888da91 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -148,4 +149,108 @@ void registerStorageObjectStorage(StorageFactory & factory) UNUSED(factory); } +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. + +void registerStorageIceberg(StorageFactory & factory) +{ + factory.registerStorage( + "Iceberg", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergS3", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergAzure", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); + + factory.registerStorage( + "IcebergLocal", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, + }); +} + +#endif + +#if USE_PARQUET +void registerStorageDeltaLake(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} +#endif + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage( + "Hudi", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h deleted file mode 100644 index db8287f97bf..00000000000 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ /dev/null @@ -1,120 +0,0 @@ -#pragma once - -#include "config.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -template -class ITableFunctionDataLake : public TableFunction -{ -public: - static constexpr auto name = Name::name; - std::string getName() const override { return name; } - -protected: - StoragePtr executeImpl( - const ASTPtr & /* ast_function */, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool /*is_insert_query*/) const override - { - ColumnsDescription columns; - auto configuration = TableFunction::getConfiguration(); - if (configuration->structure != "auto") - columns = parseColumnsListFromString(configuration->structure, context); - else if (!cached_columns.empty()) - columns = cached_columns; - - StoragePtr storage = Storage::create( - configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), - columns, ConstraintsDescription{}, String{}, std::nullopt, LoadingStrictnessLevel::CREATE); - - storage->startup(); - return storage; - } - - const char * getStorageTypeName() const override { return name; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override - { - auto configuration = TableFunction::getConfiguration(); - if (configuration->structure == "auto") - { - context->checkAccess(TableFunction::getSourceAccessType()); - auto object_storage = TableFunction::getObjectStorage(context, !is_insert_query); - return Storage::getTableStructureFromData(object_storage, configuration, std::nullopt, context); - } - else - { - return parseColumnsListFromString(configuration->structure, context); - } - } - - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override - { - auto configuration = TableFunction::getConfiguration(); - configuration->format = "Parquet"; - /// Set default format to Parquet if it's not specified in arguments. - TableFunction::parseArguments(ast_function, context); - } -}; - -struct TableFunctionIcebergName -{ - static constexpr auto name = "iceberg"; -}; - -struct TableFunctionIcebergS3Name -{ - static constexpr auto name = "icebergS3"; -}; - -struct TableFunctionIcebergAzureName -{ - static constexpr auto name = "icebergAzure"; -}; - -struct TableFunctionIcebergLocalName -{ - static constexpr auto name = "icebergLocal"; -}; - -struct TableFunctionDeltaLakeName -{ - static constexpr auto name = "deltaLake"; -}; - -struct TableFunctionHudiName -{ - static constexpr auto name = "hudi"; -}; - -#if USE_AVRO -# if USE_AWS_S3 -using TableFunctionIceberg = ITableFunctionDataLake; -using TableFunctionIcebergS3 = ITableFunctionDataLake; -# endif -# if USE_AZURE_BLOB_STORAGE -using TableFunctionIcebergAzure = ITableFunctionDataLake; -# endif -using TableFunctionIcebergLocal = ITableFunctionDataLake; -#endif -#if USE_AWS_S3 -# if USE_PARQUET -using TableFunctionDeltaLake = ITableFunctionDataLake; -#endif -using TableFunctionHudi = ITableFunctionDataLake; -#endif -} diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 9cebb91044a..60409a732c4 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -225,4 +225,94 @@ template class TableFunctionObjectStorage; #endif template class TableFunctionObjectStorage; + +#if USE_AVRO +void registerTableFunctionIceberg(TableFunctionFactory & factory) +{ +# if USE_AWS_S3 + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", + .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); + +# endif +# if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", + .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); +# endif + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", + .examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); +} +#endif + +#if USE_AWS_S3 +# if USE_PARQUET +void registerTableFunctionDeltaLake(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the DeltaLake table stored on object store.)", + .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); +} +# endif + +void registerTableFunctionHudi(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Hudi table stored on object store.)", + .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); +} +#endif + +void registerDataLakeTableFunctions(TableFunctionFactory & factory) +{ + UNUSED(factory); +#if USE_AVRO + registerTableFunctionIceberg(factory); +#endif +#if USE_AWS_S3 +# if USE_PARQUET + registerTableFunctionDeltaLake(factory); +# endif + registerTableFunctionHudi(factory); +#endif +} + +#if USE_AVRO +# if USE_AWS_S3 +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +# endif +# if USE_AZURE_BLOB_STORAGE +template class TableFunctionObjectStorage; +# endif +template class TableFunctionObjectStorage; +#endif +#if USE_AWS_S3 +# if USE_PARQUET +template class TableFunctionObjectStorage; +# endif +template class TableFunctionObjectStorage; +#endif } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 6b923f93e75..3cf86f982d1 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -61,6 +62,42 @@ struct LocalDefinition static constexpr auto storage_type_name = "Local"; }; +struct IcebergDefinition +{ + static constexpr auto name = "iceberg"; + static constexpr auto storage_type_name = "S3"; +}; + +struct IcebergS3Definition +{ + static constexpr auto name = "icebergS3"; + static constexpr auto storage_type_name = "S3"; +}; + +struct IcebergAzureDefinition +{ + static constexpr auto name = "icebergAzure"; + static constexpr auto storage_type_name = "Azure"; +}; + +struct IcebergLocalDefinition +{ + static constexpr auto name = "icebergLocal"; + static constexpr auto storage_type_name = "Local"; +}; + +struct DeltaLakeDefinition +{ + static constexpr auto name = "deltaLake"; + static constexpr auto storage_type_name = "S3"; +}; + +struct HudiDefinition +{ + static constexpr auto name = "hudi"; + static constexpr auto storage_type_name = "S3"; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -137,4 +174,22 @@ using TableFunctionHDFS = TableFunctionObjectStorage; + + +#if USE_AVRO +# if USE_AWS_S3 +using TableFunctionIceberg = TableFunctionObjectStorage; +using TableFunctionIcebergS3 = TableFunctionObjectStorage; +# endif +# if USE_AZURE_BLOB_STORAGE +using TableFunctionIcebergAzure = TableFunctionObjectStorage; +# endif +using TableFunctionIcebergLocal = TableFunctionObjectStorage; +#endif +#if USE_AWS_S3 +# if USE_PARQUET +using TableFunctionDeltaLake = TableFunctionObjectStorage; +# endif +using TableFunctionHudi = TableFunctionObjectStorage; +#endif } diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp deleted file mode 100644 index 8361d8a7977..00000000000 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ /dev/null @@ -1,88 +0,0 @@ -#include -#include - -namespace DB -{ - -#if USE_AVRO -void registerTableFunctionIceberg(TableFunctionFactory & factory) -{ -# if USE_AWS_S3 - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", - .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); - -# endif -# if USE_AZURE_BLOB_STORAGE - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", - .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -# endif - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", - .examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} -#endif - -#if USE_AWS_S3 -# if USE_PARQUET -void registerTableFunctionDeltaLake(TableFunctionFactory & factory) -{ - factory.registerFunction( - { - .documentation = - { - .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", - .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"} - }, - .allow_readonly = false - }); -} -#endif - -void registerTableFunctionHudi(TableFunctionFactory & factory) -{ - factory.registerFunction( - { - .documentation = - { - .description=R"(The table function can be used to read the Hudi table stored on object store.)", - .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"} - }, - .allow_readonly = false - }); -} -#endif - -void registerDataLakeTableFunctions(TableFunctionFactory & factory) -{ - UNUSED(factory); -#if USE_AVRO - registerTableFunctionIceberg(factory); -#endif -#if USE_AWS_S3 -# if USE_PARQUET - registerTableFunctionDeltaLake(factory); -#endif - registerTableFunctionHudi(factory); -#endif -} - -} From 7b01c19d06bf424cfcfaad154a12575a9ad81145 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 27 Sep 2024 15:06:27 +0000 Subject: [PATCH 098/816] fix header... --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 1 + src/Processors/QueryPlan/JoinStep.cpp | 32 ++++++++++++++++--- src/Processors/QueryPlan/JoinStep.h | 4 +++ .../Transforms/ColumnPermuteTransform.cpp | 4 +-- tests/integration/helpers/random_settings.py | 2 +- .../02001_join_on_const_bs_long.sql.j2 | 4 +-- 7 files changed, 39 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index bfd9be70bb5..01483b34092 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1887,6 +1887,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 543dc1a88f6..4f4d7e22022 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1641,6 +1641,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ std::move(join_algorithm), settings[Setting::max_block_size], settings[Setting::max_threads], + outer_scope_columns, false /*optimize_read_in_order*/); join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index fefb193827f..9fdfeedb111 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -45,14 +45,13 @@ size_t getPrefixLength(const NameSet & prefix, const Names & names) if (!prefix.contains(names[i])) break; } - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{}: [{}] [{}] -> {}", __FILE__, __LINE__, fmt::join(names, ", "), fmt::join(prefix, ", "), i); return i; } std::vector getPermutationToRotate(size_t prefix_size, size_t total_size) { std::vector permutation(total_size); - size_t i = prefix_size; + size_t i = prefix_size % total_size; for (auto & elem : permutation) { elem = i; @@ -92,8 +91,13 @@ JoinStep::JoinStep( JoinPtr join_, size_t max_block_size_, size_t max_streams_, + NameSet required_output_, bool keep_left_read_in_order_) - : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) + : join(std::move(join_)) + , max_block_size(max_block_size_) + , max_streams(max_streams_) + , required_output(std::move(required_output_)) + , keep_left_read_in_order(keep_left_read_in_order_) { updateInputStreams(DataStreams{left_stream_, right_stream_}); } @@ -128,9 +132,20 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines const auto & result_names = pipeline->getHeader().getNames(); size_t prefix_size = getPrefixLength(rhs_names, result_names); - if (0 < prefix_size && prefix_size < result_names.size()) + if (!columns_to_remove.empty() || (0 < prefix_size && prefix_size < result_names.size())) { auto column_permutation = getPermutationToRotate(prefix_size, result_names.size()); + size_t n = 0; + auto it = columns_to_remove.begin(); + for (size_t i = 0; i < column_permutation.size(); ++i) + { + if (it != columns_to_remove.end() && *it == i) + ++it; + else + column_permutation[n++] = column_permutation[i]; + } + column_permutation.resize(n); + pipeline->addSimpleTransform([column_perm = std::move(column_permutation)](const Block & header) { return std::make_shared(header, std::move(column_perm)); @@ -174,6 +189,15 @@ void JoinStep::updateOutputStream() if (swap_streams) result_header = rotateBlock(result_header, input_streams[1].header); + columns_to_remove.clear(); + for (size_t i = 0; i < result_header.columns(); ++i) + { + if (required_output.empty()) + break; + if (!required_output.contains(result_header.getByPosition(i).name)) + columns_to_remove.insert(i); + } + result_header.erase(columns_to_remove); output_stream = DataStream { .header = result_header }; } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 96c02f9fd19..30b20a0d3a5 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -20,6 +20,7 @@ public: JoinPtr join_, size_t max_block_size_, size_t max_streams_, + NameSet required_output_, bool keep_left_read_in_order_); String getName() const override { return "Join"; } @@ -48,6 +49,9 @@ private: JoinPtr join; size_t max_block_size; size_t max_streams; + + NameSet required_output; + std::set columns_to_remove; bool keep_left_read_in_order; }; diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp index ac7793bd136..2921bcac177 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -10,8 +10,8 @@ template void applyPermutation(std::vector & data, const std::vector & permutation) { std::vector res; - res.reserve(data.size()); - for (size_t i = 0; i < data.size(); ++i) + res.reserve(permutation.size()); + for (size_t i = 0; i < permutation.size(); ++i) res.emplace_back(std::move(data[permutation[i]])); data = std::move(res); } diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index a34d8e93c47..3a51d8cf52f 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -7,7 +7,7 @@ def randomize_settings(): yield "max_block_size", random.randint(8000, 100000) if random.random() < 0.5: yield "query_plan_join_inner_table_selection", random.choice( - ["auto", "left", "right"] + ["auto", "left"] ) diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 1726bcb7062..83548e087bd 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t1 (id Int) ENGINE = TinyLog; +CREATE TABLE t2 (id Int) ENGINE = TinyLog; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); From de6517367677773b97ddcb0820859493a8295ac0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 27 Sep 2024 16:01:52 +0000 Subject: [PATCH 099/816] Automatic style fix --- tests/integration/helpers/random_settings.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index 3a51d8cf52f..32cde54d0e7 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -6,9 +6,7 @@ def randomize_settings(): if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) if random.random() < 0.5: - yield "query_plan_join_inner_table_selection", random.choice( - ["auto", "left"] - ) + yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"]) def write_random_settings_config(destination): From decfe0b676ab4a334fd2fcc61dd5a211f5fe7d44 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 27 Sep 2024 16:52:44 +0000 Subject: [PATCH 100/816] fix build --- src/Core/Settings.cpp | 2 +- src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp | 8 +++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4e63c3ae957..dcd1d33ff27 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -345,7 +345,7 @@ namespace ErrorCodes M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ M(Bool, single_join_prefer_left_table, true, "For single JOIN in case of identifier ambiguity prefer left table", IMPORTANT) \ \ - M(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, "Select the side of the join to be the inner table in the query plan. Possible values: 'auto', 'left', 'right'.", 0) \ + M(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, "Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'.", 0) \ M(UInt64, preferred_block_size_bytes, 1000000, "This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality.", 0) \ \ M(UInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index 8074304de52..cd66a230038 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -56,11 +56,9 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) return; const auto & table_join = join->getTableJoin(); - auto kind = table_join.kind(); - if (table_join.hasUsing() - || table_join.strictness() != JoinStrictness::All - || (kind != JoinKind::Inner && kind != JoinKind::Left - && kind != JoinKind::Right && kind != JoinKind::Full)) + /// fixme: USING clause handled specially in join algorithm, so swap breaks it + /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test + if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) return; bool need_swap = false; From 2c073ec53eb70d6a10d56c60713b22a6acca10e6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 27 Sep 2024 22:22:03 -0300 Subject: [PATCH 101/816] draft, needs more testing, tests and etc --- src/Access/AuthenticationData.cpp | 59 ++++++++++++++++++- src/Access/AuthenticationData.h | 4 ++ src/Access/IAccessStorage.cpp | 4 +- src/Access/IAccessStorage.h | 1 - src/Access/User.cpp | 3 +- src/Access/User.h | 1 - .../Access/InterpreterCreateUserQuery.cpp | 45 +------------- ...InterpreterShowCreateAccessEntityQuery.cpp | 7 --- src/Interpreters/Session.cpp | 6 +- src/Parsers/ASTLiteral.cpp | 1 + src/Parsers/Access/ASTAuthenticationData.cpp | 14 +++++ src/Parsers/Access/ASTAuthenticationData.h | 1 + src/Parsers/Access/ASTCreateUserQuery.cpp | 9 --- src/Parsers/Access/ASTCreateUserQuery.h | 2 - src/Parsers/Access/ParserCreateUserQuery.cpp | 43 +++++++------- 15 files changed, 108 insertions(+), 92 deletions(-) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 97010e67c5e..b5f76e1e317 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -7,6 +7,9 @@ #include #include #include +#include +#include +#include #include #include @@ -40,6 +43,37 @@ namespace ErrorCodes extern const int OPENSSL_ERROR; } +namespace +{ + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } +} + AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_view text [[maybe_unused]]) { #if USE_SSL @@ -381,14 +415,34 @@ std::shared_ptr AuthenticationData::toAST() const throw Exception(ErrorCodes::LOGICAL_ERROR, "AST: Unexpected authentication type {}", toString(auth_type)); } + + if (valid_until) + { + WriteBufferFromOwnString out; + writeDateTimeText(*valid_until, out); + + node->valid_until = std::make_shared(out.str()); + } + return node; } AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules) { + std::optional valid_until; + + if (query.valid_until) + { + valid_until = getValidUntilFromAST(query.valid_until, context); + } + if (query.type && query.type == AuthenticationType::NO_PASSWORD) - return AuthenticationData(); + { + AuthenticationData auth_data; + auth_data.setValidUntil(valid_until); + return auth_data; + } /// For this type of authentication we have ASTPublicSSHKey as children for ASTAuthenticationData if (query.type && query.type == AuthenticationType::SSH_KEY) @@ -415,6 +469,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que } auth_data.setSSHKeys(std::move(keys)); + auth_data.setValidUntil(valid_until); return auth_data; #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); @@ -486,11 +541,13 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que #endif } + auth_data.setValidUntil(valid_until); auth_data.setPassword(value); return auth_data; } AuthenticationData auth_data(*query.type); + auth_data.setValidUntil(valid_until); if (query.contains_hash) { diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index 8093fe1d888..bdcd8cbb14d 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -74,6 +74,9 @@ public: const String & getHTTPAuthenticationServerName() const { return http_auth_server_name; } void setHTTPAuthenticationServerName(const String & name) { http_auth_server_name = name; } + std::optional getValidUntil() const { return valid_until; } + void setValidUntil(std::optional valid_until_) { valid_until = valid_until_; } + friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } @@ -106,6 +109,7 @@ private: /// HTTP authentication properties String http_auth_server_name; HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC; + std::optional valid_until; }; } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 29475461c45..209a5700f96 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -539,7 +539,7 @@ std::optional IAccessStorage::authenticateImpl( continue; } - if (areCredentialsValid(user->getName(), user->valid_until, auth_method, credentials, external_authenticators, auth_result.settings)) + if (areCredentialsValid(user->getName(), auth_method, credentials, external_authenticators, auth_result.settings)) { auth_result.authentication_data = auth_method; return auth_result; @@ -564,7 +564,6 @@ std::optional IAccessStorage::authenticateImpl( bool IAccessStorage::areCredentialsValid( const std::string & user_name, - time_t valid_until, const AuthenticationData & authentication_method, const Credentials & credentials, const ExternalAuthenticators & external_authenticators, @@ -576,6 +575,7 @@ bool IAccessStorage::areCredentialsValid( if (credentials.getUserName() != user_name) return false; + auto valid_until = authentication_method.getValidUntil(); if (valid_until) { const time_t now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index a8ac75075d3..f4e618f45c9 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -233,7 +233,6 @@ protected: bool allow_plaintext_password) const; virtual bool areCredentialsValid( const std::string & user_name, - time_t valid_until, const AuthenticationData & authentication_method, const Credentials & credentials, const ExternalAuthenticators & external_authenticators, diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 2052527f4ae..69d544d5c19 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -19,8 +19,7 @@ bool User::equal(const IAccessEntity & other) const return (authentication_methods == other_user.authentication_methods) && (allowed_client_hosts == other_user.allowed_client_hosts) && (access == other_user.access) && (granted_roles == other_user.granted_roles) && (default_roles == other_user.default_roles) - && (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database) - && (valid_until == other_user.valid_until); + && (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database); } void User::setName(const String & name_) diff --git a/src/Access/User.h b/src/Access/User.h index 7f91c1e3756..58de156db7c 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -23,7 +23,6 @@ struct User : public IAccessEntity SettingsProfileElements settings; RolesOrUsersSet grantees = RolesOrUsersSet::AllTag{}; String default_database; - time_t valid_until = 0; bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 81600b2b6eb..89478996899 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -39,7 +39,6 @@ namespace const std::optional & override_default_roles, const std::optional & override_settings, const std::optional & override_grantees, - const std::optional & valid_until, bool reset_authentication_methods, bool replace_authentication_methods, bool allow_implicit_no_password, @@ -128,9 +127,6 @@ namespace } } - if (valid_until) - user.valid_until = *valid_until; - if (override_name && !override_name->host_pattern.empty()) { user.allowed_client_hosts = AllowedClientHosts{}; @@ -170,34 +166,6 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } - - time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) - { - if (context) - valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); - - const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); - - if (valid_until_str == "infinity") - return 0; - - time_t time = 0; - ReadBufferFromString in(valid_until_str); - - if (context) - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - else - { - readDateTimeText(time, in); - } - - return time; - } } BlockIO InterpreterCreateUserQuery::execute() @@ -221,10 +189,6 @@ BlockIO InterpreterCreateUserQuery::execute() } } - std::optional valid_until; - if (query.valid_until) - valid_until = getValidUntilFromAST(query.valid_until, getContext()); - std::optional default_roles_from_query; if (query.default_roles) { @@ -269,7 +233,7 @@ BlockIO InterpreterCreateUserQuery::execute() auto updated_user = typeid_cast>(entity->clone()); updateUserFromQueryImpl( *updated_user, query, authentication_methods, {}, default_roles_from_query, settings_from_query, grantees_from_query, - valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, + query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user); return updated_user; @@ -291,7 +255,7 @@ BlockIO InterpreterCreateUserQuery::execute() auto new_user = std::make_shared(); updateUserFromQueryImpl( *new_user, query, authentication_methods, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, - valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, + query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user); new_users.emplace_back(std::move(new_user)); @@ -346,10 +310,6 @@ void InterpreterCreateUserQuery::updateUserFromQuery( } } - std::optional valid_until; - if (query.valid_until) - valid_until = getValidUntilFromAST(query.valid_until, {}); - updateUserFromQueryImpl( user, query, @@ -358,7 +318,6 @@ void InterpreterCreateUserQuery::updateUserFromQuery( {}, {}, {}, - valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, allow_no_password, diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index ef6ddf1866d..8b7cef056ed 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -69,13 +69,6 @@ namespace query->authentication_methods.push_back(authentication_method.toAST()); } - if (user.valid_until) - { - WriteBufferFromOwnString out; - writeDateTimeText(user.valid_until, out); - query->valid_until = std::make_shared(out.str()); - } - if (!user.settings.empty()) { if (attach_mode) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index ede91b0df56..9036c2af6a7 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -384,12 +384,12 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So void Session::checkIfUserIsStillValid() { - if (user && user->valid_until) + if (const auto valid_until = user_authenticated_with.getValidUntil()) { const time_t now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - if (now > user->valid_until) - throw Exception(ErrorCodes::USER_EXPIRED, "User expired"); + if (now > valid_until) + throw Exception(ErrorCodes::USER_EXPIRED, "Authentication method used has expired"); } } diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 515f4f0cb9f..fc9749e6f1e 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -1,3 +1,4 @@ + #include #include #include diff --git a/src/Parsers/Access/ASTAuthenticationData.cpp b/src/Parsers/Access/ASTAuthenticationData.cpp index 75082041161..9fa75185d32 100644 --- a/src/Parsers/Access/ASTAuthenticationData.cpp +++ b/src/Parsers/Access/ASTAuthenticationData.cpp @@ -14,6 +14,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + void formatValidUntil(const IAST & valid_until, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " VALID UNTIL " << (settings.hilite ? IAST::hilite_none : ""); + valid_until.format(settings); + } +} + std::optional ASTAuthenticationData::getPassword() const { if (contains_password) @@ -205,6 +214,11 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt children[1]->format(settings); } + if (valid_until) + { + formatValidUntil(*valid_until, settings); + } + } bool ASTAuthenticationData::hasSecretParts() const diff --git a/src/Parsers/Access/ASTAuthenticationData.h b/src/Parsers/Access/ASTAuthenticationData.h index 7f0644b3437..24c4c015efd 100644 --- a/src/Parsers/Access/ASTAuthenticationData.h +++ b/src/Parsers/Access/ASTAuthenticationData.h @@ -41,6 +41,7 @@ public: bool contains_password = false; bool contains_hash = false; + ASTPtr valid_until; protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index ec48c32b684..25c2f805781 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -40,12 +40,6 @@ namespace } } - void formatValidUntil(const IAST & valid_until, const IAST::FormatSettings & settings) - { - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " VALID UNTIL " << (settings.hilite ? IAST::hilite_none : ""); - valid_until.format(settings); - } - void formatHosts(const char * prefix, const AllowedClientHosts & hosts, const IAST::FormatSettings & settings) { if (prefix) @@ -260,9 +254,6 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & formatAuthenticationData(authentication_methods, format); } - if (valid_until) - formatValidUntil(*valid_until, format); - if (hosts) formatHosts(nullptr, *hosts, format); if (add_hosts) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index e1bae98f2f3..347552a9f11 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -62,8 +62,6 @@ public: std::shared_ptr default_database; - ASTPtr valid_until; - String getID(char) const override; ASTPtr clone() const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 8bfc84a28a6..7c171432b66 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -43,6 +43,19 @@ namespace }); } + bool parseValidUntil(IParserBase::Pos & pos, Expected & expected, ASTPtr & valid_until) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword{Keyword::VALID_UNTIL}.ignore(pos, expected)) + return false; + + ParserStringAndSubstitution until_p; + + return until_p.parse(pos, valid_until, expected); + }); + } + bool parseAuthenticationData( IParserBase::Pos & pos, Expected & expected, @@ -223,6 +236,12 @@ namespace if (http_auth_scheme) auth_data->children.push_back(std::move(http_auth_scheme)); + if (parseValidUntil(pos, expected, auth_data->valid_until)) + { + // I am still not sure why this has to be done and if it has to be done + auth_data->children.push_back(auth_data->valid_until); + } + return true; }); } @@ -471,19 +490,6 @@ namespace }); } - bool parseValidUntil(IParserBase::Pos & pos, Expected & expected, ASTPtr & valid_until) - { - return IParserBase::wrapParseImpl(pos, [&] - { - if (!ParserKeyword{Keyword::VALID_UNTIL}.ignore(pos, expected)) - return false; - - ParserStringAndSubstitution until_p; - - return until_p.parse(pos, valid_until, expected); - }); - } - bool parseAddIdentifiedWith(IParserBase::Pos & pos, Expected & expected, std::vector> & auth_data) { return IParserBase::wrapParseImpl(pos, [&] @@ -579,11 +585,6 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec reset_authentication_methods_to_new = parseResetAuthenticationMethods(pos, expected); } - if (!valid_until) - { - parseValidUntil(pos, expected, valid_until); - } - AllowedClientHosts new_hosts; if (parseHosts(pos, expected, "", new_hosts)) { @@ -674,7 +675,6 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->settings = std::move(settings); query->grantees = std::move(grantees); query->default_database = std::move(default_database); - query->valid_until = std::move(valid_until); query->storage_name = std::move(storage_name); query->reset_authentication_methods_to_new = reset_authentication_methods_to_new; query->add_identified_with = parsed_add_identified_with; @@ -685,8 +685,9 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->children.push_back(authentication_method); } - if (query->valid_until) - query->children.push_back(query->valid_until); + // todo arthur +// if (query->valid_until) +// query->children.push_back(query->valid_until); return true; } From 5e772899f69b9429f05e3257116af0652266fc5d Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 29 Sep 2024 18:38:58 +0000 Subject: [PATCH 102/816] add resource-specific workload settings --- .../Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/SchedulingSettings.cpp | 118 ++++++++---------- src/Common/Scheduler/SchedulingSettings.h | 4 +- src/Parsers/ASTCreateWorkloadQuery.cpp | 26 ++-- src/Parsers/ASTCreateWorkloadQuery.h | 22 +++- src/Parsers/ParserCreateWorkloadQuery.cpp | 88 +++++++------ .../System/StorageSystemWorkloads.cpp | 4 +- 7 files changed, 150 insertions(+), 114 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 460693e1935..cf6b041c9f1 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -43,7 +43,7 @@ IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resourc auto * create = typeid_cast(ast.get()); name = create->getWorkloadName(); parent = create->getWorkloadParent(); - settings.updateFromAST(create->settings, resource_name); + settings.updateFromChanges(create->changes, resource_name); } IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index 18a465fe930..352e61fb560 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -12,11 +12,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) +// TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage +// TODO(serxa): and probably we should add and persist version in filename for future changes +void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) { - UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST - if (auto * set = typeid_cast(settings.get())) - { + struct { std::optional new_weight; std::optional new_priority; std::optional new_max_speed; @@ -24,7 +24,8 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r std::optional new_max_requests; std::optional new_max_cost; - auto get_not_negative_float64 = [] (const String & name, const Field & field) { + static Float64 getNotNegativeFloat64(const String & name, const Field & field) + { { UInt64 val; if (field.tryGet(val)) @@ -42,9 +43,10 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r } return field.safeGet(); - }; + } - auto get_not_negative_int64 = [] (const String & name, const Field & field) { + static Int64 getNotNegativeInt64(const String & name, const Field & field) + { { UInt64 val; if (field.tryGet(val)) @@ -67,74 +69,64 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r } return field.safeGet(); - }; + } - // Read changed setting values - for (const auto & [name, value] : set->changes) + void read(const String & name, const Field & value) { - // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage - // TODO(serxa): and probably we should add and persist version in filename for future changes if (name == "weight") - new_weight = get_not_negative_float64(name, value); + new_weight = getNotNegativeFloat64(name, value); else if (name == "priority") new_priority = Priority{value.safeGet()}; else if (name == "max_speed") - new_max_speed = get_not_negative_float64(name, value); + new_max_speed = getNotNegativeFloat64(name, value); else if (name == "max_burst") - new_max_burst = get_not_negative_float64(name, value); + new_max_burst = getNotNegativeFloat64(name, value); else if (name == "max_requests") - new_max_requests = get_not_negative_int64(name, value); + new_max_requests = getNotNegativeInt64(name, value); else if (name == "max_cost") - new_max_cost = get_not_negative_int64(name, value); + new_max_cost = getNotNegativeInt64(name, value); } + } regular, specific; - // Read setting to be reset to default values - static SchedulingSettings default_settings; - bool reset_max_burst = false; - for (const String & name : set->default_settings) - { - if (name == "weight") - new_weight = default_settings.weight; - else if (name == "priority") - new_priority = default_settings.priority; - else if (name == "max_speed") - new_max_speed = default_settings.max_speed; - else if (name == "max_burst") - reset_max_burst = true; - else if (name == "max_requests") - new_max_requests = default_settings.max_requests; - else if (name == "max_cost") - new_max_cost = default_settings.max_cost; - } - if (reset_max_burst) - new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); - - // Validate we could use values we read in a scheduler node - { - SchedulerNodeInfo validating_node(new_weight ? *new_weight : weight, new_priority ? *new_priority : priority); - } - - // Save new values into the `this` object - // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value - if (new_weight) - weight = *new_weight; - if (new_priority) - priority = *new_priority; - if (new_max_speed) - { - max_speed = *new_max_speed; - // We always set max_burst if max_speed is changed. - // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed - if (!new_max_burst) - max_burst = default_burst_seconds * max_speed; - } - if (new_max_burst) - max_burst = *new_max_burst; - if (new_max_requests) - max_requests = *new_max_requests; - if (new_max_cost) - max_cost = *new_max_cost; + // Read changed setting values + for (const auto & [name, value, resource] : changes) + { + if (resource.empty()) + regular.read(name, value); + else if (resource == resource_name) + specific.read(name, value); } + + auto get_value = [] (const std::optional & specific_new, const std::optional & regular_new, T & old) + { + if (specific_new) + return *specific_new; + if (regular_new) + return *regular_new; + return old; + }; + + // Validate that we could use values read in a scheduler node + { + SchedulerNodeInfo validating_node( + get_value(specific.new_weight, regular.new_weight, weight), + get_value(specific.new_priority, regular.new_priority, priority)); + } + + // Commit new values. + // Previous values are left intentionally for ALTER query to be able to skip not mentioned setting values + weight = get_value(specific.new_weight, regular.new_weight, weight); + priority = get_value(specific.new_priority, regular.new_priority, priority); + if (specific.new_max_speed || regular.new_max_speed) + { + max_speed = get_value(specific.new_max_speed, regular.new_max_speed, max_speed); + // We always set max_burst if max_speed is changed. + // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed + max_burst = default_burst_seconds * max_speed; + } + max_burst = get_value(specific.new_max_burst, regular.new_max_burst, max_burst); + max_requests = get_value(specific.new_max_requests, regular.new_max_requests, max_requests); + max_cost = get_value(specific.new_max_cost, regular.new_max_cost, max_cost); } } diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index 6d790b34164..cda6b546dfc 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include @@ -33,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - void updateFromAST(const ASTPtr & settings, const String & resource_name); + void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name); }; } diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index 869dc64daf7..972ce733651 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -21,11 +22,7 @@ ASTPtr ASTCreateWorkloadQuery::clone() const res->children.push_back(res->workload_parent); } - if (settings) - { - res->settings = settings->clone(); - res->children.push_back(res->settings); - } + res->changes = changes; return res; } @@ -54,10 +51,25 @@ void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAS format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); } - if (settings) + if (!changes.empty()) { format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; - settings->format(format); + + bool first = true; + + for (const auto & change : changes) + { + if (!first) + format.ostr << ", "; + else + first = false; + format.ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value); + if (!change.resource.empty()) + { + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' '; + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : ""); + } + } } } diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h index a17bc4a11cd..71e27295bc1 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -1,8 +1,9 @@ #pragma once +#include #include #include - +#include namespace DB { @@ -12,7 +13,24 @@ class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster public: ASTPtr workload_name; ASTPtr workload_parent; - ASTPtr settings; + + /// Special version of settings that support optional `FOR resource` clause + struct SettingChange + { + String name; + Field value; + String resource; + + SettingChange() = default; + SettingChange(std::string_view name_, const Field & value_, std::string_view resource_) : name(name_), value(value_), resource(resource_) {} + SettingChange(std::string_view name_, Field && value_, std::string_view resource_) : name(name_), value(std::move(value_)), resource(resource_) {} + + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.resource == rhs.resource); } + friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } + }; + + using SettingsChanges = std::vector; + SettingsChanges changes; bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index 427d9aa40be..9caf474741c 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -16,54 +17,67 @@ namespace DB namespace { -bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) +bool parseWorkloadSetting( + ASTCreateWorkloadQuery::SettingChange & change, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier name_p; + ParserLiteral value_p; + ParserToken s_eq(TokenType::Equals); + ParserIdentifier resource_name_p; + + ASTPtr name_node; + ASTPtr value_node; + ASTPtr resource_name_node; + + String name; + String resource_name; + + if (!name_p.parse(pos, name_node, expected)) + return false; + tryGetIdentifierNameInto(name_node, name); + + if (!s_eq.ignore(pos, expected)) + return false; + + if (!value_p.parse(pos, value_node, expected)) + return false; + + if (ParserKeyword(Keyword::FOR).ignore(pos, expected)) + { + if (!resource_name_p.parse(pos, resource_name_node, expected)) + return false; + tryGetIdentifierNameInto(resource_name_node, resource_name); + } + + change.name = std::move(name); + change.value = value_node->as().value; + change.resource = std::move(resource_name); + + return true; +} + +bool parseSettings(IParser::Pos & pos, Expected & expected, ASTCreateWorkloadQuery::SettingsChanges & changes) { return IParserBase::wrapParseImpl(pos, [&] { if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) return false; - SettingsChanges settings_changes; - Strings default_settings; + ASTCreateWorkloadQuery::SettingsChanges res_changes; auto parse_setting = [&] { - SettingChange setting; - String default_setting; - std::pair parameter; - - if (ParserSetQuery::parseNameValuePairWithParameterOrDefault(setting, default_setting, parameter, pos, expected)) - { - if (!default_setting.empty()) - { - default_settings.push_back(std::move(default_setting)); - return true; - } - if (!setting.name.empty()) - { - settings_changes.push_back(std::move(setting)); - return true; - } - // TODO(serxa): parse optional clause: [FOR resource_name] - return false; // We do not support parameters - } - - return false; + ASTCreateWorkloadQuery::SettingChange change; + if (!parseWorkloadSetting(change, pos, expected)) + return false; + res_changes.push_back(std::move(change)); + return true; }; if (!ParserList::parseUtil(pos, expected, parse_setting, false)) return false; - ASTPtr res_settings; - if (!settings_changes.empty()) - { - auto settings_changes_ast = std::make_shared(); - settings_changes_ast->changes = std::move(settings_changes); - settings_changes_ast->is_standalone = false; - res_settings = settings_changes_ast; - } - - settings = std::move(res_settings); + changes = std::move(res_changes); return true; }); } @@ -114,8 +128,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } - ASTPtr settings; - parseSettings(pos, expected, settings); + ASTCreateWorkloadQuery::SettingsChanges changes; + parseSettings(pos, expected, changes); auto create_workload_query = std::make_shared(); node = create_workload_query; @@ -132,8 +146,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_workload_query->or_replace = or_replace; create_workload_query->if_not_exists = if_not_exists; create_workload_query->cluster = std::move(cluster_str); + create_workload_query->changes = std::move(changes); - create_workload_query->settings = std::move(settings); return true; } diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp index dad2750d8c0..ebb7e693e26 100644 --- a/src/Storages/System/StorageSystemWorkloads.cpp +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -35,13 +35,13 @@ void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr c void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) { - // TODO(serxa): add backup for workloads and resources + // TODO(serxa): add backup for workloads // storage.backup(backup_entries_collector, data_path_in_backup); } void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) { - // TODO(serxa): add restore for workloads and resources + // TODO(serxa): add restore for workloads // storage.restore(restorer, data_path_in_backup); } From a8c164ece7427dd4382fd543598cc14ff773379e Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 29 Sep 2024 18:39:30 +0000 Subject: [PATCH 103/816] add test for resource-specific settings --- tests/integration/test_scheduler/test.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 401444b2d5a..0eee9d968ba 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -591,7 +591,7 @@ def test_create_workload(): f""" create resource io_write (write disk s3_no_resource); create resource io_read (read disk s3_no_resource); - create workload all settings max_cost = 1000000; + create workload all settings max_cost = 1000000 for io_write, max_cost = 2000000 for io_read; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; @@ -629,6 +629,18 @@ def test_create_workload(): ) == "2\n" ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_write' and max_cost=1000000" + ) + == "1\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_read' and max_cost=2000000" + ) + == "1\n" + ) do_checks() node.restart_clickhouse() # Check that workloads persist From c7f662dc989833d707d15ef086edd69c1d5b64cd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 30 Sep 2024 02:43:53 +0000 Subject: [PATCH 104/816] fix build, add initial fuzzing processing --- .../data_type_deserialization_fuzzer.cpp | 1 + src/Parsers/fuzzers/CMakeLists.txt | 4 +- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- tests/fuzz/runner.py | 76 +++++++++++++++++-- 4 files changed, 75 insertions(+), 8 deletions(-) diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index f9a733647e1..216b252ad0f 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt index 903319d733c..c829c26a805 100644 --- a/src/Parsers/fuzzers/CMakeLists.txt +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -2,10 +2,10 @@ clickhouse_add_executable(lexer_fuzzer lexer_fuzzer.cpp ${SRCS}) target_link_libraries(lexer_fuzzer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser_fuzzer select_parser_fuzzer.cpp ${SRCS}) -target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers dbms) +target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms) clickhouse_add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) -target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers dbms) +target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers clickhouse_functions dbms) add_subdirectory(codegen_fuzzer) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 74fdcff79f7..ee17e03fce2 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -47,4 +47,4 @@ target_compile_options (codegen_select_fuzzer PRIVATE -Wno-newline-eof) target_link_libraries(protoc ch_contrib::fuzzer) target_include_directories(codegen_select_fuzzer SYSTEM BEFORE PRIVATE "${CMAKE_CURRENT_BINARY_DIR}") -target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc dbms) +target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc clickhouse_functions dbms) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 44259228f60..5abab282afd 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -4,19 +4,70 @@ import configparser import logging import os from pathlib import Path +import re import subprocess DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") +def report(source: str, reason: str, call_stack: list, test_unit: str): + print(f"########### REPORT: {source} {reason} {test_unit}") + for line in call_stack: + print(f" {line}") + print("########### END OF REPORT ###########") + +def process_fuzzer_output(output: str): + pass + +def process_error(error: str): + ERROR = r'^==\d+== ERROR: (\S+): (.*)' + error_source = '' + error_reason = '' + SUMMARY = r'^SUMMARY: ' + TEST_UNIT_LINE = r"artifact_prefix='.*/'; Test unit written to (.*)" + test_unit = '' + CALL_STACK_LINE = r'^\s+(#\d+.*)' + call_stack = [] + is_call_stack = False + + for line_num, line in enumerate(error.splitlines(), 1): + + if is_call_stack: + match = re.search(CALL_STACK_LINE, line) + if match: + call_stack.append(match.group(1)) + continue + else: + if re.search(SUMMARY, line): + is_call_stack = False + continue + + if not call_stack and not is_call_stack: + match = re.search(ERROR, line) + if match: + error_source = match.group(1) + error_reason = match.group(2) + is_call_stack = True + continue + + match = re.search(TEST_UNIT_LINE, line) + if match: + test_unit = match.group(1) + + report(error_source, error_reason, call_stack, test_unit) def run_fuzzer(fuzzer: str): logging.info("Running fuzzer %s...", fuzzer) - corpus_dir = f"{fuzzer}.in" - with Path(corpus_dir) as path: + seed_corpus_dir = f"{fuzzer}.in" + with Path(seed_corpus_dir) as path: if not path.exists() or not path.is_dir(): - corpus_dir = "" + seed_corpus_dir = "" + + active_corpus_dir = f"{fuzzer}.corpus" + if not os.path.exists(active_corpus_dir): + os.makedirs(active_corpus_dir) + options_file = f"{fuzzer}.options" custom_libfuzzer_options = "" @@ -53,7 +104,7 @@ def run_fuzzer(fuzzer: str): for key, value in parser["fuzzer_arguments"].items() ) - cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {active_corpus_dir} {seed_corpus_dir}" if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" if fuzzer_arguments: @@ -65,8 +116,23 @@ def run_fuzzer(fuzzer: str): cmd_line += " < /dev/null" logging.info("...will execute: %s", cmd_line) - subprocess.check_call(cmd_line, shell=True) + #subprocess.check_call(cmd_line, shell=True) + try: + result = subprocess.run( + cmd_line, + stderr=subprocess.PIPE, + stdout=subprocess.DEVNULL, + text=True, + check=True, + shell=True + ) + except subprocess.CalledProcessError as e: +# print("Command failed with error:", e) + print("Stderr output:", e.stderr) + process_error(e.stderr) + else: + process_fuzzer_output(result.stderr) def main(): logging.basicConfig(level=logging.INFO) From abd3747806dd8f3fb75eac4f0a5cea3c6eacffc2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 30 Sep 2024 03:43:34 +0000 Subject: [PATCH 105/816] fix style --- tests/fuzz/runner.py | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 5abab282afd..6825a072e2d 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -3,30 +3,33 @@ import configparser import logging import os -from pathlib import Path import re import subprocess +from pathlib import Path DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") + def report(source: str, reason: str, call_stack: list, test_unit: str): print(f"########### REPORT: {source} {reason} {test_unit}") for line in call_stack: print(f" {line}") print("########### END OF REPORT ###########") + def process_fuzzer_output(output: str): pass + def process_error(error: str): - ERROR = r'^==\d+== ERROR: (\S+): (.*)' - error_source = '' - error_reason = '' - SUMMARY = r'^SUMMARY: ' + ERROR = r"^==\d+== ERROR: (\S+): (.*)" + error_source = "" + error_reason = "" + SUMMARY = r"^SUMMARY: " TEST_UNIT_LINE = r"artifact_prefix='.*/'; Test unit written to (.*)" - test_unit = '' - CALL_STACK_LINE = r'^\s+(#\d+.*)' + test_unit = "" + CALL_STACK_LINE = r"^\s+(#\d+.*)" call_stack = [] is_call_stack = False @@ -56,6 +59,7 @@ def process_error(error: str): report(error_source, error_reason, call_stack, test_unit) + def run_fuzzer(fuzzer: str): logging.info("Running fuzzer %s...", fuzzer) @@ -68,7 +72,6 @@ def run_fuzzer(fuzzer: str): if not os.path.exists(active_corpus_dir): os.makedirs(active_corpus_dir) - options_file = f"{fuzzer}.options" custom_libfuzzer_options = "" fuzzer_arguments = "" @@ -104,7 +107,9 @@ def run_fuzzer(fuzzer: str): for key, value in parser["fuzzer_arguments"].items() ) - cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {active_corpus_dir} {seed_corpus_dir}" + cmd_line = ( + f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {active_corpus_dir} {seed_corpus_dir}" + ) if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" if fuzzer_arguments: @@ -116,7 +121,7 @@ def run_fuzzer(fuzzer: str): cmd_line += " < /dev/null" logging.info("...will execute: %s", cmd_line) - #subprocess.check_call(cmd_line, shell=True) + # subprocess.check_call(cmd_line, shell=True) try: result = subprocess.run( @@ -125,15 +130,16 @@ def run_fuzzer(fuzzer: str): stdout=subprocess.DEVNULL, text=True, check=True, - shell=True + shell=True, ) except subprocess.CalledProcessError as e: -# print("Command failed with error:", e) + # print("Command failed with error:", e) print("Stderr output:", e.stderr) process_error(e.stderr) else: process_fuzzer_output(result.stderr) + def main(): logging.basicConfig(level=logging.INFO) From 55ae792706177ce96940f23d7147914db06dcf39 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 30 Sep 2024 04:02:25 +0000 Subject: [PATCH 106/816] fix style --- tests/fuzz/runner.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 6825a072e2d..deb219baff9 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -18,6 +18,7 @@ def report(source: str, reason: str, call_stack: list, test_unit: str): print("########### END OF REPORT ###########") +# pylint: disable=unused-argument def process_fuzzer_output(output: str): pass @@ -33,6 +34,7 @@ def process_error(error: str): call_stack = [] is_call_stack = False + # pylint: disable=unused-variable for line_num, line in enumerate(error.splitlines(), 1): if is_call_stack: @@ -40,10 +42,10 @@ def process_error(error: str): if match: call_stack.append(match.group(1)) continue - else: - if re.search(SUMMARY, line): - is_call_stack = False - continue + + if re.search(SUMMARY, line): + is_call_stack = False + continue if not call_stack and not is_call_stack: match = re.search(ERROR, line) From 743ffeba2585e2de410194893d79cc1411ceffff Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 30 Sep 2024 09:38:16 +0000 Subject: [PATCH 107/816] Fix style check error --- tests/integration/test_ddl_worker_replicas/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py index fecb7d97d25..0905165611f 100644 --- a/tests/integration/test_ddl_worker_replicas/test.py +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -1,5 +1,4 @@ import pytest -import time from helpers.cluster import ClickHouseCluster From ffe61c3007ec216a9b8010f3238c351834b1e068 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 30 Sep 2024 11:18:28 +0000 Subject: [PATCH 108/816] working on validations and tests --- .../Scheduler/Nodes/IOResourceManager.cpp | 3 - src/Common/Scheduler/SchedulingSettings.h | 2 +- .../Workload/WorkloadEntityStorageBase.cpp | 76 ++++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 7 +- .../03232_workload_create_and_drop.reference | 2 +- .../03232_workload_create_and_drop.sql | 6 +- .../03232_workloads_and_resources.reference | 0 .../03232_workloads_and_resources.sql | 17 +++++ 8 files changed, 92 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.reference create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.sql diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index cf6b041c9f1..07929e855ce 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -61,12 +61,10 @@ IOResourceManager::Resource::~Resource() void IOResourceManager::Resource::createNode(const NodeInfo & info) { - // TODO(serxa): make sure all possible callers validate empty workload name! if (info.name.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", resource_name); - // TODO(serxa): make sure all possible callers validate self-reference! if (info.name == info.parent) throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", info.name, resource_name); @@ -80,7 +78,6 @@ void IOResourceManager::Resource::createNode(const NodeInfo & info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", info.parent, info.name, resource_name); - // TODO(serxa): make sure all possible callers validate second root, add tests for creating the second root if (info.parent.empty() && root_node) throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", info.name, root_node->basename, resource_name); diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index cda6b546dfc..6db3ef0dce9 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -33,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name); + void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name = {}); }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index f0c76b92870..8679c8639f6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,13 +1,14 @@ #include -#include -#include - +#include #include #include #include #include +#include +#include + #include #include @@ -52,18 +53,33 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } -void forEachReference(const ASTPtr & source_entity, std::function func) +enum class ReferenceType +{ + Parent, ForResource +}; + +void forEachReference(const ASTPtr & source_entity, std::function func) { if (auto * res = typeid_cast(source_entity.get())) { + // Parent reference String parent = res->getWorkloadParent(); if (!parent.empty()) - func(parent, res->getWorkloadName()); - // TODO(serxa): add references to RESOURCEs mentioned in SETTINGS clause after FOR keyword + func(parent, res->getWorkloadName(), ReferenceType::Parent); + + // References to RESOURCEs mentioned in SETTINGS clause after FOR keyword + std::unordered_set resources; + for (const auto & [name, value, resource] : res->changes) + { + if (!resource.empty()) + resources.insert(resource); + } + for (const String & resource : resources) + func(resource, res->getWorkloadName(), ReferenceType::ForResource); } if (auto * res = typeid_cast(source_entity.get())) { - // RESOURCE has no references to be validated + // RESOURCE has no references to be validated, we allow mentioned disks to be created later } } @@ -173,6 +189,16 @@ bool WorkloadEntityStorageBase::storeEntity( bool replace_if_exists, const Settings & settings) { + if (entity_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); + + auto * workload = typeid_cast(create_entity_query.get()); + if (workload) + { + if (entity_name == workload->getWorkloadParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Self-referencing workloads are not allowed."); + } + std::unique_lock lock{mutex}; create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); @@ -185,11 +211,35 @@ bool WorkloadEntityStorageBase::storeEntity( return false; } + std::optional new_root_name; + + // Validate workload + if (workload) + { + if (!workload->hasParent()) + { + if (!root_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); + new_root_name = workload->getWorkloadName(); + } + + SchedulingSettings validator; + validator.updateFromChanges(workload->changes); + } + forEachReference(create_entity_query, - [this] (const String & target, const String & source) + [this, workload] (const String & target, const String & source, ReferenceType type) { if (auto it = entities.find(target); it == entities.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + + // Validate that we could parse the settings for specific resource + if (type == ReferenceType::ForResource) + { + // TODO(serxa): check this is a target is a resource, not workload + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + } }); bool stored = storeEntityImpl( @@ -203,8 +253,10 @@ bool WorkloadEntityStorageBase::storeEntity( if (stored) { + if (new_root_name) + root_name = *new_root_name; forEachReference(create_entity_query, - [this] (const String & target, const String & source) + [this] (const String & target, const String & source, ReferenceType) { references[target].insert(source); }); @@ -248,8 +300,10 @@ bool WorkloadEntityStorageBase::removeEntity( if (removed) { + if (entity_name == root_name) + root_name.clear(); forEachReference(it->second, - [this] (const String & target, const String & source) + [this] (const String & target, const String & source, ReferenceType) { references[target].erase(source); if (references[target].empty()) @@ -354,7 +408,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector queue; mutable std::recursive_mutex mutex; - std::unordered_map entities; // Maps entity name into CREATE entity query - std::unordered_map> references; // Keep track of references between entities for validation + std::unordered_map entities; /// Maps entity name into CREATE entity query + + // Validation + std::unordered_map> references; /// Keep track of references between entities + String root_name; /// current root workload name ContextPtr global_context; }; diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference index 4bac2ef71f2..923e8652a35 100644 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.reference +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -1,5 +1,5 @@ all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` development all CREATE WORKLOAD development IN `all` production all CREATE WORKLOAD production IN `all` all CREATE WORKLOAD `all` -all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql index 38a7dad7cbc..1d8f97baf4c 100644 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.sql +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -1,11 +1,11 @@ -- Tags: no-parallel -- Do not run this test in parallel because `all` workload might affect other queries execution process CREATE OR REPLACE WORKLOAD all; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; CREATE WORKLOAD IF NOT EXISTS production IN all; CREATE WORKLOAD development IN all; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; DROP WORKLOAD IF EXISTS production; DROP WORKLOAD development; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; DROP WORKLOAD all; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.reference b/tests/queries/0_stateless/03232_workloads_and_resources.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql new file mode 100644 index 00000000000..1653659bcc4 --- /dev/null +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -0,0 +1,17 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +create resource 03232_write (write disk 03232_fake_disk); +create resource 03232_read (read disk 03232_fake_disk); +create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} +create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create workload admin in all settings priority = 0; +create workload production in all settings priority = 1, weight = 9; +create workload development in all settings priority = 1, weight = 1; +create workload another_root; -- {serverError BAD_ARGUMENTS} + +drop workload if exists production; +drop workload if exists development; +drop workload if exists admin; +drop workload if exists all; +drop resource if exists 03232_write; +drop resource if exists 03232_read; From ba5a0e98e3acc83531542ed6b35b57a1a0c10fee Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 30 Sep 2024 13:03:17 +0000 Subject: [PATCH 109/816] fix build --- src/AggregateFunctions/fuzzers/CMakeLists.txt | 2 +- src/Core/fuzzers/CMakeLists.txt | 2 +- src/DataTypes/fuzzers/CMakeLists.txt | 2 +- src/Formats/fuzzers/CMakeLists.txt | 2 +- src/Interpreters/fuzzers/CMakeLists.txt | 1 + src/Storages/fuzzers/CMakeLists.txt | 2 +- 6 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/CMakeLists.txt b/src/AggregateFunctions/fuzzers/CMakeLists.txt index 6a7be0d4377..f01bcb0b631 100644 --- a/src/AggregateFunctions/fuzzers/CMakeLists.txt +++ b/src/AggregateFunctions/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(aggregate_function_state_deserialization_fuzzer aggregate_function_state_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions) +target_link_libraries(aggregate_function_state_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms) diff --git a/src/Core/fuzzers/CMakeLists.txt b/src/Core/fuzzers/CMakeLists.txt index c60ce0e097f..51db6fa0b53 100644 --- a/src/Core/fuzzers/CMakeLists.txt +++ b/src/Core/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (names_and_types_fuzzer names_and_types_fuzzer.cpp) -target_link_libraries (names_and_types_fuzzer PRIVATE) +target_link_libraries (names_and_types_fuzzer PRIVATE dbms) diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 9e5b1b3f673..8dedd3470e2 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) -target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions) +target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms) diff --git a/src/Formats/fuzzers/CMakeLists.txt b/src/Formats/fuzzers/CMakeLists.txt index ee1a4fd4358..83aa5eb781a 100644 --- a/src/Formats/fuzzers/CMakeLists.txt +++ b/src/Formats/fuzzers/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable(format_fuzzer format_fuzzer.cpp ${SRCS}) -target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions) +target_link_libraries(format_fuzzer PRIVATE clickhouse_aggregate_functions dbms) diff --git a/src/Interpreters/fuzzers/CMakeLists.txt b/src/Interpreters/fuzzers/CMakeLists.txt index 3317bba7e30..174fae299b7 100644 --- a/src/Interpreters/fuzzers/CMakeLists.txt +++ b/src/Interpreters/fuzzers/CMakeLists.txt @@ -3,5 +3,6 @@ target_link_libraries(execute_query_fuzzer PRIVATE dbms clickhouse_table_functions clickhouse_aggregate_functions + clickhouse_functions clickhouse_dictionaries clickhouse_dictionaries_embedded) diff --git a/src/Storages/fuzzers/CMakeLists.txt b/src/Storages/fuzzers/CMakeLists.txt index 2c7c0c16fc2..719b9b77cd9 100644 --- a/src/Storages/fuzzers/CMakeLists.txt +++ b/src/Storages/fuzzers/CMakeLists.txt @@ -4,4 +4,4 @@ clickhouse_add_executable (mergetree_checksum_fuzzer mergetree_checksum_fuzzer.c target_link_libraries (mergetree_checksum_fuzzer PRIVATE dbms) clickhouse_add_executable (columns_description_fuzzer columns_description_fuzzer.cpp) -target_link_libraries (columns_description_fuzzer PRIVATE) +target_link_libraries (columns_description_fuzzer PRIVATE dbms) From 2117a29eb17ea81c653755396599ffaa06477cac Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 11:09:27 -0300 Subject: [PATCH 110/816] introduce the concept of global_valid_until --- src/Access/AuthenticationData.cpp | 13 ++-- src/Access/AuthenticationData.h | 12 +++- .../Access/InterpreterCreateUserQuery.cpp | 59 ++++++++++++++++--- src/Parsers/Access/ASTAuthenticationData.cpp | 6 ++ src/Parsers/Access/ASTCreateUserQuery.cpp | 13 ++++ src/Parsers/Access/ASTCreateUserQuery.h | 2 + src/Parsers/Access/ParserCreateUserQuery.cpp | 15 +++-- 7 files changed, 98 insertions(+), 22 deletions(-) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index b5f76e1e317..feff2794702 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -147,7 +147,8 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) && (lhs.ssh_keys == rhs.ssh_keys) #endif && (lhs.http_auth_scheme == rhs.http_auth_scheme) - && (lhs.http_auth_server_name == rhs.http_auth_server_name); + && (lhs.http_auth_server_name == rhs.http_auth_server_name) + && (lhs.valid_until == rhs.valid_until); } @@ -419,7 +420,7 @@ std::shared_ptr AuthenticationData::toAST() const if (valid_until) { WriteBufferFromOwnString out; - writeDateTimeText(*valid_until, out); + writeDateTimeText(valid_until, out); node->valid_until = std::make_shared(out.str()); } @@ -440,7 +441,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que if (query.type && query.type == AuthenticationType::NO_PASSWORD) { AuthenticationData auth_data; - auth_data.setValidUntil(valid_until); + auth_data.setValidUntilIfNotNull(valid_until); return auth_data; } @@ -469,7 +470,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que } auth_data.setSSHKeys(std::move(keys)); - auth_data.setValidUntil(valid_until); + auth_data.setValidUntilIfNotNull(valid_until); return auth_data; #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); @@ -541,13 +542,13 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que #endif } - auth_data.setValidUntil(valid_until); + auth_data.setValidUntilIfNotNull(valid_until); auth_data.setPassword(value); return auth_data; } AuthenticationData auth_data(*query.type); - auth_data.setValidUntil(valid_until); + auth_data.setValidUntilIfNotNull(valid_until); if (query.contains_hash) { diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index bdcd8cbb14d..43e59d1b239 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -74,8 +74,14 @@ public: const String & getHTTPAuthenticationServerName() const { return http_auth_server_name; } void setHTTPAuthenticationServerName(const String & name) { http_auth_server_name = name; } - std::optional getValidUntil() const { return valid_until; } - void setValidUntil(std::optional valid_until_) { valid_until = valid_until_; } + time_t getValidUntil() const { return valid_until; } + void setValidUntil(time_t valid_until_) { valid_until = valid_until_; } + void setValidUntilIfNotNull(std::optional valid_until_) { + if (valid_until_) + { + setValidUntil(*valid_until_); + } + } friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } @@ -109,7 +115,7 @@ private: /// HTTP authentication properties String http_auth_server_name; HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC; - std::optional valid_until; + time_t valid_until = 0; }; } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 89478996899..b7784b6a9b0 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -39,6 +39,7 @@ namespace const std::optional & override_default_roles, const std::optional & override_settings, const std::optional & override_grantees, + const std::optional & global_valid_until, bool reset_authentication_methods, bool replace_authentication_methods, bool allow_implicit_no_password, @@ -99,12 +100,17 @@ namespace user.authentication_methods.emplace_back(authentication_method); } - bool has_no_password_authentication_method = std::find_if(user.authentication_methods.begin(), - user.authentication_methods.end(), - [](const AuthenticationData & authentication_data) - { - return authentication_data.getType() == AuthenticationType::NO_PASSWORD; - }) != user.authentication_methods.end(); + bool has_no_password_authentication_method = false; + + for (auto & authentication_method : user.authentication_methods) + { + authentication_method.setValidUntilIfNotNull(global_valid_until); + + if (authentication_method.getType() == AuthenticationType::NO_PASSWORD) + { + has_no_password_authentication_method = true; + } + } if (has_no_password_authentication_method && user.authentication_methods.size() > 1) { @@ -166,6 +172,34 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } + + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } } BlockIO InterpreterCreateUserQuery::execute() @@ -189,6 +223,10 @@ BlockIO InterpreterCreateUserQuery::execute() } } + std::optional global_valid_until; + if (query.global_valid_until) + global_valid_until = getValidUntilFromAST(query.global_valid_until, getContext()); + std::optional default_roles_from_query; if (query.default_roles) { @@ -233,7 +271,7 @@ BlockIO InterpreterCreateUserQuery::execute() auto updated_user = typeid_cast>(entity->clone()); updateUserFromQueryImpl( *updated_user, query, authentication_methods, {}, default_roles_from_query, settings_from_query, grantees_from_query, - query.reset_authentication_methods_to_new, query.replace_authentication_methods, + global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user); return updated_user; @@ -255,7 +293,7 @@ BlockIO InterpreterCreateUserQuery::execute() auto new_user = std::make_shared(); updateUserFromQueryImpl( *new_user, query, authentication_methods, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, - query.reset_authentication_methods_to_new, query.replace_authentication_methods, + global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, implicit_no_password_allowed, no_password_allowed, plaintext_password_allowed, getContext()->getServerSettings().max_authentication_methods_per_user); new_users.emplace_back(std::move(new_user)); @@ -310,6 +348,10 @@ void InterpreterCreateUserQuery::updateUserFromQuery( } } + std::optional global_valid_until; + if (query.global_valid_until) + global_valid_until = getValidUntilFromAST(query.global_valid_until, {}); + updateUserFromQueryImpl( user, query, @@ -318,6 +360,7 @@ void InterpreterCreateUserQuery::updateUserFromQuery( {}, {}, {}, + global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods, allow_no_password, diff --git a/src/Parsers/Access/ASTAuthenticationData.cpp b/src/Parsers/Access/ASTAuthenticationData.cpp index 9fa75185d32..0b2eebb3311 100644 --- a/src/Parsers/Access/ASTAuthenticationData.cpp +++ b/src/Parsers/Access/ASTAuthenticationData.cpp @@ -55,6 +55,12 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " no_password" << (settings.hilite ? IAST::hilite_none : ""); + + if (valid_until) + { + formatValidUntil(*valid_until, settings); + } + return; } diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 25c2f805781..956d976014b 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -40,6 +40,12 @@ namespace } } + void formatValidUntil(const IAST & valid_until, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " VALID UNTIL " << (settings.hilite ? IAST::hilite_none : ""); + valid_until.format(settings); + } + void formatHosts(const char * prefix, const AllowedClientHosts & hosts, const IAST::FormatSettings & settings) { if (prefix) @@ -254,6 +260,13 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & formatAuthenticationData(authentication_methods, format); } + if (global_valid_until) + { + // todo arthur: is this correct? Should we actually format it? + chassert(authentication_methods.empty()); + formatValidUntil(*global_valid_until, format); + } + if (hosts) formatHosts(nullptr, *hosts, format); if (add_hosts) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 347552a9f11..8926c7cad44 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -62,6 +62,8 @@ public: std::shared_ptr default_database; + ASTPtr global_valid_until; + String getID(char) const override; ASTPtr clone() const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 7c171432b66..8ec253d5cc3 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -238,7 +238,7 @@ namespace if (parseValidUntil(pos, expected, auth_data->valid_until)) { - // I am still not sure why this has to be done and if it has to be done + // todo arthur I am still not sure why this has to be done and if it has to be done auth_data->children.push_back(auth_data->valid_until); } @@ -560,7 +560,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::shared_ptr settings; std::shared_ptr grantees; std::shared_ptr default_database; - ASTPtr valid_until; + ASTPtr global_valid_until; String cluster; String storage_name; bool reset_authentication_methods_to_new = false; @@ -641,6 +641,11 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; + if (auth_data.empty() && !global_valid_until) + { + parseValidUntil(pos, expected, global_valid_until); + } + break; } @@ -675,6 +680,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->settings = std::move(settings); query->grantees = std::move(grantees); query->default_database = std::move(default_database); + query->global_valid_until = std::move(global_valid_until); query->storage_name = std::move(storage_name); query->reset_authentication_methods_to_new = reset_authentication_methods_to_new; query->add_identified_with = parsed_add_identified_with; @@ -685,9 +691,8 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->children.push_back(authentication_method); } - // todo arthur -// if (query->valid_until) -// query->children.push_back(query->valid_until); + if (query->global_valid_until) + query->children.push_back(query->global_valid_until); return true; } From a4a1401f71a9f1b57050a8a0618c9a097d2927ce Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 11:19:05 -0300 Subject: [PATCH 111/816] removei setValidUntilIfNotNull --- src/Access/AuthenticationData.cpp | 10 +++++----- src/Access/AuthenticationData.h | 6 ------ src/Interpreters/Access/InterpreterCreateUserQuery.cpp | 5 ++++- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index feff2794702..01d10f9ebe6 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -431,7 +431,7 @@ std::shared_ptr AuthenticationData::toAST() const AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool check_password_rules) { - std::optional valid_until; + time_t valid_until = 0; if (query.valid_until) { @@ -441,7 +441,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que if (query.type && query.type == AuthenticationType::NO_PASSWORD) { AuthenticationData auth_data; - auth_data.setValidUntilIfNotNull(valid_until); + auth_data.setValidUntil(valid_until); return auth_data; } @@ -470,7 +470,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que } auth_data.setSSHKeys(std::move(keys)); - auth_data.setValidUntilIfNotNull(valid_until); + auth_data.setValidUntil(valid_until); return auth_data; #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); @@ -542,13 +542,13 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que #endif } - auth_data.setValidUntilIfNotNull(valid_until); + auth_data.setValidUntil(valid_until); auth_data.setPassword(value); return auth_data; } AuthenticationData auth_data(*query.type); - auth_data.setValidUntilIfNotNull(valid_until); + auth_data.setValidUntil(valid_until); if (query.contains_hash) { diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index 43e59d1b239..3c601144eb8 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -76,12 +76,6 @@ public: time_t getValidUntil() const { return valid_until; } void setValidUntil(time_t valid_until_) { valid_until = valid_until_; } - void setValidUntilIfNotNull(std::optional valid_until_) { - if (valid_until_) - { - setValidUntil(*valid_until_); - } - } friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index b7784b6a9b0..851d5c18c94 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -104,7 +104,10 @@ namespace for (auto & authentication_method : user.authentication_methods) { - authentication_method.setValidUntilIfNotNull(global_valid_until); + if (global_valid_until) + { + authentication_method.setValidUntil(*global_valid_until); + } if (authentication_method.getType() == AuthenticationType::NO_PASSWORD) { From 3af5bd6a49c6f5bfb49658138eee6835ae1fc5a5 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 30 Sep 2024 14:56:34 +0000 Subject: [PATCH 112/816] ignore encoding errors in fuzzers output --- tests/fuzz/runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index deb219baff9..6f229725d4e 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -133,6 +133,7 @@ def run_fuzzer(fuzzer: str): text=True, check=True, shell=True, + errors='replace', ) except subprocess.CalledProcessError as e: # print("Command failed with error:", e) From 07fd719c8b2be80d08f088c2849a5fc150b98bc5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 30 Sep 2024 15:03:00 +0000 Subject: [PATCH 113/816] Automatic style fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 6f229725d4e..e6eff430d1b 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -133,7 +133,7 @@ def run_fuzzer(fuzzer: str): text=True, check=True, shell=True, - errors='replace', + errors="replace", ) except subprocess.CalledProcessError as e: # print("Command failed with error:", e) From 46ada08197d1dab15116485cf85739a710685b5e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 Sep 2024 16:27:27 +0000 Subject: [PATCH 114/816] fix tests --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +- src/Planner/PlannerJoinTree.cpp | 3 +- src/Processors/QueryPlan/JoinStep.cpp | 24 +++++++++-- src/Processors/QueryPlan/JoinStep.h | 4 +- .../0_stateless/00826_cross_to_inner_join.sql | 3 +- .../01107_join_right_table_totals.reference | 7 ++++ .../01107_join_right_table_totals.sql | 10 ++++- .../01881_join_on_conditions_hash.sql.j2 | 10 ++--- ...oin_with_nullable_lowcardinality_crash.sql | 5 ++- .../0_stateless/02282_array_distance.sql | 12 ++++-- .../02381_join_dup_columns_in_plan.reference | 1 - .../0_stateless/02461_join_lc_issue_42380.sql | 3 +- .../02514_analyzer_drop_join_on.reference | 1 - .../02835_join_step_explain.reference | 10 ++--- ...filter_push_down_equivalent_sets.reference | 40 ++++++++++++++----- ..._join_filter_push_down_equivalent_sets.sql | 40 ++++++++++++++----- .../03038_recursive_cte_postgres_4.reference | 4 +- .../03038_recursive_cte_postgres_4.sql | 4 +- ...03130_convert_outer_join_to_inner_join.sql | 13 ++++-- ...ter_push_down_equivalent_columns.reference | 3 +- .../03236_squashing_high_memory.sql | 1 + 21 files changed, 145 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 547f8d63c7f..c830d95eada 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1886,7 +1886,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 3c540c3ef81..720f0a380ab 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1641,7 +1641,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ settings[Setting::max_block_size], settings[Setting::max_threads], outer_scope_columns, - false /*optimize_read_in_order*/); + false /*optimize_read_in_order*/, + true /*optimize_skip_unused_shards*/); join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 9fdfeedb111..8365af4e589 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -92,12 +92,14 @@ JoinStep::JoinStep( size_t max_block_size_, size_t max_streams_, NameSet required_output_, - bool keep_left_read_in_order_) + bool keep_left_read_in_order_, + bool use_new_analyzer_) : join(std::move(join_)) , max_block_size(max_block_size_) , max_streams(max_streams_) , required_output(std::move(required_output_)) , keep_left_read_in_order(keep_left_read_in_order_) + , use_new_analyzer(use_new_analyzer_) { updateInputStreams(DataStreams{left_stream_, right_stream_}); } @@ -130,6 +132,9 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines keep_left_read_in_order, &processors); + if (!use_new_analyzer) + return pipeline; + const auto & result_names = pipeline->getHeader().getNames(); size_t prefix_size = getPrefixLength(rhs_names, result_names); if (!columns_to_remove.empty() || (0 < prefix_size && prefix_size < result_names.size())) @@ -184,19 +189,30 @@ void JoinStep::updateOutputStream() const auto & header = swap_streams ? input_streams[1].header : input_streams[0].header; Block result_header = JoiningTransform::transformHeader(header, join); - join_algorithm_header = result_header; + + if (!use_new_analyzer) + { + if (swap_streams) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer"); + output_stream = DataStream { .header = result_header }; + return; + } + + if (swap_streams) result_header = rotateBlock(result_header, input_streams[1].header); columns_to_remove.clear(); for (size_t i = 0; i < result_header.columns(); ++i) { - if (required_output.empty()) - break; if (!required_output.contains(result_header.getByPosition(i).name)) columns_to_remove.insert(i); } + /// Do not remove all columns, keep at least one + if (!columns_to_remove.empty() && columns_to_remove.size() == result_header.columns()) + columns_to_remove.erase(columns_to_remove.begin()); + result_header.erase(columns_to_remove); output_stream = DataStream { .header = result_header }; } diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 30b20a0d3a5..b0947cb6be7 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -21,7 +21,8 @@ public: size_t max_block_size_, size_t max_streams_, NameSet required_output_, - bool keep_left_read_in_order_); + bool keep_left_read_in_order_, + bool use_new_analyzer_); String getName() const override { return "Join"; } @@ -53,6 +54,7 @@ private: NameSet required_output; std::set columns_to_remove; bool keep_left_read_in_order; + bool use_new_analyzer = false; }; /// Special step for the case when Join is already filled. diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index e9f9e13e2d3..f81832a4109 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -48,7 +48,8 @@ SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- comma nullable ---'; SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) +ORDER BY ALL; SELECT '--- cross ---'; diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.reference b/tests/queries/0_stateless/01107_join_right_table_totals.reference index daf503b776d..aa569ff9331 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.reference +++ b/tests/queries/0_stateless/01107_join_right_table_totals.reference @@ -18,28 +18,35 @@ 0 0 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 0 0 +- 1 foo 1 1 300 0 foo 1 0 300 +- 1 100 1970-01-01 1 100 1970-01-01 1 100 1970-01-01 1 200 1970-01-02 1 200 1970-01-02 1 100 1970-01-01 diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.sql b/tests/queries/0_stateless/01107_join_right_table_totals.sql index ad8954d5d70..7e549282489 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.sql +++ b/tests/queries/0_stateless/01107_join_right_table_totals.sql @@ -64,39 +64,47 @@ USING (id); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l RIGHT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r -ON l.item_id = r.item_id; +ON l.item_id = r.item_id +ORDER BY ALL; DROP TABLE t; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c2d85cefb18..c13722f431a 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; {% endfor -%} diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql index abc2ee41402..c3c84ebaded 100644 --- a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -12,8 +12,9 @@ CREATE TABLE without_nullable insert into with_nullable values(0,'f'),(0,'usa'); insert into without_nullable values(0,'usa'),(0,'us2a'); -select if(t0.country is null ,t2.country,t0.country) "country" -from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country +ORDER BY 1 DESC; drop table with_nullable; drop table without_nullable; diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 2cca853fd67..85abc8fa381 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -48,7 +48,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2 v1, vec2 v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -61,7 +62,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2f v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -74,7 +76,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2d v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT v1.id, @@ -86,7 +89,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 365725f8ffe..90aab0a0eb2 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -148,7 +148,6 @@ Header: key String value String Join Header: __table1.key String - __table3.key String __table3.value String Sorting Header: __table1.key String diff --git a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql index f0ecbf64e58..8b5c6846bd0 100644 --- a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql +++ b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql @@ -9,4 +9,5 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B INSERT INTO t1__fuzz_13 VALUES (1); INSERT INTO t2__fuzz_47 VALUES (1); -SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2; +SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2 +ORDER BY ALL; diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 59983fff778..d407a4c7985 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -50,7 +50,6 @@ Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table1.k UInt64 __table4.d2 String Expression (DROP unused columns after JOIN) Header: __table1.a2 String diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 31205956662..2f641d4aa44 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -58,18 +58,16 @@ Header: id UInt64 Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 INPUT : 2 -> __table2.value_1 String : 2 - INPUT :: 3 -> __table2.value_2 UInt64 : 3 - INPUT : 4 -> __table2.id UInt64 : 4 - ALIAS __table1.id :: 0 -> id UInt64 : 5 + INPUT : 3 -> __table2.id UInt64 : 3 + ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 4 -> rhs.id UInt64 : 2 -Positions: 5 0 2 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String __table2.value_1 String - __table2.value_2 UInt64 __table2.id UInt64 Type: INNER Strictness: ASOF diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 80f4e309505..c98a98b236c 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -2,7 +2,9 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -69,7 +71,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right'; +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -136,7 +140,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -206,7 +212,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -273,7 +281,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -347,7 +357,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -421,7 +433,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -488,7 +502,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -562,7 +578,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -636,7 +654,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index e1a13d1ce71..d6dcc34c796 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -22,7 +22,9 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -33,7 +35,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right'; +; SELECT '--'; @@ -44,7 +48,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6; @@ -53,7 +59,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -64,7 +72,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -75,7 +85,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -86,7 +98,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -97,7 +111,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -108,7 +124,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -119,7 +137,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index cf070eebc38..7df38e855f6 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -52,7 +52,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_inner_table_selection = 'right' +; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] 2 3 arc 2 -> 3 false [(2,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index 7dad74893b9..d33ca7b078e 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -55,7 +55,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_inner_table_selection = 'right' +; -- ordering by the path column has same effect as SEARCH DEPTH FIRST WITH RECURSIVE search_graph AS ( diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index b3d1827d98f..ddefc322b4f 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -22,7 +22,10 @@ SETTINGS index_granularity = 16 INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0; + +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -30,7 +33,9 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -38,7 +43,9 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference index 7058d36aaf9..1c82e76cc65 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference @@ -65,8 +65,7 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice'; Expression ((Project names + (Projection + ))) Header: name String Join (JOIN FillRightFirst) - Header: __table1.name String - __table2.name String + Header: __table2.name String Filter (( + Change column names to column identifiers)) Header: __table1.name String ReadFromMergeTree (default.users) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index f6e5dbdef03..eeb3ae85e84 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,6 +11,7 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; +SET query_plan_join_inner_table_selection = 'right'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, From 417a0a8017649502de262414043de00dce6413c4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 13:47:08 -0300 Subject: [PATCH 115/816] retrigger ci, something is off From 16fb8d883cbe3c2c8a9c468b4799a8fc7d7b5563 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 14:58:42 -0300 Subject: [PATCH 116/816] remove validuntil --- src/Access/User.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 76f6059be13..1c92f467003 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -87,7 +87,6 @@ void User::clearAllExceptDependencies() access = {}; settings.removeSettingsKeepProfiles(); default_database = {}; - valid_until = 0; } } From e077e0c7fef593d933c8b0e5eb9a46d07056564e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 15:28:21 -0300 Subject: [PATCH 117/816] tests for expiration time for each auth metho --- .../integration/test_user_valid_until/test.py | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index eea05af9e45..8e0bcb8b353 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -124,3 +124,41 @@ def test_restart(started_cluster): assert error in node.query_and_get_error("SELECT 1", user="user_restart") node.query("DROP USER IF EXISTS user_restart") + + +def test_multiple_authentication_methods(started_cluster): + node.query("DROP USER IF EXISTS user_basic") + + node.query( + "CREATE USER user_basic IDENTIFIED WITH plaintext_password BY 'no_expiration'," + "plaintext_password by 'not_expired' VALID UNTIL '06/11/2040', plaintext_password by 'expired' VALID UNTIL '06/11/2010'," + "plaintext_password by 'infinity' VALID UNTIL 'infinity'" + ) + + assert ( + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH plaintext_password, plaintext_password VALID UNTIL \\'2040-11-06 00:00:00\\', " + "plaintext_password VALID UNTIL \\'2010-11-06 00:00:00\\', plaintext_password\n" + ) + assert node.query("SELECT 1", user="user_basic", password="no_expiration") == "1\n" + assert node.query("SELECT 1", user="user_basic", password="not_expired") == "1\n" + assert node.query("SELECT 1", user="user_basic", password="infinity") == "1\n" + + error = "Authentication failed" + assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="expired") + + # Expire them all + node.query("ALTER USER user_basic VALID UNTIL '06/11/2010 08:03:20'") + + assert ( + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'\n" + ) + + assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="no_expiration") + assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="not_expired") + assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="infinity") + assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="expired") From d5a371e5bc5a8c30483ac2767372b812e57de076 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 15:44:02 -0300 Subject: [PATCH 118/816] fix style and add docs --- .../en/sql-reference/statements/alter/user.md | 14 +++++++- .../sql-reference/statements/create/user.md | 6 ++-- .../integration/test_user_valid_until/test.py | 36 ++++++++++++------- 3 files changed, 40 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/user.md b/docs/en/sql-reference/statements/alter/user.md index a56532e2ab0..1514b16a657 100644 --- a/docs/en/sql-reference/statements/alter/user.md +++ b/docs/en/sql-reference/statements/alter/user.md @@ -12,7 +12,7 @@ Syntax: ``` sql ALTER USER [IF EXISTS] name1 [RENAME TO new_name |, name2 [,...]] [ON CLUSTER cluster_name] - [NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} + [NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} [VALID UNTIL datetime] [, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]] [[ADD | DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [VALID UNTIL datetime] @@ -91,3 +91,15 @@ Reset authentication methods and keep the most recent added one: ``` sql ALTER USER user1 RESET AUTHENTICATION METHODS TO NEW ``` + +## VALID UNTIL Clause + +Allows you to specify the expiration date and, optionally, the time for an authentication method. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`. +The `VALID UNTIL` clause can only be specified along with an authentication method, except for the case where no authentication method has been specified in the query. In this scenario, the `VALID UNTIL` clause will be applied to all existing authentication methods. + +Examples: + +- `ALTER USER name1 VALID UNTIL '2025-01-01'` +- `ALTER USER name1 VALID UNTIL '2025-01-01 12:00:00 UTC'` +- `ALTER USER name1 VALID UNTIL 'infinity'` +- `ALTER USER name1 IDENTIFIED WITH plaintext_password BY 'no_expiration', bcrypt_password BY 'expiration_set' VALID UNTIL'2025-01-01''` diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index a018e28306c..afc679580dc 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -11,7 +11,7 @@ Syntax: ``` sql CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [, name2 [,...]] [ON CLUSTER cluster_name] - [NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} + [NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} [VALID UNTIL datetime] [, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [VALID UNTIL datetime] @@ -178,13 +178,15 @@ ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technica ## VALID UNTIL Clause -Allows you to specify the expiration date and, optionally, the time for user credentials. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`. +Allows you to specify the expiration date and, optionally, the time for an authentication method. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`. +The `VALID UNTIL` clause can only be specified along with an authentication method, except for the case where no authentication method has been specified in the query. In this scenario, the `VALID UNTIL` clause will be applied to all existing authentication methods. Examples: - `CREATE USER name1 VALID UNTIL '2025-01-01'` - `CREATE USER name1 VALID UNTIL '2025-01-01 12:00:00 UTC'` - `CREATE USER name1 VALID UNTIL 'infinity'` +- `CREATE USER name1 IDENTIFIED WITH plaintext_password BY 'no_expiration', bcrypt_password BY 'expiration_set' VALID UNTIL'2025-01-01''` ## GRANTEES Clause diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index 8e0bcb8b353..565790457b2 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -136,29 +136,39 @@ def test_multiple_authentication_methods(started_cluster): ) assert ( - node.query("SHOW CREATE USER user_basic") - == "CREATE USER user_basic IDENTIFIED WITH plaintext_password, plaintext_password VALID UNTIL \\'2040-11-06 00:00:00\\', " - "plaintext_password VALID UNTIL \\'2010-11-06 00:00:00\\', plaintext_password\n" + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH plaintext_password, plaintext_password VALID UNTIL \\'2040-11-06 00:00:00\\', " + "plaintext_password VALID UNTIL \\'2010-11-06 00:00:00\\', plaintext_password\n" ) assert node.query("SELECT 1", user="user_basic", password="no_expiration") == "1\n" assert node.query("SELECT 1", user="user_basic", password="not_expired") == "1\n" assert node.query("SELECT 1", user="user_basic", password="infinity") == "1\n" error = "Authentication failed" - assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="expired") + assert error in node.query_and_get_error( + "SELECT 1", user="user_basic", password="expired" + ) # Expire them all node.query("ALTER USER user_basic VALID UNTIL '06/11/2010 08:03:20'") assert ( - node.query("SHOW CREATE USER user_basic") - == "CREATE USER user_basic IDENTIFIED WITH plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," - " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," - " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," - " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'\n" + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'," + " plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'\n" ) - assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="no_expiration") - assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="not_expired") - assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="infinity") - assert error in node.query_and_get_error("SELECT 1", user="user_basic", password="expired") + assert error in node.query_and_get_error( + "SELECT 1", user="user_basic", password="no_expiration" + ) + assert error in node.query_and_get_error( + "SELECT 1", user="user_basic", password="not_expired" + ) + assert error in node.query_and_get_error( + "SELECT 1", user="user_basic", password="infinity" + ) + assert error in node.query_and_get_error( + "SELECT 1", user="user_basic", password="expired" + ) From 7a568e8d0ced8a9f4cff5795a3618975477d6724 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 16:00:35 -0300 Subject: [PATCH 119/816] unify getvaliduntilfromast impl --- src/Access/AuthenticationData.cpp | 32 +--------------- .../Access/InterpreterCreateUserQuery.cpp | 29 +-------------- .../Access/getValidUntilFromAST.cpp | 37 +++++++++++++++++++ .../Access/getValidUntilFromAST.h | 9 +++++ 4 files changed, 48 insertions(+), 59 deletions(-) create mode 100644 src/Interpreters/Access/getValidUntilFromAST.cpp create mode 100644 src/Interpreters/Access/getValidUntilFromAST.h diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 01d10f9ebe6..9223c3a8059 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -43,37 +44,6 @@ namespace ErrorCodes extern const int OPENSSL_ERROR; } -namespace -{ - time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) - { - if (context) - valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); - - const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); - - if (valid_until_str == "infinity") - return 0; - - time_t time = 0; - ReadBufferFromString in(valid_until_str); - - if (context) - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - else - { - readDateTimeText(time, in); - } - - return time; - } -} - AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(std::string_view text [[maybe_unused]]) { #if USE_SSL diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 7d4693228cf..fc0f7610c90 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -175,34 +176,6 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } - - time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) - { - if (context) - valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); - - const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); - - if (valid_until_str == "infinity") - return 0; - - time_t time = 0; - ReadBufferFromString in(valid_until_str); - - if (context) - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - else - { - readDateTimeText(time, in); - } - - return time; - } } BlockIO InterpreterCreateUserQuery::execute() diff --git a/src/Interpreters/Access/getValidUntilFromAST.cpp b/src/Interpreters/Access/getValidUntilFromAST.cpp new file mode 100644 index 00000000000..caf831e61ee --- /dev/null +++ b/src/Interpreters/Access/getValidUntilFromAST.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } +} diff --git a/src/Interpreters/Access/getValidUntilFromAST.h b/src/Interpreters/Access/getValidUntilFromAST.h new file mode 100644 index 00000000000..ab0c6c8c9b6 --- /dev/null +++ b/src/Interpreters/Access/getValidUntilFromAST.h @@ -0,0 +1,9 @@ +#pragma once + +#include +#include + +namespace DB +{ + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context); +} From 21931b1d99c618227e66a83cd719e1492e0e7dc0 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 30 Sep 2024 21:47:46 +0000 Subject: [PATCH 120/816] add more worklaod validation and testing --- src/Common/Scheduler/ISchedulerNode.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 -- .../Scheduler/Workload/WorkloadEntityStorageBase.cpp | 4 +++- .../0_stateless/03232_workloads_and_resources.sql | 12 +++++++++++- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index d68a32e8290..73fd0759c6a 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -74,7 +74,7 @@ struct SchedulerNodeInfo if (value <= 0 || !isfinite(value)) throw Exception( ErrorCodes::INVALID_SCHEDULER_NODE, - "Negative and non-finite node weights are not allowed: {}", + "Zero, negative and non-finite node weights are not allowed: {}", value); weight = value; } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 07929e855ce..55defbd2432 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -73,7 +73,6 @@ void IOResourceManager::Resource::createNode(const NodeInfo & info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", info.name, resource_name); - // TODO(serxa): make sure all possible callers validate parent existence, add tests for creating workload with invalid parent if (!info.parent.empty() && !node_for_workload.contains(info.parent)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", info.parent, info.name, resource_name); @@ -111,7 +110,6 @@ void IOResourceManager::Resource::deleteNode(const NodeInfo & info) auto node = node_for_workload[info.name]; - // TODO(serxa): make sure all possible callers validate that removing workload has no children workloads if (node->hasUnifiedChildren()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", info.name, resource_name); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 8679c8639f6..6f633893d70 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -236,7 +236,9 @@ bool WorkloadEntityStorageBase::storeEntity( // Validate that we could parse the settings for specific resource if (type == ReferenceType::ForResource) { - // TODO(serxa): check this is a target is a resource, not workload + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); + SchedulingSettings validator; validator.updateFromChanges(workload->changes, target); } diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index 1653659bcc4..ae0061b3bd5 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -2,12 +2,22 @@ -- Do not run this test in parallel because `all` workload might affect other queries execution process create resource 03232_write (write disk 03232_fake_disk); create resource 03232_read (read disk 03232_fake_disk); -create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; + create workload another_root; -- {serverError BAD_ARGUMENTS} +create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} +drop workload all; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} +create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} +create workload invalid in all settings weight = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} drop workload if exists production; drop workload if exists development; From 860587c6c03624c35a5893e5d1be28349433ab23 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 1 Oct 2024 08:34:01 +0000 Subject: [PATCH 121/816] Tidy the test file --- .../test.py | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py index d7dc1618802..cce8928e5d5 100644 --- a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py +++ b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py @@ -43,33 +43,29 @@ def test_stop_waiting_for_offline_hosts(started_cluster): ) assert time.time() - start < timeout - start = time.time() node1.query( "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", settings=settings, ) - assert time.time() - start < timeout node4.stop() start = time.time() - with pytest.raises(Exception) as err: - node1.query( - "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", - settings=settings, - ) - assert "Return code: 159" in str(err.value) + assert "Code: 159. DB::Exception" in node1.query_and_get_error( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) + assert time.time() - start >= timeout start = time.time() - with pytest.raises(Exception) as err: - node1.query( - "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", - settings=settings, - ) - assert "Return code: 159" in str(err.value) + assert "Code: 159. DB::Exception" in node1.query_and_get_error( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) assert time.time() - start >= timeout + # set `distributed_ddl_output_mode` = `throw_only_active`` settings = { "distributed_ddl_task_timeout": timeout, "distributed_ddl_output_mode": "throw_only_active", From b25b711a7135fa82bd3bd41a2547d316bb5c9ff1 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 10:52:16 +0000 Subject: [PATCH 122/816] implement detach of a unified node --- src/Common/Scheduler/SchedulingSettings.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index 352e61fb560..60319cdd54c 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -12,8 +12,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -// TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage -// TODO(serxa): and probably we should add and persist version in filename for future changes void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) { struct { From 1ccdc196b702a0c02ba91501fbf7716c8905c03a Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 10:53:09 +0000 Subject: [PATCH 123/816] implement detach of a unified node --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 98 ++++++++++++++++++- 1 file changed, 97 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 76685319c34..2b2eb320e0a 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -87,6 +87,8 @@ private: SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached std::unordered_map children; // basename -> child + bool empty() const { return children.empty(); } + SchedulerNodePtr getRoot() { chassert(!children.empty()); @@ -122,6 +124,29 @@ private: reparent(child, root); return {}; // Root is the same } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue *, const UnifiedSchedulerNodePtr & child) + { + auto it = children.find(child->basename); + if (it == children.end()) + return {}; // unknown child + + children.erase(it); + if (children.size() == 1) + { + // Remove fair if the only child has left + chassert(root); + root.reset(); // it will be still alive because it is attached to hierarchy for now + return children.begin()->second; // The last child is a new root now + } + else if (children.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children have left) + } }; /// Handles all the children nodes with intermediate fair and/or priority nodes @@ -130,6 +155,9 @@ private: SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached std::unordered_map branches; /// Branches for different priority values + // Returns true iff there are no unified children attached + bool empty() const { return branches.empty(); } + /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) @@ -169,6 +197,42 @@ private: return {}; // Root is the same } } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + auto it = branches.find(child->info.priority); + if (it == branches.end()) + return {}; // unknown child + + auto & child_branch = it->second; + auto branch_root = child_branch.detachUnifiedChild(event_queue_, child); + if (child_branch.empty()) + { + branches.erase(it); + if (branches.size() == 1) + { + // Remove priority node if the only child-branch has left + chassert(root); + root.reset(); // it will be still alive because it is attached to hierarchy for now + return branches.begin()->second.getRoot(); // The last child-branch is a new root now + } + else if (branches.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children-branches have left) + } + if (branch_root) + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } + return {}; // Root is the same + } }; /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. @@ -193,6 +257,21 @@ private: return branch.attachUnifiedChild(event_queue_, child); } + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (queue) + return {}; // No-op, it already has no children + auto branch_root = branch.detachUnifiedChild(event_queue_, child); + if (branch.empty()) + { + createQueue(event_queue_); + return queue; + } + return branch_root; + } + private: void createQueue(EventQueue * event_queue_) { @@ -256,6 +335,22 @@ private: } return {}; } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (auto branch_root = branch.detachUnifiedChild(event_queue_, child)) + { + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } }; public: @@ -279,7 +374,8 @@ public: /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) { - UNUSED(child); // TODO(serxa): implement detachUnifiedChild() + if (auto new_child = impl.detachUnifiedChild(event_queue, child)) + reparent(new_child, this); } /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) From e28171d2b6ea1ffb8783f6141f59763684b4dfd4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Oct 2024 11:38:38 +0000 Subject: [PATCH 124/816] fix clang tidy --- src/Processors/Transforms/ColumnPermuteTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp index 2921bcac177..169dd2dc67e 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -11,8 +11,8 @@ void applyPermutation(std::vector & data, const std::vector & permuta { std::vector res; res.reserve(permutation.size()); - for (size_t i = 0; i < permutation.size(); ++i) - res.emplace_back(std::move(data[permutation[i]])); + for (size_t i : permutation) + res.emplace_back(std::move(data[i])); data = std::move(res); } From 335e1847fee258ce75639dbdce34fd0bdf5b040a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Oct 2024 11:39:51 +0000 Subject: [PATCH 125/816] up src/Core/SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d1f90f378e6..54c9f53f41b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -71,10 +71,10 @@ static std::initializer_list Date: Tue, 1 Oct 2024 12:00:11 +0000 Subject: [PATCH 126/816] randomize only latest version settings --- tests/integration/helpers/cluster.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f5f87947c0f..1687f049b25 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4592,7 +4592,12 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: + if ( + self.randomize_settings + and self.image == "clickhouse/integration-test" + and self.tag == "latest" + and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR + ): # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") From 3117224c7429a8cda989bac7b7877ed25a55e096 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 1 Oct 2024 14:02:17 +0000 Subject: [PATCH 127/816] add timeout for every fuzzer --- tests/fuzz/runner.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index e6eff430d1b..cfd60d8f259 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -62,7 +62,7 @@ def process_error(error: str): report(error_source, error_reason, call_stack, test_unit) -def run_fuzzer(fuzzer: str): +def run_fuzzer(fuzzer: str, timeout: int): logging.info("Running fuzzer %s...", fuzzer) seed_corpus_dir = f"{fuzzer}.in" @@ -134,6 +134,7 @@ def run_fuzzer(fuzzer: str): check=True, shell=True, errors="replace", + timeout=timeout, ) except subprocess.CalledProcessError as e: # print("Command failed with error:", e) @@ -148,10 +149,16 @@ def main(): subprocess.check_call("ls -al", shell=True) + timeout = 30 + + match = re.search(r"(^|\s+)-max_total_time=(\d+)($|\s)", FUZZER_ARGS) + if match: + timeout += match.group(2) + with Path() as current: for fuzzer in current.iterdir(): if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): - run_fuzzer(fuzzer) + run_fuzzer(fuzzer, timeout) if __name__ == "__main__": From 77e13544d6d5641a68a765c7e15f7af4b9bfec00 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 1 Oct 2024 14:03:05 +0000 Subject: [PATCH 128/816] Parallel relicas: use local plan for local replica by default --- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index d0ce90e6fdd..dfba3b128bb 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -965,7 +965,7 @@ namespace ErrorCodes M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 0, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384]", 0) \ M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ - M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \ + M(Bool, parallel_replicas_local_plan, true, "If true, use local plan for local replica in a query with parallel replicas, otherwise all replicas in a used cluster considered as remote", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 560f144866b..92cf586b9c6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -67,6 +67,7 @@ static std::initializer_list Date: Tue, 1 Oct 2024 11:21:54 -0300 Subject: [PATCH 129/816] remove comments --- src/Parsers/Access/ASTCreateUserQuery.cpp | 2 -- src/Parsers/Access/ParserCreateUserQuery.cpp | 1 - 2 files changed, 3 deletions(-) diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 956d976014b..eb4503acf82 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -262,8 +262,6 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & if (global_valid_until) { - // todo arthur: is this correct? Should we actually format it? - chassert(authentication_methods.empty()); formatValidUntil(*global_valid_until, format); } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 8ec253d5cc3..9643bf2d8fb 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -238,7 +238,6 @@ namespace if (parseValidUntil(pos, expected, auth_data->valid_until)) { - // todo arthur I am still not sure why this has to be done and if it has to be done auth_data->children.push_back(auth_data->valid_until); } From 73101bf4b237937751adac877d750c26ba347650 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Oct 2024 14:26:44 +0000 Subject: [PATCH 130/816] w --- src/Parsers/IAST.cpp | 1 - src/Processors/QueryPlan/JoinStep.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index d6daf9bd78b..ad95f69b220 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -174,7 +174,6 @@ String IAST::formatWithPossiblyHidingSensitiveData( IdentifierQuotingRule identifier_quoting_rule, IdentifierQuotingStyle identifier_quoting_style) const { - WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 8365af4e589..2d7dd689149 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -151,7 +151,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines } column_permutation.resize(n); - pipeline->addSimpleTransform([column_perm = std::move(column_permutation)](const Block & header) + pipeline->addSimpleTransform([column_perm = std::move(column_permutation)](const Block & header) mutable { return std::make_shared(header, std::move(column_perm)); }); From a7da67069ab92c06e069d0f91132b8b12e0c2eda Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 1 Oct 2024 15:49:26 +0000 Subject: [PATCH 131/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index cfd60d8f259..ccc5a4b7465 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -153,7 +153,7 @@ def main(): match = re.search(r"(^|\s+)-max_total_time=(\d+)($|\s)", FUZZER_ARGS) if match: - timeout += match.group(2) + timeout += int(match.group(2)) with Path() as current: for fuzzer in current.iterdir(): From 809f0ee0a2b8f928cb655b98e654a1304717abb6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 16:36:32 +0000 Subject: [PATCH 132/816] fix test: correct drop order --- tests/integration/test_scheduler/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 0eee9d968ba..58d8ab44457 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -58,12 +58,12 @@ def set_default_configs(): def clear_workloads_and_resources(): node.query( f""" - drop resource if exists io_write; - drop resource if exists io_read; drop workload if exists production; drop workload if exists development; drop workload if exists admin; drop workload if exists all; + drop resource if exists io_write; + drop resource if exists io_read; """ ) yield From da525b6ab5b752c5029433e3513007e6b5e8759b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 1 Oct 2024 18:25:22 +0000 Subject: [PATCH 133/816] process timeout --- tests/fuzz/runner.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index ccc5a4b7465..f4a6a67e1f8 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -138,8 +138,11 @@ def run_fuzzer(fuzzer: str, timeout: int): ) except subprocess.CalledProcessError as e: # print("Command failed with error:", e) - print("Stderr output:", e.stderr) + print("Stderr output: ", e.stderr) process_error(e.stderr) + except subprocess.TimeoutExpired as e: + print("Timeout: ", e.stderr) + process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) From 6413b15d811d91588cbb9c2cd280f7b7734983e5 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 2 Oct 2024 11:55:26 +0000 Subject: [PATCH 134/816] 1) Extend stop_start_wait_sec and reduce connection_timeout_ms in test_replicated_database when restarting an instance. 2) Make createReplicaDirs and markReplicasActive virtual functions --- src/Databases/DatabaseReplicatedWorker.h | 4 ++++ src/Interpreters/DDLWorker.cpp | 9 ++++++++- src/Interpreters/DDLWorker.h | 4 ++-- tests/integration/helpers/cluster.py | 10 +++++----- .../test_replicated_database/configs/config.xml | 3 +++ .../test_replicated_database/configs/config2.xml | 3 +++ tests/integration/test_replicated_database/test.py | 10 +++++----- 7 files changed, 30 insertions(+), 13 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 51ff0f96e6d..820e55b17f0 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -41,6 +41,10 @@ public: private: bool initializeMainThread() override; void initializeReplication() override; + + void createReplicaDirs(const ZooKeeperPtr &, const NameSet &) override { } + void markReplicasActive(bool) override { } + void initializeLogPointer(const String & processed_entry_name); DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index e29826c6c54..e79bb4716ff 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -1176,7 +1176,14 @@ void DDLWorker::runMainThread() } cleanup_event->set(); - markReplicasActive(reinitialized); + try + { + markReplicasActive(reinitialized); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred when markReplicasActive: "); + } scheduleTasks(reinitialized); subsequent_errors_count = 0; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 53434d18861..649b56def4b 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -158,8 +158,8 @@ protected: virtual bool initializeMainThread(); virtual void initializeReplication(); - void createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids); - void markReplicasActive(bool reinitialized); + virtual void createReplicaDirs(const ZooKeeperPtr & zookeeper, const NameSet & host_ids); + virtual void markReplicasActive(bool reinitialized); void runMainThread(); void runCleanupThread(); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f5f87947c0f..a0bed265723 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4028,11 +4028,11 @@ class ClickHouseInstance: ) logging.info(f"PS RESULT:\n{ps_clickhouse}") pid = self.get_process_pid("clickhouse") - if pid is not None: - self.exec_in_container( - ["bash", "-c", f"gdb -batch -ex 'thread apply all bt full' -p {pid}"], - user="root", - ) + # if pid is not None: + # self.exec_in_container( + # ["bash", "-c", f"gdb -batch -ex 'thread apply all bt full' -p {pid}"], + # user="root", + # ) if last_err is not None: raise last_err diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 706628cf93b..13a8f58cd8a 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -7,4 +7,7 @@ 50 42 false + + 200 + diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml index 8192c191952..5f3e933753d 100644 --- a/tests/integration/test_replicated_database/configs/config2.xml +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -8,4 +8,7 @@ 42 group false + + 200 + diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 20e2dbca7dd..a219010d3eb 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -617,7 +617,7 @@ def test_alters_from_different_replicas(started_cluster): ) # test_replica_restart - main_node.restart_clickhouse() + main_node.restart_clickhouse(stop_start_wait_sec=120) expected = ( "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" @@ -1125,7 +1125,7 @@ def test_startup_without_zk(started_cluster): main_node.query("INSERT INTO startup.rmt VALUES (42)") with PartitionManager() as pm: pm.drop_instance_zk_connections(main_node) - main_node.restart_clickhouse(stop_start_wait_sec=60) + main_node.restart_clickhouse(stop_start_wait_sec=120) assert main_node.query("SELECT (*,).1 FROM startup.rmt") == "42\n" # we need to wait until the table is not readonly @@ -1143,7 +1143,7 @@ def test_server_uuid(started_cluster): uuid1 = main_node.query("select serverUUID()") uuid2 = dummy_node.query("select serverUUID()") assert uuid1 != uuid2 - main_node.restart_clickhouse() + main_node.restart_clickhouse(stop_start_wait_sec=120) uuid1_after_restart = main_node.query("select serverUUID()") assert uuid1 == uuid1_after_restart @@ -1408,14 +1408,14 @@ def test_modify_comment(started_cluster): ) def restart_verify_not_readonly(): - main_node.restart_clickhouse() + main_node.restart_clickhouse(stop_start_wait_sec=120) assert ( main_node.query( "SELECT is_readonly FROM system.replicas WHERE table = 'modify_comment_table'" ) == "0\n" ) - dummy_node.restart_clickhouse() + dummy_node.restart_clickhouse(stop_start_wait_sec=120) assert ( dummy_node.query( "SELECT is_readonly FROM system.replicas WHERE table = 'modify_comment_table'" From fec1b32a79987767618e44dc06a04ac8f6762a09 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 14:01:02 +0000 Subject: [PATCH 135/816] fix parser --- tests/fuzz/runner.py | 34 +++++++++++++--------------------- 1 file changed, 13 insertions(+), 21 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index f4a6a67e1f8..4099ff940e8 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -24,42 +24,34 @@ def process_fuzzer_output(output: str): def process_error(error: str): - ERROR = r"^==\d+== ERROR: (\S+): (.*)" + ERROR = r"^==\d+==\s?ERROR: (\S+): (.*)" error_source = "" error_reason = "" - SUMMARY = r"^SUMMARY: " TEST_UNIT_LINE = r"artifact_prefix='.*/'; Test unit written to (.*)" - test_unit = "" - CALL_STACK_LINE = r"^\s+(#\d+.*)" call_stack = [] is_call_stack = False # pylint: disable=unused-variable for line_num, line in enumerate(error.splitlines(), 1): - if is_call_stack: - match = re.search(CALL_STACK_LINE, line) - if match: - call_stack.append(match.group(1)) - continue - - if re.search(SUMMARY, line): + if re.search(r"^==\d+==", line): is_call_stack = False + continue + call_stack.append(line) continue - if not call_stack and not is_call_stack: - match = re.search(ERROR, line) + if call_stack: + match = re.search(TEST_UNIT_LINE, line) if match: - error_source = match.group(1) - error_reason = match.group(2) - is_call_stack = True - continue + report(error_source, error_reason, call_stack, match.group(1)) + call_stack.clear() + continue - match = re.search(TEST_UNIT_LINE, line) + match = re.search(ERROR, line) if match: - test_unit = match.group(1) - - report(error_source, error_reason, call_stack, test_unit) + error_source = match.group(1) + error_reason = match.group(2) + is_call_stack = True def run_fuzzer(fuzzer: str, timeout: int): From c367d63c5089cb1f1810bd4f3f767f551b6fed7f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 2 Oct 2024 15:46:40 +0000 Subject: [PATCH 136/816] fix --- src/Processors/QueryPlan/JoinStep.cpp | 5 +++-- src/Processors/Transforms/ColumnPermuteTransform.cpp | 4 ++-- src/Processors/Transforms/ColumnPermuteTransform.h | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 2d7dd689149..d6f9590d240 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -151,12 +151,13 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines } column_permutation.resize(n); - pipeline->addSimpleTransform([column_perm = std::move(column_permutation)](const Block & header) mutable + pipeline->addSimpleTransform([&column_permutation](const Block & header) { - return std::make_shared(header, std::move(column_perm)); + return std::make_shared(header, column_permutation); }); } + return pipeline; } diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp index 169dd2dc67e..eb2a691d6d1 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -33,9 +33,9 @@ void permuteChunk(Chunk & chunk, const std::vector & permutation) } -ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, std::vector permutation_) +ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) : ISimpleTransform(header_, permuteBlock(header_, permutation_), false) - , permutation(std::move(permutation_)) + , permutation(permutation_) { } diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h index b2e3c469833..f4d68850193 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.h +++ b/src/Processors/Transforms/ColumnPermuteTransform.h @@ -13,7 +13,7 @@ namespace DB class ColumnPermuteTransform : public ISimpleTransform { public: - ColumnPermuteTransform(const Block & header_, std::vector permutation_); + ColumnPermuteTransform(const Block & header_, const std::vector & permutation_); String getName() const override { return "ColumnPermuteTransform"; } From ab89e4daa0fe9cf6035c030b1863d64c4c2d8ce0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 15:51:41 +0000 Subject: [PATCH 137/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 4099ff940e8..d752fce1bd0 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -133,7 +133,7 @@ def run_fuzzer(fuzzer: str, timeout: int): print("Stderr output: ", e.stderr) process_error(e.stderr) except subprocess.TimeoutExpired as e: - print("Timeout: ", e.stderr) + print("Timeout") process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) From 228b01331d1099f68bc086945a3924e981634cfa Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 2 Oct 2024 15:56:26 +0000 Subject: [PATCH 138/816] fix conflict in src/Core/SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 1769eebbe8b..a488e6dd203 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -96,7 +96,6 @@ static std::initializer_list Date: Wed, 2 Oct 2024 17:14:11 +0000 Subject: [PATCH 139/816] debugging timeouts --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index d752fce1bd0..05b8faa96a2 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -133,7 +133,7 @@ def run_fuzzer(fuzzer: str, timeout: int): print("Stderr output: ", e.stderr) process_error(e.stderr) except subprocess.TimeoutExpired as e: - print("Timeout") + print("Timeout for %s", cmd_line) process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) From 7722a5e4fa78763a7d88f69b826b7103d0f306a3 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 2 Oct 2024 17:54:43 +0000 Subject: [PATCH 140/816] fixes and tests for DROP WORKLOAD query --- src/Common/Scheduler/Nodes/FairPolicy.h | 7 +++ .../Scheduler/Nodes/IOResourceManager.cpp | 2 + src/Common/Scheduler/Nodes/PriorityPolicy.h | 7 +++ .../Scheduler/Nodes/SemaphoreConstraint.h | 7 +++ .../Scheduler/Nodes/ThrottlerConstraint.h | 4 ++ .../Scheduler/Nodes/UnifiedSchedulerNode.h | 14 ++++- src/Common/Scheduler/SchedulerRoot.h | 24 +++---- tests/integration/test_scheduler/test.py | 63 +++++++++++++++++++ 8 files changed, 115 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index 81bfaaadf19..007a2416fae 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -52,6 +52,13 @@ public: : ISchedulerNode(event_queue_, info_) {} + ~FairPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + const String & getTypeName() const override { static String type_name("fair"); diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 55defbd2432..cf67bf2dfcb 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -125,6 +125,8 @@ void IOResourceManager::Resource::deleteNode(const NodeInfo & info) root_node.reset(); } + node_for_workload.erase(info.name); + updateCurrentVersion(); }); } diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index ea8bde718a2..f7d4ee93a36 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -43,6 +43,13 @@ public: : ISchedulerNode(event_queue_, node_info) {} + ~PriorityPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + const String & getTypeName() const override { static String type_name("priority"); diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index eab093f6b00..20a10f5da7d 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -31,6 +31,13 @@ public: , max_cost(max_cost_) {} + ~SemaphoreConstraint() override + { + // We need to clear `parent` in child to avoid dangling references + if (child) + removeChild(child.get()); + } + const String & getTypeName() const override { static String type_name("inflight_limit"); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 40b51f24b98..eaa26b2da54 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -38,6 +38,10 @@ public: { // We should cancel event on destruction to avoid dangling references from event queue event_queue->cancelPostponed(postponed); + + // We need to clear `parent` in child to avoid dangling reference + if (child) + removeChild(child.get()); } const String & getTypeName() const override diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 2b2eb320e0a..2de5131efbb 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -67,6 +67,7 @@ private: /// Helper function for managing a parent of a node static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) { + chassert(node); chassert(new_parent); if (new_parent == node->parent) return; @@ -139,7 +140,8 @@ private: { // Remove fair if the only child has left chassert(root); - root.reset(); // it will be still alive because it is attached to hierarchy for now + detach(root); + root.reset(); return children.begin()->second; // The last child is a new root now } else if (children.empty()) @@ -216,7 +218,8 @@ private: { // Remove priority node if the only child-branch has left chassert(root); - root.reset(); // it will be still alive because it is attached to hierarchy for now + detach(root); + root.reset(); return branches.begin()->second.getRoot(); // The last child-branch is a new root now } else if (branches.empty()) @@ -361,6 +364,13 @@ public: reparent(immediate_child, this); } + ~UnifiedSchedulerNode() override + { + // We need to clear `parent` in child to avoid dangling references + if (immediate_child) + removeChild(immediate_child.get()); + } + /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index e2ed133f662..8bb25f80139 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -31,24 +31,24 @@ namespace ErrorCodes class SchedulerRoot : public ISchedulerNode { private: - struct TResource + struct Resource { SchedulerNodePtr root; // Intrusive cyclic list of active resources - TResource * next = nullptr; - TResource * prev = nullptr; + Resource * next = nullptr; + Resource * prev = nullptr; - explicit TResource(const SchedulerNodePtr & root_) + explicit Resource(const SchedulerNodePtr & root_) : root(root_) { root->info.parent.ptr = this; } // Get pointer stored by ctor in info - static TResource * get(SchedulerNodeInfo & info) + static Resource * get(SchedulerNodeInfo & info) { - return reinterpret_cast(info.parent.ptr); + return reinterpret_cast(info.parent.ptr); } }; @@ -60,6 +60,8 @@ public: ~SchedulerRoot() override { stop(); + while (!children.empty()) + removeChild(children.begin()->first); } /// Runs separate scheduler thread @@ -185,7 +187,7 @@ public: void activateChild(ISchedulerNode * child) override { - activate(TResource::get(child->info)); + activate(Resource::get(child->info)); } void setParent(ISchedulerNode *) override @@ -194,7 +196,7 @@ public: } private: - void activate(TResource * value) + void activate(Resource * value) { assert(value->next == nullptr && value->prev == nullptr); if (current == nullptr) // No active children @@ -212,7 +214,7 @@ private: } } - void deactivate(TResource * value) + void deactivate(Resource * value) { if (value->next == nullptr) return; // Already deactivated @@ -257,8 +259,8 @@ private: request->execute(); } - TResource * current = nullptr; // round-robin pointer - std::unordered_map children; // resources by pointer + Resource * current = nullptr; // round-robin pointer + std::unordered_map children; // resources by pointer std::atomic stop_flag = false; EventQueue events; ThreadFromGlobalPool scheduler; diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 58d8ab44457..79d9466eb59 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -5,6 +5,7 @@ import time import threading import pytest +import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -647,6 +648,68 @@ def test_create_workload(): do_checks() +def test_workload_hierarchy_changes(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + for iteration in range(3): + split_idx = random.randint(1, len(queries) - 2) + for query_idx in range(0, split_idx): + node.query(queries[query_idx]) + node.query("create resource io_test (write disk non_existent_disk, read disk non_existent_disk);") + node.query("drop resource io_test;") + for query_idx in range(split_idx, len(queries)): + node.query(queries[query_idx]) + def test_resource_read_and_write(): node.query( From 0f8fed3d83bac3f9a91225c5c190fa1d6624ebe3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 20:07:02 +0000 Subject: [PATCH 141/816] add s3 corpus --- tests/fuzz/runner.py | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 05b8faa96a2..3b916145e0c 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -6,6 +6,8 @@ import os import re import subprocess from pathlib import Path +from tests.ci.env_helper import S3_BUILDS_BUCKET +from tests.ci.s3_helper import S3Helper DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") @@ -55,6 +57,8 @@ def process_error(error: str): def run_fuzzer(fuzzer: str, timeout: int): + s3 = S3Helper() + logging.info("Running fuzzer %s...", fuzzer) seed_corpus_dir = f"{fuzzer}.in" @@ -63,8 +67,14 @@ def run_fuzzer(fuzzer: str, timeout: int): seed_corpus_dir = "" active_corpus_dir = f"{fuzzer}.corpus" - if not os.path.exists(active_corpus_dir): - os.makedirs(active_corpus_dir) + s3.download_files(bucket=S3_BUILDS_BUCKET, + s3_path=f"fuzzer/corpus/{fuzzer}/", + file_suffix="", + local_directory=active_corpus_dir,) + + new_corpus_dir = f"{fuzzer}.corpus_new" + if not os.path.exists(new_corpus_dir): + os.makedirs(new_corpus_dir) options_file = f"{fuzzer}.options" custom_libfuzzer_options = "" @@ -102,7 +112,7 @@ def run_fuzzer(fuzzer: str, timeout: int): ) cmd_line = ( - f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {active_corpus_dir} {seed_corpus_dir}" + f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {new_corpus_dir} {active_corpus_dir} {seed_corpus_dir}" ) if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" @@ -133,11 +143,17 @@ def run_fuzzer(fuzzer: str, timeout: int): print("Stderr output: ", e.stderr) process_error(e.stderr) except subprocess.TimeoutExpired as e: - print("Timeout for %s", cmd_line) + print("Timeout for ", cmd_line) process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) + f = open(f"{new_corpus_dir}/testfile", "a") + f.write("Now the file has more content!") + f.close() + + s3.upload_build_directory_to_s3(new_corpus_dir, "fuzzer/corpus/") + def main(): logging.basicConfig(level=logging.INFO) From f43ebf004f334ec782fdccd2aa38c1846288fe4a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 20:24:13 +0000 Subject: [PATCH 142/816] fix style --- tests/fuzz/runner.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 3b916145e0c..8e1de7ca38d 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -6,6 +6,7 @@ import os import re import subprocess from pathlib import Path + from tests.ci.env_helper import S3_BUILDS_BUCKET from tests.ci.s3_helper import S3Helper @@ -68,9 +69,10 @@ def run_fuzzer(fuzzer: str, timeout: int): active_corpus_dir = f"{fuzzer}.corpus" s3.download_files(bucket=S3_BUILDS_BUCKET, - s3_path=f"fuzzer/corpus/{fuzzer}/", - file_suffix="", - local_directory=active_corpus_dir,) + s3_path=f"fuzzer/corpus/{fuzzer}/", + file_suffix="", + local_directory=active_corpus_dir, + ) new_corpus_dir = f"{fuzzer}.corpus_new" if not os.path.exists(new_corpus_dir): @@ -111,9 +113,8 @@ def run_fuzzer(fuzzer: str, timeout: int): for key, value in parser["fuzzer_arguments"].items() ) - cmd_line = ( - f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {new_corpus_dir} {active_corpus_dir} {seed_corpus_dir}" - ) + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {new_corpus_dir} {active_corpus_dir} {seed_corpus_dir}" + if custom_libfuzzer_options: cmd_line += f" {custom_libfuzzer_options}" if fuzzer_arguments: From 245e76a5d3be2dd78cf072ef9c4810da4a497d29 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 20:36:31 +0000 Subject: [PATCH 143/816] fix style --- tests/fuzz/runner.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 8e1de7ca38d..7f398d2124a 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -68,7 +68,8 @@ def run_fuzzer(fuzzer: str, timeout: int): seed_corpus_dir = "" active_corpus_dir = f"{fuzzer}.corpus" - s3.download_files(bucket=S3_BUILDS_BUCKET, + s3.download_files( + bucket=S3_BUILDS_BUCKET, s3_path=f"fuzzer/corpus/{fuzzer}/", file_suffix="", local_directory=active_corpus_dir, From 55fd44935d70195fa969941ee3d98b636bdcfe42 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 20:57:16 +0000 Subject: [PATCH 144/816] fix style --- tests/fuzz/runner.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 7f398d2124a..dbe9511b85c 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -7,8 +7,8 @@ import re import subprocess from pathlib import Path -from tests.ci.env_helper import S3_BUILDS_BUCKET -from tests.ci.s3_helper import S3Helper +from ci.env_helper import S3_BUILDS_BUCKET +from ci.s3_helper import S3Helper DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") @@ -150,9 +150,8 @@ def run_fuzzer(fuzzer: str, timeout: int): else: process_fuzzer_output(result.stderr) - f = open(f"{new_corpus_dir}/testfile", "a") - f.write("Now the file has more content!") - f.close() + with open(f"{new_corpus_dir}/testfile", "a", encoding='ascii') as f: + f.write("Now the file has more content!") s3.upload_build_directory_to_s3(new_corpus_dir, "fuzzer/corpus/") From f490d835136e0e28557ffc654e6cb87e13bde65e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 21:09:31 +0000 Subject: [PATCH 145/816] fix style --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index dbe9511b85c..ac6cbc56a7e 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -150,7 +150,7 @@ def run_fuzzer(fuzzer: str, timeout: int): else: process_fuzzer_output(result.stderr) - with open(f"{new_corpus_dir}/testfile", "a", encoding='ascii') as f: + with open(f"{new_corpus_dir}/testfile", "a", encoding="ascii") as f: f.write("Now the file has more content!") s3.upload_build_directory_to_s3(new_corpus_dir, "fuzzer/corpus/") From 4f23f16417c62057f721273492a0d60441588477 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 22:39:20 +0000 Subject: [PATCH 146/816] fix --- tests/fuzz/runner.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index ac6cbc56a7e..d85bc018739 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -7,9 +7,6 @@ import re import subprocess from pathlib import Path -from ci.env_helper import S3_BUILDS_BUCKET -from ci.s3_helper import S3Helper - DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") @@ -174,4 +171,9 @@ def main(): if __name__ == "__main__": + from os import sys, path + ACTIVE_DIR = path.dirname(path.abspath(__file__)) + sys.path.append(path.dirname(ACTIVE_DIR)) + from ci.env_helper import S3_BUILDS_BUCKET + from ci.s3_helper import S3Helper main() From 5e95ce8a485f1497af06b144c3754941fb1fba93 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 23:03:08 +0000 Subject: [PATCH 147/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index d85bc018739..fc93c7437ca 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -171,7 +171,7 @@ def main(): if __name__ == "__main__": - from os import sys, path + from os import path, sys ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) from ci.env_helper import S3_BUILDS_BUCKET From dff243a132c5014c1485133d92812bfb3750e67d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 2 Oct 2024 23:19:06 +0000 Subject: [PATCH 148/816] fix --- tests/fuzz/runner.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index fc93c7437ca..d03bc6f5bed 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -172,8 +172,10 @@ def main(): if __name__ == "__main__": from os import path, sys + ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) from ci.env_helper import S3_BUILDS_BUCKET from ci.s3_helper import S3Helper + main() From d022c4615b851b58aaa0f5dbdb1ab3b05b22ab83 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 00:10:59 +0000 Subject: [PATCH 149/816] fix --- tests/fuzz/runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index d03bc6f5bed..ffd319cf16c 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -175,7 +175,7 @@ if __name__ == "__main__": ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) - from ci.env_helper import S3_BUILDS_BUCKET - from ci.s3_helper import S3Helper + from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error + from ci.s3_helper import S3Helper # pylint: disable=import-error main() From 0d8ff26706c1879debbb4cfa029fbaa9239cd004 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 00:18:42 +0000 Subject: [PATCH 150/816] add unittests for IOResourceManager --- .../Scheduler/Nodes/tests/ResourceTest.h | 6 + .../Nodes/tests/gtest_io_resource_manager.cpp | 237 ++++++++++++++++++ .../Workload/WorkloadEntityStorageBase.cpp | 7 +- 3 files changed, 246 insertions(+), 4 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index c8cc0ed0e57..aa490b38f47 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -328,6 +328,12 @@ struct ResourceTestManager : public ResourceTestBase , busy_period(thread_count) {} + enum DoNotInitManagerEnum { DoNotInitManager }; + + explicit ResourceTestManager(size_t thread_count, DoNotInitManagerEnum) + : busy_period(thread_count) + {} + ~ResourceTestManager() { for (auto & thread : threads) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp new file mode 100644 index 00000000000..f8c973deb3b --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -0,0 +1,237 @@ +#include + +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +class WorkloadEntityTestStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityTestStorage() + : WorkloadEntityStorageBase(Context::getGlobalContextInstance()) + {} + + virtual void loadEntities() override {} + + void executeQuery(const String & query) + { + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; + + auto parse = [&] (IParser & parser) + { + String error; + const char * end = query.data(); + return tryParseQuery( + parser, + end, + query.data() + query.size(), + error, + false, + "", + false, + 0, + DBMS_DEFAULT_MAX_PARSER_DEPTH, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, + true); + }; + + if (ASTPtr create_workload = parse(create_workload_p)) + { + auto & parsed = create_workload->as(); + auto workload_name = parsed.getWorkloadName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Workload, + workload_name, + create_workload, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr create_resource = parse(create_resource_p)) + { + auto & parsed = create_resource->as(); + auto resource_name = parsed.getResourceName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Resource, + resource_name, + create_resource, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr drop_workload = parse(drop_workload_p)) + { + auto & parsed = drop_workload->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Workload, + parsed.workload_name, + throw_if_not_exists); + } + else if (ASTPtr drop_resource = parse(drop_resource_p)) + { + auto & parsed = drop_resource->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Resource, + parsed.resource_name, + throw_if_not_exists); + } + FAIL(); + } + +private: + bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override + { + UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); + return true; + } + + bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override + { + UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); + return true; + } +}; + +struct ResourceTest : ResourceTestManager +{ + WorkloadEntityTestStorage storage; + + explicit ResourceTest(size_t thread_count = 1) + : ResourceTestManager(thread_count, DoNotInitManager) + { + manager = std::make_shared(storage); + } + + void query(const String & query_str) + { + storage.executeQuery(query_str); + } +}; + +using TestGuard = ResourceTest::Guard; + +TEST(SchedulerIOResourceManager, Smoke) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); + t.query("CREATE WORKLOAD A in all"); + t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); + + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); + + for (int i = 0; i < 10; i++) + { + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); + + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); + + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); + } +} + +TEST(SchedulerIOResourceManager, Fairness) +{ + // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). + // Requests from A use `value = 1` and from B `value = -1` is used. + std::atomic unfairness = 0; + auto fairness_diff = [&] (Int64 value) + { + Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; + EXPECT_NEAR(cur_unfairness, 0, 1); + }; + + constexpr size_t threads_per_queue = 2; + int requests_per_thread = 100; + ResourceTest t(2 * threads_per_queue + 1); + + t.query("CREATE RESOURCE res1"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD A in all"); + t.query("CREATE WORKLOAD B in all"); + t.query("CREATE WORKLOAD leader in all"); + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("A"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(1); + } + }); + } + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("B"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(-1); + } + }); + } + + ClassifierPtr c = t.manager->acquire("leader"); + ResourceLink link = c->get("res1"); + t.blockResource(link); +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 6f633893d70..91f418449ed 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -26,9 +26,8 @@ namespace ErrorCodes namespace { -ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const ContextPtr & context) +ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) { - UNUSED(context); auto ptr = create_query.clone(); if (auto * res = typeid_cast(ptr.get())) { @@ -201,7 +200,7 @@ bool WorkloadEntityStorageBase::storeEntity( std::unique_lock lock{mutex}; - create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); if (auto it = entities.find(entity_name); it != entities.end()) { @@ -400,7 +399,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector normalized_entities; for (const auto & [entity_name, create_query] : new_entities) - normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); + normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); // TODO(serxa): do validation and throw LOGICAL_ERROR if failed From 4c02ddcf3bf68398a210f0492d5aba6d15943571 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 00:18:58 +0000 Subject: [PATCH 151/816] style --- .../tests/gtest_custom_resource_manager.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp index 495654d45ce..37432128606 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp @@ -31,21 +31,21 @@ TEST(SchedulerCustomResourceManager, Smoke) )CONFIG"); - ClassifierPtr cA = t.manager->acquire("A"); - ClassifierPtr cB = t.manager->acquire("B"); + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); - gA.lock(); - gA.consume(1); - gA.unlock(); + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); - ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); - gB.unlock(); + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); - ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); - gB.consume(2); + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); } } From f009d1e7d5c7c605874c637977e1639455086b67 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 00:28:15 +0000 Subject: [PATCH 152/816] fix --- tests/fuzz/runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index ffd319cf16c..171c99698a7 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -175,7 +175,7 @@ if __name__ == "__main__": ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) - from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error - from ci.s3_helper import S3Helper # pylint: disable=import-error + from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error + from ci.s3_helper import S3Helper # pylint: disable=import-error main() From 4a7de86089ac2bdcad31791d1db717f25c656b5d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 00:42:53 +0000 Subject: [PATCH 153/816] fix --- tests/fuzz/runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 171c99698a7..af3f2ff6040 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -175,7 +175,7 @@ if __name__ == "__main__": ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) - from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error - from ci.s3_helper import S3Helper # pylint: disable=import-error + from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error,no-name-in-module + from ci.s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module main() From bf292bcc45a131a589bbb0ba113bcc80db380b07 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Oct 2024 00:52:51 +0000 Subject: [PATCH 154/816] Automatic style fix --- tests/fuzz/runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index af3f2ff6040..718799a7f63 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -175,7 +175,9 @@ if __name__ == "__main__": ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append(path.dirname(ACTIVE_DIR)) - from ci.env_helper import S3_BUILDS_BUCKET # pylint: disable=import-error,no-name-in-module + from ci.env_helper import ( # pylint: disable=import-error,no-name-in-module + S3_BUILDS_BUCKET, + ) from ci.s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module main() From 709ac588400aa8187dcbb2929379c846433d966f Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 3 Oct 2024 01:02:09 +0000 Subject: [PATCH 155/816] Fix flaky test --- .../test_ddl_worker_replicas/test.py | 34 +++++++++++-------- 1 file changed, 20 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py index 0905165611f..db2c89127bc 100644 --- a/tests/integration/test_ddl_worker_replicas/test.py +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -17,7 +17,10 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True ) node4 = cluster.add_instance( - "node4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + "node4", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, ) @@ -47,25 +50,28 @@ def test_ddl_worker_replicas(started_cluster): lines = list(result.split("\n")) assert len(lines) == 1 - + print(f"Test: {replica} {lines[0]}") parts = list(lines[0].split("\t")) assert len(parts) == 3 assert parts[0] == "active" assert len(parts[1]) != 0 assert len(parts[2]) != 0 - node4.stop() + try: + node4.stop_clickhouse() - # wait for node4 active path is removed - node1.query_with_retry( - sql=f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'", - check_callback=lambda result: result == 0, - ) + # wait for node4 active path is removed + node1.query_with_retry( + sql=f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'", + check_callback=lambda result: result == 0, + ) - result = node1.query_with_retry( - f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" - ).strip() + result = node1.query_with_retry( + f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" + ).strip() - lines = list(result.split("\n")) - assert len(lines) == 1 - assert len(lines[0]) == 0 + lines = list(result.split("\n")) + assert len(lines) == 1 + assert len(lines[0]) == 0 + finally: + node4.start_clickhouse() From d279be6ac2683cfebe56399b8d1e60cca085eb1e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 02:10:07 +0000 Subject: [PATCH 156/816] add boto3 to requirements --- docker/test/fuzzer/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fuzzer/requirements.txt b/docker/test/fuzzer/requirements.txt index 3dce93e023b..74147513e76 100644 --- a/docker/test/fuzzer/requirements.txt +++ b/docker/test/fuzzer/requirements.txt @@ -25,3 +25,4 @@ six==1.16.0 wadllib==1.3.6 wheel==0.37.1 zipp==1.0.0 +boto3 From 39c0fa2ea4b93aedf43c763cd4eaa66d31f7de67 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 3 Oct 2024 02:18:21 +0000 Subject: [PATCH 157/816] Revert changes in test_replicated_database --- .../test_replicated_database/configs/config.xml | 3 --- .../test_replicated_database/configs/config2.xml | 3 --- tests/integration/test_replicated_database/test.py | 8 ++++---- 3 files changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 13a8f58cd8a..706628cf93b 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -7,7 +7,4 @@ 50 42 false - - 200 - diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml index 5f3e933753d..8192c191952 100644 --- a/tests/integration/test_replicated_database/configs/config2.xml +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -8,7 +8,4 @@ 42 group false - - 200 - diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index f5670557c22..6fd337cf214 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -617,7 +617,7 @@ def test_alters_from_different_replicas(started_cluster): ) # test_replica_restart - main_node.restart_clickhouse(stop_start_wait_sec=120) + main_node.restart_clickhouse() expected = ( "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" @@ -1145,7 +1145,7 @@ def test_server_uuid(started_cluster): uuid1 = main_node.query("select serverUUID()") uuid2 = dummy_node.query("select serverUUID()") assert uuid1 != uuid2 - main_node.restart_clickhouse(stop_start_wait_sec=120) + main_node.restart_clickhouse() uuid1_after_restart = main_node.query("select serverUUID()") assert uuid1 == uuid1_after_restart @@ -1416,14 +1416,14 @@ def test_modify_comment(started_cluster): ) def restart_verify_not_readonly(): - main_node.restart_clickhouse(stop_start_wait_sec=120) + main_node.restart_clickhouse() assert ( main_node.query( "SELECT is_readonly FROM system.replicas WHERE table = 'modify_comment_table'" ) == "0\n" ) - dummy_node.restart_clickhouse(stop_start_wait_sec=120) + dummy_node.restart_clickhouse() assert ( dummy_node.query( "SELECT is_readonly FROM system.replicas WHERE table = 'modify_comment_table'" From ce3983d757e032cdcbd3af81f0a79a959bf036bc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 02:20:14 +0000 Subject: [PATCH 158/816] fix --- docker/test/fuzzer/requirements.txt | 1 - docker/test/libfuzzer/requirements.txt | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/requirements.txt b/docker/test/fuzzer/requirements.txt index 74147513e76..3dce93e023b 100644 --- a/docker/test/fuzzer/requirements.txt +++ b/docker/test/fuzzer/requirements.txt @@ -25,4 +25,3 @@ six==1.16.0 wadllib==1.3.6 wheel==0.37.1 zipp==1.0.0 -boto3 diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt index 3dce93e023b..74147513e76 100644 --- a/docker/test/libfuzzer/requirements.txt +++ b/docker/test/libfuzzer/requirements.txt @@ -25,3 +25,4 @@ six==1.16.0 wadllib==1.3.6 wheel==0.37.1 zipp==1.0.0 +boto3 From c7b8a98fa6a2d0c914112562834c52f4acd04b9a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 03:12:58 +0000 Subject: [PATCH 159/816] fix --- tests/fuzz/runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 718799a7f63..6c4c2930a90 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -174,10 +174,10 @@ if __name__ == "__main__": from os import path, sys ACTIVE_DIR = path.dirname(path.abspath(__file__)) - sys.path.append(path.dirname(ACTIVE_DIR)) - from ci.env_helper import ( # pylint: disable=import-error,no-name-in-module + sys.path.append(path.dirname(ACTIVE_DIR) / "ci") + from env_helper import ( # pylint: disable=import-error,no-name-in-module S3_BUILDS_BUCKET, ) - from ci.s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module + from s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module main() From 2bb3dd7cbc6c860849add0adcd32c296a00d349c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 04:09:00 +0000 Subject: [PATCH 160/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 6c4c2930a90..a64af5bab66 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -174,7 +174,7 @@ if __name__ == "__main__": from os import path, sys ACTIVE_DIR = path.dirname(path.abspath(__file__)) - sys.path.append(path.dirname(ACTIVE_DIR) / "ci") + sys.path.append(Path(path.dirname(ACTIVE_DIR)) / "ci") from env_helper import ( # pylint: disable=import-error,no-name-in-module S3_BUILDS_BUCKET, ) From 582e01ba57218480a2ef485ccc5f8c4ff440bfc3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 05:39:42 +0000 Subject: [PATCH 161/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index a64af5bab66..51201e85224 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -174,7 +174,7 @@ if __name__ == "__main__": from os import path, sys ACTIVE_DIR = path.dirname(path.abspath(__file__)) - sys.path.append(Path(path.dirname(ACTIVE_DIR)) / "ci") + sys.path.append((Path(path.dirname(ACTIVE_DIR)) / "ci").as_posix()) from env_helper import ( # pylint: disable=import-error,no-name-in-module S3_BUILDS_BUCKET, ) From 1dc67425bdc084346bafa1264828e979b7909071 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 06:45:03 +0000 Subject: [PATCH 162/816] fix --- tests/fuzz/runner.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 51201e85224..e11a5415227 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +import botocore import configparser import logging import os @@ -65,12 +66,15 @@ def run_fuzzer(fuzzer: str, timeout: int): seed_corpus_dir = "" active_corpus_dir = f"{fuzzer}.corpus" - s3.download_files( - bucket=S3_BUILDS_BUCKET, - s3_path=f"fuzzer/corpus/{fuzzer}/", - file_suffix="", - local_directory=active_corpus_dir, - ) + try: + s3.download_files( + bucket=S3_BUILDS_BUCKET, + s3_path=f"fuzzer/corpus/{fuzzer}/", + file_suffix="", + local_directory=active_corpus_dir, + ) + except botocore.errorfactory.NoSuchKey as e: + logging.debug("No active corpus exists for %s", fuzzer) new_corpus_dir = f"{fuzzer}.corpus_new" if not os.path.exists(new_corpus_dir): From 0a08ec018a1626a823d4496f57843e24816bf12c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 07:02:11 +0000 Subject: [PATCH 163/816] fix --- tests/fuzz/runner.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index e11a5415227..06a232a0e5a 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -import botocore import configparser import logging import os @@ -8,6 +7,8 @@ import re import subprocess from pathlib import Path +import botocore + DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") From 86045e0f0913f4d746a721bc365d52b0efddebee Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 3 Oct 2024 07:09:41 +0000 Subject: [PATCH 164/816] Tidy test_ddl_on_cluster_stop_waiting_for_offline_hosts --- src/Databases/DatabaseReplicatedWorker.h | 5 -- .../test.py | 72 ++++++++++--------- 2 files changed, 37 insertions(+), 40 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index 0d4b68d6b23..7c3726b339b 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -39,11 +39,6 @@ public: UInt64 getCurrentInitializationDurationMs() const; -protected: - // No need to `createReplicaDirs` and `markReplicasActive` - void createReplicaDirs(const ZooKeeperPtr &, const NameSet &) override { } - void markReplicasActive(bool) override { } - private: bool initializeMainThread() override; void initializeReplication() override; diff --git a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py index cce8928e5d5..06bdd6f2198 100644 --- a/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py +++ b/tests/integration/test_ddl_on_cluster_stop_waiting_for_offline_hosts/test.py @@ -1,6 +1,7 @@ -import pytest import time +import pytest + from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -18,7 +19,10 @@ node3 = cluster.add_instance( "node3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True ) node4 = cluster.add_instance( - "node4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + "node4", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + stay_alive=True, ) @@ -36,51 +40,49 @@ def test_stop_waiting_for_offline_hosts(started_cluster): timeout = 10 settings = {"distributed_ddl_task_timeout": timeout} - start = time.time() node1.query( "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", settings=settings, ) - assert time.time() - start < timeout node1.query( "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", settings=settings, ) - node4.stop() + try: + node4.stop_clickhouse() - start = time.time() - assert "Code: 159. DB::Exception" in node1.query_and_get_error( - "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", - settings=settings, - ) + start = time.time() + assert "Code: 159. DB::Exception" in node1.query_and_get_error( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) + assert time.time() - start >= timeout - assert time.time() - start >= timeout + start = time.time() + assert "Code: 159. DB::Exception" in node1.query_and_get_error( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) + assert time.time() - start >= timeout - start = time.time() - assert "Code: 159. DB::Exception" in node1.query_and_get_error( - "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", - settings=settings, - ) - assert time.time() - start >= timeout + # set `distributed_ddl_output_mode` = `throw_only_active`` + settings = { + "distributed_ddl_task_timeout": timeout, + "distributed_ddl_output_mode": "throw_only_active", + } - # set `distributed_ddl_output_mode` = `throw_only_active`` - settings = { - "distributed_ddl_task_timeout": timeout, - "distributed_ddl_output_mode": "throw_only_active", - } + start = time.time() + node1.query( + "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", + settings=settings, + ) - start = time.time() - node1.query( - "DROP TABLE IF EXISTS test_table ON CLUSTER test_cluster SYNC", - settings=settings, - ) - assert time.time() - start < timeout - - start = time.time() - node1.query( - "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", - settings=settings, - ) - assert time.time() - start < timeout + start = time.time() + node1.query( + "CREATE TABLE test_table ON CLUSTER test_cluster (x Int) Engine=Memory", + settings=settings, + ) + finally: + node4.start_clickhouse() From 55ff81518f9a35dc3797b1c80acd6d4ef990c5d3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 07:24:50 +0000 Subject: [PATCH 165/816] fix --- tests/fuzz/runner.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 06a232a0e5a..ccd7cbc475a 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -7,7 +7,7 @@ import re import subprocess from pathlib import Path -import botocore +from botocore.exceptions import ClientError DEBUGGER = os.getenv("DEBUGGER", "") FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") @@ -74,8 +74,11 @@ def run_fuzzer(fuzzer: str, timeout: int): file_suffix="", local_directory=active_corpus_dir, ) - except botocore.errorfactory.NoSuchKey as e: - logging.debug("No active corpus exists for %s", fuzzer) + except ClientError as e: + if e.response['Error']['Code'] == 'NoSuchKey': + logging.debug("No active corpus exists for %s", fuzzer) + else: + raise new_corpus_dir = f"{fuzzer}.corpus_new" if not os.path.exists(new_corpus_dir): From 3008330afec6c45fd3badf335cca57cb173ecadc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Oct 2024 07:33:39 +0000 Subject: [PATCH 166/816] Automatic style fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index ccd7cbc475a..e1860d60081 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -75,7 +75,7 @@ def run_fuzzer(fuzzer: str, timeout: int): local_directory=active_corpus_dir, ) except ClientError as e: - if e.response['Error']['Code'] == 'NoSuchKey': + if e.response["Error"]["Code"] == "NoSuchKey": logging.debug("No active corpus exists for %s", fuzzer) else: raise From d6b3c106db864f9ae1b471c84c67a8032862d52f Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 07:51:47 +0000 Subject: [PATCH 167/816] fix unittests --- .../Scheduler/Nodes/tests/gtest_io_resource_manager.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index f8c973deb3b..31dd98eafc5 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -107,7 +107,8 @@ public: parsed.resource_name, throw_if_not_exists); } - FAIL(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query in WorkloadEntityTestStorage: {}", query); } private: @@ -157,7 +158,7 @@ TEST(SchedulerIOResourceManager, Smoke) { ResourceTest t; - t.query("CREATE RESOURCE res1"); + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); t.query("CREATE WORKLOAD A in all"); t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); @@ -195,7 +196,7 @@ TEST(SchedulerIOResourceManager, Fairness) int requests_per_thread = 100; ResourceTest t(2 * threads_per_queue + 1); - t.query("CREATE RESOURCE res1"); + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); t.query("CREATE WORKLOAD A in all"); t.query("CREATE WORKLOAD B in all"); From ad05a454352c882e1e81250d99c8d73669d9c2c9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Oct 2024 11:44:35 +0000 Subject: [PATCH 168/816] upd tests --- .../0_stateless/00826_cross_to_inner_join.sql | 10 +++++----- .../00847_multiple_join_same_column.sql | 14 +++++++------- .../01015_empty_in_inner_right_join.sql.j2 | 2 ++ .../0_stateless/02000_join_on_const.reference | 18 +++++++++--------- .../0_stateless/02000_join_on_const.sql | 16 ++++++++-------- .../0_stateless/03094_one_thousand_joins.sql | 1 + 6 files changed, 32 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index f81832a4109..5ab7a2d0626 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,12 +41,12 @@ SELECT '--- is null or ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- comma and or ---'; SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) ORDER BY ALL; diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index c7f0c6383c2..bbb4eb12466 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) left join y on (y.a = s.a and y.b = s.b) order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a as t_a from t left join s on s.a = t_a order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, s.a as s_a from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; drop table t; drop table s; diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index cdb9d253b9b..cdbb0542ffb 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,5 +1,7 @@ SET joined_subquery_requires_alias = 0; +SET query_plan_join_inner_table_selection = 'auto'; + {% for join_algorithm in ['partial_merge', 'hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 3bd1633ce32..f8e46a2b976 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -2 2 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +2 2 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 @@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 1 ('',0) SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 0 ('b',256) -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 0 ('b',256) SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 2 4 2 Nullable(UInt64) UInt8 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index da70973ed87..33638edafa5 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; -- { echoOff } diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 6ae4e4d4d3c..69c4fb42a6b 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,6 +3,7 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES +SET query_plan_join_inner_table_selection = 'auto'; -- 'left' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From e60ae9c64a237d0a7c9fba5a1e83ff611e0f8c58 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 12:44:02 +0000 Subject: [PATCH 169/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index e1860d60081..2e779401e0b 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -158,7 +158,7 @@ def run_fuzzer(fuzzer: str, timeout: int): with open(f"{new_corpus_dir}/testfile", "a", encoding="ascii") as f: f.write("Now the file has more content!") - s3.upload_build_directory_to_s3(new_corpus_dir, "fuzzer/corpus/") + s3.upload_build_directory_to_s3(new_corpus_dir, Path("fuzzer/corpus/")) def main(): From 11fabc1a18c8579e2775fe9b8bfec9e5656d09ec Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 3 Oct 2024 12:44:18 +0000 Subject: [PATCH 170/816] Make test_ddl_worker_replicas predictable --- .../test_ddl_worker_replicas/test.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_ddl_worker_replicas/test.py b/tests/integration/test_ddl_worker_replicas/test.py index db2c89127bc..5ba5f406e4f 100644 --- a/tests/integration/test_ddl_worker_replicas/test.py +++ b/tests/integration/test_ddl_worker_replicas/test.py @@ -35,22 +35,20 @@ def started_cluster(): def test_ddl_worker_replicas(started_cluster): - replica_list = node1.query( - "SELECT name FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas'" - ).strip() + for replica in ["node1:9000", "node2:9000", "node3:9000", "node4:9000"]: + # wait until the replicas path is created + node1.query_with_retry( + sql=f"SELECT count() FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/{replica}'", + check_callback=lambda result: result == 1, + ) - replica_list = list(replica_list.split("\n")) - expected_replicas = ["node1:9000", "node2:9000", "node3:9000", "node4:9000"] - assert expected_replicas.sort() == replica_list.sort() - - for replica in replica_list: result = node1.query( f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/{replica}'" ).strip() + print(f"result: {replica} {result}") lines = list(result.split("\n")) assert len(lines) == 1 - print(f"Test: {replica} {lines[0]}") parts = list(lines[0].split("\t")) assert len(parts) == 3 assert parts[0] == "active" @@ -70,6 +68,8 @@ def test_ddl_worker_replicas(started_cluster): f"SELECT name, value, ephemeralOwner FROM system.zookeeper WHERE path='/clickhouse/task_queue/replicas/node4:9000'" ).strip() + print(f"result: {replica} {result}") + lines = list(result.split("\n")) assert len(lines) == 1 assert len(lines[0]) == 0 From de69aa8c946258ebd25fc4e0a131b0244f5cbac1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 13:42:24 +0000 Subject: [PATCH 171/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 2e779401e0b..c6b1b2a623b 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -158,7 +158,7 @@ def run_fuzzer(fuzzer: str, timeout: int): with open(f"{new_corpus_dir}/testfile", "a", encoding="ascii") as f: f.write("Now the file has more content!") - s3.upload_build_directory_to_s3(new_corpus_dir, Path("fuzzer/corpus/")) + s3.upload_build_directory_to_s3(Path(new_corpus_dir), "fuzzer/corpus/") def main(): From da5ebde4d5db8d2838c4473fe21e69d3b5a9ae4e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 16:16:39 +0000 Subject: [PATCH 172/816] add CI env --- tests/ci/libfuzzer_test_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 8f19dd7d023..46406dc3557 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -133,6 +133,8 @@ def main(): check_name, run_by_hash_num, run_by_hash_total ) + additional_envs.append("CI=1") + ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh") ci_logs_args = ci_logs_credentials.get_docker_arguments( pr_info, stopwatch.start_time_str, check_name From 3fb92a61a0c115fd564913fa918acf1c0e5db987 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Oct 2024 16:49:19 +0000 Subject: [PATCH 173/816] t --- src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp | 5 +++++ tests/integration/helpers/cluster.py | 2 ++ 2 files changed, 7 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index cd66a230038..d0f4371fac6 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -56,6 +56,11 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) return; const auto & table_join = join->getTableJoin(); + + /// Algorithms other than HashJoin may not support OUTER JOINs + if (table_join.kind() != JoinKind::Inner && !typeid_cast(join.get())) + return; + /// fixme: USING clause handled specially in join algorithm, so swap breaks it /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8cf3e318797..8fe2932137c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4524,6 +4524,8 @@ class ClickHouseInstance: ): # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") + else: + print(f"XXXX Skip random settings for {self.name}, {self.randomize_settings} {self.image}:{self.tag} @ {self.base_config_dir} ?= {DEFAULT_BASE_CONFIG_DIR}") version = None version_parts = self.tag.split(".") From 2c8c5629d95941da2cef30ce373751a83a95b8d4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Oct 2024 16:57:19 +0000 Subject: [PATCH 174/816] Automatic style fix --- tests/integration/helpers/cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8fe2932137c..f9d3746b2d9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4525,7 +4525,9 @@ class ClickHouseInstance: # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") else: - print(f"XXXX Skip random settings for {self.name}, {self.randomize_settings} {self.image}:{self.tag} @ {self.base_config_dir} ?= {DEFAULT_BASE_CONFIG_DIR}") + print( + f"XXXX Skip random settings for {self.name}, {self.randomize_settings} {self.image}:{self.tag} @ {self.base_config_dir} ?= {DEFAULT_BASE_CONFIG_DIR}" + ) version = None version_parts = self.tag.split(".") From 4d917d80b42f8dedbd4ddbfecc1c6d9c5fa87c01 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 17:32:05 +0000 Subject: [PATCH 175/816] fix --- tests/ci/libfuzzer_test_check.py | 2 +- tests/fuzz/runner.py | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 46406dc3557..5de28d5641a 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -59,7 +59,7 @@ def get_run_command( envs = [ # a static link, don't use S3_URL or S3_DOWNLOAD - '-e S3_URL="https://s3.amazonaws.com/clickhouse-datasets"', + '-e S3_URL="https://s3.amazonaws.com"', ] envs += [f"-e {e}" for e in additional_envs] diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index c6b1b2a623b..5d0f2865422 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -155,10 +155,7 @@ def run_fuzzer(fuzzer: str, timeout: int): else: process_fuzzer_output(result.stderr) - with open(f"{new_corpus_dir}/testfile", "a", encoding="ascii") as f: - f.write("Now the file has more content!") - - s3.upload_build_directory_to_s3(Path(new_corpus_dir), "fuzzer/corpus/") + s3.upload_build_directory_to_s3(Path(new_corpus_dir), f"fuzzer/corpus/{fuzzer}", False) def main(): From f66bc05c0188d5873696d01b2d80486c73625bb2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Oct 2024 17:39:14 +0000 Subject: [PATCH 176/816] Automatic style fix --- tests/fuzz/runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 5d0f2865422..2e7c1184bcc 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -155,7 +155,9 @@ def run_fuzzer(fuzzer: str, timeout: int): else: process_fuzzer_output(result.stderr) - s3.upload_build_directory_to_s3(Path(new_corpus_dir), f"fuzzer/corpus/{fuzzer}", False) + s3.upload_build_directory_to_s3( + Path(new_corpus_dir), f"fuzzer/corpus/{fuzzer}", False + ) def main(): From 6fa23c4b72747293d58aebb11d1bb7d2a15b4647 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 23:44:40 +0000 Subject: [PATCH 177/816] kill all fuzzers on timeout --- tests/fuzz/runner.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 2e7c1184bcc..42e54acfecc 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -4,6 +4,7 @@ import configparser import logging import os import re +import signal import subprocess from pathlib import Path @@ -56,6 +57,15 @@ def process_error(error: str): is_call_stack = True +def kill_fuzzer(fuzzer: str): + p = subprocess.Popen(['ps', '-A'], stdout=subprocess.PIPE) + out, err = p.communicate() + for line in out.splitlines(): + if fuzzer in line: + pid = int(line.split(None, 1)[0]) + os.kill(pid, signal.SIGKILL) + + def run_fuzzer(fuzzer: str, timeout: int): s3 = S3Helper() @@ -151,6 +161,7 @@ def run_fuzzer(fuzzer: str, timeout: int): process_error(e.stderr) except subprocess.TimeoutExpired as e: print("Timeout for ", cmd_line) + kill_fuzzer(fuzzer) process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) From a0d2f2085d56252eb689a72909b567db9325fdc1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 3 Oct 2024 23:57:05 +0000 Subject: [PATCH 178/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 42e54acfecc..512a20e58c5 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -58,7 +58,7 @@ def process_error(error: str): def kill_fuzzer(fuzzer: str): - p = subprocess.Popen(['ps', '-A'], stdout=subprocess.PIPE) + p = subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) out, err = p.communicate() for line in out.splitlines(): if fuzzer in line: From 08d098a2f486fab845fa46459b5e842132028ea4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 00:15:36 +0000 Subject: [PATCH 179/816] fix --- tests/fuzz/runner.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 512a20e58c5..8e05625a6d9 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -58,12 +58,12 @@ def process_error(error: str): def kill_fuzzer(fuzzer: str): - p = subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) - out, err = p.communicate() - for line in out.splitlines(): - if fuzzer in line: - pid = int(line.split(None, 1)[0]) - os.kill(pid, signal.SIGKILL) + with subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) as p + out, _ = p.communicate() + for line in out.splitlines(): + if fuzzer in line: + pid = int(line.split(None, 1)[0]) + os.kill(pid, signal.SIGKILL) def run_fuzzer(fuzzer: str, timeout: int): From bfb2e7c04413f467e310231830f6701b39739e5e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 00:16:16 +0000 Subject: [PATCH 180/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 8e05625a6d9..81a76fbcdb9 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -58,7 +58,7 @@ def process_error(error: str): def kill_fuzzer(fuzzer: str): - with subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) as p + with subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) as p: out, _ = p.communicate() for line in out.splitlines(): if fuzzer in line: From 9d81ff0a8906ed5549ca3a75a0540b4fb0e13dfc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 01:22:26 +0000 Subject: [PATCH 181/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 81a76fbcdb9..702014ce04f 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -185,7 +185,7 @@ def main(): with Path() as current: for fuzzer in current.iterdir(): if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): - run_fuzzer(fuzzer, timeout) + run_fuzzer(fuzzer.name, timeout) if __name__ == "__main__": From 5cf7a777a2b7bf80c9a3eba1d89a5a3bbfa2c86f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 02:31:34 +0000 Subject: [PATCH 182/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 702014ce04f..b51b0f99abc 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -61,7 +61,7 @@ def kill_fuzzer(fuzzer: str): with subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) as p: out, _ = p.communicate() for line in out.splitlines(): - if fuzzer in line: + if fuzzer.encode("utf-8") in line: pid = int(line.split(None, 1)[0]) os.kill(pid, signal.SIGKILL) From db69e018bf31acf0ec0c22e63bebe1448429e4fc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 03:18:01 +0000 Subject: [PATCH 183/816] fix --- tests/fuzz/runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index b51b0f99abc..bcfc7e6146f 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -63,6 +63,7 @@ def kill_fuzzer(fuzzer: str): for line in out.splitlines(): if fuzzer.encode("utf-8") in line: pid = int(line.split(None, 1)[0]) + logging.info("Killing fuzzer %s, pid %d", fuzzer, pid) os.kill(pid, signal.SIGKILL) From 530d034302720ec3c479e38ba18ac432e27f6ab3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 04:35:35 +0000 Subject: [PATCH 184/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index bcfc7e6146f..948bc9d48ed 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -161,7 +161,7 @@ def run_fuzzer(fuzzer: str, timeout: int): print("Stderr output: ", e.stderr) process_error(e.stderr) except subprocess.TimeoutExpired as e: - print("Timeout for ", cmd_line) + logging.info("Timeout for %s", cmd_line) kill_fuzzer(fuzzer) process_fuzzer_output(e.stderr) else: From e9e35eb118f35ecfa0b6d21fe4a9be7e87443a1f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 05:31:17 +0000 Subject: [PATCH 185/816] fix --- tests/fuzz/runner.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 948bc9d48ed..ac2bb78b7f0 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -7,6 +7,7 @@ import re import signal import subprocess from pathlib import Path +from time import sleep from botocore.exceptions import ClientError @@ -163,6 +164,7 @@ def run_fuzzer(fuzzer: str, timeout: int): except subprocess.TimeoutExpired as e: logging.info("Timeout for %s", cmd_line) kill_fuzzer(fuzzer) + sleep(10) process_fuzzer_output(e.stderr) else: process_fuzzer_output(result.stderr) From 0872cc0dd7a78b26fb16c98c04894bf168bd199a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Oct 2024 10:05:16 +0000 Subject: [PATCH 186/816] fix inegration settings randomization with non default tag --- tests/integration/helpers/cluster.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f9d3746b2d9..5fa4c5dfce3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -67,6 +67,7 @@ DEFAULT_ENV_NAME = ".env" DEFAULT_BASE_CONFIG_DIR = os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" ) +DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest") SANITIZER_SIGN = "==================" @@ -504,7 +505,6 @@ class ClickHouseCluster: "CLICKHOUSE_TESTS_DOCKERD_HOST" ) self.docker_api_version = os.environ.get("DOCKER_API_VERSION") - self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") self.base_cmd = ["docker", "compose"] if custom_dockerd_host: @@ -1079,7 +1079,7 @@ class ClickHouseCluster: env_variables["keeper_binary"] = binary_path env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix - env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag + env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG env_variables["user"] = str(os.getuid()) env_variables["keeper_fs"] = "bind" for i in range(1, 4): @@ -1672,7 +1672,7 @@ class ClickHouseCluster: ) if tag is None: - tag = self.docker_base_tag + tag = DOCKER_BASE_TAG if not env_variables: env_variables = {} self.use_keeper = use_keeper @@ -4519,15 +4519,11 @@ class ClickHouseInstance: if ( self.randomize_settings and self.image == "clickhouse/integration-test" - and self.tag == "latest" + and self.tag == DOCKER_BASE_TAG and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR ): # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") - else: - print( - f"XXXX Skip random settings for {self.name}, {self.randomize_settings} {self.image}:{self.tag} @ {self.base_config_dir} ?= {DEFAULT_BASE_CONFIG_DIR}" - ) version = None version_parts = self.tag.split(".") From c91b0563de2ffb81fa5c10655c8711c894792aac Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Oct 2024 11:05:21 +0000 Subject: [PATCH 187/816] materialize block in JoiningTransform::transformHeader --- src/Processors/Transforms/JoiningTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f2fb6327129..187f4bf6728 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -19,6 +19,7 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); + materializeBlockInplace(header); LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } From eb8ae504db5b7d04ff1d9f04f6068e91472153eb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 12:03:21 +0000 Subject: [PATCH 188/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index ac2bb78b7f0..e842f40f8d8 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -59,7 +59,7 @@ def process_error(error: str): def kill_fuzzer(fuzzer: str): - with subprocess.Popen(["ps", "-A"], stdout=subprocess.PIPE) as p: + with subprocess.Popen(["ps", "-A", "u"], stdout=subprocess.PIPE) as p: out, _ = p.communicate() for line in out.splitlines(): if fuzzer.encode("utf-8") in line: From c7902255ba868af3903e075bb69e27381f062351 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 4 Oct 2024 12:54:13 +0000 Subject: [PATCH 189/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index e842f40f8d8..b3c19fbb0a4 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -63,7 +63,7 @@ def kill_fuzzer(fuzzer: str): out, _ = p.communicate() for line in out.splitlines(): if fuzzer.encode("utf-8") in line: - pid = int(line.split(None, 1)[0]) + pid = int(line.split(None, 2)[1]) logging.info("Killing fuzzer %s, pid %d", fuzzer, pid) os.kill(pid, signal.SIGKILL) From c555eb4ba50734f9c3a760af44bd1edb702d26f1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Oct 2024 13:24:54 +0000 Subject: [PATCH 190/816] optimize join step planning a bit --- src/Planner/PlannerJoinTree.cpp | 92 +++++++++++-------- src/Processors/QueryPlan/JoinStep.cpp | 11 ++- src/Processors/QueryPlan/JoinStep.h | 6 +- .../QueryPlan/Optimizations/optimizeJoin.cpp | 3 +- 4 files changed, 70 insertions(+), 42 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 720f0a380ab..19fd896f9a8 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1241,6 +1241,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } +std::optional createStepToDropColumns( + const Block & header, + const ColumnIdentifierSet & outer_scope_columns, + const PlannerContextPtr & planner_context) +{ + ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; + std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; + std::optional first_skipped_column_node_index; + + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); + size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); + + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + + for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + { + const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; + + if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) + || !global_planner_context->hasColumnIdentifier(output->result_name)) + continue; + + if (!outer_scope_columns.contains(output->result_name)) + { + if (!first_skipped_column_node_index) + first_skipped_column_node_index = i; + continue; + } + + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); + drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); + } + + if (!first_skipped_column_node_index) + return {}; + + /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. + * + * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; + */ + if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); + + drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); + + return drop_unused_columns_after_join_actions_dag; +} + JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, @@ -1654,47 +1703,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; - std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; - std::optional first_skipped_column_node_index; - - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); - size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); - - for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + const auto & header_after_join = result_plan.getCurrentDataStream().header; + if (header_after_join.columns() > outer_scope_columns.size()) { - const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; - - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) - || !global_planner_context->hasColumnIdentifier(output->result_name)) - continue; - - if (!outer_scope_columns.contains(output->result_name)) + auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context); + if (drop_unused_columns_after_join_actions_dag) { - if (!first_skipped_column_node_index) - first_skipped_column_node_index = i; - continue; + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(*drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); } - - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); - drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); } - /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. - * - * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; - */ - if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); - - drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); - - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); - for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index d6f9590d240..3edc64ef967 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -185,8 +185,18 @@ void JoinStep::describeActions(JSONBuilder::JSONMap & map) const map.add(name, value); } +void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) +{ + join_algorithm_header.clear(); + swap_streams = swap_streams_; + join = std::move(join_); +} + void JoinStep::updateOutputStream() { + if (join_algorithm_header) + return; + const auto & header = swap_streams ? input_streams[1].header : input_streams[0].header; Block result_header = JoiningTransform::transformHeader(header, join); @@ -200,7 +210,6 @@ void JoinStep::updateOutputStream() return; } - if (swap_streams) result_header = rotateBlock(result_header, input_streams[1].header); diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index b0947cb6be7..bf6560d5a07 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -34,13 +34,12 @@ public: void describeActions(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } - void setJoin(JoinPtr join_) { join = std::move(join_); } + void setJoin(JoinPtr join_, bool swap_streams_ = false); bool allowPushDownToRight() const; bool canUpdateInputStream() const override { return true; } JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right; - bool swap_streams = false; private: void updateOutputStream() override; @@ -51,10 +50,11 @@ private: size_t max_block_size; size_t max_streams; - NameSet required_output; + const NameSet required_output; std::set columns_to_remove; bool keep_left_read_in_order; bool use_new_analyzer = false; + bool swap_streams = false; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index d0f4371fac6..ced3b987b64 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -92,12 +92,11 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) const auto & left_stream_input_header = streams.front().header; const auto & right_stream_input_header = streams.back().header; - join_step->swap_streams = true; auto updated_table_join = std::make_shared(table_join); updated_table_join->swapSides(); auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); - join_step->setJoin(std::move(updated_join)); + join_step->setJoin(std::move(updated_join), /* swap_streams= */ true); } } From 1fee0534d654dc201b4e2479cdb044cd15887105 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 4 Oct 2024 18:19:35 -0300 Subject: [PATCH 191/816] workaround valid until --- src/Access/AuthenticationData.cpp | 3 ++- src/Parsers/Access/ParserCreateUserQuery.cpp | 5 +---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index 9223c3a8059..f44e883a9b4 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -474,6 +474,8 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que AuthenticationData auth_data(current_type); + auth_data.setValidUntil(valid_until); + if (check_password_rules) context->getAccessControl().checkPasswordComplexityRules(value); @@ -512,7 +514,6 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que #endif } - auth_data.setValidUntil(valid_until); auth_data.setPassword(value); return auth_data; } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 9643bf2d8fb..68ebb8a983f 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -236,10 +236,7 @@ namespace if (http_auth_scheme) auth_data->children.push_back(std::move(http_auth_scheme)); - if (parseValidUntil(pos, expected, auth_data->valid_until)) - { - auth_data->children.push_back(auth_data->valid_until); - } + parseValidUntil(pos, expected, auth_data->valid_until); return true; }); From ee200fa3d965d88ae5a23e0186b9cf42c7c23d12 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 5 Oct 2024 13:48:19 +0000 Subject: [PATCH 192/816] test resource request failure --- .../Scheduler/Nodes/tests/ResourceTest.h | 43 +++++++- .../Nodes/tests/gtest_io_resource_manager.cpp | 97 ++++++++++++++++++- 2 files changed, 134 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index aa490b38f47..bbe0df4872e 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,6 +1,8 @@ #pragma once -#include "Common/Scheduler/SchedulingSettings.h" +#include + +#include #include #include #include @@ -283,6 +285,8 @@ private: ResourceCost failed_cost = 0; }; +enum EnqueueOnlyEnum { EnqueueOnly }; + template struct ResourceTestManager : public ResourceTestBase { @@ -294,16 +298,49 @@ struct ResourceTestManager : public ResourceTestBase struct Guard : public ResourceGuard { ResourceTestManager & t; + ResourceCost cost; - Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) + /// Works like regular ResourceGuard, ready for consumption after constructor + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) , t(t_) + , cost(cost_) { t.onEnqueue(link); + waitExecute(); + } + + /// Just enqueue resource request, do not block (neede for tests to sync). Call `waitExecuted()` afterwards + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) + , t(t_) + , cost(cost_) + { + t.onEnqueue(link); + } + + /// Waits for ResourceRequest::execute() to be called for enqueued requet + void waitExecute() + { lock(); t.onExecute(link); consume(cost); } + + /// Waits for ResourceRequest::failure() to be called for enqueued request + void waitFailed(const String & pattern) + { + try + { + lock(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::RESOURCE_ACCESS_DENIED); + ASSERT_TRUE(e.message().contains(pattern)); + } + } }; struct TItem diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 31dd98eafc5..93c8439bdae 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -150,6 +150,27 @@ struct ResourceTest : ResourceTestManager { storage.executeQuery(query_str); } + + template + void async(const String & workload, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + func2(classifier); + }); + } + + template + void async(const String & workload, const String & resource, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + ResourceLink link = classifier->get(resource); + func2(link); + }); + } }; using TestGuard = ResourceTest::Guard; @@ -198,9 +219,9 @@ TEST(SchedulerIOResourceManager, Fairness) t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); - t.query("CREATE WORKLOAD A in all"); - t.query("CREATE WORKLOAD B in all"); - t.query("CREATE WORKLOAD leader in all"); + t.query("CREATE WORKLOAD A IN all"); + t.query("CREATE WORKLOAD B IN all"); + t.query("CREATE WORKLOAD leader IN all"); for (int thread = 0; thread < threads_per_queue; thread++) { @@ -236,3 +257,73 @@ TEST(SchedulerIOResourceManager, Fairness) ResourceLink link = c->get("res1"); t.blockResource(link); } + +TEST(SchedulerIOResourceManager, DropNotEmptyQueue) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(3); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // 2nd resource request is enqueued + g.waitFailed("is about to be destructed"); + }); + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); +} + +TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + static constexpr int queue_size = 100; + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(2 + queue_size); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + for (int i = 0; i < queue_size; i++) + { + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // many resource requests are enqueued + g.waitFailed("is about to be destructed"); + }); + } + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); +} From 37bb566be1189f52f5fbf6148b8e70811118a5d5 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 5 Oct 2024 14:13:54 +0000 Subject: [PATCH 193/816] fix misleading comments --- src/Common/Scheduler/ResourceRequest.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index 24afcc98b57..03bdaec6a2b 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -45,8 +45,7 @@ constexpr size_t ResourceMaxConstraints = 8; * * Request can also be canceled before (3) using ISchedulerQueue::cancelRequest(). * Returning false means it is too late for request to be canceled. It should be processed in a regular way. - * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen - * and step (6) MUST be omitted. + * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen. */ class ResourceRequest : public boost::intrusive::list_base_hook<> { @@ -88,7 +87,7 @@ public: /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// WARNING: this function MUST not be called if request was canceled or failed. + /// It is okay to call finish() even for failed and canceled requests (it will be no-op) void finish(); /// Is called from the scheduler thread to fill `constraints` chain From a74185806cd488a61dabbe385e8a0b8d7dee465f Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 6 Oct 2024 12:36:38 +0000 Subject: [PATCH 194/816] CREATE OR REPLACE WORKLOAD support --- src/Common/ErrorCodes.cpp | 2 - .../Workload/WorkloadEntityDiskStorage.cpp | 7 +- .../Workload/WorkloadEntityStorageBase.cpp | 156 +++++++++++++----- .../Workload/WorkloadEntityStorageBase.h | 11 +- .../03232_workloads_and_resources.sql | 1 + 5 files changed, 129 insertions(+), 48 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f441426e2f0..9f07c3ed5d5 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -611,8 +611,6 @@ M(730, REFRESH_FAILED) \ M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \ M(733, TABLE_IS_BEING_RESTARTED) \ - M(734, WORKLOAD_ENTITY_ALREADY_EXISTS) \ - M(735, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 5ffec270610..190b2928fe0 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -41,8 +41,7 @@ namespace Setting namespace ErrorCodes { extern const int DIRECTORY_DOESNT_EXIST; - extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; - extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int BAD_ARGUMENTS; } @@ -215,7 +214,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( if (fs::exists(file_path)) { if (throw_if_exists) - throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) return false; } @@ -266,7 +265,7 @@ bool WorkloadEntityDiskStorage::removeEntityImpl( if (!existed) { if (throw_if_not_exists) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else return false; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 91f418449ed..4e0c4f8dbbd 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -18,8 +19,7 @@ namespace DB namespace ErrorCodes { - extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; - extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } @@ -123,7 +123,7 @@ ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const auto it = entities.find(entity_name); if (it == entities.end()) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The workload entity name '{}' is not saved", entity_name); @@ -191,23 +191,34 @@ bool WorkloadEntityStorageBase::storeEntity( if (entity_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); auto * workload = typeid_cast(create_entity_query.get()); - if (workload) - { - if (entity_name == workload->getWorkloadParent()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Self-referencing workloads are not allowed."); - } + auto * resource = typeid_cast(create_entity_query.get()); std::unique_lock lock{mutex}; - create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); - + ASTPtr old_entity; // entity to be REPLACED if (auto it = entities.find(entity_name); it != entities.end()) { if (throw_if_exists) - throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) return false; + else + old_entity = it->second; + } + + // Validate CREATE OR REPLACE + if (old_entity) + { + auto * old_workload = typeid_cast(old_entity.get()); + auto * old_resource = typeid_cast(old_entity.get()); + if (workload && !old_workload) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); + if (resource && !old_resource) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); + if (workload && !old_workload->hasParent() && workload->hasParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); } std::optional new_root_name; @@ -217,7 +228,7 @@ bool WorkloadEntityStorageBase::storeEntity( { if (!workload->hasParent()) { - if (!root_name.empty()) + if (!root_name.empty() && root_name != workload->getWorkloadName()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); new_root_name = workload->getWorkloadName(); } @@ -232,15 +243,31 @@ bool WorkloadEntityStorageBase::storeEntity( if (auto it = entities.find(target); it == entities.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); - // Validate that we could parse the settings for specific resource - if (type == ReferenceType::ForResource) + switch (type) { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); + case ReferenceType::Parent: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); + break; + } + case ReferenceType::ForResource: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - SchedulingSettings validator; - validator.updateFromChanges(workload->changes, target); + // Validate that we could parse the settings for specific resource + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + break; + } } + + // Detect reference cycles. + // The only way to create a cycle is to add an edge that will be a part of a new cycle. + // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. + if (isIndirectlyReferenced(source, target)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); }); bool stored = storeEntityImpl( @@ -256,12 +283,17 @@ bool WorkloadEntityStorageBase::storeEntity( { if (new_root_name) root_name = *new_root_name; - forEachReference(create_entity_query, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].insert(source); - }); + + // Remove references of a replaced entity (only for CREATE OR REPLACE) + removeReferences(old_entity); + + // Insert references of created entity + insertReferences(create_entity_query); + + // Store in memory entities[entity_name] = create_entity_query; + + // Process notifications onEntityAdded(entity_type, entity_name, create_entity_query); unlockAndNotify(lock); } @@ -280,7 +312,7 @@ bool WorkloadEntityStorageBase::removeEntity( if (it == entities.end()) { if (throw_if_not_exists) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else return false; } @@ -303,16 +335,15 @@ bool WorkloadEntityStorageBase::removeEntity( { if (entity_name == root_name) root_name.clear(); - forEachReference(it->second, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].erase(source); - if (references[target].empty()) - references.erase(target); - }); - entities.erase(it); - onEntityRemoved(entity_type, entity_name); + // Clean up references + removeReferences(it->second); + + // Remove from memory + entities.erase(it); + + // Process notifications + onEntityRemoved(entity_type, entity_name); unlockAndNotify(lock); } @@ -407,14 +438,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector bfs; + std::unordered_set visited; + visited.insert(target); + bfs.push(target); + while (!bfs.empty()) + { + String current = bfs.front(); + bfs.pop(); + if (current == source) + return true; + if (auto it = references.find(current); it != references.end()) + { + for (const String & node : it->second) + { + if (visited.contains(node)) + continue; + visited.insert(node); + bfs.push(node); + } + } + } + return false; +} + +void WorkloadEntityStorageBase::insertReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].insert(source); + }); +} + +void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].erase(source); + if (references[target].empty()) + references.erase(target); + }); +} + } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 9b81e5bdff6..7bfc28b3263 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -83,6 +83,15 @@ protected: /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). void unlockAndNotify(std::unique_lock & lock); + /// Return true iff `references` has a path from `source` to `target` + bool isIndirectlyReferenced(const String & target, const String & source); + + /// Adds references that are described by `entity` to `references` + void insertReferences(const ASTPtr & entity); + + /// Removes references that are described by `entity` from `references` + void removeReferences(const ASTPtr & entity); + struct Handlers { std::mutex mutex; @@ -97,7 +106,7 @@ protected: std::unordered_map entities; /// Maps entity name into CREATE entity query // Validation - std::unordered_map> references; /// Keep track of references between entities + std::unordered_map> references; /// Keep track of references between entities. Key is target. Values is set of sources String root_name; /// current root workload name ContextPtr global_context; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index ae0061b3bd5..3e12d70b7ff 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -10,6 +10,7 @@ create workload development in all settings priority = 1, weight = 1; create workload another_root; -- {serverError BAD_ARGUMENTS} create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} drop workload all; -- {serverError BAD_ARGUMENTS} +create workload invalid in 03232_write; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} From 93d0ed126a4c489ab36bc46ffbf34b8721716094 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:08:29 +0000 Subject: [PATCH 195/816] support update of constraints --- .../Scheduler/Nodes/SemaphoreConstraint.h | 32 +++++++++++++++++-- .../Scheduler/Nodes/ThrottlerConstraint.h | 23 ++++++++++--- 2 files changed, 48 insertions(+), 7 deletions(-) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 20a10f5da7d..feac9654e70 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -123,6 +123,32 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation or deactivation + void updateConstraints(const SchedulerNodePtr & self, Int64 new_max_requests, UInt64 new_max_cost) + { + std::unique_lock lock(mutex); + bool was_active = active(); + max_requests = new_max_requests; + max_cost = new_max_cost; + + if (parent) + { + // Activate on transition from inactive state + if (!was_active && active()) + parent->activateChild(this); + // Deactivate on transition into inactive state + else if (was_active && !active()) + { + // Node deactivation is usually done in dequeueRequest(), but we do not want to + // do extra call to active() on every request just to make sure there was no update(). + // There is no interface method to do deactivation, so we do the following trick. + parent->removeChild(this); + parent->attachChild(self); // This call is the only reason we have `recursive_mutex` + } + } + } + bool isActive() override { std::unique_lock lock(mutex); @@ -164,10 +190,10 @@ private: return satisfied() && child_active; } - const Int64 max_requests = default_max_requests; - const Int64 max_cost = default_max_cost; + Int64 max_requests = default_max_requests; + Int64 max_cost = default_max_cost; - std::mutex mutex; + std::recursive_mutex mutex; Int64 requests = 0; Int64 cost = 0; bool child_active = false; diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index eaa26b2da54..7071b0221ae 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -118,6 +118,21 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation + void updateConstraints(double new_max_speed, double new_max_burst) + { + event_queue->cancelPostponed(postponed); + postponed = EventQueue::not_postponed; + bool was_active = active(); + updateBucket(0, true); // To apply previous params for duration since `last_update` + max_speed = new_max_speed; + max_burst = new_max_burst; + updateBucket(0, false); // To postpone (if needed) using new params + if (!was_active && active() && parent) + parent->activateChild(this); + } + bool isActive() override { return active(); @@ -160,7 +175,7 @@ private: parent->activateChild(this); } - void updateBucket(ResourceCost use = 0) + void updateBucket(ResourceCost use = 0, bool do_not_postpone = false) { auto now = event_queue->now(); if (max_speed > 0.0) @@ -170,7 +185,7 @@ private: tokens -= use; // This is done outside min() to avoid passing large requests w/o token consumption after long idle period // Postpone activation until there is positive amount of tokens - if (tokens < 0.0) + if (!do_not_postpone && tokens < 0.0) { auto delay_ns = std::chrono::nanoseconds(static_cast(-tokens / max_speed * 1e9)); if (postponed == EventQueue::not_postponed) @@ -194,8 +209,8 @@ private: return satisfied() && child_active; } - const double max_speed{0}; /// in tokens per second - const double max_burst{0}; /// in tokens + double max_speed{0}; /// in tokens per second + double max_burst{0}; /// in tokens EventQueue::TimePoint last_update; UInt64 postponed = EventQueue::not_postponed; From b3b0e4fef643fba711f6c11519634ab8354d1869 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:21:14 +0000 Subject: [PATCH 196/816] manager support for CREATE OR REPLACE --- .../Scheduler/Nodes/IOResourceManager.cpp | 33 ++++---- .../Scheduler/Nodes/IOResourceManager.h | 7 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 76 +++++++++++++++++-- 3 files changed, 90 insertions(+), 26 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index cf67bf2dfcb..101a0fa4c32 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -137,7 +137,7 @@ void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const No throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", old_info.name, new_info.name, resource_name); - if (old_info.parent != new_info.parent && (old_info.parent.empty() || old_info.parent.empty())) + if (old_info.parent != new_info.parent && (old_info.parent.empty() || new_info.parent.empty())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", old_info.name, old_info.parent, new_info.parent, resource_name); @@ -157,22 +157,20 @@ void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const No { auto node = node_for_workload[old_info.name]; bool detached = false; - if (old_info.parent != new_info.parent) + if (UnifiedSchedulerNode::updateRequiresDetach(old_info.parent, new_info.parent, old_info.settings, new_info.settings)) { - node_for_workload[old_info.parent]->detachUnifiedChild(node); + if (!old_info.parent.empty()) + node_for_workload[old_info.parent]->detachUnifiedChild(node); detached = true; } node->updateSchedulingSettings(new_info.settings); - if (!detached && !old_info.parent.empty() && old_info.settings.priority != new_info.settings.priority) - node_for_workload[old_info.parent]->updateUnifiedChildPriority( - node, - old_info.settings.priority, - new_info.settings.priority); if (detached) - node_for_workload[new_info.parent]->attachUnifiedChild(node); - + { + if (!new_info.parent.empty()) + node_for_workload[new_info.parent]->attachUnifiedChild(node); + } updateCurrentVersion(); }); } @@ -268,7 +266,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) case WorkloadEntityType::Resource: { if (entity) - createResource(entity_name, entity); + createOrUpdateResource(entity_name, entity); else deleteResource(entity_name); break; @@ -315,14 +313,11 @@ void IOResourceManager::deleteWorkload(const String & workload_name) } } -void IOResourceManager::createResource(const String & resource_name, const ASTPtr & ast) +void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) { std::unique_lock lock{mutex}; if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) - { - // Resource to be created already exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + resource_iter->second->updateResource(ast); else { // Add all workloads into the new resource @@ -420,6 +415,12 @@ void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const V attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; } +void IOResourceManager::Resource::updateResource(const ASTPtr & new_resource_entity) +{ + chassert(getEntityName(new_resource_entity) == resource_name); + resource_entity = new_resource_entity; +} + std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) { auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index f4871379456..dc57b985455 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -173,6 +173,9 @@ private: void deleteNode(const NodeInfo & info); void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); + /// Updates resource entity + void updateResource(const ASTPtr & new_resource_entity); + /// Updates a classifier to contain a reference for specified workload std::future attachClassifier(Classifier & classifier, const String & workload_name); @@ -205,7 +208,7 @@ private: future.get(); // Blocks until execution is done in the scheduler thread } - const ASTPtr resource_entity; + ASTPtr resource_entity; const String resource_name; SchedulerRoot scheduler; @@ -256,7 +259,7 @@ private: void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); void deleteWorkload(const String & workload_name); - void createResource(const String & resource_name, const ASTPtr & ast); + void createOrUpdateResource(const String & resource_name, const ASTPtr & ast); void deleteResource(const String & resource_name); // Topological sorting of worklaods diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 2de5131efbb..f0ec17a8dca 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -160,6 +160,14 @@ private: // Returns true iff there are no unified children attached bool empty() const { return branches.empty(); } + SchedulerNodePtr getRoot() + { + chassert(!branches.empty()); + if (root) + return root; + return branches.begin()->second.getRoot(); // There should be exactly one child-branch + } + /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) @@ -244,6 +252,14 @@ private: SchedulerNodePtr queue; /// FifoQueue node is used if there are no children ChildrenBranch branch; /// Used if there is at least one child + SchedulerNodePtr getRoot() + { + if (queue) + return queue; + else + return branch.getRoot(); + } + // Should be called after constructor, before any other methods [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) { @@ -354,6 +370,52 @@ private: } return {}; } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr updateSchedulingSettings(EventQueue * event_queue_, const SchedulingSettings & new_settings) + { + SchedulerNodePtr node = branch.getRoot(); + + if (!settings.hasSemaphore() && new_settings.hasSemaphore()) // Add semaphore + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_requests, new_settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + else if (settings.hasSemaphore() && !new_settings.hasSemaphore()) // Remove semaphore + { + detach(semaphore); + semaphore.reset(); + } + else if (settings.hasSemaphore() && new_settings.hasSemaphore()) // Update semaphore + { + static_cast(*semaphore).updateConstraints(semaphore, new_settings.max_requests, new_settings.max_cost); + node = semaphore; + } + + if (!settings.hasThrottler() && new_settings.hasThrottler()) // Add throttler + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_speed, new_settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + else if (settings.hasThrottler() && !new_settings.hasThrottler()) // Remove throttler + { + detach(throttler); + throttler.reset(); + } + else if (settings.hasThrottler() && new_settings.hasThrottler()) // Update throttler + { + static_cast(*throttler).updateConstraints(new_settings.max_speed, new_settings.max_burst); + node = throttler; + } + + settings = new_settings; + return node; + } }; public: @@ -388,20 +450,19 @@ public: reparent(new_child, this); } - /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) - /// NOTE: Changing a priority of a unified child may lead to change of its parent. - void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) + static bool updateRequiresDetach(const String & old_parent, const String & new_parent, const SchedulingSettings & old_settings, const SchedulingSettings & new_settings) { - UNUSED(child, old_priority, new_priority); // TODO(serxa): implement updateUnifiedChildPriority() + return old_parent != new_parent || old_settings.priority != new_settings.priority; } /// Updates scheduling settings. Set of constraints might change. - /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) + /// NOTE: Caller is responsible for detaching and attaching if `updateRequiresDetach` returns true void updateSchedulingSettings(const SchedulingSettings & new_settings) { - UNUSED(new_settings); // TODO(serxa): implement updateSchedulingSettings() info.setPriority(new_settings.priority); info.setWeight(new_settings.weight); + if (auto new_child = impl.updateSchedulingSettings(event_queue, new_settings)) + reparent(new_child, this); } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children @@ -418,8 +479,7 @@ public: /// all unified nodes. Such a version control is done by `IOResourceManager`. void addRawPointerNodes(std::vector & nodes) { - if (impl.throttler) - nodes.push_back(impl.throttler); + // NOTE: `impl.throttler` could be skipped, because ThrottlerConstraint does not call `request->addConstraint()` if (impl.semaphore) nodes.push_back(impl.semaphore); if (impl.branch.queue) From 869ac2a20b87489821c3b1426d5ca60561de47e8 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:21:58 +0000 Subject: [PATCH 197/816] clean up --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 7bfc28b3263..51c5d3f0ac6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -73,14 +73,11 @@ protected: /// Called by derived class after a new workload entity has been added. void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); - /// Called by derived class after an workload entity has been changed. - void onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity); - /// Called by derived class after an workload entity has been removed. void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); /// Sends notifications to subscribers about changes in workload entities - /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). + /// (added with previous calls onEntityAdded(), onEntityRemoved()). void unlockAndNotify(std::unique_lock & lock); /// Return true iff `references` has a path from `source` to `target` From cf500575dae2c88563df1211aad4f4a9b03352f9 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:26:10 +0000 Subject: [PATCH 198/816] clean up --- .../Workload/WorkloadEntityStorageBase.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4e0c4f8dbbd..b1e426d363e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -382,11 +382,6 @@ void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, co queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); } -void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) -{ - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = changed_entity}); -} - void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) { queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); @@ -465,11 +460,13 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lockgetID()); } - for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) - onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); - + // Resources should be created first becase workloads could reference them for (auto & [entity_name, ast] : resources) onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); + + // Workloads should be created in an order such that children are created only after its parent is created + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); } std::vector> WorkloadEntityStorageBase::getAllEntities() const From b20536f7ea3ed6a2f7c51a659ff7a206cfe35ea4 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 12:00:42 +0000 Subject: [PATCH 199/816] add stateless tests for CREATE OR REPLACE --- .../03232_workloads_and_resources.sql | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index 3e12d70b7ff..a3e46166396 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel -- Do not run this test in parallel because `all` workload might affect other queries execution process + +-- Test simple resource and workload hierarchy creation create resource 03232_write (write disk 03232_fake_disk); create resource 03232_read (read disk 03232_fake_disk); create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; @@ -7,6 +9,7 @@ create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; +-- Test that illegal actions are not allowed create workload another_root; -- {serverError BAD_ARGUMENTS} create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} drop workload all; -- {serverError BAD_ARGUMENTS} @@ -19,7 +22,44 @@ create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGU create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} +create or replace workload all in production; -- {serverError BAD_ARGUMENTS} +-- Test CREATE OR REPLACE WORKLOAD +create or replace workload all settings max_requests = 200 for 03232_write, max_requests = 100 for 03232_read; +create or replace workload admin in all settings priority = 1; +create or replace workload admin in all settings priority = 2; +create or replace workload admin in all settings priority = 0; +create or replace workload production in all settings priority = 1, weight = 90; +create or replace workload production in all settings priority = 0, weight = 9; +create or replace workload production in all settings priority = 2, weight = 9; +create or replace workload development in all settings priority = 1; +create or replace workload development in all settings priority = 0; +create or replace workload development in all settings priority = 2; + +-- Test CREATE OR REPLACE RESOURCE +create or replace resource 03232_write (write disk 03232_fake_disk_2); +create or replace resource 03232_read (read disk 03232_fake_disk_2); + +-- Test update settings with CREATE OR REPLACE WORKLOAD +create or replace workload production in all settings priority = 1, weight = 9, max_requests = 100; +create or replace workload development in all settings priority = 1, weight = 1, max_requests = 10; +create or replace workload production in all settings priority = 1, weight = 9, max_cost = 100000; +create or replace workload development in all settings priority = 1, weight = 1, max_cost = 10000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000, max_burst = 10000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000, max_burst = 1000000; +create or replace workload all settings max_cost = 1000000, max_speed = 100000 for 03232_write, max_speed = 200000 for 03232_read; +create or replace workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create or replace workload production in all settings priority = 1, weight = 9; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Test change parent with CREATE OR REPLACE WORKLOAD +create or replace workload development in production settings priority = 1, weight = 1; +create or replace workload development in admin settings priority = 1, weight = 1; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Clean up drop workload if exists production; drop workload if exists development; drop workload if exists admin; From d28e41f712dc83d160bcd788eb99115ef5b8e517 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 12:46:54 +0000 Subject: [PATCH 200/816] add unit tests for weight and priority updates --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 5 ++ .../Scheduler/Nodes/tests/ResourceTest.h | 21 +++++ .../tests/gtest_throttler_constraint.cpp | 20 ++--- .../tests/gtest_unified_scheduler_node.cpp | 80 ++++++++++++++++--- 4 files changed, 106 insertions(+), 20 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index f0ec17a8dca..f7b5d1a2056 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -465,6 +465,11 @@ public: reparent(new_child, this); } + const SchedulingSettings & getSettings() const + { + return impl.settings; + } + /// Returns the queue to be used for resource requests or `nullptr` if it has unified children std::shared_ptr getQueue() { diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index bbe0df4872e..704f7119300 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -179,6 +179,27 @@ public: return node; } + // Updates the parent and/or scheduling settings for a specidfied `node`. + // Unit test implementation must make sure that all needed queues and constraints are not going to be destroyed. + // Normally it is the responsibility of IOResourceManager, but we do not use it here, so manual version control is required. + // (see IOResourceManager::Resource::updateCurrentVersion() fo details) + void updateUnifiedNode(const UnifiedSchedulerNodePtr & node, const UnifiedSchedulerNodePtr & old_parent, const UnifiedSchedulerNodePtr & new_parent, const SchedulingSettings & new_settings) + { + EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported + bool detached = false; + if (UnifiedSchedulerNode::updateRequiresDetach(old_parent->basename, new_parent->basename, node->getSettings(), new_settings)) { + if (old_parent) + old_parent->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_settings); + + if (detached && new_parent) + new_parent->attachUnifiedChild(node); + } + + void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { enqueueImpl(node->getQueue().get(), costs, node->basename); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 2bc24cdb292..9bb1bc572b8 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -160,22 +160,22 @@ TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) t.enqueue("/fair/B", {req_cost}); } - double shareA = 0.1; - double shareB = 0.9; + double share_a = 0.1; + double share_b = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); - auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - double consumedA = 0; - double consumedB = 0; + double consumed_a = 0; + double consumed_b = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); - t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); - consumedA = arrival_curve * shareA; - consumedB = arrival_curve * shareB; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index faebaa72b71..cfd837d4f1a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -403,23 +403,23 @@ TEST(SchedulerUnifiedNode, ThrottlerAndFairness) t.enqueue(b, {req_cost}); } - double shareA = 0.1; - double shareB = 0.9; + double share_a = 0.1; + double share_b = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); - auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - double consumedA = 0; - double consumedB = 0; + double consumed_a = 0; + double consumed_b = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); - t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); - consumedA = arrival_curve * shareA; - consumedB = arrival_curve * shareB; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; } } @@ -493,3 +493,63 @@ TEST(SchedulerUnifiedNode, ResourceGuardException) t.dequeue(2); t.consumed("A", 20); } + +TEST(SchedulerUnifiedNode, UpdateWeight) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{}}); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, UpdatePriority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-1}}); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); +} From ee2a5dc4b5794200fb899087b9ff2b75d3b9ee9b Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:11:42 +0000 Subject: [PATCH 201/816] unittest for parent update --- .../tests/gtest_unified_scheduler_node.cpp | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index cfd837d4f1a..5b86b54e9b0 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -553,3 +553,42 @@ TEST(SchedulerUnifiedNode, UpdatePriority) t.consumed("A", 10); t.consumed("B", 10); } + +TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + + auto x = t.createUnifiedNode("X", a, {}); + auto y = t.createUnifiedNode("Y", b, {}); + + t.enqueue(x, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("X", 20); + t.consumed("Y", 0); + + t.updateUnifiedNode(x, a, b, {}); + + t.dequeue(2); + t.consumed("X", 10); + t.consumed("Y", 10); + + t.updateUnifiedNode(y, b, a, {}); + + t.dequeue(2); + t.consumed("X", 0); + t.consumed("Y", 20); + + t.updateUnifiedNode(y, a, all, {}); + t.updateUnifiedNode(x, b, all, {}); + + t.dequeue(4); + t.consumed("X", 20); + t.consumed("Y", 20); +} + From 6f243450cc2fd4ada3a04c50272f7b369b168682 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:12:26 +0000 Subject: [PATCH 202/816] fix parent detaching --- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index f7b5d1a2056..ef10458df0d 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -135,6 +135,7 @@ private: if (it == children.end()) return {}; // unknown child + detach(child); children.erase(it); if (children.size() == 1) { From cd393c7f9d6549a72465d57a7eeee64f4b0b0706 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:31:15 +0000 Subject: [PATCH 203/816] add unit tests for intermediate node updates --- .../tests/gtest_unified_scheduler_node.cpp | 83 ++++++++++++++++++- 1 file changed, 81 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 5b86b54e9b0..10f92bc43c3 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -451,7 +451,6 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.consumed("A", 20); } - TEST(SchedulerUnifiedNode, ResourceGuardException) { ResourceTest t; @@ -561,7 +560,6 @@ TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) auto all = t.createUnifiedNode("all"); auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); - auto x = t.createUnifiedNode("X", a, {}); auto y = t.createUnifiedNode("Y", b, {}); @@ -592,3 +590,84 @@ TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) t.consumed("Y", 20); } +TEST(SchedulerUnifiedNode, UpdatePriorityOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto x1 = t.createUnifiedNode("X1", a, {}); + auto y1 = t.createUnifiedNode("Y1", b, {}); + auto x2 = t.createUnifiedNode("X2", a, {}); + auto y2 = t.createUnifiedNode("Y2", b, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{2}}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{1}}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} + +TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto c = t.createUnifiedNode("C", a, {}); + auto d = t.createUnifiedNode("D", b, {}); + auto x1 = t.createUnifiedNode("X1", c, {}); + auto y1 = t.createUnifiedNode("Y1", d, {}); + auto x2 = t.createUnifiedNode("X2", c, {}); + auto y2 = t.createUnifiedNode("Y2", d, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(c, a, b, {}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(d, b, a, {}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} From a803c56ae2943bbb46a87572448d6434d1ef4337 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Oct 2024 14:06:08 +0000 Subject: [PATCH 204/816] fix JoinStep permute columns --- src/Processors/QueryPlan/JoinStep.cpp | 106 ++++------- .../Transforms/ColumnPermuteTransform.cpp | 16 +- .../Transforms/ColumnPermuteTransform.h | 2 + .../01763_filter_push_down_bugs.reference | 2 +- ...emove_redundant_sorting_analyzer.reference | 4 +- ...move_redundant_distinct_analyzer.reference | 18 +- .../02514_analyzer_drop_join_on.reference | 44 ++--- ...oin_with_totals_and_subquery_bug.reference | 2 +- .../02835_join_step_explain.reference | 28 +-- ...filter_push_down_equivalent_sets.reference | 166 +++++++++--------- 10 files changed, 175 insertions(+), 213 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 3edc64ef967..dcedc57713d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes namespace { -std::vector> describeJoinActions(const JoinPtr & join) +static std::vector> describeJoinActions(const JoinPtr & join) { std::vector> description; const auto & table_join = join->getTableJoin(); @@ -37,52 +37,37 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } -size_t getPrefixLength(const NameSet & prefix, const Names & names) +std::vector getPermutationForBlock( + const Block & block, + const Block & lhs_block, + const Block & rhs_block, + const NameSet & name_filter) { - size_t i = 0; - for (; i < names.size(); ++i) - { - if (!prefix.contains(names[i])) - break; - } - return i; -} + std::vector permutation; + permutation.reserve(block.columns()); + Block::NameMap name_map = block.getNamesToIndexesMap(); -std::vector getPermutationToRotate(size_t prefix_size, size_t total_size) -{ - std::vector permutation(total_size); - size_t i = prefix_size % total_size; - for (auto & elem : permutation) + bool is_trivial = true; + for (const auto & other_block : {lhs_block, rhs_block}) { - elem = i; - i = (i + 1) % total_size; + for (const auto & col : other_block) + { + if (!name_filter.contains(col.name)) + continue; + if (auto it = name_map.find(col.name); it != name_map.end()) + { + is_trivial = is_trivial && it->second == permutation.size(); + permutation.push_back(it->second); + } + } } + + if (is_trivial && permutation.size() == block.columns()) + return {}; + return permutation; } -Block rotateBlock(const Block & block, size_t prefix_size) -{ - auto columns = block.getColumnsWithTypeAndName(); - std::rotate(columns.begin(), columns.begin() + prefix_size, columns.end()); - auto res = Block(std::move(columns)); - return res; -} - -NameSet getNameSetFromBlock(const Block & block) -{ - NameSet names; - for (const auto & column : block) - names.insert(column.name); - return names; -} - -Block rotateBlock(const Block & block, const Block & prefix_block) -{ - NameSet prefix_names_set = getNameSetFromBlock(prefix_block); - size_t prefix_size = getPrefixLength(prefix_names_set, block.getNames()); - return rotateBlock(block, prefix_size); -} - } JoinStep::JoinStep( @@ -109,7 +94,8 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); - NameSet rhs_names = getNameSetFromBlock(pipelines[1]->getHeader()); + Block lhs_header = pipelines[0]->getHeader(); + Block rhs_header = pipelines[1]->getHeader(); if (swap_streams) std::swap(pipelines[0], pipelines[1]); @@ -135,29 +121,15 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (!use_new_analyzer) return pipeline; - const auto & result_names = pipeline->getHeader().getNames(); - size_t prefix_size = getPrefixLength(rhs_names, result_names); - if (!columns_to_remove.empty() || (0 < prefix_size && prefix_size < result_names.size())) + auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output); + if (!column_permutation.empty()) { - auto column_permutation = getPermutationToRotate(prefix_size, result_names.size()); - size_t n = 0; - auto it = columns_to_remove.begin(); - for (size_t i = 0; i < column_permutation.size(); ++i) - { - if (it != columns_to_remove.end() && *it == i) - ++it; - else - column_permutation[n++] = column_permutation[i]; - } - column_permutation.resize(n); - pipeline->addSimpleTransform([&column_permutation](const Block & header) { return std::make_shared(header, column_permutation); }); } - return pipeline; } @@ -177,12 +149,16 @@ void JoinStep::describeActions(FormatSettings & settings) const for (const auto & [name, value] : describeJoinActions(join)) settings.out << prefix << name << ": " << value << '\n'; + if (swap_streams) + settings.out << prefix << "Swapped: true\n"; } void JoinStep::describeActions(JSONBuilder::JSONMap & map) const { for (const auto & [name, value] : describeJoinActions(join)) map.add(name, value); + if (swap_streams) + map.add("Swapped", true); } void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) @@ -210,20 +186,10 @@ void JoinStep::updateOutputStream() return; } - if (swap_streams) - result_header = rotateBlock(result_header, input_streams[1].header); + auto column_permutation = getPermutationForBlock(result_header, input_streams[0].header, input_streams[1].header, required_output); + if (!column_permutation.empty()) + result_header = ColumnPermuteTransform::permute(std::move(result_header), column_permutation); - columns_to_remove.clear(); - for (size_t i = 0; i < result_header.columns(); ++i) - { - if (!required_output.contains(result_header.getByPosition(i).name)) - columns_to_remove.insert(i); - } - /// Do not remove all columns, keep at least one - if (!columns_to_remove.empty() && columns_to_remove.size() == result_header.columns()) - columns_to_remove.erase(columns_to_remove.begin()); - - result_header.erase(columns_to_remove); output_stream = DataStream { .header = result_header }; } diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp index eb2a691d6d1..67c7996cbe0 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -16,13 +16,6 @@ void applyPermutation(std::vector & data, const std::vector & permuta data = std::move(res); } -Block permuteBlock(const Block & block, const std::vector & permutation) -{ - auto columns = block.getColumnsWithTypeAndName(); - applyPermutation(columns, permutation); - return Block(columns); -} - void permuteChunk(Chunk & chunk, const std::vector & permutation) { size_t num_rows = chunk.getNumRows(); @@ -33,8 +26,15 @@ void permuteChunk(Chunk & chunk, const std::vector & permutation) } +Block ColumnPermuteTransform::permute(const Block & block, const std::vector & permutation) +{ + auto columns = block.getColumnsWithTypeAndName(); + applyPermutation(columns, permutation); + return Block(columns); +} + ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) - : ISimpleTransform(header_, permuteBlock(header_, permutation_), false) + : ISimpleTransform(header_, permute(header_, permutation_), false) , permutation(permutation_) { } diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h index f4d68850193..25f3a8d0825 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.h +++ b/src/Processors/Transforms/ColumnPermuteTransform.h @@ -19,6 +19,8 @@ public: void transform(Chunk & chunk) override; + static Block permute(const Block & block, const std::vector & permutation); + private: Names column_names; std::vector permutation; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 19018a610b7..229ac6eae09 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter ((WHERE + DROP unused columns after JOIN)) + Filter (WHERE) Join (JOIN FillRightFirst) Expression ReadFromMergeTree (default.t1) diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 3c68d14fdf2..c9bf36f88ea 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers @@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index 867ae394c1f..baa2be9dfdb 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -79,7 +79,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) Distinct (Preliminary DISTINCT) - Expression ((Projection + DROP unused columns after JOIN)) + Expression (Projection) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + Project names)) Distinct (DISTINCT) @@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -280,7 +280,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -315,7 +315,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -386,7 +386,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -457,7 +457,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index d407a4c7985..bbfdf1ad5f4 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -8,17 +8,17 @@ Header: count() UInt64 Aggregating Header: __table1.a2 String count() UInt64 - Expression ((Before GROUP BY + DROP unused columns after JOIN)) + Expression (Before GROUP BY) Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table3.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) @@ -45,38 +45,32 @@ Header: count() UInt64 EXPLAIN PLAN header = 1 SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) ; -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String __table4.d2 String - Expression (DROP unused columns after JOIN) + Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression (DROP unused columns after JOIN) + Expression (Change column names to column identifiers) Header: __table1.a2 String __table1.k UInt64 - Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 - Expression (Change column names to column identifiers) - Header: __table1.a2 String - __table1.k UInt64 - ReadFromMemoryStorage - Header: a2 String - k UInt64 - Expression (Change column names to column identifiers) - Header: __table2.k UInt64 - ReadFromMemoryStorage - Header: k UInt64 + ReadFromMemoryStorage + Header: a2 String + k UInt64 Expression (Change column names to column identifiers) - Header: __table3.k UInt64 + Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 + Expression (Change column names to column identifiers) + Header: __table3.k UInt64 + ReadFromMemoryStorage + Header: k UInt64 Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 @@ -105,21 +99,21 @@ Header: bx String Expression Header: __table1.a2 String __table2.bx String - __table4.c2 String __table4.c1 UInt64 + __table4.c2 String Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table4.c2 String __table4.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + __table4.c2 String + Expression (JOIN actions) Header: __table1.a2 String - __table2.bx String __table2.b1 UInt64 + __table2.bx String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table2.bx String __table2.b1 UInt64 + __table2.bx String Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 __table1.a2 String diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 86e7e2a6a49..116c78a15e4 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -5,7 +5,7 @@ 1 1 -1 +0 \N 100000000000000000000 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 2f641d4aa44..bdbc019d4f8 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -1,22 +1,22 @@ -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.value_1 String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.value_1 String __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -50,25 +50,25 @@ Positions: 4 0 2 1 Parts: 1 Granules: 1 -- -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.value_1 String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.value_1 String __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ASOF Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index c98a98b236c..b7718d926c6 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -12,18 +12,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -81,18 +81,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -150,18 +150,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -222,18 +222,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -291,31 +291,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -367,31 +367,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -443,18 +443,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -512,31 +512,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -588,31 +588,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -664,34 +664,34 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6 - FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 + FUNCTION equals(__table2.id : 2, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin From a019dd0410adff0b0e64eb0d818b5f25056fc764 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Oct 2024 17:12:28 +0000 Subject: [PATCH 205/816] fix clang tidy --- src/Processors/QueryPlan/JoinStep.cpp | 4 ++-- src/Processors/Transforms/ColumnPermuteTransform.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index dcedc57713d..9cb06042cf6 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes namespace { -static std::vector> describeJoinActions(const JoinPtr & join) +std::vector> describeJoinActions(const JoinPtr & join) { std::vector> description; const auto & table_join = join->getTableJoin(); @@ -188,7 +188,7 @@ void JoinStep::updateOutputStream() auto column_permutation = getPermutationForBlock(result_header, input_streams[0].header, input_streams[1].header, required_output); if (!column_permutation.empty()) - result_header = ColumnPermuteTransform::permute(std::move(result_header), column_permutation); + result_header = ColumnPermuteTransform::permute(result_header, column_permutation); output_stream = DataStream { .header = result_header }; } diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp index 67c7996cbe0..f371689814c 100644 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -12,7 +12,7 @@ void applyPermutation(std::vector & data, const std::vector & permuta std::vector res; res.reserve(permutation.size()); for (size_t i : permutation) - res.emplace_back(std::move(data[i])); + res.push_back(data[i]); data = std::move(res); } From dca5c250fe0a70e8d0bb88714fd42cb7b1e85168 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 18:19:05 +0000 Subject: [PATCH 206/816] add tests for throttler updates --- .../Scheduler/Nodes/tests/ResourceTest.h | 7 +- .../tests/gtest_unified_scheduler_node.cpp | 73 +++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 704f7119300..1bd7824911d 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -187,7 +187,12 @@ public: { EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported bool detached = false; - if (UnifiedSchedulerNode::updateRequiresDetach(old_parent->basename, new_parent->basename, node->getSettings(), new_settings)) { + if (UnifiedSchedulerNode::updateRequiresDetach( + old_parent ? old_parent->basename : "", + new_parent ? new_parent->basename : "", + node->getSettings(), + new_settings)) + { if (old_parent) old_parent->detachUnifiedChild(node); detached = true; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 10f92bc43c3..dcc29422d7f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -671,3 +671,76 @@ TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) t.consumed("X2", 0); t.consumed("Y2", 20); } + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxSpeed) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 1.0, .max_burst = 20.0}); + + t.process(start + std::chrono::seconds(12)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(22)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxBurst) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 0); // There was nothing to consume + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 30.0}); + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 40); // min(30 tokens, 5 sec * 10 tokens/sec) = 30 tokens + 1 extra request to go below zero + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 60); // Consume rest + + t.process(start + std::chrono::seconds(150)); + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 100.0, .max_burst = 200.0}); + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {195, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 201); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} From d307f266378e4c2c0fa910b5d4cae75b7df3d537 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 8 Oct 2024 09:03:23 -0300 Subject: [PATCH 207/816] make sure valid until is parsed for not identified as well --- src/Parsers/Access/ParserCreateUserQuery.cpp | 2 ++ tests/integration/test_user_valid_until/test.py | 12 ++++++++++++ 2 files changed, 14 insertions(+) diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 68ebb8a983f..fe53a0ef657 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -298,6 +298,8 @@ namespace authentication_methods.emplace_back(std::make_shared()); authentication_methods.back()->type = AuthenticationType::NO_PASSWORD; + parseValidUntil(pos, expected, authentication_methods.back()->valid_until); + return true; } diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index 565790457b2..64071491200 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -76,6 +76,18 @@ def test_basic(started_cluster): node.query("DROP USER IF EXISTS user_basic") + # NOT IDENTIFIED test to make sure valid until is also parsed on its short-circuit + node.query("CREATE USER user_basic NOT IDENTIFIED VALID UNTIL '01/01/2010'") + + assert ( + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH no_password VALID UNTIL \\'2010-01-01 00:00:00\\'\n" + ) + + assert error in node.query_and_get_error("SELECT 1", user="user_basic") + + node.query("DROP USER IF EXISTS user_basic") + def test_details(started_cluster): node.query("DROP USER IF EXISTS user_details_infinity, user_details_time_only") From e85672d97cfecefb659cdb6711ab83a904269da9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 8 Oct 2024 09:19:02 -0300 Subject: [PATCH 208/816] black check --- tests/integration/test_user_valid_until/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index 64071491200..828432f223e 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -80,8 +80,8 @@ def test_basic(started_cluster): node.query("CREATE USER user_basic NOT IDENTIFIED VALID UNTIL '01/01/2010'") assert ( - node.query("SHOW CREATE USER user_basic") - == "CREATE USER user_basic IDENTIFIED WITH no_password VALID UNTIL \\'2010-01-01 00:00:00\\'\n" + node.query("SHOW CREATE USER user_basic") + == "CREATE USER user_basic IDENTIFIED WITH no_password VALID UNTIL \\'2010-01-01 00:00:00\\'\n" ) assert error in node.query_and_get_error("SELECT 1", user="user_basic") From 58ec00aea869318a2af81cc5189728376fd3cfb0 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:18:41 +0000 Subject: [PATCH 209/816] fix typos --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.h | 6 +++--- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 2 +- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 4 ++-- .../Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp | 4 ++-- src/Common/Scheduler/ResourceRequest.h | 4 ++-- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 2 +- .../Scheduler/Workload/createWorkloadEntityStorage.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 4 ++-- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 101a0fa4c32..91c54bade3c 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -189,7 +189,7 @@ void IOResourceManager::Resource::updateCurrentVersion() { previous_version->newer_version = current_version; // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. - // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems impossible) // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index dc57b985455..f33251ad37c 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -30,7 +30,7 @@ namespace DB * * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). - * After DROP RESOURCE parts of hierarchy might be keept alive while at least one query uses it. + * After DROP RESOURCE parts of hierarchy might be kept alive while at least one query uses it. * * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. @@ -46,7 +46,7 @@ namespace DB * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). * - * Here is an example of SQL and corresponding heirarchy of scheduler nodes: + * Here is an example of SQL and corresponding hierarchy of scheduler nodes: * CREATE RESOURCE my_io_resource (...) * CREATE WORKLOAD all * CREATE WORKLOAD production PARENT all @@ -85,7 +85,7 @@ namespace DB * * Previous version should hold reference to a newer version. It is required for proper handling of updates. * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). - * It may move a queue to a new position in the hierarchy or create/destry constraints, thus resource requests + * It may move a queue to a new position in the hierarchy or create/destroy constraints, thus resource requests * created by old classifier may reference constraints of newer versions through `request->constraints` which * is filled during dequeueRequst(). * diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index ef10458df0d..1ab187b388e 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -29,7 +29,7 @@ using UnifiedSchedulerNodePtr = std::shared_ptr; /* * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes - * are also present. This approch is easiers for manipulations in runtime than using multiple types of nodes. + * are also present. This approach is easiers for manipulations in runtime than using multiple types of nodes. * * Unified node is capable of updating its internal structure based on: * 1. Number of children (fifo if =0 or fairness/priority if >0). diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 1bd7824911d..05797189837 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -336,7 +336,7 @@ struct ResourceTestManager : public ResourceTestBase waitExecute(); } - /// Just enqueue resource request, do not block (neede for tests to sync). Call `waitExecuted()` afterwards + /// Just enqueue resource request, do not block (needed for tests to sync). Call `waitExecuted()` afterwards Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) , t(t_) @@ -345,7 +345,7 @@ struct ResourceTestManager : public ResourceTestBase t.onEnqueue(link); } - /// Waits for ResourceRequest::execute() to be called for enqueued requet + /// Waits for ResourceRequest::execute() to be called for enqueued request void waitExecute() { lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index dcc29422d7f..159ccc616f4 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -431,7 +431,7 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled - // This will destory the queue and fail both requests + // This will destroy the queue and fail both requests auto a = t.createUnifiedNode("A", all); t.failed(20); @@ -472,7 +472,7 @@ TEST(SchedulerUnifiedNode, ResourceGuardException) } }); - // This will destory the queue and fail both requests + // This will destroy the queue and fail both requests auto a = t.createUnifiedNode("A", all); t.failed(20); consumer.join(); diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index 03bdaec6a2b..e633af15157 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -17,8 +17,8 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -// TODO(serxa): validate hierarchy to avoid too many constrants -/// Max number of constraints for a request to pass though (depth of constaints chain) +// TODO(serxa): validate hierarchy to avoid too many constraints +/// Max number of constraints for a request to pass though (depth of constraints chain) constexpr size_t ResourceMaxConstraints = 8; /* diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index b1e426d363e..f29d0f45f22 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -460,7 +460,7 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lockgetID()); } - // Resources should be created first becase workloads could reference them + // Resources should be created first because workloads could reference them for (auto & [entity_name, ast] : resources) onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp index dde995db6e1..8475fe21455 100644 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -34,7 +34,7 @@ std::unique_ptr createWorkloadEntityStorage(const Contex zookeeper_path_key, disk_path_key); } - abort(); // TODO(serxa): crate WorkloadEntityKeeperStorage object + abort(); // TODO(serxa): create WorkloadEntityKeeperStorage object //return std::make_unique(global_context, config.getString(zookeeper_path_key)); } else diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index cd5f1e375d9..03ab0fd8572 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -110,7 +110,7 @@ DiskObjectStorage::DiskObjectStorage( if (read_resource_name_from_config.empty()) LOG_INFO(log, "Using resource '{}' for READ", resource_name); else - LOG_INFO(log, "Resource '{}' should be used for READ, but it is overriden by config to resource '{}'", + LOG_INFO(log, "Resource '{}' should be used for READ, but it is overridden by config to resource '{}'", resource_name, read_resource_name_from_config); read_resource_name_from_sql = resource_name; break; @@ -120,7 +120,7 @@ DiskObjectStorage::DiskObjectStorage( if (write_resource_name_from_config.empty()) LOG_INFO(log, "Using resource '{}' for WRITE", resource_name); else - LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overriden by config to resource '{}'", + LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overridden by config to resource '{}'", resource_name, write_resource_name_from_config); write_resource_name_from_sql = resource_name; break; From 6170b02c23512ae226c5179f4015b8cec29d0600 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:24:05 +0000 Subject: [PATCH 210/816] fix style --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 1 - src/Common/Scheduler/Nodes/IOResourceManager.h | 3 ++- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 5 +++++ src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h | 1 + src/Parsers/ASTCreateResourceQuery.h | 6 ++++-- tests/integration/test_scheduler/test.py | 2 +- 7 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 91c54bade3c..26f7c65ef55 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { - extern const int RESOURCE_ACCESS_DENIED; extern const int RESOURCE_NOT_FOUND; extern const int INVALID_SCHEDULER_NODE; extern const int LOGICAL_ERROR; diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index f33251ad37c..0bbd14c2ca9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -249,7 +249,8 @@ private: private: IOResourceManager * resource_manager; std::mutex mutex; - struct Attachment { + struct Attachment + { ResourcePtr resource; VersionPtr version; ResourceLink link; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 1ab187b388e..3edca1f70c1 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { extern const int INVALID_SCHEDULER_NODE; - extern const int LOGICAL_ERROR; } class UnifiedSchedulerNode; @@ -84,7 +83,8 @@ private: } /// A branch of the tree for a specific priority value - struct FairnessBranch { + struct FairnessBranch + { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached std::unordered_map children; // basename -> child diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 05797189837..b02aa00588a 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -29,6 +29,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + struct ResourceTestBase { ResourceTestBase() diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index e69de29bb2d..6f70f09beec 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -0,0 +1 @@ +#pragma once diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index e1713e6b063..5cf29b31ed0 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -10,11 +10,13 @@ namespace DB class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster { public: - enum class AccessMode { + enum class AccessMode + { Read, Write }; - struct Operation { + struct Operation + { AccessMode mode; String disk; }; diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 2c2fa043f28..24071a29bd6 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -2,11 +2,11 @@ # pylint: disable=redefined-outer-name # pylint: disable=line-too-long +import random import threading import time import pytest -import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster From 02867ca2e7630d866be8b9bc8ad90b48af18a3a0 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:58:06 +0000 Subject: [PATCH 211/816] style --- src/Common/Scheduler/Nodes/ClassifiersConfig.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 4b0b0eaccfa..455d0880aa6 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int RESOURCE_NOT_FOUND; -} - ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { Poco::Util::AbstractConfiguration::Keys keys; From 511054f92d524b59a2bade3e34965300e69949ee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 9 Oct 2024 01:06:04 +0000 Subject: [PATCH 212/816] Automatic style fix --- tests/integration/test_scheduler/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 24071a29bd6..b78376bffe2 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -645,7 +645,7 @@ def test_create_workload(): ) do_checks() - node.restart_clickhouse() # Check that workloads persist + node.restart_clickhouse() # Check that workloads persist do_checks() @@ -706,7 +706,9 @@ def test_workload_hierarchy_changes(): split_idx = random.randint(1, len(queries) - 2) for query_idx in range(0, split_idx): node.query(queries[query_idx]) - node.query("create resource io_test (write disk non_existent_disk, read disk non_existent_disk);") + node.query( + "create resource io_test (write disk non_existent_disk, read disk non_existent_disk);" + ) node.query("drop resource io_test;") for query_idx in range(split_idx, len(queries)): node.query(queries[query_idx]) From cde94e7afa85d842d9ff6b760acc55ba23a78a03 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 08:20:03 +0000 Subject: [PATCH 213/816] adjust test 01271_show_privileges --- tests/queries/0_stateless/01271_show_privileges.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..feeef5e89fa 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -59,6 +59,8 @@ CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE ARBITRARY TEMPORARY TABLE CREATE ARBITRARY TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE +CREATE WORKLOAD [] GLOBAL CREATE +CREATE RESOURCE [] GLOBAL CREATE CREATE NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN CREATE [] \N ALL DROP DATABASE [] DATABASE DROP @@ -66,6 +68,8 @@ DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP +DROP WORKLOAD [] GLOBAL DROP +DROP RESOURCE [] GLOBAL DROP DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN DROP [] \N ALL UNDROP TABLE [] TABLE ALL From fd0a9511eb912011d0de530dfe3be94de27d5836 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 16:47:31 +0000 Subject: [PATCH 214/816] fix stateless test --- tests/queries/0_stateless/03232_resource_create_and_drop.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql index eb356e12448..ceebd557a51 100644 --- a/tests/queries/0_stateless/03232_resource_create_and_drop.sql +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because creating the same resource twice will fail CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); From b86f3481d1ebf82601b38a12343fb4b055765cda Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Oct 2024 00:45:45 +0000 Subject: [PATCH 215/816] exclude jobs option for fuzzers --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index b3c19fbb0a4..e4a8c691ded 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -122,7 +122,7 @@ def run_fuzzer(fuzzer: str, timeout: int): if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - f"-{key}={value}" for key, value in parser["libfuzzer"].items() + f"-{key}={value}" for key, value in parser["libfuzzer"].items() if key != "jobs" ) if parser.has_section("fuzzer_arguments"): From c6d6ee27f4e7feaa2dbcedcf2a3c98faef041345 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Oct 2024 00:52:58 +0000 Subject: [PATCH 216/816] Automatic style fix --- tests/fuzz/runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index e4a8c691ded..f398b33308e 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -122,7 +122,9 @@ def run_fuzzer(fuzzer: str, timeout: int): if parser.has_section("libfuzzer"): custom_libfuzzer_options = " ".join( - f"-{key}={value}" for key, value in parser["libfuzzer"].items() if key != "jobs" + f"-{key}={value}" + for key, value in parser["libfuzzer"].items() + if key != "jobs" ) if parser.has_section("fuzzer_arguments"): From 384ba4217dcb8bc526e50dea7cbf88b9ed7e734d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Oct 2024 05:15:29 +0000 Subject: [PATCH 217/816] Fix logical error in JSONExtract with LowCardinality(Nullable) --- src/Formats/JSONExtractTree.cpp | 2 +- .../0_stateless/03247_json_extract_lc_nullable.reference | 1 + tests/queries/0_stateless/03247_json_extract_lc_nullable.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03247_json_extract_lc_nullable.reference create mode 100644 tests/queries/0_stateless/03247_json_extract_lc_nullable.sql diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 9ea335ee7fe..ae6051823b7 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1085,7 +1085,7 @@ public: } auto & col_lc = assert_cast(column); - auto tmp_nested = col_lc.getDictionary().getNestedColumn()->cloneEmpty(); + auto tmp_nested = removeNullable(col_lc.getDictionary().getNestedColumn()->cloneEmpty())->assumeMutable(); if (!nested->insertResultToColumn(*tmp_nested, element, insert_settings, format_settings, error)) return false; diff --git a/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference b/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference new file mode 100644 index 00000000000..a949a93dfcc --- /dev/null +++ b/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference @@ -0,0 +1 @@ +128 diff --git a/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql b/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql new file mode 100644 index 00000000000..bac1e34c1ab --- /dev/null +++ b/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql @@ -0,0 +1,2 @@ +select JSONExtract('{"a" : 128}', 'a', 'LowCardinality(Nullable(Int128))'); + From a5853ee23022969e8250b4a83ed2ba9a02bcaf77 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 10 Oct 2024 11:52:34 +0000 Subject: [PATCH 218/816] fix empty outer_scope_columns in JoinStep --- src/Planner/PlannerJoinTree.cpp | 12 ++++++- ...convert_outer_join_to_inner_join.reference | 36 +++++++++---------- 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0e82215c12a..ee0b68f4b63 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1684,13 +1684,23 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } auto join_pipeline_type = join_algorithm->pipelineType(); + + ColumnIdentifierSet outer_scope_columns_nonempty; + if (outer_scope_columns.empty()) + { + if (left_header.columns() > 1) + outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); + else if (right_header.columns() > 1) + outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); + } + auto join_step = std::make_unique( left_plan.getCurrentDataStream(), right_plan.getCurrentDataStream(), std::move(join_algorithm), settings[Setting::max_block_size], settings[Setting::max_threads], - outer_scope_columns, + outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, false /*optimize_read_in_order*/, true /*optimize_skip_unused_shards*/); join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index d35bdeff98b..5fde4f80c5d 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -5,18 +5,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -75,18 +75,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -145,18 +145,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin From 46faeaafd2bc93a60b3ad9671a476e9e66830675 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 10 Oct 2024 11:17:03 -0300 Subject: [PATCH 219/816] add missing continue and other two continue statements to be on the safe side --- src/Parsers/Access/ParserCreateUserQuery.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index fe53a0ef657..3b0a8823b8a 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -576,11 +576,13 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { parsed_add_identified_with = parseAddIdentifiedWith(pos, expected, auth_data); } + continue; } if (!reset_authentication_methods_to_new && alter && auth_data.empty()) { reset_authentication_methods_to_new = parseResetAuthenticationMethods(pos, expected); + continue; } AllowedClientHosts new_hosts; @@ -642,6 +644,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (auth_data.empty() && !global_valid_until) { parseValidUntil(pos, expected, global_valid_until); + continue; } break; From 845c4a543c091f5951b5e5b2063531ad264da6d1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Oct 2024 18:59:48 +0000 Subject: [PATCH 220/816] add test for libfuzzer --- utils/CMakeLists.txt | 4 ++++ utils/libfuzzer-test/CMakeLists.txt | 1 + utils/libfuzzer-test/README.md | 1 + utils/libfuzzer-test/test_basic_fuzzer/CMakeLists.txt | 1 + utils/libfuzzer-test/test_basic_fuzzer/main.cpp | 11 +++++++++++ 5 files changed, 18 insertions(+) create mode 100644 utils/libfuzzer-test/CMakeLists.txt create mode 100644 utils/libfuzzer-test/README.md create mode 100644 utils/libfuzzer-test/test_basic_fuzzer/CMakeLists.txt create mode 100644 utils/libfuzzer-test/test_basic_fuzzer/main.cpp diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index ec44a1e1de9..8c706ee6b67 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -23,3 +23,7 @@ if (ENABLE_UTILS) add_subdirectory (keeper-data-dumper) add_subdirectory (memcpy-bench) endif () + +if (ENABLE_FUZZING) + add_subdirectory (libfuzzer-test) +endif () diff --git a/utils/libfuzzer-test/CMakeLists.txt b/utils/libfuzzer-test/CMakeLists.txt new file mode 100644 index 00000000000..8765787ff8a --- /dev/null +++ b/utils/libfuzzer-test/CMakeLists.txt @@ -0,0 +1 @@ +add_subdirectory (test_basic_fuzzer) diff --git a/utils/libfuzzer-test/README.md b/utils/libfuzzer-test/README.md new file mode 100644 index 00000000000..5598cbdb961 --- /dev/null +++ b/utils/libfuzzer-test/README.md @@ -0,0 +1 @@ +This folder contains various stuff intended to test libfuzzer functionality. diff --git a/utils/libfuzzer-test/test_basic_fuzzer/CMakeLists.txt b/utils/libfuzzer-test/test_basic_fuzzer/CMakeLists.txt new file mode 100644 index 00000000000..dc927f35a4b --- /dev/null +++ b/utils/libfuzzer-test/test_basic_fuzzer/CMakeLists.txt @@ -0,0 +1 @@ +add_executable (test_basic_fuzzer main.cpp) diff --git a/utils/libfuzzer-test/test_basic_fuzzer/main.cpp b/utils/libfuzzer-test/test_basic_fuzzer/main.cpp new file mode 100644 index 00000000000..7ccad63273d --- /dev/null +++ b/utils/libfuzzer-test/test_basic_fuzzer/main.cpp @@ -0,0 +1,11 @@ +#include +#include + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t *data, size_t size) +{ + if (size > 0 && data[0] == 'H') + if (size > 1 && data[1] == 'I') + if (size > 2 && data[2] == '!') + __builtin_trap(); + return 0; +} From 6d8125d520a1c00efde8377f27a096aec56a41db Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 10 Oct 2024 15:38:22 -0400 Subject: [PATCH 221/816] trigger build From b064d757ca0af321e1a4929d6be1fe3b12dd200f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 10 Oct 2024 15:48:33 -0400 Subject: [PATCH 222/816] trigger build From ca5f3c50d2e9a74a0a5a7cf9b5ef7f42e171fba7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 10 Oct 2024 16:10:02 -0400 Subject: [PATCH 223/816] trigger build --- src/DataTypes/fuzzers/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 8dedd3470e2..8940586fc70 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,3 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) + target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms) From 8f9ccdf69c983440d698deb0497250a92dcf76ec Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Oct 2024 23:08:52 +0000 Subject: [PATCH 224/816] fix parser --- tests/fuzz/runner.py | 30 +++++++++++++----------------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index f398b33308e..c6c978c3508 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -17,8 +17,7 @@ FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") def report(source: str, reason: str, call_stack: list, test_unit: str): print(f"########### REPORT: {source} {reason} {test_unit}") - for line in call_stack: - print(f" {line}") + print("".join(call_stack)) print("########### END OF REPORT ###########") @@ -31,31 +30,28 @@ def process_error(error: str): ERROR = r"^==\d+==\s?ERROR: (\S+): (.*)" error_source = "" error_reason = "" - TEST_UNIT_LINE = r"artifact_prefix='.*/'; Test unit written to (.*)" - call_stack = [] - is_call_stack = False + test_unit = "" + TEST_UNIT_LINE = r"artifact_prefix='.*\/'; Test unit written to (.*)" + error_info = [] + is_error = False # pylint: disable=unused-variable - for line_num, line in enumerate(error.splitlines(), 1): - if is_call_stack: - if re.search(r"^==\d+==", line): - is_call_stack = False - continue - call_stack.append(line) - continue - - if call_stack: + for line_num, line in enumerate(sys.stdin, 1): + if is_error: + error_info.append(line) match = re.search(TEST_UNIT_LINE, line) if match: - report(error_source, error_reason, call_stack, match.group(1)) - call_stack.clear() + test_unit = match.group(1) continue match = re.search(ERROR, line) if match: + error_info.append(line) error_source = match.group(1) error_reason = match.group(2) - is_call_stack = True + is_error = True + + report(error_source, error_reason, error_info, test_unit) def kill_fuzzer(fuzzer: str): From 85a6bb1d1fc4024d57139008953fb35b5be51288 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 11 Oct 2024 03:11:39 +0000 Subject: [PATCH 225/816] fix parser --- tests/fuzz/runner.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index c6c978c3508..3a462d11172 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -16,9 +16,9 @@ FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") def report(source: str, reason: str, call_stack: list, test_unit: str): - print(f"########### REPORT: {source} {reason} {test_unit}") - print("".join(call_stack)) - print("########### END OF REPORT ###########") + logging.info("########### REPORT: %s %s %s", source, reason, test_unit) + logging.info("".join(call_stack)) + logging.info("########### END OF REPORT ###########") # pylint: disable=unused-argument @@ -157,7 +157,7 @@ def run_fuzzer(fuzzer: str, timeout: int): ) except subprocess.CalledProcessError as e: # print("Command failed with error:", e) - print("Stderr output: ", e.stderr) + logging.info("Stderr output: %s", e.stderr) process_error(e.stderr) except subprocess.TimeoutExpired as e: logging.info("Timeout for %s", cmd_line) From 5e99f63e7e5b825813f01ac56a0094d6c95c276a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 11 Oct 2024 04:05:08 +0000 Subject: [PATCH 226/816] fix parser --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 3a462d11172..1d3829598c3 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -36,7 +36,7 @@ def process_error(error: str): is_error = False # pylint: disable=unused-variable - for line_num, line in enumerate(sys.stdin, 1): + for line_num, line in enumerate(error.splitlines(), 1): if is_error: error_info.append(line) match = re.search(TEST_UNIT_LINE, line) From 164cc1211a9d322e4f8842b3fb21b6bfa3755d36 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 11 Oct 2024 14:11:54 -0300 Subject: [PATCH 227/816] continue only on success.. --- src/Parsers/Access/ParserCreateUserQuery.cpp | 22 +++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 3b0a8823b8a..657302574c2 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -572,17 +572,27 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { parsed_identified_with = parseIdentifiedOrNotIdentified(pos, expected, auth_data); - if (!parsed_identified_with && alter) + if (parsed_identified_with) + { + continue; + } + else if (alter) { parsed_add_identified_with = parseAddIdentifiedWith(pos, expected, auth_data); + if (parsed_add_identified_with) + { + continue; + } } - continue; } if (!reset_authentication_methods_to_new && alter && auth_data.empty()) { reset_authentication_methods_to_new = parseResetAuthenticationMethods(pos, expected); - continue; + if (reset_authentication_methods_to_new) + { + continue; + } } AllowedClientHosts new_hosts; @@ -643,8 +653,10 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (auth_data.empty() && !global_valid_until) { - parseValidUntil(pos, expected, global_valid_until); - continue; + if (parseValidUntil(pos, expected, global_valid_until)) + { + continue; + } } break; From c4763389c145416186d128aa5bd03633ab5b383b Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 12 Oct 2024 10:53:16 +0000 Subject: [PATCH 228/816] fix tidy build --- src/Common/Scheduler/Nodes/FairPolicy.h | 2 +- src/Common/Scheduler/Nodes/FifoQueue.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/PriorityPolicy.h | 2 +- src/Common/Scheduler/Nodes/SemaphoreConstraint.h | 2 +- src/Common/Scheduler/Nodes/ThrottlerConstraint.h | 2 +- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- src/Common/Scheduler/SchedulerRoot.h | 2 +- .../Workload/WorkloadEntityStorageBase.cpp | 15 +-------------- .../Workload/WorkloadEntityStorageBase.h | 3 +-- src/Parsers/ASTCreateResourceQuery.h | 2 +- 11 files changed, 12 insertions(+), 26 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index c6151c5727b..a865711c460 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -28,7 +28,7 @@ namespace ErrorCodes * of a child is set to vruntime of "start" of the last request. This guarantees immediate processing * of at least single request of newly activated children and thus best isolation and scheduling latency. */ -class FairPolicy : public ISchedulerNode +class FairPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 3372864402c..ea8985e314f 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -23,7 +23,7 @@ namespace ErrorCodes /* * FIFO queue to hold pending resource requests */ -class FifoQueue : public ISchedulerQueue +class FifoQueue final : public ISchedulerQueue { public: FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 26f7c65ef55..0f015dd22b6 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -250,7 +250,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) { - for (auto [entity_type, entity_name, entity] : events) + for (const auto & [entity_type, entity_name, entity] : events) { switch (entity_type) { diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index bb2c98d28b1..cfbe242c13e 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -19,7 +19,7 @@ namespace ErrorCodes * Scheduler node that implements priority scheduling policy. * Requests are scheduled in order of priorities. */ -class PriorityPolicy : public ISchedulerNode +class PriorityPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index c7dccbcdfbb..2454c1ec5bf 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -14,7 +14,7 @@ namespace DB * Limited concurrency constraint. * Blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost` */ -class SemaphoreConstraint : public ISchedulerConstraint +class SemaphoreConstraint final : public ISchedulerConstraint { static constexpr Int64 default_max_requests = std::numeric_limits::max(); static constexpr Int64 default_max_cost = std::numeric_limits::max(); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index e9e4c15b18f..a2594b7ff2e 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -13,7 +13,7 @@ namespace DB * Limited throughput constraint. Blocks if token-bucket constraint is violated: * i.e. more than `max_burst + duration * max_speed` cost units (aka tokens) dequeued from this node in last `duration` seconds. */ -class ThrottlerConstraint : public ISchedulerConstraint +class ThrottlerConstraint final : public ISchedulerConstraint { public: static constexpr double default_burst_seconds = 1.0; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 3edca1f70c1..e8e568c9acb 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -54,7 +54,7 @@ using UnifiedSchedulerNodePtr = std::shared_ptr; * - unified child: leaf of this "internal" subtree (CHILD[p,w]); * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) */ -class UnifiedSchedulerNode : public ISchedulerNode +class UnifiedSchedulerNode final : public ISchedulerNode { private: /// Helper function for managing a parent of a node @@ -472,7 +472,7 @@ public: } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children - std::shared_ptr getQueue() + std::shared_ptr getQueue() const { return static_pointer_cast(impl.branch.queue); } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 8549a1880fb..45e4309fc81 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -28,7 +28,7 @@ namespace ErrorCodes * Resource scheduler root node with a dedicated thread. * Immediate children correspond to different resources. */ -class SchedulerRoot : public ISchedulerNode +class SchedulerRoot final : public ISchedulerNode { private: struct Resource diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index f29d0f45f22..060bbbd6f87 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -430,7 +430,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector> WorkloadEntityStorageBase::getAllEntities return all_entities; } -// TODO(serxa): add notifications or remove this function -void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) -{ - boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; - std::lock_guard lock(mutex); - for (auto it = entities.begin(); it != entities.end();) - { - auto current = it++; - if (!names_set_to_keep.contains(current->first)) - entities.erase(current); - } -} - bool WorkloadEntityStorageBase::isIndirectlyReferenced(const String & target, const String & source) { std::queue bfs; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 51c5d3f0ac6..e1f43181a0c 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -45,7 +45,7 @@ public: const String & entity_name, bool throw_if_not_exists) override; - virtual scope_guard getAllEntitiesAndSubscribe( + scope_guard getAllEntitiesAndSubscribe( const OnChangedHandler & handler) override; protected: @@ -68,7 +68,6 @@ protected: void setAllEntities(const std::vector> & new_entities); void makeEventsForAllEntities(std::unique_lock & lock); - void removeAllEntitiesExcept(const Strings & entity_names_to_keep); /// Called by derived class after a new workload entity has been added. void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index 5cf29b31ed0..b05176837bc 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -33,7 +33,7 @@ public: ASTPtr clone() const override; - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } From d58395c834c12349e9f7e0774cdede1d593c704e Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 12 Oct 2024 15:53:31 +0000 Subject: [PATCH 229/816] fix tidy build --- .../Scheduler/Nodes/tests/ResourceTest.h | 61 ++++++++++++++++--- 1 file changed, 53 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index b02aa00588a..681453817c6 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -91,15 +91,56 @@ struct ResourceTestBase }; -struct ConstraintTest : public SemaphoreConstraint +struct ConstraintTest final : public ISchedulerConstraint { explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : SemaphoreConstraint(event_queue_, config, config_prefix) + : ISchedulerConstraint(event_queue_, config, config_prefix) + , impl(event_queue_, config, config_prefix) {} + const String & getTypeName() const override + { + return impl.getTypeName(); + } + + bool equals(ISchedulerNode * other) override + { + return impl.equals(other); + } + + void attachChild(const std::shared_ptr & child) override + { + impl.attachChild(child); + } + + void removeChild(ISchedulerNode * child) override + { + impl.removeChild(child); + } + + ISchedulerNode * getChild(const String & child_name) override + { + return impl.getChild(child_name); + } + + void activateChild(ISchedulerNode * child) override + { + impl.activateChild(child); + } + + bool isActive() override + { + return impl.isActive(); + } + + size_t activeChildren() override + { + return impl.activeChildren(); + } + std::pair dequeueRequest() override { - auto [request, active] = SemaphoreConstraint::dequeueRequest(); + auto [request, active] = impl.dequeueRequest(); if (request) { std::unique_lock lock(mutex); @@ -110,13 +151,17 @@ struct ConstraintTest : public SemaphoreConstraint void finishRequest(ResourceRequest * request) override { - { - std::unique_lock lock(mutex); - requests.erase(request); - } - SemaphoreConstraint::finishRequest(request); + impl.finishRequest(request); + std::unique_lock lock(mutex); + requests.erase(request); } + bool isSatisfied() override + { + return impl.isSatisfied(); + } + + SemaphoreConstraint impl; std::mutex mutex; std::set requests; }; From 912d59d2c89e1cd1f0f72280e2fc01c9c9d28255 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 13 Oct 2024 11:14:08 +0000 Subject: [PATCH 230/816] fix unittests --- .../Scheduler/Nodes/tests/ResourceTest.h | 76 ------------------- .../Nodes/tests/gtest_resource_scheduler.cpp | 21 ++--- 2 files changed, 11 insertions(+), 86 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 681453817c6..3fcbea55ee1 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -90,82 +90,6 @@ struct ResourceTestBase } }; - -struct ConstraintTest final : public ISchedulerConstraint -{ - explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : ISchedulerConstraint(event_queue_, config, config_prefix) - , impl(event_queue_, config, config_prefix) - {} - - const String & getTypeName() const override - { - return impl.getTypeName(); - } - - bool equals(ISchedulerNode * other) override - { - return impl.equals(other); - } - - void attachChild(const std::shared_ptr & child) override - { - impl.attachChild(child); - } - - void removeChild(ISchedulerNode * child) override - { - impl.removeChild(child); - } - - ISchedulerNode * getChild(const String & child_name) override - { - return impl.getChild(child_name); - } - - void activateChild(ISchedulerNode * child) override - { - impl.activateChild(child); - } - - bool isActive() override - { - return impl.isActive(); - } - - size_t activeChildren() override - { - return impl.activeChildren(); - } - - std::pair dequeueRequest() override - { - auto [request, active] = impl.dequeueRequest(); - if (request) - { - std::unique_lock lock(mutex); - requests.insert(request); - } - return {request, active}; - } - - void finishRequest(ResourceRequest * request) override - { - impl.finishRequest(request); - std::unique_lock lock(mutex); - requests.erase(request); - } - - bool isSatisfied() override - { - return impl.isSatisfied(); - } - - SemaphoreConstraint impl; - std::mutex mutex; - std::set requests; -}; - class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index 8eaa4ebb840..85d35fab0a6 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,5 +1,6 @@ #include +#include #include #include @@ -113,14 +114,14 @@ TEST(SchedulerRoot, Smoke) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); r1.registerResource(); ResourceHolder r2(t); - auto * fc2 = r2.add("/", "1"); + auto * fc2 = r2.add("/", "1"); r2.add("/prio"); auto c = r2.addQueue("/prio/C", "-1"); auto d = r2.addQueue("/prio/D", "-2"); @@ -128,25 +129,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } } @@ -156,7 +157,7 @@ TEST(SchedulerRoot, Budget) ResourceTest t; ResourceHolder r1(t); - r1.add("/", "1"); + r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", ""); r1.registerResource(); @@ -181,7 +182,7 @@ TEST(SchedulerRoot, Cancel) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); @@ -194,7 +195,7 @@ TEST(SchedulerRoot, Cancel) MyRequest request(1,[&] { sync.arrive_and_wait(); // (A) - EXPECT_TRUE(fc1->requests.contains(&request)); + EXPECT_TRUE(fc1->getInflights().first == 1); sync.arrive_and_wait(); // (B) request.finish(); destruct_sync.arrive_and_wait(); // (C) @@ -219,5 +220,5 @@ TEST(SchedulerRoot, Cancel) consumer1.join(); consumer2.join(); - EXPECT_TRUE(fc1->requests.empty()); + EXPECT_TRUE(fc1->getInflights().first == 0); } From bec2db7b79c8b2d7343be7d16d7eac0d732cee20 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 14 Oct 2024 21:17:01 +0000 Subject: [PATCH 231/816] implement workload entity storage based on keeper --- .../Nodes/tests/gtest_io_resource_manager.cpp | 8 +- .../Workload/WorkloadEntityDiskStorage.cpp | 12 +- .../Workload/WorkloadEntityDiskStorage.h | 4 +- .../Workload/WorkloadEntityKeeperStorage.cpp | 274 ++++++++ .../Workload/WorkloadEntityKeeperStorage.h | 69 ++ .../Workload/WorkloadEntityStorageBase.cpp | 624 ++++++++++++------ .../Workload/WorkloadEntityStorageBase.h | 40 +- .../Workload/createWorkloadEntityStorage.cpp | 13 +- src/Parsers/ASTCreateResourceQuery.h | 3 + src/Parsers/ParserCreateWorkloadEntity.cpp | 16 + src/Parsers/ParserCreateWorkloadEntity.h | 17 + .../configs/storage_configuration.xml | 1 + tests/integration/test_scheduler/test.py | 1 + 13 files changed, 852 insertions(+), 230 deletions(-) create mode 100644 src/Parsers/ParserCreateWorkloadEntity.cpp create mode 100644 src/Parsers/ParserCreateWorkloadEntity.h diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 93c8439bdae..15cd6436c47 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -112,7 +112,7 @@ public: } private: - bool storeEntityImpl( + WorkloadEntityStorageBase::OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -122,17 +122,17 @@ private: const Settings & settings) override { UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); - return true; + return OperationResult::Ok; } - bool removeEntityImpl( + WorkloadEntityStorageBase::OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, bool throw_if_not_exists) override { UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); - return true; + return OperationResult::Ok; } }; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 190b2928fe0..0e67074c84b 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -198,7 +198,7 @@ void WorkloadEntityDiskStorage::createDirectory() } -bool WorkloadEntityDiskStorage::storeEntityImpl( +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::storeEntityImpl( const ContextPtr & /*current_context*/, WorkloadEntityType entity_type, const String & entity_name, @@ -216,7 +216,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( if (throw_if_exists) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) - return false; + return OperationResult::Failed; } WriteBufferFromOwnString create_statement_buf; @@ -247,11 +247,11 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( } LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); - return true; + return OperationResult::Ok; } -bool WorkloadEntityDiskStorage::removeEntityImpl( +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::removeEntityImpl( const ContextPtr & /*current_context*/, WorkloadEntityType entity_type, const String & entity_name, @@ -267,11 +267,11 @@ bool WorkloadEntityDiskStorage::removeEntityImpl( if (throw_if_not_exists) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else - return false; + return OperationResult::Failed; } LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); - return true; + return OperationResult::Ok; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index ceb736372ae..b60a5075a02 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -16,7 +16,7 @@ public: void loadEntities() override; private: - bool storeEntityImpl( + OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -25,7 +25,7 @@ private: bool replace_if_exists, const Settings & settings) override; - bool removeEntityImpl( + OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index e69de29bb2d..37d1cc568ec 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -0,0 +1,274 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace Setting +{ +extern const SettingsUInt64 max_parser_backtracks; +extern const SettingsUInt64 max_parser_depth; +} + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; +} + +WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( + const ContextPtr & global_context_, const String & zookeeper_path_) + : WorkloadEntityStorageBase(global_context_) + , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} + , zookeeper_path{zookeeper_path_} + , watch_queue{std::make_shared>(std::numeric_limits::max())} + , log{getLogger("WorkloadEntityKeeperStorage")} +{ + if (zookeeper_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); + + if (zookeeper_path.back() == '/') + zookeeper_path.resize(zookeeper_path.size() - 1); + + /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. + if (zookeeper_path.front() != '/') + zookeeper_path = "/" + zookeeper_path; +} + +WorkloadEntityKeeperStorage::~WorkloadEntityKeeperStorage() +{ + SCOPE_EXIT_SAFE(stopWatchingThread()); +} + +void WorkloadEntityKeeperStorage::startWatchingThread() +{ + if (!watching_flag.exchange(true)) + watching_thread = ThreadFromGlobalPool(&WorkloadEntityKeeperStorage::processWatchQueue, this); +} + +void WorkloadEntityKeeperStorage::stopWatchingThread() +{ + if (watching_flag.exchange(false)) + { + watch_queue->finish(); + if (watching_thread.joinable()) + watching_thread.join(); + } +} + +zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() +{ + auto [zookeeper, session_status] = zookeeper_getter.getZooKeeper(); + + if (session_status == zkutil::ZooKeeperCachingGetter::SessionStatus::New) + { + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + + createRootNodes(zookeeper); + refreshAllEntities(zookeeper); + } + + return zookeeper; +} + +void WorkloadEntityKeeperStorage::loadEntities() +{ + /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. + /// However the watching thread must be started anyway in case the connection will be established later. + if (!entities_loaded) + { + try + { + refreshAllEntities(getZooKeeper()); + startWatchingThread(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load workload entities"); + } + } + startWatchingThread(); +} + + +void WorkloadEntityKeeperStorage::processWatchQueue() +{ + LOG_DEBUG(log, "Started watching thread"); + setThreadName("WrkldEntWatch"); + + while (watching_flag) + { + try + { + /// Re-initialize ZooKeeper session if expired + getZooKeeper(); + + bool queued = false; + if (!watch_queue->tryPop(queued, /* timeout_ms: */ 10000)) + continue; + + refreshAllEntities(getZooKeeper()); + } + catch (...) + { + tryLogCurrentException(log, "Will try to restart watching thread after error"); + zookeeper_getter.resetCache(); + sleepForSeconds(5); + } + } + + LOG_DEBUG(log, "Stopped watching thread"); +} + + +void WorkloadEntityKeeperStorage::stopWatching() +{ + stopWatchingThread(); +} + +void WorkloadEntityKeeperStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper) +{ + zookeeper->createAncestors(zookeeper_path); + // If node does not exist we consider it to be equal to empty node: no workload entities + zookeeper->createIfNotExists(zookeeper_path, ""); +} + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool /*throw_if_exists*/, + bool /*replace_if_exists*/, + const Settings &) +{ + LOG_DEBUG(log, "Storing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, create_entity_query}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshAllEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} stored", backQuote(entity_name)); + + return OperationResult::Ok; +} + + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool /*throw_if_not_exists*/) +{ + LOG_DEBUG(log, "Removing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, {}}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshAllEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} removed", backQuote(entity_name)); + + return OperationResult::Ok; +} + +std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) +{ + const auto data_watcher = [my_watch_queue = watch_queue](const Coordination::WatchResponse & response) + { + if (response.type == Coordination::Event::CHANGED) + { + [[maybe_unused]] bool inserted = my_watch_queue->emplace(true); + /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). + } + }; + + Coordination::Stat stat; + String data; + bool exists = zookeeper->tryGetWatch(zookeeper_path, data, &stat, data_watcher); + if (!exists) + { + createRootNodes(zookeeper); + data = zookeeper->getWatch(zookeeper_path, &stat, data_watcher); + } + return {data, stat.version}; +} + +void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper) +{ + /// It doesn't make sense to keep the old watch events because we will reread everything in this function. + watch_queue->clear(); + + refreshEntities(zookeeper); + entities_loaded = true; +} + +void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) +{ + LOG_DEBUG(log, "Refreshing workload entities"); + auto [data, version] = getDataAndSetWatch(zookeeper); + + ASTs queries; + ParserCreateWorkloadEntity parser; + const char * begin = data.data(); /// begin of current query + const char * pos = begin; /// parser moves pos from begin to the end of current query + const char * end = begin + data.size(); + while (pos < end) + { + queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS)); + while (isWhitespaceASCII(*pos) || *pos == ';') + ++pos; + } + + /// Read & parse all SQL entities from data we just read from ZooKeeper + std::vector> new_entities; + for (const auto & query : queries) + { + if (auto * create_workload_query = query->as()) + new_entities.emplace_back(create_workload_query->getWorkloadName(), query); + else if (auto * create_resource_query = query->as()) + new_entities.emplace_back(create_resource_query->getResourceName(), query); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity query in keeper storage: {}", query->getID()); + } + + setAllEntities(new_entities); + current_version = version; + + LOG_DEBUG(log, "Workload entities refreshing is done"); +} + +} + diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 6f70f09beec..523be850d8d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -1 +1,70 @@ #pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Loads RESOURCE and WORKLOAD sql objects from Keeper. +class WorkloadEntityKeeperStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityKeeperStorage(const ContextPtr & global_context_, const String & zookeeper_path_); + ~WorkloadEntityKeeperStorage() override; + + bool isReplicated() const override { return true; } + String getReplicationID() const override { return zookeeper_path; } + + void loadEntities() override; + void stopWatching() override; + +private: + OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void processWatchQueue(); + + zkutil::ZooKeeperPtr getZooKeeper(); + + void startWatchingThread(); + void stopWatchingThread(); + + void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); + + std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); + + void refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper); // TODO(serxa): get rid of it + void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); + + zkutil::ZooKeeperCachingGetter zookeeper_getter; + String zookeeper_path; + Int32 current_version = 0; + + ThreadFromGlobalPool watching_thread; + std::atomic entities_loaded = false; + std::atomic watching_flag = false; + + std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue + + LoggerPtr log; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 060bbbd6f87..0cd872f4890 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include @@ -13,7 +15,6 @@ #include #include - namespace DB { @@ -26,6 +27,7 @@ namespace ErrorCodes namespace { +/// Removes details from a CREATE query to be used as workload entity definition ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) { auto ptr = create_query.clone(); @@ -42,6 +44,7 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) return ptr; } +/// Returns a type of a workload entity `ptr` WorkloadEntityType getEntityType(const ASTPtr & ptr) { if (auto * res = typeid_cast(ptr.get())) @@ -52,12 +55,38 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +bool entityEquals(const ASTPtr & lhs, const ASTPtr & rhs) +{ + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + { + return std::forward_as_tuple(a->getWorkloadName(), a->getWorkloadParent(), a->changes) + == std::forward_as_tuple(b->getWorkloadName(), b->getWorkloadParent(), b->changes); + } + } + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + return std::forward_as_tuple(a->getResourceName(), a->operations) + == std::forward_as_tuple(b->getResourceName(), b->operations); + } + return false; +} + +/// Workload entities could reference each other. +/// This enum defines all possible reference types enum class ReferenceType { - Parent, ForResource + Parent, // Source workload references target workload as a parent + ForResource // Source workload references target resource in its `SETTINGS x = y FOR resource` clause }; -void forEachReference(const ASTPtr & source_entity, std::function func) +/// Runs a `func` callback for every reference from `source` to `target`. +/// This function is the source of truth defining what `target` references are stored in a workload `source_entity` +void forEachReference( + const ASTPtr & source_entity, + std::function func) { if (auto * res = typeid_cast(source_entity.get())) { @@ -82,6 +111,7 @@ void forEachReference(const ASTPtr & source_entity, std::function & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) { if (visited.contains(name)) @@ -101,6 +131,7 @@ void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, c sorted_workloads.emplace_back(name, ast); } +/// Returns pairs {worload_name, create_workload_ast} in order that respect child-parent relation (parent first, then children) std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) { std::vector> sorted_workloads; @@ -110,6 +141,143 @@ std::vector> topologicallySortedWorkloads(const std::u return sorted_workloads; } +/// Helper for recursive DFS +void topologicallySortedDependenciesImpl( + const String & name, + const std::unordered_map> & dependencies, + std::unordered_set & visited, + std::vector & result) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + if (auto it = dependencies.find(name); it != dependencies.end()) + { + for (const String & dep : it->second) + topologicallySortedDependenciesImpl(dep, dependencies, visited, result); + } + + result.emplace_back(name); +} + +/// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) +std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) { + std::unordered_set visited; // Set to track visited nodes + std::vector result; // Result to store nodes in topologically sorted order + + // Perform DFS for each node in the graph + for (const auto & [name, _] : dependencies) + topologicallySortedDependenciesImpl(name, dependencies, visited, result); + + // Reverse the result to get the correct topological order + std::reverse(result.begin(), result.end()); + + return result; +} + +/// Represents a change of a workload entity (WORKLOAD or RESOURCE) +struct EntityChange +{ + String name; /// Name of entity + ASTPtr before; /// Entity before change (CREATE if not set) + ASTPtr after; /// Entity after change (DROP if not set) + + std::vector toEvents() const + { + if (!after) + return {{getEntityType(before), name, {}}}; + else if (!before) + return {{getEntityType(after), name, after}}; + else + { + auto type_before = getEntityType(before); + auto type_after = getEntityType(after); + // If type changed, we have to remove an old entity and add a new one + if (type_before != type_after) + return {{type_before, name, {}}, {type_after, name, after}}; + else + return {{type_after, name, after}}; + } + } +}; + +/// Returns `changes` ordered for execution. +/// Every intemediate state during execution will be consistent (i.e. all references will be valid) +/// NOTE: It does not validate changes, any problem will be detected during execution. +/// NOTE: There will be no error if valid order does not exist. +std::vector topologicallySortedChanges(const std::vector & changes) +{ + // Construct map from entity name into entity change + std::unordered_map change_by_name; + for (const auto & change : changes) + change_by_name[change.name] = &change; + + // Construct references maps (before changes and after changes) + std::unordered_map> old_sources; // Key is target. Value is set of names of source entities. + std::unordered_map> new_targets; // Key is source. Value is set of names of target entities. + for (const auto & change : changes) + { + if (change.before) + { + forEachReference(change.before, + [&] (const String & target, const String & source, ReferenceType) + { + old_sources[target].insert(source); + }); + } + if (change.after) + { + forEachReference(change.after, + [&] (const String & target, const String & source, ReferenceType) + { + new_targets[source].insert(target); + }); + } + } + + // There are consistency rules that regulate order in which changes must be applied (see below). + // Construct DAG of dependencies between changes. + std::unordered_map> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. + for (const auto & change : changes) + { + for (const auto & event : change.toEvents()) + { + if (!event.entity) // DROP + { + // Rule 1: Entity can only be removed after all existing references to it are removed as well. + for (const String & source : old_sources[event.name]) + { + if (change_by_name.contains(source)) + dependencies[event.name].insert(source); + } + } + else // CREATE || CREATE OR REPLACE + { + // Rule 2: Entity can only be created after all entities it references are created as well. + for (const String & target : new_targets[event.name]) + { + if (auto it = change_by_name.find(target); it != change_by_name.end()) + { + const EntityChange & target_change = *it->second; + // If target is creating, it should be created first. + // (But if target is updating, there is no dependency). + if (!target_change.before) + dependencies[event.name].insert(target); + } + } + } + } + } + + // Topological sort of changes to respect consistency rules + std::vector result; + for (const String & name : topologicallySortedDependencies(dependencies)) + result.push_back(*change_by_name[name]); + + return result; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -130,7 +298,7 @@ ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const return it->second; } -ASTPtr WorkloadEntityStorageBase::tryGet(const std::string & entity_name) const +ASTPtr WorkloadEntityStorageBase::tryGet(const String & entity_name) const { std::lock_guard lock(mutex); @@ -146,9 +314,9 @@ bool WorkloadEntityStorageBase::has(const String & entity_name) const return tryGet(entity_name) != nullptr; } -std::vector WorkloadEntityStorageBase::getAllEntityNames() const +std::vector WorkloadEntityStorageBase::getAllEntityNames() const { - std::vector entity_names; + std::vector entity_names; std::lock_guard lock(mutex); entity_names.reserve(entities.size()); @@ -159,9 +327,9 @@ std::vector WorkloadEntityStorageBase::getAllEntityNames() const return entity_names; } -std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const { - std::vector entity_names; + std::vector entity_names; std::lock_guard lock(mutex); for (const auto & [name, entity] : entities) @@ -195,110 +363,101 @@ bool WorkloadEntityStorageBase::storeEntity( auto * workload = typeid_cast(create_entity_query.get()); auto * resource = typeid_cast(create_entity_query.get()); - std::unique_lock lock{mutex}; - - ASTPtr old_entity; // entity to be REPLACED - if (auto it = entities.find(entity_name); it != entities.end()) + while (true) { - if (throw_if_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); - else if (!replace_if_exists) - return false; - else - old_entity = it->second; - } + std::unique_lock lock{mutex}; - // Validate CREATE OR REPLACE - if (old_entity) - { - auto * old_workload = typeid_cast(old_entity.get()); - auto * old_resource = typeid_cast(old_entity.get()); - if (workload && !old_workload) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); - if (resource && !old_resource) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); - if (workload && !old_workload->hasParent() && workload->hasParent()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); - } - - std::optional new_root_name; - - // Validate workload - if (workload) - { - if (!workload->hasParent()) + ASTPtr old_entity; // entity to be REPLACED + if (auto it = entities.find(entity_name); it != entities.end()) { - if (!root_name.empty() && root_name != workload->getWorkloadName()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); - new_root_name = workload->getWorkloadName(); + if (throw_if_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + else + old_entity = it->second; } - SchedulingSettings validator; - validator.updateFromChanges(workload->changes); - } - - forEachReference(create_entity_query, - [this, workload] (const String & target, const String & source, ReferenceType type) + // Validate CREATE OR REPLACE + if (old_entity) { - if (auto it = entities.find(target); it == entities.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + auto * old_workload = typeid_cast(old_entity.get()); + auto * old_resource = typeid_cast(old_entity.get()); + if (workload && !old_workload) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); + if (resource && !old_resource) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); + if (workload && !old_workload->hasParent() && workload->hasParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); + } - switch (type) + // Validate workload + if (workload) + { + if (!workload->hasParent()) { - case ReferenceType::Parent: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); - break; - } - case ReferenceType::ForResource: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - - // Validate that we could parse the settings for specific resource - SchedulingSettings validator; - validator.updateFromChanges(workload->changes, target); - break; - } + if (!root_name.empty() && root_name != workload->getWorkloadName()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); } - // Detect reference cycles. - // The only way to create a cycle is to add an edge that will be a part of a new cycle. - // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. - if (isIndirectlyReferenced(source, target)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); - }); + SchedulingSettings validator; + validator.updateFromChanges(workload->changes); + } - bool stored = storeEntityImpl( - current_context, - entity_type, - entity_name, - create_entity_query, - throw_if_exists, - replace_if_exists, - settings); + forEachReference(create_entity_query, + [this, workload] (const String & target, const String & source, ReferenceType type) + { + if (auto it = entities.find(target); it == entities.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); - if (stored) - { - if (new_root_name) - root_name = *new_root_name; + switch (type) + { + case ReferenceType::Parent: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); + break; + } + case ReferenceType::ForResource: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - // Remove references of a replaced entity (only for CREATE OR REPLACE) - removeReferences(old_entity); + // Validate that we could parse the settings for specific resource + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + break; + } + } - // Insert references of created entity - insertReferences(create_entity_query); + // Detect reference cycles. + // The only way to create a cycle is to add an edge that will be a part of a new cycle. + // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. + if (isIndirectlyReferenced(source, target)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); + }); - // Store in memory - entities[entity_name] = create_entity_query; + auto result = storeEntityImpl( + current_context, + entity_type, + entity_name, + create_entity_query, + throw_if_exists, + replace_if_exists, + settings); - // Process notifications - onEntityAdded(entity_type, entity_name, create_entity_query); - unlockAndNotify(lock); + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, create_entity_query}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; } - - return stored; } bool WorkloadEntityStorageBase::removeEntity( @@ -307,47 +466,44 @@ bool WorkloadEntityStorageBase::removeEntity( const String & entity_name, bool throw_if_not_exists) { - std::unique_lock lock(mutex); - auto it = entities.find(entity_name); - if (it == entities.end()) + while (true) { - if (throw_if_not_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); - else - return false; + std::unique_lock lock(mutex); + auto it = entities.find(entity_name); + if (it == entities.end()) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + if (auto reference_it = references.find(entity_name); reference_it != references.end()) + { + String names; + for (const String & name : reference_it->second) + names += " " + name; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); + } + + auto result = removeEntityImpl( + current_context, + entity_type, + entity_name, + throw_if_not_exists); + + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, {}}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; } - - if (auto reference_it = references.find(entity_name); reference_it != references.end()) - { - String names; - for (const String & name : reference_it->second) - names += " " + name; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); - } - - bool removed = removeEntityImpl( - current_context, - entity_type, - entity_name, - throw_if_not_exists); - - if (removed) - { - if (entity_name == root_name) - root_name.clear(); - - // Clean up references - removeReferences(it->second); - - // Remove from memory - entities.erase(it); - - // Process notifications - onEntityRemoved(entity_type, entity_name); - unlockAndNotify(lock); - } - - return removed; } scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) @@ -357,9 +513,7 @@ scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChange std::vector current_state; { std::unique_lock lock{mutex}; - chassert(queue.empty()); - makeEventsForAllEntities(lock); - current_state = std::move(queue); + current_state = orderEntities(entities); std::lock_guard lock2{handlers->mutex}; handlers->list.push_back(handler); @@ -377,41 +531,30 @@ scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChange return result; } -void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) +void WorkloadEntityStorageBase::unlockAndNotify( + std::unique_lock & lock, + std::vector tx) { - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); -} + if (tx.empty()) + return; -void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) -{ - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); -} - -void WorkloadEntityStorageBase::unlockAndNotify(std::unique_lock & mutex_lock) -{ - /// Only one thread can send notification at any time, that is why we need `mutex_lock` - if (!queue.empty()) + std::vector current_handlers; { - auto events = std::move(queue); + std::lock_guard handlers_lock{handlers->mutex}; + boost::range::copy(handlers->list, std::back_inserter(current_handlers)); + } - std::vector current_handlers; + lock.unlock(); + + for (const auto & handler : current_handlers) + { + try { - std::lock_guard handlers_lock{handlers->mutex}; - boost::range::copy(handlers->list, std::back_inserter(current_handlers)); + handler(tx); } - - mutex_lock.unlock(); - - for (const auto & handler : current_handlers) + catch (...) { - try - { - handler(events); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + tryLogCurrentException(__PRETTY_FUNCTION__); } } } @@ -421,52 +564,84 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } -void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) +void WorkloadEntityStorageBase::setAllEntities(const std::vector> & raw_new_entities) { - std::unordered_map normalized_entities; - for (const auto & [entity_name, create_query] : new_entities) - normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); - - // TODO(serxa): do validation and throw LOGICAL_ERROR if failed + std::unordered_map new_entities; + for (const auto & [entity_name, create_query] : raw_new_entities) + new_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); std::unique_lock lock(mutex); - chassert(entities.empty()); // TODO(serxa): keeper storage could do full refresh, so we should support it here - entities = std::move(normalized_entities); - for (const auto & [entity_name, entity] : entities) - insertReferences(entity); - // Quick check to avoid extra work + // Fill vector of `changes` based on difference between current `entities` and `new_entities` + std::vector changes; + for (const auto & [entity_name, entity] : entities) { - std::lock_guard lock2(handlers->mutex); - if (handlers->list.empty()) - return; + if (auto it = new_entities.find(entity_name); it != new_entities.end()) + { + if (!entityEquals(entity, it->second)) + changes.emplace_back(entity_name, entity, it->second); // Remove entities that are not present in `new_entities` + } + else + changes.emplace_back(entity_name, entity, ASTPtr{}); // Update entities that are present in both `new_entities` and `entities` + } + for (const auto & [entity_name, entity] : new_entities) + { + if (!entities.contains(entity_name)) + changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` } - makeEventsForAllEntities(lock); - unlockAndNotify(lock); + // Sort `changes` to respect consistency of references and apply them one by one. + std::vector tx; + for (const auto & change : topologicallySortedChanges(changes)) + { + for (const auto & event : change.toEvents()) + { + // TODO(serxa): do validation and throw LOGICAL_ERROR if failed + applyEvent(lock, event); + tx.push_back(event); + } + } + + // Notify subscribers + unlockAndNotify(lock, tx); } -void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock &) +void WorkloadEntityStorageBase::applyEvent( + std::unique_lock &, + const Event & event) { - std::unordered_map workloads; - std::unordered_map resources; - for (auto & [entity_name, ast] : entities) + if (event.entity) // CREATE || CREATE OR REPLACE { - if (typeid_cast(ast.get())) - workloads.emplace(entity_name, ast); - else if (typeid_cast(ast.get())) - resources.emplace(entity_name, ast); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + auto * workload = typeid_cast(event.entity.get()); + + // Validate workload + if (workload && !workload->hasParent()) + root_name = workload->getWorkloadName(); + + // Remove references of a replaced entity (only for CREATE OR REPLACE) + if (auto it = entities.find(event.name); it != entities.end()) + removeReferences(it->second); + + // Insert references of created entity + insertReferences(event.entity); + + // Store in memory + entities[event.name] = event.entity; } + else // DROP + { + auto it = entities.find(event.name); + chassert(it != entities.end()); - // Resources should be created first because workloads could reference them - for (auto & [entity_name, ast] : resources) - onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); + if (event.name == root_name) + root_name.clear(); - // Workloads should be created in an order such that children are created only after its parent is created - for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) - onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); + // Clean up references + removeReferences(it->second); + + // Remove from memory + entities.erase(it); + } } std::vector> WorkloadEntityStorageBase::getAllEntities() const @@ -528,4 +703,59 @@ void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) }); } +std::vector WorkloadEntityStorageBase::orderEntities( + const std::unordered_map & all_entities, + std::optional change) +{ + std::vector result; + + std::unordered_map workloads; + for (auto & [entity_name, ast] : all_entities) + { + if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this workload if it is removed or updated + workloads.emplace(entity_name, ast); + } + else if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this resource if it is removed or updated + // Resources should go first because workloads could reference them + result.emplace_back(WorkloadEntityType::Resource, entity_name, ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + } + + // Introduce new entity described by `change` + if (change && change->entity) + { + if (change->type == WorkloadEntityType::Workload) + workloads.emplace(change->name, change->entity); + else if (change->type == WorkloadEntityType::Resource) + result.emplace_back(WorkloadEntityType::Resource, change->name, change->entity); + } + + // Workloads should go in an order such that children are enlisted only after its parent + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + result.emplace_back(WorkloadEntityType::Workload, entity_name, ast); + + return result; +} + +String WorkloadEntityStorageBase::serializeAllEntities(std::optional change) +{ + std::unique_lock lock; + auto ordered_entities = orderEntities(entities, change); + WriteBufferFromOwnString buf; + for (const auto & event : ordered_entities) + { + formatAST(*event.entity, buf, false, true); + buf.write(";\n", 2); + } + return buf.str(); +} + } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index e1f43181a0c..905c80610c2 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -49,7 +49,14 @@ public: const OnChangedHandler & handler) override; protected: - virtual bool storeEntityImpl( + enum class OperationResult + { + Ok, + Failed, + Retry + }; + + virtual OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -58,7 +65,7 @@ protected: bool replace_if_exists, const Settings & settings) = 0; - virtual bool removeEntityImpl( + virtual OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -66,18 +73,21 @@ protected: std::unique_lock getLock() const; + /// Replace current `entities` with `new_entities` and notifies subscribers. + /// Note that subscribers will be notified with a sequence of events. + /// It is guaranteed that all itermediate states (between every pair of consecutive events) + /// will be consistent (all references between entities will be valid) void setAllEntities(const std::vector> & new_entities); - void makeEventsForAllEntities(std::unique_lock & lock); - /// Called by derived class after a new workload entity has been added. - void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); + /// Serialize `entities` stored in memory plus one optional `change` into multiline string + String serializeAllEntities(std::optional change = {}); - /// Called by derived class after an workload entity has been removed. - void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); +private: + /// Change state in memory + void applyEvent(std::unique_lock & lock, const Event & event); - /// Sends notifications to subscribers about changes in workload entities - /// (added with previous calls onEntityAdded(), onEntityRemoved()). - void unlockAndNotify(std::unique_lock & lock); + /// Notify subscribers about changes describe by vector of events `tx` + void unlockAndNotify(std::unique_lock & lock, std::vector tx); /// Return true iff `references` has a path from `source` to `target` bool isIndirectlyReferenced(const String & target, const String & source); @@ -88,6 +98,11 @@ protected: /// Removes references that are described by `entity` from `references` void removeReferences(const ASTPtr & entity); + /// Returns an ordered vector of `entities` + std::vector orderEntities( + const std::unordered_map & all_entitites, + std::optional change = {}); + struct Handlers { std::mutex mutex; @@ -96,15 +111,14 @@ protected: /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. std::shared_ptr handlers; - std::vector queue; - mutable std::recursive_mutex mutex; std::unordered_map entities; /// Maps entity name into CREATE entity query // Validation - std::unordered_map> references; /// Keep track of references between entities. Key is target. Values is set of sources + std::unordered_map> references; /// Keep track of references between entities. Key is target. Value is set of sources String root_name; /// current root workload name +protected: ContextPtr global_context; }; diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp index 8475fe21455..5dc1265e31d 100644 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -34,15 +34,12 @@ std::unique_ptr createWorkloadEntityStorage(const Contex zookeeper_path_key, disk_path_key); } - abort(); // TODO(serxa): create WorkloadEntityKeeperStorage object - //return std::make_unique(global_context, config.getString(zookeeper_path_key)); - } - else - { - String default_path = fs::path{global_context->getPath()} / "workload" / ""; - String path = config.getString(disk_path_key, default_path); - return std::make_unique(global_context, path); + return std::make_unique(global_context, config.getString(zookeeper_path_key)); } + + String default_path = fs::path{global_context->getPath()} / "workload" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); } } diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index b05176837bc..f1c762e5bcd 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -19,6 +19,9 @@ public: { AccessMode mode; String disk; + + friend bool operator ==(const Operation & lhs, const Operation & rhs) { return lhs.mode == rhs.mode && lhs.disk == rhs.disk; } + friend bool operator !=(const Operation & lhs, const Operation & rhs) { return !(lhs == rhs); } }; using Operations = std::vector; diff --git a/src/Parsers/ParserCreateWorkloadEntity.cpp b/src/Parsers/ParserCreateWorkloadEntity.cpp new file mode 100644 index 00000000000..013210a6d87 --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +bool ParserCreateWorkloadEntity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserCreateWorkloadQuery create_workload_p; + ParserCreateResourceQuery create_resource_p; + + return create_workload_p.parse(pos, node, expected) || create_resource_p.parse(pos, node, expected); +} + +} diff --git a/src/Parsers/ParserCreateWorkloadEntity.h b/src/Parsers/ParserCreateWorkloadEntity.h new file mode 100644 index 00000000000..1e7b78b3ccc --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +/// Special parser for the CREATE WORKLOAD and CREATE RESOURCE queries. +class ParserCreateWorkloadEntity : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE workload entity query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 16cdf4a5b15..9498044c836 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -1,4 +1,5 @@ + /clickhouse/workload/definitions.sql diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index b78376bffe2..40c5f7e11ed 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -24,6 +24,7 @@ node = cluster.add_instance( "configs/workloads.xml.default", ], with_minio=True, + with_zookeeper=True, ) From 05eb1ef42b92f4a6150d6bc2f4bb5764d39f9fe8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 15 Oct 2024 01:01:01 +0000 Subject: [PATCH 232/816] style --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 0cd872f4890..4d09d49c927 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -162,7 +162,8 @@ void topologicallySortedDependenciesImpl( } /// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) -std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) { +std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) +{ std::unordered_set visited; // Set to track visited nodes std::vector result; // Result to store nodes in topologically sorted order From 32d892c9db424b7df8616ca5d6459cc78ba0cfde Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 15 Oct 2024 15:06:57 +0000 Subject: [PATCH 233/816] tests fix --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- tests/integration/test_drop_replica/test.py | 19 ++++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b743095e6f6..3016f62f20d 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1010,7 +1010,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) { ReplicatedTableStatus status; storage_replicated->getStatus(status); - if (status.zookeeper_info.path == query.replica_zk_path) + if (status.replica_path == remote_replica_path) throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "There is a local table {}, which has the same table path in ZooKeeper. " "Please check the path in query. " diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index e0928c6ab08..b959e80fc19 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -141,11 +141,7 @@ def test_drop_replica(start_cluster): shard=1 ) ) - assert "There is a local table" in node_1_2.query_and_get_error( - "SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format( - shard=1 - ) - ) + assert "There is a local table" in node_1_1.query_and_get_error( "SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format( shard=1 @@ -221,11 +217,16 @@ def test_drop_replica(start_cluster): ) assert exists_replica_1_1 == None - node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'") - exists_replica_1_1 = check_exists( + node_1_2.query("DETACH TABLE test4.test_table") + node_1_1.query( + "SYSTEM DROP REPLICA 'node_1_2' FROM ZKPATH '/clickhouse/tables/test4/{shard}/replicated/test_table'".format( + shard=1 + ) + ) + exists_replica_1_2 = check_exists( zk, "/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format( - shard=1, replica="node_1_1" + shard=1, replica="node_1_2" ), ) - assert exists_replica_1_1 == None + assert exists_replica_1_2 == None From 42dd97b78c189ffdd1dcba13c5bc3ca84e1388f6 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 15 Oct 2024 15:23:47 +0000 Subject: [PATCH 234/816] Empty From 7d861825c50fd7edb0996fffdc53f731a758b7e4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 15 Oct 2024 19:14:55 +0200 Subject: [PATCH 235/816] Rewrite the logic completely --- src/Interpreters/Set.cpp | 114 +++++++++++++++++++++++++++++++-------- 1 file changed, 92 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index c0f14f3a4c7..fee5b6fcc1e 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -6,8 +6,7 @@ #include #include -#include -#include +#include #include #include @@ -280,19 +279,95 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } -ColumnPtr returnColumnOrFilter(const ColumnPtr & first, const ColumnPtr & second) +void Set::checkIsCreated() const { - ConstantFilterDescription second_const_descr(*second); - if (second_const_descr.always_true) - return nullptr; + if (!is_created.load()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); +} - if (second_const_descr.always_false) - return first; +ColumnPtr checkDateTimePrecision(const ColumnPtr & column_to_cast, const ColumnPtr & column_after_cast, const size_t vec_res_size) +{ + /// Handle nullable columns + const ColumnNullable * original_nullable_column = typeid_cast(column_to_cast.get()); + const IColumn * original_nested_column = original_nullable_column ? &original_nullable_column->getNestedColumn() : column_to_cast.get(); - FilterDescription filter_descr(*second); - if (!filter_descr.data) - return nullptr; - return first->filter(*filter_descr.data, 0); + const ColumnNullable * result_nullable_column = typeid_cast(column_after_cast.get()); + + /// Check if the original column is of ColumnDecimal type + const auto * original_decimal_column = typeid_cast *>(original_nested_column); + + if (!original_decimal_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ColumnDecimal for DateTime64"); + + /// Get the data array from the original column + const auto & original_data = original_decimal_column->getData(); + + /// Prepare the final column + MutableColumnPtr final_column = column_after_cast->cloneEmpty(); + final_column->reserve(vec_res_size); + + /// Handle null maps + const NullMap * original_null_map = original_nullable_column ? &original_nullable_column->getNullMapData() : nullptr; + const NullMap * result_null_map = result_nullable_column ? &result_nullable_column->getNullMapData() : nullptr; + + /// Create a combined null map if necessary + NullMap combined_null_map(vec_res_size, false); + if (original_null_map || result_null_map) + { + for (size_t row = 0; row < vec_res_size; ++row) + { + bool is_null = false; + if (original_null_map && (*original_null_map)[row]) + is_null = true; + if (result_null_map && (*result_null_map)[row]) + is_null = true; + combined_null_map[row] = is_null; + } + } + + /// Decide which value to use for each row + for (size_t row = 0; row < vec_res_size; ++row) + { + bool is_null = combined_null_map.empty() ? false : combined_null_map[row]; + + if (is_null) + final_column->insertDefault(); + else + { + Int64 value = original_data[row]; + + if (value % result_nullable_column->getInt(row) != 0) + { + /// Sub-second precision exists; use the original value + /// We need to convert the value to the data type of final_column + + if (isDateTime64(result_nullable_column->getNestedColumn().getDataType())) + { + final_column->insertData(reinterpret_cast(&value), 0); + } + else if (isUInt32(result_nullable_column->getNestedColumn().getDataType())) + { + final_column->insert(static_cast(value)); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported final column type"); + } + else + final_column->insertFrom(*column_after_cast, row); /// Didn't lost precision, don't do anything + } + } + + /// If the original column was nullable, make the final column nullable + if (original_nullable_column) + { + /// Create the null map column as MutableColumnPtr + auto null_map_column = ColumnUInt8::create(); + null_map_column->getData().swap(combined_null_map); + + /// Wrap the final column and null map into a ColumnNullable + final_column = ColumnNullable::create(std::move(final_column), std::move(null_map_column)); + } + return final_column; } ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const @@ -348,17 +423,12 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co result = castColumnAccurate(column_to_cast, data_types[i], cast_cache.get()); } - ColumnPtr col_to_emplace; /// If we cast DateTime64 column to other type, we lose its precision. if we have this case, we should not let this cast happen - if (isDateTime64(column_before_cast.column->getDataType())) - col_to_emplace = returnColumnOrFilter(column_before_cast.column, res->getPtr()); - else - col_to_emplace = result; + /// If the original column is DateTime64, check for sub-second precision + if (isDateTime64(column_to_cast.column->getDataType())) + result = checkDateTimePrecision(column_to_cast.column, result, vec_res.size()); - if (!col_to_emplace) - col_to_emplace = column_before_cast.column; - - materialized_columns.emplace_back() = col_to_emplace; - key_columns.emplace_back() = materialized_columns.back().get(); + materialized_columns.emplace_back(result); + key_columns.emplace_back(materialized_columns.back().get()); } /// We will check existence in Set only for keys whose components do not contain any NULL value. From 1cb938f761b28244f6eceb9a71f12bb59bbfee43 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 15 Oct 2024 19:56:16 +0200 Subject: [PATCH 236/816] Fix build --- src/Interpreters/Set.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index fee5b6fcc1e..8115251f78b 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -279,12 +279,6 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } -void Set::checkIsCreated() const -{ - if (!is_created.load()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); -} - ColumnPtr checkDateTimePrecision(const ColumnPtr & column_to_cast, const ColumnPtr & column_after_cast, const size_t vec_res_size) { /// Handle nullable columns From 1bd4be3df127fdc42e4df01dd3c3da938ce6d327 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 01:10:57 +0000 Subject: [PATCH 237/816] prepare for database upload --- tests/fuzz/runner.py | 44 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 39 insertions(+), 5 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 1d3829598c3..bc6d3864810 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -8,6 +8,7 @@ import signal import subprocess from pathlib import Path from time import sleep +from typing import List from botocore.exceptions import ClientError @@ -26,7 +27,7 @@ def process_fuzzer_output(output: str): pass -def process_error(error: str): +def process_error(error: str) -> list: ERROR = r"^==\d+==\s?ERROR: (\S+): (.*)" error_source = "" error_reason = "" @@ -52,6 +53,7 @@ def process_error(error: str): is_error = True report(error_source, error_reason, error_info, test_unit) + return error_info def kill_fuzzer(fuzzer: str): @@ -64,7 +66,7 @@ def kill_fuzzer(fuzzer: str): os.kill(pid, signal.SIGKILL) -def run_fuzzer(fuzzer: str, timeout: int): +def run_fuzzer(fuzzer: str, timeout: int) -> TestResult: s3 = S3Helper() logging.info("Running fuzzer %s...", fuzzer) @@ -142,8 +144,9 @@ def run_fuzzer(fuzzer: str, timeout: int): cmd_line += " < /dev/null" logging.info("...will execute: %s", cmd_line) - # subprocess.check_call(cmd_line, shell=True) + test_result = TestResult(fuzzer, "OK") + stopwatch = Stopwatch() try: result = subprocess.run( cmd_line, @@ -158,19 +161,36 @@ def run_fuzzer(fuzzer: str, timeout: int): except subprocess.CalledProcessError as e: # print("Command failed with error:", e) logging.info("Stderr output: %s", e.stderr) - process_error(e.stderr) + test_result = TestResult( + fuzzer, + "FAIL", + stopwatch.duration_seconds, + "", + "\n".join(process_error(e.stderr)), + ) except subprocess.TimeoutExpired as e: logging.info("Timeout for %s", cmd_line) kill_fuzzer(fuzzer) sleep(10) process_fuzzer_output(e.stderr) + test_result = TestResult( + fuzzer, + "Timeout", + stopwatch.duration_seconds, + "", + "", + ) else: process_fuzzer_output(result.stderr) + test_result.time = stopwatch.duration_seconds s3.upload_build_directory_to_s3( Path(new_corpus_dir), f"fuzzer/corpus/{fuzzer}", False ) + logging.info("test_result: %s", test_result) + return test_result + def main(): logging.basicConfig(level=logging.INFO) @@ -183,10 +203,17 @@ def main(): if match: timeout += int(match.group(2)) + test_results = [] + stopwatch = Stopwatch() with Path() as current: for fuzzer in current.iterdir(): if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): - run_fuzzer(fuzzer.name, timeout) + test_results.append(run_fuzzer(fuzzer.name, timeout)) + + prepared_results = prepare_tests_results_for_clickhouse(PRInfo(), test_results, "failure", stopwatch.duration_seconds, stopwatch.start_time_str, "", "libFuzzer") + # ch_helper = ClickHouseHelper() + # ch_helper.insert_events_into(db="default", table="checks", events=prepared_results) + logging.info("prepared_results: %s", prepared_results) if __name__ == "__main__": @@ -198,5 +225,12 @@ if __name__ == "__main__": S3_BUILDS_BUCKET, ) from s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module + from clickhouse_helper import ( # pylint: disable=import-error,no-name-in-module + ClickHouseHelper, + prepare_tests_results_for_clickhouse, + ) + from pr_info import PRInfo # pylint: disable=import-error,no-name-in-module + from stopwatch import Stopwatch # pylint: disable=import-error,no-name-in-module + from report import TestResult # pylint: disable=import-error,no-name-in-module main() From e590d036fed24a126a63c226d4ee6e01d7a66957 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 01:26:24 +0000 Subject: [PATCH 238/816] fix style --- tests/fuzz/runner.py | 23 +++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index bc6d3864810..313b38d2d86 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -8,7 +8,6 @@ import signal import subprocess from pathlib import Path from time import sleep -from typing import List from botocore.exceptions import ClientError @@ -210,7 +209,15 @@ def main(): if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): test_results.append(run_fuzzer(fuzzer.name, timeout)) - prepared_results = prepare_tests_results_for_clickhouse(PRInfo(), test_results, "failure", stopwatch.duration_seconds, stopwatch.start_time_str, "", "libFuzzer") + prepared_results = prepare_tests_results_for_clickhouse( + PRInfo(), + test_results, + "failure", + stopwatch.duration_seconds, + stopwatch.start_time_str, + "", + "libFuzzer", + ) # ch_helper = ClickHouseHelper() # ch_helper.insert_events_into(db="default", table="checks", events=prepared_results) logging.info("prepared_results: %s", prepared_results) @@ -221,16 +228,16 @@ if __name__ == "__main__": ACTIVE_DIR = path.dirname(path.abspath(__file__)) sys.path.append((Path(path.dirname(ACTIVE_DIR)) / "ci").as_posix()) - from env_helper import ( # pylint: disable=import-error,no-name-in-module - S3_BUILDS_BUCKET, - ) - from s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module - from clickhouse_helper import ( # pylint: disable=import-error,no-name-in-module + from clickhouse_helper import ( # pylint: disable=import-error,no-name-in-module,unused-import ClickHouseHelper, prepare_tests_results_for_clickhouse, ) + from env_helper import ( # pylint: disable=import-error,no-name-in-module + S3_BUILDS_BUCKET, + ) from pr_info import PRInfo # pylint: disable=import-error,no-name-in-module - from stopwatch import Stopwatch # pylint: disable=import-error,no-name-in-module from report import TestResult # pylint: disable=import-error,no-name-in-module + from s3_helper import S3Helper # pylint: disable=import-error,no-name-in-module + from stopwatch import Stopwatch # pylint: disable=import-error,no-name-in-module main() From 9c790785d63695e16773192c4cdad3ddd27f2a3e Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 02:15:04 +0000 Subject: [PATCH 239/816] fix --- tests/fuzz/runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 313b38d2d86..8dd510a8f6e 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -65,7 +65,7 @@ def kill_fuzzer(fuzzer: str): os.kill(pid, signal.SIGKILL) -def run_fuzzer(fuzzer: str, timeout: int) -> TestResult: +def run_fuzzer(fuzzer: str, timeout: int): s3 = S3Helper() logging.info("Running fuzzer %s...", fuzzer) From fbbac87299ed8a6cec447786eed5afb628c48b66 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 02:57:58 +0000 Subject: [PATCH 240/816] add requests --- docker/test/libfuzzer/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt index 74147513e76..fd19ad04d8f 100644 --- a/docker/test/libfuzzer/requirements.txt +++ b/docker/test/libfuzzer/requirements.txt @@ -26,3 +26,4 @@ wadllib==1.3.6 wheel==0.37.1 zipp==1.0.0 boto3 +requests From 7ed274559330501da9f3d570cc7460ec22926e79 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 03:57:15 +0000 Subject: [PATCH 241/816] add github --- docker/test/libfuzzer/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt index fd19ad04d8f..bebf26db0bf 100644 --- a/docker/test/libfuzzer/requirements.txt +++ b/docker/test/libfuzzer/requirements.txt @@ -27,3 +27,4 @@ wheel==0.37.1 zipp==1.0.0 boto3 requests +github From c1956d4458b9722371610047fda01cccc7278fbb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 04:49:53 +0000 Subject: [PATCH 242/816] add pygithub --- docker/test/libfuzzer/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt index bebf26db0bf..d73af2861e6 100644 --- a/docker/test/libfuzzer/requirements.txt +++ b/docker/test/libfuzzer/requirements.txt @@ -27,4 +27,4 @@ wheel==0.37.1 zipp==1.0.0 boto3 requests -github +pygithub From 9ebd2fc4dbd3c6407b9bfb1cc9ce9b0c4708cb0f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 05:42:19 +0000 Subject: [PATCH 243/816] add unidiff --- docker/test/libfuzzer/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt index d73af2861e6..3fd33058a6b 100644 --- a/docker/test/libfuzzer/requirements.txt +++ b/docker/test/libfuzzer/requirements.txt @@ -28,3 +28,4 @@ zipp==1.0.0 boto3 requests pygithub +unidiff From d1b3f364fb55404427c756dafe959b8d05b31c99 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 16 Oct 2024 08:59:11 +0000 Subject: [PATCH 244/816] Fix flaky check --- tests/integration/test_drop_replica/test.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index b959e80fc19..b70a0725039 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -8,6 +8,7 @@ def fill_nodes(nodes, shard): for node in nodes: node.query( """ + DROP DATABASE IF EXISTS test SYNC; CREATE DATABASE test; CREATE TABLE test.test_table(date Date, id UInt32) @@ -20,6 +21,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test1 SYNC; CREATE DATABASE test1; CREATE TABLE test1.test_table(date Date, id UInt32) @@ -32,6 +34,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test2 SYNC; CREATE DATABASE test2; CREATE TABLE test2.test_table(date Date, id UInt32) @@ -44,7 +47,8 @@ def fill_nodes(nodes, shard): node.query( """ - CREATE DATABASE test3; + DROP DATABASE IF EXISTS test3 SYNC; + CREATE DATABASE test3; CREATE TABLE test3.test_table(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) @@ -56,6 +60,7 @@ def fill_nodes(nodes, shard): node.query( """ + DROP DATABASE IF EXISTS test4 SYNC; CREATE DATABASE test4; CREATE TABLE test4.test_table(date Date, id UInt32) @@ -83,9 +88,6 @@ node_1_3 = cluster.add_instance( def start_cluster(): try: cluster.start() - - fill_nodes([node_1_1, node_1_2], 1) - yield cluster except Exception as ex: @@ -101,6 +103,8 @@ def check_exists(zk, path): def test_drop_replica(start_cluster): + fill_nodes([node_1_1, node_1_2], 1) + node_1_1.query( "INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)" ) @@ -230,3 +234,7 @@ def test_drop_replica(start_cluster): ), ) assert exists_replica_1_2 == None + + node_1_1.query("ATTACH DATABASE test") + for i in range(1, 5): + node_1_1.query("ATTACH DATABASE test{}".format(i)) From c552179b7f09ab750c7da9e0e3e06560f2cfeea9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 16 Oct 2024 13:14:44 +0200 Subject: [PATCH 245/816] Update Set.cpp --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 8115251f78b..04ae08e9b40 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -418,7 +418,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } /// If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && isDateTimeOrDateTime64(result->getDataType())) result = checkDateTimePrecision(column_to_cast.column, result, vec_res.size()); materialized_columns.emplace_back(result); From 7ccb4ccb5838cf2e4581aaa9dc16760c024c00ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 16 Oct 2024 14:00:22 +0000 Subject: [PATCH 246/816] simplier rollback of Dynamic --- src/Columns/ColumnDynamic.cpp | 52 ++++------------------ src/Columns/ColumnDynamic.h | 2 - src/Columns/ColumnObject.cpp | 4 +- src/Columns/tests/gtest_column_dynamic.cpp | 5 +-- 4 files changed, 12 insertions(+), 51 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index ee42c7eccfe..41a9096bc0c 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1024,32 +1024,6 @@ void ColumnDynamic::updateCheckpoint(ColumnCheckpoint & checkpoint) const checkpoint.size = size(); } - -DataTypePtr ColumnDynamic::popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n) -{ - const auto & type_variant = assert_cast(*info.variant_type); - - std::unordered_map discriminator_to_name; - std::unordered_map name_to_data_type; - - for (const auto & [name, discriminator] : info.variant_name_to_discriminator) - discriminator_to_name.emplace(discriminator, name); - - for (const auto & type : type_variant.getVariants()) - name_to_data_type.emplace(type->getName(), type); - - /// Remove last n variants according to global discriminators. - /// This code relies on invariant that new variants are always added to the end in ColumnVariant. - for (auto it = local_to_global_discriminators.rbegin(); it < local_to_global_discriminators.rbegin() + n; ++it) - discriminator_to_name.erase(*it); - - DataTypes new_variants; - for (const auto & [d, name] : discriminator_to_name) - new_variants.push_back(name_to_data_type.at(name)); - - return std::make_shared(std::move(new_variants)); -} - void ColumnDynamic::rollback(const ColumnCheckpoint & checkpoint) { const auto & nested = assert_cast(checkpoint).nested; @@ -1062,28 +1036,18 @@ void ColumnDynamic::rollback(const ColumnCheckpoint & checkpoint) return; } - auto new_subcolumns = variant_column_ptr->getVariants(); - auto new_discriminators_map = variant_column_ptr->getLocalToGlobalDiscriminatorsMapping(); - auto new_discriminators_column = variant_column_ptr->getLocalDiscriminatorsPtr(); - auto new_offses_column = variant_column_ptr->getOffsetsPtr(); - - /// Remove new variants that were added since last checkpoint. - auto new_variant_type = popBackVariants(variant_info, new_discriminators_map, variant_column_ptr->getNumVariants() - nested.size()); - createVariantInfo(new_variant_type); - variant_mappings_cache.clear(); - - new_subcolumns.resize(nested.size()); - new_discriminators_map.resize(nested.size()); - /// Manually rollback internals of Variant column - new_discriminators_column->assumeMutable()->popBack(new_discriminators_column->size() - checkpoint.size); - new_offses_column->assumeMutable()->popBack(new_offses_column->size() - checkpoint.size); + variant_column_ptr->getOffsets().resize_assume_reserved(checkpoint.size); + variant_column_ptr->getLocalDiscriminators().resize_assume_reserved(checkpoint.size); + auto & variants = variant_column_ptr->getVariants(); for (size_t i = 0; i < nested.size(); ++i) - new_subcolumns[i]->rollback(*nested[i]); + variants[i]->rollback(*nested[i]); - variant_column = ColumnVariant::create(new_discriminators_column, new_offses_column, Columns(new_subcolumns.begin(), new_subcolumns.end()), new_discriminators_map); - variant_column_ptr = assert_cast(variant_column.get()); + /// Keep the structure of variant as is but rollback + /// to 0 variants that are not in the checkpoint. + for (size_t i = nested.size(); i < variants.size(); ++i) + variants[i] = variants[i]->cloneEmpty(); } String ColumnDynamic::getTypeNameAt(size_t row_num) const diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 8d8097905bf..57a1545a832 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -453,8 +453,6 @@ private: void updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type); - static DataTypePtr popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n); - WrappedPtr variant_column; /// Store and use pointer to ColumnVariant to avoid virtual calls. /// ColumnDynamic is widely used inside ColumnObject for each path and diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 1c66a09e99c..18ba8ed36ee 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -32,7 +32,7 @@ const std::shared_ptr & getDynamicSerialization() struct ColumnObjectCheckpoint : public ColumnCheckpoint { - using CheckpointsMap = std::unordered_map; + using CheckpointsMap = std::unordered_map; ColumnObjectCheckpoint(size_t size_, CheckpointsMap typed_paths_, CheckpointsMap dynamic_paths_, ColumnCheckpointPtr shared_data_) : ColumnCheckpoint(size_) @@ -719,7 +719,7 @@ ColumnCheckpointPtr ColumnObject::getCheckpoint() const { auto get_checkpoints = [](const auto & columns) { - std::unordered_map checkpoints; + ColumnObjectCheckpoint::CheckpointsMap checkpoints; for (const auto & [name, column] : columns) checkpoints[name] = column->getCheckpoint(); diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index f956f60b378..9a435a97a07 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -940,10 +940,9 @@ TEST(ColumnDynamic, rollback) auto check_checkpoint = [&](const ColumnCheckpoint & cp, std::vector sizes) { const auto & nested = assert_cast(cp).nested; - ASSERT_EQ(nested.size(), sizes.size()); size_t num_rows = 0; - for (size_t i = 0; i < sizes.size(); ++i) + for (size_t i = 0; i < nested.size(); ++i) { ASSERT_EQ(nested[i]->size, sizes[i]); num_rows += sizes[i]; @@ -960,7 +959,7 @@ TEST(ColumnDynamic, rollback) column->insert(Field(42)); column->updateCheckpoint(*checkpoint); - checkpoints.emplace_back(checkpoint, std::vector{0, 1}); + checkpoints.emplace_back(checkpoint, std::vector{0, 1, 0}); column->insert(Field("str1")); column->rollback(*checkpoint); From a02367e1ed28ef517b0683e5c4904d08195da37c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 16 Oct 2024 15:16:47 +0100 Subject: [PATCH 247/816] fix --- src/Functions/IFunction.cpp | 26 ++++++++++--------- ..._number_of_arguments_for_dynamic.reference | 0 ..._check_number_of_arguments_for_dynamic.sql | 17 ++++++++++++ 3 files changed, 31 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.reference create mode 100644 tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 10a25cfe0d0..68d4f25f08d 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -1,27 +1,28 @@ #include #include -#include -#include -#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include #include #include #include -#include #include -#include -#include +#include +#include +#include #include "config.h" +#include +#include + #if USE_EMBEDDED_COMPILER # include #endif @@ -451,6 +452,7 @@ FunctionBasePtr IFunctionOverloadResolver::build(const ColumnsWithTypeAndName & /// Use FunctionBaseDynamicAdaptor if default implementation for Dynamic is enabled and we have Dynamic type in arguments. if (useDefaultImplementationForDynamic()) { + checkNumberOfArguments(arguments.size()); for (const auto & arg : arguments) { if (isDynamic(arg.type)) diff --git a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.reference b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql new file mode 100644 index 00000000000..86b74b22175 --- /dev/null +++ b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql @@ -0,0 +1,17 @@ +set allow_experimental_json_type=1; + +CREATE TABLE t +( + `a` JSON +) +ENGINE = MergeTree() +ORDER BY tuple(); + +insert into t values ('{"a":1}'), ('{"a":2.0}'); + +SELECT 1 +FROM +( + SELECT 1 AS c0 +) AS tx +FULL OUTER JOIN t AS t2 ON equals(t2.a.Float32); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 7981e99bee1c0f4a6f79ddcace1c53183c883d18 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 16 Oct 2024 14:18:19 +0000 Subject: [PATCH 248/816] use func-tester --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a34ef624ce3..7637c096474 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -523,7 +523,7 @@ class CI: run_by_labels=[Tags.libFuzzer], timeout=10800, run_command='libfuzzer_test_check.py "$CHECK_NAME"', - runner_type=Runners.STYLE_CHECKER, + runner_type=Runners.FUNC_TESTER, ), JobNames.DOCKER_SERVER: CommonJobConfigs.DOCKER_SERVER.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE, BuildNames.PACKAGE_AARCH64] From 8a854bd5ac225f2cbb1bef441b1d8c930fd1356f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:10:05 +0000 Subject: [PATCH 249/816] add test for keeper entity storage with partitioning --- tests/integration/test_scheduler/test.py | 121 +++++++++++++++++++++++ 1 file changed, 121 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 40c5f7e11ed..05f38d09245 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -10,6 +10,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) @@ -27,6 +28,20 @@ node = cluster.add_instance( with_zookeeper=True, ) +node2 = cluster.add_instance( + "node2", + stay_alive=True, + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], + with_minio=True, + with_zookeeper=True, +) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): @@ -809,3 +824,109 @@ def test_resource_read_and_write(): ) == "1\n" ) + + +def test_workload_entity_keeper_storage(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + + def check_consistency(): + checks = [ + "select name, create_query from system.workloads order by all", + "select name, create_query from system.resources order by all", + "select resource, path, type, weight, priority, max_requests, max_cost, max_speed, max_burst from system.scheduler where resource not in ['network_read', 'network_write'] order by all", + ] + attempts = 10 + value1 = "" + value2 = "" + error_query = "" + for attempt in range(attempts): + for query in checks: + value1 = node.query(query) + value2 = node2.query(query) + if value1 != value2: + error_query = query + break # error + else: + break # success + time.sleep(0.5) + else: + raise Exception( + f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" + ) + + + for iteration in range(3): + split_idx_1 = random.randint(1, len(queries) - 3) + split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(0, split_idx_1): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_1, split_idx_2): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_2, len(queries)): + node.query(queries[query_idx]) + + check_consistency() From 23f90fe778ee30b5928d3076becbdc64957b6424 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:12:17 +0000 Subject: [PATCH 250/816] fix bug in topological sorting of entity changes, more logs --- .../Workload/WorkloadEntityDiskStorage.cpp | 2 +- .../Workload/WorkloadEntityDiskStorage.h | 1 - .../Workload/WorkloadEntityKeeperStorage.cpp | 7 ++++-- .../Workload/WorkloadEntityKeeperStorage.h | 2 -- .../Workload/WorkloadEntityStorageBase.cpp | 25 +++++++++++++++---- .../Workload/WorkloadEntityStorageBase.h | 1 + 6 files changed, 27 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 0e67074c84b..209d6f06100 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -60,8 +60,8 @@ namespace WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) : WorkloadEntityStorageBase(global_context_) , dir_path{makeDirectoryPathCanonical(dir_path_)} - , log{getLogger("WorkloadEntityDiskStorage")} { + log = getLogger("WorkloadEntityDiskStorage"); } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index b60a5075a02..cb3fb600182 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -38,7 +38,6 @@ private: String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; String dir_path; - LoggerPtr log; std::atomic entities_loaded = false; }; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index 37d1cc568ec..4aa087e029d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -35,8 +35,8 @@ WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} , watch_queue{std::make_shared>(std::numeric_limits::max())} - , log{getLogger("WorkloadEntityKeeperStorage")} { + log = getLogger("WorkloadEntityKeeperStorage"); if (zookeeper_path.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); @@ -237,9 +237,11 @@ void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) { - LOG_DEBUG(log, "Refreshing workload entities"); auto [data, version] = getDataAndSetWatch(zookeeper); + if (version == current_version) + return; + LOG_DEBUG(log, "Refreshing workload entities from keeper"); ASTs queries; ParserCreateWorkloadEntity parser; const char * begin = data.data(); /// begin of current query @@ -256,6 +258,7 @@ void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & z std::vector> new_entities; for (const auto & query : queries) { + LOG_TRACE(log, "Read keeper entity definition: {}", serializeAST(*query)); if (auto * create_workload_query = query->as()) new_entities.emplace_back(create_workload_query->getWorkloadName(), query); else if (auto * create_resource_query = query->as()) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 523be850d8d..deda5ba909b 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -63,8 +63,6 @@ private: std::atomic watching_flag = false; std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue - - LoggerPtr log; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4d09d49c927..dd4f5365191 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -171,9 +172,6 @@ std::vector topologicallySortedDependencies(const std::unordered_map topologicallySortedChanges(const std::vector> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. for (const auto & change : changes) { + dependencies.emplace(change.name, std::unordered_set{}); // Make sure we create nodes that have no dependencies for (const auto & event : change.toEvents()) { if (!event.entity) // DROP @@ -284,6 +283,7 @@ std::vector topologicallySortedChanges(const std::vector()) , global_context(std::move(global_context_)) + , log{getLogger("WorkloadEntityStorage")} // could be overriden in derived class {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const @@ -580,15 +580,26 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vectorsecond)) - changes.emplace_back(entity_name, entity, it->second); // Remove entities that are not present in `new_entities` + { + changes.emplace_back(entity_name, entity, it->second); // Update entities that are present in both `new_entities` and `entities` + LOG_TRACE(log, "Entity {} was updated", entity_name); + } + else + LOG_TRACE(log, "Entity {} is the same", entity_name); } else - changes.emplace_back(entity_name, entity, ASTPtr{}); // Update entities that are present in both `new_entities` and `entities` + { + changes.emplace_back(entity_name, entity, ASTPtr{}); // Remove entities that are not present in `new_entities` + LOG_TRACE(log, "Entity {} was dropped", entity_name); + } } for (const auto & [entity_name, entity] : new_entities) { if (!entities.contains(entity_name)) + { changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` + LOG_TRACE(log, "Entity {} was created", entity_name); + } } // Sort `changes` to respect consistency of references and apply them one by one. @@ -613,6 +624,8 @@ void WorkloadEntityStorageBase::applyEvent( { if (event.entity) // CREATE || CREATE OR REPLACE { + LOG_DEBUG(log, "Create or replace entity: {}", serializeAST(*event.entity)); + auto * workload = typeid_cast(event.entity.get()); // Validate workload @@ -634,6 +647,8 @@ void WorkloadEntityStorageBase::applyEvent( auto it = entities.find(event.name); chassert(it != entities.end()); + LOG_DEBUG(log, "Drop entity: {}", event.name); + if (event.name == root_name) root_name.clear(); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 905c80610c2..f1ef4124e98 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -120,6 +120,7 @@ private: protected: ContextPtr global_context; + LoggerPtr log; }; } From 79a6225a9c2fa4da62cdf752b8553e240c6f2312 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:20:17 +0000 Subject: [PATCH 251/816] abort all requests in queue before destruction --- src/Common/Scheduler/Nodes/FifoQueue.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index ea8985e314f..9502fae1a45 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -36,7 +36,7 @@ public: ~FifoQueue() override { - chassert(requests.empty()); + purgeQueue(); } const String & getTypeName() const override diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 0f015dd22b6..6e681632f68 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -54,7 +54,6 @@ IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) IOResourceManager::Resource::~Resource() { - // TODO(serxa): destroy all workloads, purge all queue, abort all resource requests scheduler.stop(); } From 318215d766d5c787bae0a8f4d5047a3458fc835c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:29:25 +0000 Subject: [PATCH 252/816] log unexpected errors in IOResourceManager --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 16 ++++++---------- src/Common/Scheduler/Nodes/IOResourceManager.h | 3 +++ 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 6e681632f68..812a49ace60 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -245,6 +246,7 @@ String IOResourceManager::Workload::getParent() const IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) : storage(storage_) + , log{getLogger("IOResourceManager")} { subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) @@ -304,11 +306,8 @@ void IOResourceManager::deleteWorkload(const String & workload_name) // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent workloads.erase(workload_iter); } - else - { - // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + else // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete workload that doesn't exist: {}", workload_name); } void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) @@ -335,11 +334,8 @@ void IOResourceManager::deleteResource(const String & resource_name) { resources.erase(resource_iter); } - else - { - // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + else // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete resource that doesn't exist: {}", resource_name); } IOResourceManager::Classifier::~Classifier() diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 0bbd14c2ca9..d336f012cd1 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -273,6 +274,8 @@ private: mutable std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); std::unordered_map resources; // TSA_GUARDED_BY(mutex); + + LoggerPtr log; }; } From b0c8430a61401e2d9e20fe56ac0b54700fc6f939 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 16 Oct 2024 17:30:57 +0200 Subject: [PATCH 253/816] Update Set.cpp --- src/Interpreters/Set.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 83ed0420a58..0407e453b13 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -286,6 +286,7 @@ ColumnPtr checkDateTimePrecision(const ColumnPtr & column_to_cast, const ColumnP const IColumn * original_nested_column = original_nullable_column ? &original_nullable_column->getNestedColumn() : column_to_cast.get(); const ColumnNullable * result_nullable_column = typeid_cast(column_after_cast.get()); + const IColumn * result_nested_column = result_nullable_column ? &result_nullable_column->getNestedColumn() : column_after_cast.get(); /// Check if the original column is of ColumnDecimal type const auto * original_decimal_column = typeid_cast *>(original_nested_column); @@ -335,16 +336,24 @@ ColumnPtr checkDateTimePrecision(const ColumnPtr & column_to_cast, const ColumnP /// Sub-second precision exists; use the original value /// We need to convert the value to the data type of final_column - if (isDateTime64(result_nullable_column->getNestedColumn().getDataType())) + if (isDateTime64(result_nested_column->getDataType())) { final_column->insertData(reinterpret_cast(&value), 0); } - else if (isUInt32(result_nullable_column->getNestedColumn().getDataType())) + else if (isUInt32(result_nested_column->getDataType())) // DateTime { final_column->insert(static_cast(value)); } + else if (isInt32(result_nested_column->getDataType())) // Date32 + { + final_column->insert(static_cast(value)); + } + else if (isUInt16(result_nested_column->getDataType())) // Date + { + final_column->insert(static_cast(value)); + } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported final column type"); + return column_after_cast; } else final_column->insertFrom(*column_after_cast, row); /// Didn't lost precision, don't do anything From 1c5a8e0008c444a3f70a84e796e6bc99bfd30882 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:41:44 +0000 Subject: [PATCH 254/816] ignore constraints beyond limit supported by resource request --- src/Common/Scheduler/Nodes/SemaphoreConstraint.h | 12 +++++++----- src/Common/Scheduler/ResourceRequest.cpp | 9 +++------ src/Common/Scheduler/ResourceRequest.h | 4 ++-- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 2454c1ec5bf..e223100a646 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -88,12 +88,14 @@ public: if (!request) return {nullptr, false}; - request->addConstraint(this); - - // Update state on request arrival std::unique_lock lock(mutex); - requests++; - cost += request->cost; + if (request->addConstraint(this)) + { + // Update state on request arrival + requests++; + cost += request->cost; + } + child_active = child_now_active; if (!active()) busy_periods++; diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 91394108f5d..71731fde9ce 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -23,20 +23,17 @@ void ResourceRequest::finish() } } -void ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +bool ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) { for (auto & constraint : constraints) { if (!constraint) { constraint = new_constraint; - return; + return true; } } - // TODO(serxa): is it possible to validate it during enqueue of resource request to avoid LOGICAL_ERRORs in the scheduler thread? possible but will not cover case of moving queue with requests inside to invalid position - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Max number of simultaneous workload constraints exceeded ({}). Remove extra constraints before using this workload.", - ResourceMaxConstraints); + return false; } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index e633af15157..bb9bfbfc8fd 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -17,7 +17,6 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -// TODO(serxa): validate hierarchy to avoid too many constraints /// Max number of constraints for a request to pass though (depth of constraints chain) constexpr size_t ResourceMaxConstraints = 8; @@ -91,7 +90,8 @@ public: void finish(); /// Is called from the scheduler thread to fill `constraints` chain - void addConstraint(ISchedulerConstraint * new_constraint); + /// Returns `true` iff constraint was added successfully + bool addConstraint(ISchedulerConstraint * new_constraint); }; } From 06aba8741d18bb5a34eb3c0934a76ee264b55f07 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 16 Oct 2024 17:49:02 +0200 Subject: [PATCH 255/816] Update Set.cpp --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 0407e453b13..0e2da497720 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -427,7 +427,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } /// If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType()) && isDateTimeOrDateTime64(result->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType())) result = checkDateTimePrecision(column_to_cast.column, result, vec_res.size()); materialized_columns.emplace_back(result); From 882ddb132ec3c360c6f3249ccbf657e0134d5b84 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 16:22:14 +0000 Subject: [PATCH 256/816] simplify keeper entity storage --- .../Workload/WorkloadEntityKeeperStorage.cpp | 57 ++++++++----------- .../Workload/WorkloadEntityKeeperStorage.h | 14 +++-- 2 files changed, 33 insertions(+), 38 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index 4aa087e029d..5b1c5d78f86 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -34,7 +34,7 @@ WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( : WorkloadEntityStorageBase(global_context_) , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} - , watch_queue{std::make_shared>(std::numeric_limits::max())} + , watch{std::make_shared()} { log = getLogger("WorkloadEntityKeeperStorage"); if (zookeeper_path.empty()) @@ -63,7 +63,7 @@ void WorkloadEntityKeeperStorage::stopWatchingThread() { if (watching_flag.exchange(false)) { - watch_queue->finish(); + watch->cv.notify_one(); if (watching_thread.joinable()) watching_thread.join(); } @@ -80,7 +80,7 @@ zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() zookeeper->sync(zookeeper_path); createRootNodes(zookeeper); - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); } return zookeeper; @@ -90,17 +90,14 @@ void WorkloadEntityKeeperStorage::loadEntities() { /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. /// However the watching thread must be started anyway in case the connection will be established later. - if (!entities_loaded) + try { - try - { - refreshAllEntities(getZooKeeper()); - startWatchingThread(); - } - catch (...) - { - tryLogCurrentException(log, "Failed to load workload entities"); - } + refreshEntities(getZooKeeper()); + startWatchingThread(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load workload entities"); } startWatchingThread(); } @@ -111,6 +108,7 @@ void WorkloadEntityKeeperStorage::processWatchQueue() LOG_DEBUG(log, "Started watching thread"); setThreadName("WrkldEntWatch"); + UInt64 handled = 0; while (watching_flag) { try @@ -118,11 +116,14 @@ void WorkloadEntityKeeperStorage::processWatchQueue() /// Re-initialize ZooKeeper session if expired getZooKeeper(); - bool queued = false; - if (!watch_queue->tryPop(queued, /* timeout_ms: */ 10000)) - continue; + { + std::unique_lock lock{watch->mutex}; + if (!watch->cv.wait_for(lock, std::chrono::seconds(10), [&] { return !watching_flag || handled != watch->triggered; })) + continue; + handled = watch->triggered; + } - refreshAllEntities(getZooKeeper()); + refreshEntities(getZooKeeper()); } catch (...) { @@ -166,7 +167,7 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEnt auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); if (code != Coordination::Error::ZOK) { - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); return OperationResult::Retry; } @@ -193,7 +194,7 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEn auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); if (code != Coordination::Error::ZOK) { - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); return OperationResult::Retry; } @@ -206,12 +207,13 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEn std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) { - const auto data_watcher = [my_watch_queue = watch_queue](const Coordination::WatchResponse & response) + const auto data_watcher = [my_watch = watch](const Coordination::WatchResponse & response) { if (response.type == Coordination::Event::CHANGED) { - [[maybe_unused]] bool inserted = my_watch_queue->emplace(true); - /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). + std::unique_lock lock{my_watch->mutex}; + my_watch->triggered++; + my_watch->cv.notify_one(); } }; @@ -226,15 +228,6 @@ std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const z return {data, stat.version}; } -void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper) -{ - /// It doesn't make sense to keep the old watch events because we will reread everything in this function. - watch_queue->clear(); - - refreshEntities(zookeeper); - entities_loaded = true; -} - void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) { auto [data, version] = getDataAndSetWatch(zookeeper); @@ -254,7 +247,7 @@ void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & z ++pos; } - /// Read & parse all SQL entities from data we just read from ZooKeeper + /// Read and parse all SQL entities from data we just read from ZooKeeper std::vector> new_entities; for (const auto & query : queries) { diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index deda5ba909b..0b55f501423 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -3,10 +3,11 @@ #include #include #include -#include #include #include +#include +#include namespace DB { @@ -48,10 +49,7 @@ private: void stopWatchingThread(); void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); - std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); - - void refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper); // TODO(serxa): get rid of it void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); zkutil::ZooKeeperCachingGetter zookeeper_getter; @@ -59,10 +57,14 @@ private: Int32 current_version = 0; ThreadFromGlobalPool watching_thread; - std::atomic entities_loaded = false; std::atomic watching_flag = false; - std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue + struct WatchEvent { + std::mutex mutex; + std::condition_variable cv; + UInt64 triggered = 0; + }; + std::shared_ptr watch; }; } From 49eaf646298d6e41245ac4cee6c3e77f51228074 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 17:13:40 +0000 Subject: [PATCH 257/816] fix rare race between queue activation and destruction --- src/Common/Scheduler/ISchedulerNode.h | 30 +++++++++++++++---- .../Scheduler/Nodes/IOResourceManager.cpp | 3 -- src/Common/Scheduler/SchedulerRoot.h | 5 ---- 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index d13e0b02c53..5e1239de274 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -139,7 +139,7 @@ public: , info(info_) {} - virtual ~ISchedulerNode() = default; + virtual ~ISchedulerNode(); virtual const String & getTypeName() const = 0; @@ -187,10 +187,7 @@ public: } /// Attach to a parent (used by attachChild) - virtual void setParent(ISchedulerNode * parent_) - { - parent = parent_; - } + void setParent(ISchedulerNode * parent_); protected: /// Notify parents about the first pending request or constraint becoming satisfied. @@ -326,6 +323,15 @@ public: pending.notify_one(); } + /// Removes an activation from queue + void cancelActivation(ISchedulerNode * node) + { + std::unique_lock lock{mutex}; + if (node->is_linked()) + activations.erase(activations.iterator_to(*node)); + node->activation_event_id = 0; + } + /// Process single event if it exists /// Note that postponing constraint are ignored, use it to empty the queue including postponed events on shutdown /// Returns `true` iff event has been processed @@ -490,6 +496,20 @@ private: std::atomic manual_time{TimePoint()}; // for tests only }; +inline ISchedulerNode::~ISchedulerNode() +{ + // Make sure there is no dangling reference in activations queue + event_queue->cancelActivation(this); +} + +inline void ISchedulerNode::setParent(ISchedulerNode * parent_) +{ + parent = parent_; + // Avoid activation of a detached node + if (parent == nullptr) + event_queue->cancelActivation(this); +} + inline void ISchedulerNode::scheduleActivation() { if (likely(parent)) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 812a49ace60..80d3650b1b9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -187,9 +187,6 @@ void IOResourceManager::Resource::updateCurrentVersion() if (previous_version) { previous_version->newer_version = current_version; - // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. - // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems impossible) - // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 45e4309fc81..451f29f33f2 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -190,11 +190,6 @@ public: activate(Resource::get(child->info)); } - void setParent(ISchedulerNode *) override - { - abort(); // scheduler must be the root and this function should not be called - } - private: void activate(Resource * value) { From 710bab4ca9e840b9e7772871335d4f725c176166 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 16 Oct 2024 20:01:38 +0200 Subject: [PATCH 258/816] Update 03208_datetime_cast_losing_precision.sql --- .../0_stateless/03208_datetime_cast_losing_precision.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql index f58c6a2d6a1..74826d24e6d 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql @@ -1 +1 @@ -SELECT now64() IN (SELECT now()); +SELECT toDateTime64(1729101630001, 3) IN (SELECT toDateTime(1729101630)); From d0fe70dc1a2023499e651b6aaac10f441d57b71d Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 18:55:42 +0000 Subject: [PATCH 259/816] style --- src/Common/Scheduler/ResourceRequest.cpp | 5 ----- src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h | 3 ++- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 2 +- tests/integration/test_scheduler/test.py | 5 ++--- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 71731fde9ce..674c7650adf 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - void ResourceRequest::finish() { // Iterate over constraints in reverse order diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 0b55f501423..25dcd6d8c9a 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -59,7 +59,8 @@ private: ThreadFromGlobalPool watching_thread; std::atomic watching_flag = false; - struct WatchEvent { + struct WatchEvent + { std::mutex mutex; std::condition_variable cv; UInt64 triggered = 0; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index dd4f5365191..a42252b1b8e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -283,7 +283,7 @@ std::vector topologicallySortedChanges(const std::vector()) , global_context(std::move(global_context_)) - , log{getLogger("WorkloadEntityStorage")} // could be overriden in derived class + , log{getLogger("WorkloadEntityStorage")} // could be overridden in derived class {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 05f38d09245..3c755860bdb 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -896,16 +896,15 @@ def test_workload_entity_keeper_storage(): value2 = node2.query(query) if value1 != value2: error_query = query - break # error + break # error else: - break # success + break # success time.sleep(0.5) else: raise Exception( f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" ) - for iteration in range(3): split_idx_1 = random.randint(1, len(queries) - 3) split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) From 3a8bea0932a633f6daf032267b401c8cef94af45 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 20:29:52 +0000 Subject: [PATCH 260/816] add docs --- .../settings.md | 30 +++++++++++- docs/en/operations/system-tables/resources.md | 37 +++++++++++++++ docs/en/operations/system-tables/workloads.md | 40 ++++++++++++++++ docs/en/operations/workload-scheduling.md | 47 +++++++++++++++++++ programs/server/config.xml | 2 +- 5 files changed, 154 insertions(+), 2 deletions(-) create mode 100644 docs/en/operations/system-tables/resources.md create mode 100644 docs/en/operations/system-tables/workloads.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 79407d46ce0..c032f1bac43 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3085,7 +3085,7 @@ By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests ove ### no_proxy By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set. -It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. +It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does. Example: @@ -3151,6 +3151,34 @@ Default value: "default" **See Also** - [Workload Scheduling](/docs/en/operations/workload-scheduling.md) +## workload_path {#workload_path} + +The directory used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. By default `/workload/` folder under server working directory is used. + +**Example** + +``` xml +/var/lib/clickhouse/workload/ +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_zookeeper_path](#workload_zookeeper_path) + +## workload_zookeeper_path {#workload_zookeeper_path} + +The path to a ZooKeeper node, which is used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. For consistency all SQL definitions are stored as a value of this single znode. By default ZooKeeper is not used and definitions are stored on [disk](#workload_path). + +**Example** + +``` xml +/clickhouse/workload/definitions.sql +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_path](#workload_path) + ## max_authentication_methods_per_user {#max_authentication_methods_per_user} The maximum number of authentication methods a user can be created with or altered to. diff --git a/docs/en/operations/system-tables/resources.md b/docs/en/operations/system-tables/resources.md new file mode 100644 index 00000000000..6329f05f610 --- /dev/null +++ b/docs/en/operations/system-tables/resources.md @@ -0,0 +1,37 @@ +--- +slug: /en/operations/system-tables/resources +--- +# resources + +Contains information for [resources](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every resource. + +Example: + +``` sql +SELECT * +FROM system.resources +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: io_read +read_disks: ['s3'] +write_disks: [] +create_query: CREATE RESOURCE io_read (READ DISK s3) + +Row 2: +────── +name: io_write +read_disks: [] +write_disks: ['s3'] +create_query: CREATE RESOURCE io_write (WRITE DISK s3) +``` + +Columns: + +- `name` (`String`) - Resource name. +- `read_disks` (`Array(String)`) - The array of disk names that uses this resource for read operations. +- `write_disks` (`Array(String)`) - The array of disk names that uses this resource for write operations. +- `create_query` (`String`) - The definition of the resource. diff --git a/docs/en/operations/system-tables/workloads.md b/docs/en/operations/system-tables/workloads.md new file mode 100644 index 00000000000..d9c62372044 --- /dev/null +++ b/docs/en/operations/system-tables/workloads.md @@ -0,0 +1,40 @@ +--- +slug: /en/operations/system-tables/workloads +--- +# workloads + +Contains information for [workloads](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every workload. + +Example: + +``` sql +SELECT * +FROM system.workloads +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: production +parent: all +create_query: CREATE WORKLOAD production IN `all` SETTINGS weight = 9 + +Row 2: +────── +name: development +parent: all +create_query: CREATE WORKLOAD development IN `all` + +Row 3: +────── +name: all +parent: +create_query: CREATE WORKLOAD `all` +``` + +Columns: + +- `name` (`String`) - Workload name. +- `parent` (`String`) - Parent workload name. +- `create_query` (`String`) - The definition of the workload. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 08629492ec6..7dc726d75f3 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -43,6 +43,14 @@ Example: ``` +Alternative way to express which disks are used by a resource is SQL syntax: + +```sql +CREATE RESOURCE (WRITE DISK disk1, READ DISK disk2) +``` + +Note that server configuration options have priority over SQL way to define resources. + ## Workload markup {#workload_markup} Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. @@ -153,9 +161,48 @@ Example: ``` +## Workload hierarchy (SQL only) {#workloads} + +Defining resources and classifiers in XML could be challenging. ClickHouse provides SQL syntax that is much more convenient. All resources that were created with `CREATE RESOURCE` share the same structure of the hierarchy, but could differ in some aspects. Every workload created with `CREATE WORKLOAD` maintain a few automatically created scheduling nodes for every resource. A child workload can be created inside another parent workload. Here is the example that defines exactly the same hierarchy as XML configuration above: + +```sql +CREATE RESOURCE network_write (WRITE DISK s3) +CREATE RESOURCE network_read (READ DISK s3) +CREATE WORKLOAD all SETTINGS max_requests = 100 +CREATE WORKLOAD development IN all +CREATE WORKLOAD production IN all SETTINGS weight = 3 +``` + +Name of a leaf workload without children could be used in query settings `SETTINGS workload = 'name'`. Note that workload classifiers are also created automatically when using SQL syntax. + +To customize workload the following settings could be used: +* `priority` - sibling workloads are served according to static priority values (lower value means higher priority). +* `weight` - sibling workloads having the same static priority share resources according to weights. +* `max_requests` - the limit on the number of concurrent resource requests in this workload. +* `max_cost` - the limit on the total inflight bytes count of concurrent resource requests in this workload. +* `max_speed` - the limit on byte processing rate of this workload (the limit is independent for every resource). +* `max_burst` - maximum number of bytes that could be processed by the workload without being throttled (for every resource independently). + +Note that workload settings are translated into proper set of scheduling nodes. For more details, see description of scheduling node [types and options](#hierarchy). + +There is no way to specify different hierarchy of workloads for different resources. But there is a way to specify differet workload setting value for a specific resource: + +```sql +CREATE OR REPLACE WORKLOAD all SETTINGS max_requests = 100, max_speed = 1000000 FOR network_read, max_speed = 2000000 FOR network_write +``` + +Also note that workload or resource could not be dropped if it is referenced from another workload. To update a definition of a workload use `CREATE OR REPLACE WORKLOAD` query. + +## Workloads and resources storage {#workload_entity_storage} +Definitions of all workload and resource in form of `CREATE WORKLOAD` and `CREATE RESOURCE` queries are stored persistently either on disk at `workload_path` or in ZooKeeper at `workload_zookeeper_path`. ZooKeeper storage is recommended to achieve consistency between nodes. Alternatively `ON CLUSTER` clause could be used along with a disk storage. + ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) + - [system.workloads](/docs/en/operations/system-tables/worklaods.md) + - [system.resources](/docs/en/operations/system-tables/resources.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting + - [workload_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_path) global server setting + - [workload_zookeeper_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_zookeeper_path) global server setting diff --git a/programs/server/config.xml b/programs/server/config.xml index b41f0344bb2..4e05e053d8a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1388,7 +1388,7 @@ - + + + + + GET + /test_dynamic_handler_auth_without_password + + dynamic_query_handler + without_password + + diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index a7804721f12..3c0ee3cd09a 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -33,5 +33,35 @@ INSERT INTO test_table(id, data) SELECT {id:UInt32}, {_request_body:String} + + + GET + /test_predefined_handler_auth_with_password + + predefined_query_handler + with_password + password + SELECT currentUser() + + + + GET + /test_predefined_handler_auth_with_password_fail + + predefined_query_handler + with_password + + SELECT currentUser() + + + + GET + /test_predefined_handler_auth_without_password + + predefined_query_handler + without_password + SELECT currentUser() + + diff --git a/tests/integration/test_http_handlers_config/users.d/users.yaml b/tests/integration/test_http_handlers_config/users.d/users.yaml new file mode 100644 index 00000000000..9ab8a84ae5a --- /dev/null +++ b/tests/integration/test_http_handlers_config/users.d/users.yaml @@ -0,0 +1,7 @@ +users: + with_password: + profile: default + password: password + without_password: + profile: default + no_password: 1 From 1b6979c5cd80666ba6c5164dae23c54b762a0d58 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 24 Oct 2024 15:28:57 +0000 Subject: [PATCH 406/816] Correct ifdefs --- .../DataLakes/DataLakeConfiguration.h | 23 ++++++++++++++++--- .../DataLakes/DeltaLakeMetadata.h | 6 +++++ .../ObjectStorage/DataLakes/HudiMetadata.cpp | 9 ++++---- 3 files changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 27599452a59..69968dff942 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -83,12 +83,29 @@ private: } }; +#if USE_AVRO +# if USE_AWS_S3 using StorageS3IcebergConfiguration = DataLakeConfiguration; +# endif + +# if USE_AZURE_BLOB_STORAGE using StorageAzureIcebergConfiguration = DataLakeConfiguration; -using StorageLocalIcebergConfiguration = DataLakeConfiguration; +# endif + +# if USE_HDFS using StorageHDFSIcebergConfiguration = DataLakeConfiguration; +# endif + +using StorageLocalIcebergConfiguration = DataLakeConfiguration; +#endif + +#if USE_PARQUET +# if USE_AWS_S3 using StorageS3DeltaLakeConfiguration = DataLakeConfiguration; +# endif +#endif + +#if USE_AWS_S3 using StorageS3HudiConfiguration = DataLakeConfiguration; - - +#endif } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index caa637cec75..031d1fb9e96 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -1,5 +1,9 @@ #pragma once +#include "config.h" + +#if USE_PARQUET + #include #include #include @@ -46,3 +50,5 @@ private: }; } + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index 40730f6d057..77ef769ed0e 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -1,11 +1,10 @@ -#include -#include #include -#include +#include +#include +#include #include #include -#include "config.h" -#include +#include namespace DB { From 52091f4ff81ea53ef3d73432ce02714a13f3a64d Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 24 Oct 2024 15:57:14 +0000 Subject: [PATCH 407/816] add docs and test --- .../settings.md | 18 ++++- .../configs/async_load_system_database.html | 3 + .../test_async_load_databases/test.py | 73 ++++++++++++++----- 3 files changed, 75 insertions(+), 19 deletions(-) create mode 100644 tests/integration/test_async_load_databases/configs/async_load_system_database.html diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b6238487725..b1d0de21046 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1975,6 +1975,22 @@ The default is `false`. true ``` +## async_load_system_database {#async_load_system_database} + +Asynchronous loading of system tables. Helpful if there is high amount of log tables and parts in system database. Independent of `async_load_databases` setting. + +If `true` all system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.asynchronous_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a system table, that is not yet loaded, will wait for exactly this table to be started up. The table that is waited for by at least one query will be loaded with higher priority. Also consider setting a limit `max_waiting_queries` for the total number of waiting queries. + +If `false`, system database loads before server start. + +The default is `false`. + +**Example** + +``` xml +true +``` + ## tables_loader_foreground_pool_size {#tables_loader_foreground_pool_size} Sets the number of threads performing load jobs in foreground pool. The foreground pool is used for loading table synchronously before server start listening on a port and for loading tables that are waited for. Foreground pool has higher priority than background pool. It means that no job starts in background pool while there are jobs running in foreground pool. @@ -3109,7 +3125,7 @@ By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests ove ### no_proxy By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set. -It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. +It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does. Example: diff --git a/tests/integration/test_async_load_databases/configs/async_load_system_database.html b/tests/integration/test_async_load_databases/configs/async_load_system_database.html new file mode 100644 index 00000000000..79823f5fbee --- /dev/null +++ b/tests/integration/test_async_load_databases/configs/async_load_system_database.html @@ -0,0 +1,3 @@ + + true + diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index 7fc6fd222d1..dd11067dfd4 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -1,4 +1,5 @@ import random +import time import pytest @@ -13,25 +14,35 @@ DICTIONARY_FILES = [ ] cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", +node1 = cluster.add_instance( + "node1", main_configs=["configs/config.xml"], dictionaries=DICTIONARY_FILES, stay_alive=True, ) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/async_load_system_database.xml", + ], + dictionaries=DICTIONARY_FILES, + stay_alive=True, +) + @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - instance.query( - """ - CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; - CREATE DATABASE IF NOT EXISTS test; - """ - ) + for node in [node1, node2]: + node.query( + """ + CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; + CREATE DATABASE IF NOT EXISTS test; + """ + ) yield cluster @@ -40,13 +51,13 @@ def started_cluster(): def get_status(dictionary_name): - return instance.query( + return node1.query( "SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'" ).rstrip("\n") def test_dict_get_data(started_cluster): - query = instance.query + query = node1.query query( "CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log;" @@ -80,7 +91,7 @@ def test_dict_get_data(started_cluster): # Wait for dictionaries to be reloaded. assert_eq_with_retry( - instance, + node1, "SELECT dictHas('dep_x', toUInt64(3))", "1", sleep_time=2, @@ -94,7 +105,7 @@ def test_dict_get_data(started_cluster): # so dep_x and dep_z are not going to be updated after the following INSERT. query("INSERT INTO test.elements VALUES (4, 'ether', 404, 0.001)") assert_eq_with_retry( - instance, + node1, "SELECT dictHas('dep_y', toUInt64(4))", "1", sleep_time=2, @@ -104,11 +115,11 @@ def test_dict_get_data(started_cluster): assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ZZ\n" query("DROP TABLE IF EXISTS test.elements;") - instance.restart_clickhouse() + node1.restart_clickhouse() def dependent_tables_assert(): - res = instance.query("select database || '.' || name from system.tables") + res = node1.query("select database || '.' || name from system.tables") assert "system.join" in res assert "default.src" in res assert "dict.dep_y" in res @@ -119,7 +130,7 @@ def dependent_tables_assert(): def test_dependent_tables(started_cluster): - query = instance.query + query = node1.query query("create database lazy engine=Lazy(10)") query("create database a") query("create table lazy.src (n int, m int) engine=Log") @@ -157,7 +168,7 @@ def test_dependent_tables(started_cluster): ) dependent_tables_assert() - instance.restart_clickhouse() + node1.restart_clickhouse() dependent_tables_assert() query("drop table a.t") query("drop table lazy.log") @@ -170,14 +181,14 @@ def test_dependent_tables(started_cluster): def test_multiple_tables(started_cluster): - query = instance.query + query = node1.query tables_count = 20 for i in range(tables_count): query( f"create table test.table_{i} (n UInt64, s String) engine=MergeTree order by n as select number, randomString(100) from numbers(100)" ) - instance.restart_clickhouse() + node1.restart_clickhouse() order = [i for i in range(tables_count)] random.shuffle(order) @@ -185,3 +196,29 @@ def test_multiple_tables(started_cluster): assert query(f"select count() from test.table_{i}") == "100\n" for i in range(tables_count): query(f"drop table test.table_{i} sync") + + +def test_async_load_system_database(started_cluster): + id = 1 + for i in range(4): + # Access some system tables that might be still loading + if id > 1: + for j in range(3): + node2.query(f"select count() from system.text_log_{random.randint(1, id - 1)}") + node2.query(f"select count() from system.query_log_{random.randint(1, id - 1)}") + + # Generate more system tables + for j in range(30): + while True: + count = int( + node2.query("select count() from system.tables where database = 'system' and name in ['query_log', 'text_log']") + ) + if count == 2: + break + time.sleep(0.1) + node2.query(f"rename table system.text_log to system.text_log_{id}") + node2.query(f"rename table system.query_log to system.query_log_{id}") + id += 1 + + # Trigger async load of system database + node2.restart_clickhouse() From 294ee94c4d7d5ec0c101cd1ff4cfbef2d4db5cb9 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 24 Oct 2024 15:59:18 +0000 Subject: [PATCH 408/816] fix --- ...c_load_system_database.html => async_load_system_database.xml} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/integration/test_async_load_databases/configs/{async_load_system_database.html => async_load_system_database.xml} (100%) diff --git a/tests/integration/test_async_load_databases/configs/async_load_system_database.html b/tests/integration/test_async_load_databases/configs/async_load_system_database.xml similarity index 100% rename from tests/integration/test_async_load_databases/configs/async_load_system_database.html rename to tests/integration/test_async_load_databases/configs/async_load_system_database.xml From 615976def8c2caaf8244bdc009a6792a7ea9f686 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 24 Oct 2024 16:05:37 +0000 Subject: [PATCH 409/816] Automatic style fix --- tests/integration/test_async_load_databases/test.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index dd11067dfd4..bda6a18d96d 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -204,14 +204,20 @@ def test_async_load_system_database(started_cluster): # Access some system tables that might be still loading if id > 1: for j in range(3): - node2.query(f"select count() from system.text_log_{random.randint(1, id - 1)}") - node2.query(f"select count() from system.query_log_{random.randint(1, id - 1)}") + node2.query( + f"select count() from system.text_log_{random.randint(1, id - 1)}" + ) + node2.query( + f"select count() from system.query_log_{random.randint(1, id - 1)}" + ) # Generate more system tables for j in range(30): while True: count = int( - node2.query("select count() from system.tables where database = 'system' and name in ['query_log', 'text_log']") + node2.query( + "select count() from system.tables where database = 'system' and name in ['query_log', 'text_log']" + ) ) if count == 2: break From 26d2a00eea1f4a6d5bf94d78065727da34f91a12 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 24 Oct 2024 17:12:21 +0000 Subject: [PATCH 410/816] fix build and test --- programs/server/Server.cpp | 3 ++- tests/integration/test_async_load_databases/test.py | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fd5ec9b2a5b..6944bacfbf2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -171,6 +171,7 @@ namespace ServerSetting extern const ServerSettingsBool async_insert_queue_flush_on_shutdown; extern const ServerSettingsUInt64 async_insert_threads; extern const ServerSettingsBool async_load_databases; + extern const ServerSettingsBool async_load_system_database; extern const ServerSettingsUInt64 background_buffer_flush_schedule_pool_size; extern const ServerSettingsUInt64 background_common_pool_size; extern const ServerSettingsUInt64 background_distributed_schedule_pool_size; @@ -2223,7 +2224,7 @@ try auto & database_catalog = DatabaseCatalog::instance(); /// We load temporary database first, because projections need it. database_catalog.initializeAndLoadTemporaryDatabase(); - load_system_metadata_tasks = loadMetadataSystem(global_context, server_settings.async_load_system_database); + load_system_metadata_tasks = loadMetadataSystem(global_context, server_settings[ServerSetting::async_load_system_database]); maybeConvertSystemDatabase(global_context, load_system_metadata_tasks); /// Startup scripts can depend on the system log tables. diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index dd11067dfd4..8c9ab607f07 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -208,8 +208,9 @@ def test_async_load_system_database(started_cluster): node2.query(f"select count() from system.query_log_{random.randint(1, id - 1)}") # Generate more system tables - for j in range(30): + for j in range(10): while True: + node2.query("system flush logs") count = int( node2.query("select count() from system.tables where database = 'system' and name in ['query_log', 'text_log']") ) From 4e9f9ac2f557b7233e09545c6b69dc66b0cf7de8 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 24 Oct 2024 17:32:18 +0000 Subject: [PATCH 411/816] more testing --- .../test_async_load_databases/test.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index 8c9ab607f07..e8311b60ce2 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -204,8 +204,16 @@ def test_async_load_system_database(started_cluster): # Access some system tables that might be still loading if id > 1: for j in range(3): - node2.query(f"select count() from system.text_log_{random.randint(1, id - 1)}") - node2.query(f"select count() from system.query_log_{random.randint(1, id - 1)}") + num = random.randint(1, id - 1) + node2.query(f"select count() from system.text_log_{num}_test") + node2.query(f"select count() from system.query_log_{num}_test") + + assert ( + int( + node2.query(f"select count() from system.asynchronous_loader where job ilike '%_log_%_test' and execution_pool = 'BackgroundLoad'") + ) + > 0 + ) # Generate more system tables for j in range(10): @@ -217,8 +225,8 @@ def test_async_load_system_database(started_cluster): if count == 2: break time.sleep(0.1) - node2.query(f"rename table system.text_log to system.text_log_{id}") - node2.query(f"rename table system.query_log to system.query_log_{id}") + node2.query(f"rename table system.text_log to system.text_log_{id}_test") + node2.query(f"rename table system.query_log to system.query_log_{id}_test") id += 1 # Trigger async load of system database From f648411c988587616d77ee3aacd47a39274b729a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 24 Oct 2024 17:44:17 +0000 Subject: [PATCH 412/816] Automatic style fix --- tests/integration/test_async_load_databases/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index 2bcfb896187..55ab5132798 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -213,7 +213,9 @@ def test_async_load_system_database(started_cluster): assert ( int( - node2.query(f"select count() from system.asynchronous_loader where job ilike '%_log_%_test' and execution_pool = 'BackgroundLoad'") + node2.query( + f"select count() from system.asynchronous_loader where job ilike '%_log_%_test' and execution_pool = 'BackgroundLoad'" + ) ) > 0 ) From 222e30a6301df71a78514ccf67ae9922cf8670eb Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 24 Oct 2024 17:45:51 +0000 Subject: [PATCH 413/816] fix --- tests/integration/test_async_load_databases/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index 2bcfb896187..c8da539cbc5 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -205,10 +205,10 @@ def test_async_load_system_database(started_cluster): if id > 1: for j in range(3): node2.query( - f"select count() from system.text_log_{random.randint(1, id - 1)}" + f"select count() from system.text_log_{random.randint(1, id - 1)}_test" ) node2.query( - f"select count() from system.query_log_{random.randint(1, id - 1)}" + f"select count() from system.query_log_{random.randint(1, id - 1)}_test" ) assert ( From ba02e7b3ddf8c1d77f248d9ae8f4001cb2bfc349 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Oct 2024 19:54:30 +0200 Subject: [PATCH 414/816] Fix bad test `01524_do_not_merge_across_partitions_select_final.sql` --- ...t_merge_across_partitions_select_final.reference | 1 - ..._do_not_merge_across_partitions_select_final.sql | 13 +------------ 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index 540137d4887..3e9f1c4c52c 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -6,4 +6,3 @@ 2020-01-01 00:00:00 2 1 499999 -18 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index 12700d9db53..9aeda582464 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -1,5 +1,4 @@ --- Tags: no-parallel, no-fasttest --- no-parallel: it checks the number of threads, which can be lowered in presence of other queries +-- Tags: no-fasttest DROP TABLE IF EXISTS select_final; @@ -32,17 +31,7 @@ SELECT max(x) FROM select_final FINAL where string = 'updated'; TRUNCATE TABLE select_final; INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(500000); - OPTIMIZE TABLE select_final FINAL; - -SET remote_filesystem_read_method = 'read'; -SET local_filesystem_read_method = 'pread'; -set load_marks_asynchronously = 0; - SELECT max(x) FROM select_final FINAL; -SYSTEM FLUSH LOGS; - -SELECT length(thread_ids) FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' AND current_database = currentDatabase() ORDER BY event_time DESC LIMIT 1; - DROP TABLE select_final; From 0eddccbcc543a9bf894b48b262b0d488ade73cb5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 24 Oct 2024 20:17:47 +0200 Subject: [PATCH 415/816] Resolve conflicts with master, support reset setting --- .../ObjectStorageQueueMetadata.cpp | 4 +- .../ObjectStorageQueueSettings.cpp | 18 ++- .../ObjectStorageQueueSettings.h | 5 + .../ObjectStorageQueueSource.cpp | 2 +- .../ObjectStorageQueueTableMetadata.cpp | 22 ++-- .../ObjectStorageQueueTableMetadata.h | 10 +- .../StorageObjectStorageQueue.cpp | 63 +++++---- .../StorageObjectStorageQueue.h | 4 - .../integration/test_storage_s3_queue/test.py | 124 +++++++++++++++++- 9 files changed, 200 insertions(+), 52 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 2389a960a81..525ca1e484b 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -292,12 +292,12 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) } new_table_metadata.tracked_files_limit = value; } - else if (endsWith(change.name, "tracked_files_ttl_sec")) + else if (endsWith(change.name, "tracked_file_ttl_sec")) { const auto value = change.value.safeGet(); if (table_metadata.tracked_files_ttl_sec == value) { - LOG_TRACE(log, "Setting `tracked_files_ttl_sec` already equals {}. " + LOG_TRACE(log, "Setting `tracked_file_ttl_sec` already equals {}. " "Will do nothing", value); return; } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp index d47e7b97404..338f575721a 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp @@ -23,15 +23,15 @@ namespace ErrorCodes 0) \ DECLARE(ObjectStorageQueueAction, after_processing, ObjectStorageQueueAction::KEEP, "Delete or keep file in after successful processing", 0) \ DECLARE(String, keeper_path, "", "Zookeeper node path", 0) \ - DECLARE(UInt32, loading_retries, 10, "Retry loading up to specified number of times", 0) \ - DECLARE(UInt32, processing_threads_num, 1, "Number of processing threads", 0) \ + DECLARE(UInt64, loading_retries, 10, "Retry loading up to specified number of times", 0) \ + DECLARE(UInt64, processing_threads_num, 1, "Number of processing threads", 0) \ DECLARE(UInt32, enable_logging_to_queue_log, 1, "Enable logging to system table system.(s3/azure_)queue_log", 0) \ DECLARE(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \ - DECLARE(UInt32, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ + DECLARE(UInt64, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ + DECLARE(UInt64, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ DECLARE(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ DECLARE(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ DECLARE(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \ - DECLARE(UInt32, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ DECLARE(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \ DECLARE(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \ DECLARE(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \ @@ -112,6 +112,11 @@ ObjectStorageQueueSettings::~ObjectStorageQueueSettings() = default; OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(ObjectStorageQueueSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR) +void ObjectStorageQueueSettings::applyChanges(const SettingsChanges & changes) +{ + impl->applyChanges(changes); +} + void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def) { if (storage_def.settings) @@ -156,4 +161,9 @@ void ObjectStorageQueueSettings::loadFromQuery(ASTStorage & storage_def) } } +Field ObjectStorageQueueSettings::get(const std::string & name) +{ + return impl->get(name); +} + } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h index c2929ac27fb..06bb78a95a2 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.h @@ -12,6 +12,7 @@ class ASTStorage; struct ObjectStorageQueueSettingsImpl; struct MutableColumnsAndConstraints; class StorageObjectStorageQueue; +class SettingsChanges; /// List of available types supported in ObjectStorageQueueSettings object #define OBJECT_STORAGE_QUEUE_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ @@ -61,6 +62,10 @@ struct ObjectStorageQueueSettings void loadFromQuery(ASTStorage & storage_def); + void applyChanges(const SettingsChanges & changes); + + Field get(const std::string & name); + private: std::unique_ptr impl; }; diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index c55287d2177..ba1a97bc2fb 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -657,7 +657,7 @@ void ObjectStorageQueueSource::commit(bool success, const std::string & exceptio void ObjectStorageQueueSource::applyActionAfterProcessing(const String & path) { - if (files_metadata->getTableMetadata().after_processing == "delete") + if (files_metadata->getTableMetadata().after_processing == ObjectStorageQueueAction::DELETE) { object_storage->removeObject(StoredObject(path)); } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index 81e7da82ebc..1c024fa09b8 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -17,11 +17,11 @@ namespace ObjectStorageQueueSetting extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing; extern const ObjectStorageQueueSettingsUInt32 buckets; extern const ObjectStorageQueueSettingsString last_processed_path; - extern const ObjectStorageQueueSettingsUInt32 loading_retries; extern const ObjectStorageQueueSettingsObjectStorageQueueMode mode; - extern const ObjectStorageQueueSettingsUInt32 processing_threads_num; - extern const ObjectStorageQueueSettingsUInt32 tracked_files_limit; - extern const ObjectStorageQueueSettingsUInt32 tracked_file_ttl_sec; + extern const ObjectStorageQueueSettingsUInt64 loading_retries; + extern const ObjectStorageQueueSettingsUInt64 processing_threads_num; + extern const ObjectStorageQueueSettingsUInt64 tracked_files_limit; + extern const ObjectStorageQueueSettingsUInt64 tracked_file_ttl_sec; } @@ -56,13 +56,13 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata( const std::string & format_) : format_name(format_) , columns(columns_.toString()) - , mode(engine_settings.mode.toString()) - , buckets(engine_settings.buckets) - , last_processed_path(engine_settings.last_processed_path) - , after_processing(engine_settings.after_processing) - , loading_retries(engine_settings.loading_retries) - , tracked_files_limit(engine_settings.tracked_files_limit) - , tracked_files_ttl_sec(engine_settings.tracked_file_ttl_sec) + , mode(engine_settings[ObjectStorageQueueSetting::mode].toString()) + , buckets(engine_settings[ObjectStorageQueueSetting::buckets]) + , last_processed_path(engine_settings[ObjectStorageQueueSetting::last_processed_path]) + , after_processing(engine_settings[ObjectStorageQueueSetting::after_processing]) + , loading_retries(engine_settings[ObjectStorageQueueSetting::loading_retries]) + , tracked_files_limit(engine_settings[ObjectStorageQueueSetting::tracked_files_limit]) + , tracked_files_ttl_sec(engine_settings[ObjectStorageQueueSetting::tracked_file_ttl_sec]) { processing_threads_num_changed = engine_settings[ObjectStorageQueueSetting::processing_threads_num].changed; if (!processing_threads_num_changed && engine_settings[ObjectStorageQueueSetting::processing_threads_num] <= 1) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h index fc1bfd1945c..3a07d4690fc 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h @@ -23,14 +23,14 @@ struct ObjectStorageQueueTableMetadata const String format_name; const String columns; const String mode; - const UInt64 buckets; + const UInt32 buckets; const String last_processed_path; /// Changeable settings. std::atomic after_processing; - std::atomic loading_retries; - std::atomic processing_threads_num; - std::atomic tracked_files_limit; - std::atomic tracked_files_ttl_sec; + std::atomic loading_retries; + std::atomic processing_threads_num; + std::atomic tracked_files_limit; + std::atomic tracked_files_ttl_sec; bool processing_threads_num_changed = false; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index efe5d5c8005..5124a4a7641 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -55,12 +55,12 @@ namespace ObjectStorageQueueSetting extern const ObjectStorageQueueSettingsUInt32 polling_min_timeout_ms; extern const ObjectStorageQueueSettingsUInt32 polling_max_timeout_ms; extern const ObjectStorageQueueSettingsUInt32 polling_backoff_ms; - extern const ObjectStorageQueueSettingsUInt32 processing_threads_num; + extern const ObjectStorageQueueSettingsUInt64 processing_threads_num; extern const ObjectStorageQueueSettingsUInt32 buckets; - extern const ObjectStorageQueueSettingsUInt32 tracked_file_ttl_sec; - extern const ObjectStorageQueueSettingsUInt32 tracked_files_limit; + extern const ObjectStorageQueueSettingsUInt64 tracked_file_ttl_sec; + extern const ObjectStorageQueueSettingsUInt64 tracked_files_limit; extern const ObjectStorageQueueSettingsString last_processed_path; - extern const ObjectStorageQueueSettingsUInt32 loading_retries; + extern const ObjectStorageQueueSettingsUInt64 loading_retries; extern const ObjectStorageQueueSettingsObjectStorageQueueAction after_processing; } @@ -356,10 +356,10 @@ void ReadFromObjectStorageQueue::initializePipeline(QueryPipelineBuilder & pipel { Pipes pipes; - size_t adjusted_num_stream = storage->getTableMetadata().processing_threads_num.load(); + size_t processing_threads_num = storage->getTableMetadata().processing_threads_num; createIterator(nullptr); - for (size_t i = 0; i < adjusted_num_streams; ++i) + for (size_t i = 0; i < processing_threads_num; ++i) pipes.emplace_back(storage->createSource( i/* processor_id */, info, @@ -490,12 +490,6 @@ bool StorageObjectStorageQueue::streamToViews() LOG_TEST(log, "Using {} processing threads", processing_threads_num); - size_t adjusted_num_streams; - { - std::lock_guard lock(changeable_settings_mutex); - adjusted_num_streams = queue_settings->processing_threads_num; - } - while (!shutdown_called && !file_iterator->isFinished()) { InterpreterInsertQuery interpreter( @@ -515,10 +509,10 @@ bool StorageObjectStorageQueue::streamToViews() Pipes pipes; std::vector> sources; - pipes.reserve(adjusted_num_streams); - sources.reserve(adjusted_num_streams); + pipes.reserve(processing_threads_num); + sources.reserve(processing_threads_num); - for (size_t i = 0; i < adjusted_num_streams; ++i) + for (size_t i = 0; i < processing_threads_num; ++i) { auto source = createSource( i/* processor_id */, @@ -534,7 +528,7 @@ bool StorageObjectStorageQueue::streamToViews() auto pipe = Pipe::unitePipes(std::move(pipes)); block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setNumThreads(adjusted_num_streams); + block_io.pipeline.setNumThreads(processing_threads_num); block_io.pipeline.setConcurrencyControl(queue_context->getSettingsRef()[Setting::use_concurrency_control]); std::atomic_size_t rows = 0; @@ -570,13 +564,13 @@ static const std::unordered_set changeable_settings_unordered_ "loading_retries", "after_processing", "tracked_files_limit", - "tracked_files_ttl_sec", + "tracked_file_ttl_sec", /// For compatibility. "s3queue_processing_threads_num", "s3queue_loading_retries", "s3queue_after_processing", "s3queue_tracked_files_limit", - "s3queue_tracked_files_ttl_sec", + "s3queue_tracked_file_ttl_sec", }; static const std::unordered_set changeable_settings_ordered_mode @@ -600,7 +594,7 @@ void StorageObjectStorageQueue::checkAlterIsPossible(const AlterCommands & comma { for (const auto & command : commands) { - if (command.type != AlterCommand::MODIFY_SETTING) + if (command.type != AlterCommand::MODIFY_SETTING && command.type != AlterCommand::RESET_SETTING) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only MODIFY SETTING alter is allowed for {}", getName()); } @@ -613,6 +607,7 @@ void StorageObjectStorageQueue::checkAlterIsPossible(const AlterCommands & comma const auto & new_changes = new_metadata.settings_changes->as().changes; const auto & old_changes = old_metadata.settings_changes->as().changes; + const auto mode = getTableMetadata().getMode(); for (const auto & changed_setting : new_changes) { auto it = std::find_if( @@ -623,12 +618,12 @@ void StorageObjectStorageQueue::checkAlterIsPossible(const AlterCommands & comma if (setting_changed) { - if (!isSettingChangeable(changed_setting.name, queue_settings->mode)) + if (!isSettingChangeable(changed_setting.name, mode)) { throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, "Changing setting {} is not allowed for {} mode of {}", - changed_setting.name, magic_enum::enum_name(queue_settings->mode.value), getName()); + changed_setting.name, magic_enum::enum_name(mode), getName()); } } } @@ -648,9 +643,24 @@ void StorageObjectStorageQueue::alter( StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); commands.apply(new_metadata, local_context); - const auto & new_settings = new_metadata.settings_changes->as().changes; + auto new_settings = new_metadata.settings_changes->as().changes; + + ObjectStorageQueueSettings default_settings; + for (const auto & setting : old_settings) + { + auto it = std::find_if( + new_settings.begin(), new_settings.end(), + [&](const SettingChange & change) { return change.name == setting.name; }); + + if (it == new_settings.end()) + { + /// Setting was reset. + new_settings.push_back(SettingChange(setting.name, default_settings.get(setting.name))); + } + } SettingsChanges changed_settings; + const auto mode = getTableMetadata().getMode(); for (const auto & setting : new_settings) { auto it = std::find_if( @@ -661,18 +671,23 @@ void StorageObjectStorageQueue::alter( if (!setting_changed) continue; - if (!isSettingChangeable(setting.name, queue_settings->mode)) + if (!isSettingChangeable(setting.name, mode)) { throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, "Changing setting {} is not allowed for {} mode of {}", - setting.name, magic_enum::enum_name(queue_settings->mode.value), getName()); + setting.name, magic_enum::enum_name(mode), getName()); } changed_settings.push_back(setting); } files_metadata->alterSettings(changed_settings); + + StorageInMemoryMetadata metadata = getInMemoryMetadata(); + metadata.setSettingsChanges(new_metadata.settings_changes); + setInMemoryMetadata(metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); } } diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h index b37ac7aa90f..08eb32928b3 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h @@ -70,10 +70,6 @@ private: ObjectStorageType type; const std::string engine_name; - - const std::unique_ptr queue_settings; - std::mutex changeable_settings_mutex; - const fs::path zk_path; const bool enable_logging_to_queue_log; const UInt32 polling_min_timeout_ms; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 3c5677e16d2..f8db75560d7 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1652,7 +1652,7 @@ def test_processed_file_setting(started_cluster, processing_threads): values_csv = ( "\n".join((",".join(map(str, row)) for row in correct_values)) + "\n" ).encode() - file_path = f"{files_path}/99.csv" + file_path = f"{files_path}/test_99.csv" put_s3_file_content(started_cluster, file_path, values_csv) expected_rows += 1 @@ -2118,3 +2118,125 @@ def test_processing_threads(started_cluster): assert node.contains_in_log( f"StorageS3Queue (default.{table_name}): Using 16 processing threads" ) + + +def test_alter_settings(started_cluster): + node1 = started_cluster.instances["node1"] + node2 = started_cluster.instances["node2"] + + table_name = f"test_alter_settings_{uuid.uuid4().hex[:8]}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + files_to_generate = 1000 + + node1.query("DROP DATABASE IF EXISTS r") + node2.query("DROP DATABASE IF EXISTS r") + + node1.query( + f"CREATE DATABASE r ENGINE=Replicated('/clickhouse/databases/{table_name}', 'shard1', 'node1')" + ) + node2.query( + f"CREATE DATABASE r ENGINE=Replicated('/clickhouse/databases/{table_name}', 'shard1', 'node2')" + ) + + create_table( + started_cluster, + node1, + table_name, + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + "processing_threads_num": 10, + "loading_retries": 20, + }, + database_name="r", + ) + + assert '"processing_threads_num":10' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"loading_retries":20' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"after_processing":"keep"' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + total_values = generate_random_files( + started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 + ) + + create_mv(node1, f"r.{table_name}", dst_table_name) + create_mv(node2, f"r.{table_name}", dst_table_name) + + def get_count(): + return int( + node1.query( + f"SELECT count() FROM clusterAllReplicas(cluster, default.{dst_table_name})" + ) + ) + + expected_rows = files_to_generate + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + assert expected_rows == get_count() + + node1.query( + f""" + ALTER TABLE r.{table_name} + MODIFY SETTING processing_threads_num=5, loading_retries=10, after_processing='delete', tracked_files_limit=50, tracked_file_ttl_sec=10000 + """ + ) + + assert '"processing_threads_num":5' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"loading_retries":10' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"after_processing":"delete"' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + node1.restart_clickhouse() + + assert '"processing_threads_num":5' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"loading_retries":10' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"after_processing":"delete"' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + node1.query( + f""" + ALTER TABLE r.{table_name} RESET SETTING after_processing + """ + ) + + assert '"processing_threads_num":5' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"loading_retries":10' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"after_processing":"keep"' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + node1.restart_clickhouse() + assert expected_rows == get_count() From 0f66694468768f4b5a7cbb5e9ca2b1e2f4aa3b02 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 24 Oct 2024 20:19:46 +0200 Subject: [PATCH 416/816] Remove change --- .../integration/test_storage_s3_queue/test.py | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index f8db75560d7..647a54ff95a 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1644,24 +1644,6 @@ def test_processed_file_setting(started_cluster, processing_threads): assert expected_rows == get_count() - node.restart_clickhouse() - - correct_values = [ - [1, 1, 1], - ] - values_csv = ( - "\n".join((",".join(map(str, row)) for row in correct_values)) + "\n" - ).encode() - file_path = f"{files_path}/test_99.csv" - put_s3_file_content(started_cluster, file_path, values_csv) - - expected_rows += 1 - for _ in range(20): - if expected_rows == get_count(): - break - time.sleep(1) - assert expected_rows == get_count() - @pytest.mark.parametrize("processing_threads", [1, 5]) def test_processed_file_setting_distributed(started_cluster, processing_threads): From 70e644ab5d10b2f6ee919989d8676ee5e378d986 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 21 Oct 2024 19:00:19 +0200 Subject: [PATCH 417/816] make numactl respect EPERM error, when get_mempolicy is is restricted by seccomp --- contrib/numactl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/numactl b/contrib/numactl index 8d13d63a05f..a1bebe8fe6f 160000 --- a/contrib/numactl +++ b/contrib/numactl @@ -1 +1 @@ -Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 +Subproject commit a1bebe8fe6f6efebb23168bc561d240f0f64ca4b From 87ec91bb103f0e7c0e2380f897bc1a4a0033e111 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 24 Oct 2024 20:58:12 +0200 Subject: [PATCH 418/816] Rewrite the code so we calculate the nullmap first. --- src/Interpreters/Set.cpp | 102 +++++++++++++++++++++------------------ 1 file changed, 56 insertions(+), 46 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 923789bafbb..42a92bc7809 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -279,22 +280,13 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } -ColumnPtr checkDateTimePrecision( - const ColumnPtr & column_to_cast, - const ColumnPtr & column_after_cast, - const size_t num_rows, - const bool transform_null_in) +ColumnPtr checkDateTimePrecision(const ColumnWithTypeAndName & column_to_cast) { // Handle nullable columns - const ColumnNullable * original_nullable_column = typeid_cast(column_to_cast.get()); + const ColumnNullable * original_nullable_column = typeid_cast(column_to_cast.column.get()); const IColumn * original_nested_column = original_nullable_column ? &original_nullable_column->getNestedColumn() - : column_to_cast.get(); - - const ColumnNullable * result_nullable_column = typeid_cast(column_after_cast.get()); - const IColumn * result_nested_column = result_nullable_column - ? &result_nullable_column->getNestedColumn() - : column_after_cast.get(); + : column_to_cast.column.get(); // Check if the original column is of ColumnDecimal type const auto * original_decimal_column = typeid_cast *>(original_nested_column); @@ -306,46 +298,49 @@ ColumnPtr checkDateTimePrecision( size_t vec_res_size = original_data.size(); // Prepare the precision null map - auto precision_null_map_column = ColumnUInt8::create(vec_res_size); + auto precision_null_map_column = ColumnUInt8::create(vec_res_size, 0); NullMap & precision_null_map = precision_null_map_column->getData(); // Determine which rows should be null based on precision loss - for (size_t row = 0; row < vec_res_size; ++row) + const auto * datetime64_type = assert_cast(column_to_cast.type.get()); + auto scale = datetime64_type->getScale(); + if (scale >= 1) { - Int64 value = original_data[row]; - Int64 result_value = result_nested_column->getInt(row); - - if (value % result_value != 0) - precision_null_map[row] = 1; // Mark as null due to precision loss - else - precision_null_map[row] = 0; // No precision loss + Int64 scale_multiplier = common::exp10_i32(scale); + for (size_t row = 0; row < vec_res_size; ++row) + { + Int64 value = original_data[row]; + if (value % scale_multiplier != 0) + precision_null_map[row] = 1; // Mark as null due to precision loss + else + precision_null_map[row] = 0; + } } - if (transform_null_in) - return ColumnNullable::create(result_nested_column->getPtr(), std::move(precision_null_map_column)); + return precision_null_map_column; +} - const NullMap * result_null_map = result_nullable_column - ? &result_nullable_column->getNullMapData() - : nullptr; +ColumnPtr mergeNullMaps(const ColumnPtr & null_map_column1, const ColumnPtr & null_map_column2) +{ + if (!null_map_column1) + return null_map_column2; + if (!null_map_column2) + return null_map_column1; - // Merge null maps - auto merged_null_map_column = ColumnUInt8::create(num_rows); - NullMap & merged_null_map = merged_null_map_column->getData(); + const auto & null_map1 = assert_cast(*null_map_column1).getData(); + const auto & null_map2 = assert_cast(*null_map_column2).getData(); - const UInt8 * result_null_map_data = result_null_map ? result_null_map->data() : nullptr; - const UInt8 * precision_null_map_data = assert_cast(*precision_null_map_column).getData().data(); + size_t size = null_map1.size(); + if (size != null_map2.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Null maps have different sizes"); - for (size_t row = 0; row < num_rows; ++row) - { - UInt8 is_null = 0; - if (result_null_map_data && result_null_map_data[row]) - is_null = 1; - if (precision_null_map_data[row]) - is_null = 1; - merged_null_map[row] = is_null; - } + auto merged_null_map_column = ColumnUInt8::create(size); + auto & merged_null_map = merged_null_map_column->getData(); - return ColumnNullable::create(result_nested_column->getPtr(), std::move(merged_null_map_column)); + for (size_t i = 0; i < size; ++i) + merged_null_map[i] = null_map1[i] || null_map2[i]; + + return merged_null_map_column; } ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const @@ -392,6 +387,9 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co { ColumnPtr result; + null_map = ConstNullMapPtr(); + null_map_holder = nullptr; + const auto & column_before_cast = columns.at(i); ColumnWithTypeAndName column_to_cast = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name}; @@ -408,21 +406,33 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co // If the original column is DateTime64, check for sub-second precision if (isDateTime64(column_to_cast.column->getDataType())) { - // Get the precision null map - result = checkDateTimePrecision(column_to_cast.column, result, vec_res.size(), transform_null_in); + ColumnPtr filtered_null_map_column = checkDateTimePrecision(column_to_cast); + + // Extract existing null map and nested column from the result + const ColumnNullable * result_nullable_column = typeid_cast(result.get()); + const IColumn * nested_result_column = result_nullable_column + ? &result_nullable_column->getNestedColumn() + : result.get(); + + ColumnPtr existing_null_map_column = result_nullable_column + ? result_nullable_column->getNullMapColumnPtr() + : nullptr; + + ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); + + result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); if (transform_null_in) { ColumnRawPtrs key_cols{result.get()}; null_map_holder = extractNestedColumnsAndNullMap(key_cols, null_map); - result = typeid_cast(result.get())->getNestedColumnPtr(); // In case of transform_null_in, result column - // is considered as not nullable in HashMethodOneNumber + result = nested_result_column->getPtr(); /// The result is considered not nullable in HashMethodOneNumber } } // Append the result to materialized columns - materialized_columns.emplace_back(result); + materialized_columns.emplace_back(std::move(result)); key_columns.emplace_back(materialized_columns.back().get()); } From 03e0e9a14b6099bf41f402b57065aedfc6515f0b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 24 Oct 2024 16:05:19 -0300 Subject: [PATCH 419/816] draft --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 43 +++++++++++++++++++ .../Impl/Parquet/ParquetDataValuesReader.h | 21 +++++++++ .../Impl/Parquet/ParquetLeafColReader.cpp | 23 ++++++++++ .../Impl/Parquet/ParquetRecordReader.cpp | 2 +- 4 files changed, 88 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index b8e4db8700c..977f2ad298b 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -296,6 +296,40 @@ void ParquetPlainValuesReader::readBatch( ); } +template <> +void ParquetBitPlainReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto & column = *assert_cast(col_ptr.get()); + auto cursor = column.size(); + + auto & container = column.getData(); + + container.resize(cursor + num_values); + + def_level_reader->visitNullableValues( + cursor, + num_values, + max_def_level, + null_map, + /* individual_visitor */ [&](size_t nest_cursor) + { + uint8_t byte; + bit_reader->GetValue(1, &byte); + container[nest_cursor] = byte; + }, + /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) + { + for (UInt32 i = 0; i < count; i++) + { + uint8_t byte; + bit_reader->GetValue(1, &byte); + container[nest_cursor++] = byte; + } + } + ); +} + template <> void ParquetPlainValuesReader, ParquetReaderTypes::TimestampInt96>::readBatch( @@ -515,6 +549,13 @@ void ParquetRleDictReader::readBatch( ); } +template <> +void ParquetRleDictReader::readBatch( + MutableColumnPtr & , LazyNullMap &, UInt32) +{ + assert(false); +} + template void ParquetRleDictReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) @@ -561,6 +602,7 @@ template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; template class ParquetFixedLenPlainReader>; template class ParquetFixedLenPlainReader>; @@ -569,6 +611,7 @@ template class ParquetRleLCReader; template class ParquetRleLCReader; template class ParquetRleLCReader; +template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index fbccb612b3c..db55f7e2d6a 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -172,6 +172,27 @@ private: ParquetDataBuffer plain_data_buffer; }; +template +class ParquetBitPlainReader : public ParquetDataValuesReader +{ +public: + ParquetBitPlainReader( + Int32 max_def_level_, + std::unique_ptr def_level_reader_, + std::unique_ptr bit_reader_) + : max_def_level(max_def_level_) + , def_level_reader(std::move(def_level_reader_)) + , bit_reader(std::move(bit_reader_)) + {} + + void readBatch(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + std::unique_ptr def_level_reader; + std::unique_ptr bit_reader; +}; + /** * The data and definition level encoding are same as ParquetPlainValuesReader. * But the element size is const and bigger than primitive data type. diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 4b5880eba37..f32d7e61062 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -463,6 +463,28 @@ void ParquetLeafColReader::initDataReader( } } +template <> +void ParquetLeafColReader::initDataReader( + parquet::Encoding::type enconding_type, + const uint8_t * buffer, + std::size_t max_size, + std::unique_ptr && def_level_reader) +{ + switch (enconding_type) + { + case parquet::Encoding::PLAIN: + { + auto bit_reader = std::make_unique(buffer, max_size); + data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(bit_reader)); + break; + } + default: + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unknown encoding type: {}", enconding_type); + } +} + template void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) { @@ -620,6 +642,7 @@ std::unique_ptr ParquetLeafColReader::createDi } +template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index acf11a30162..971bb9e1be5 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -263,7 +263,7 @@ std::unique_ptr ColReaderFactory::makeReader() switch (col_descriptor.physical_type()) { case parquet::Type::BOOLEAN: - break; + return makeLeafReader(); case parquet::Type::INT32: return fromInt32(); case parquet::Type::INT64: From e60297a638ad2a69534aaecc967fc4c3790a50db Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 25 Oct 2024 08:16:48 +0000 Subject: [PATCH 420/816] fix tidy build --- src/Interpreters/loadMetadata.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 84ca829462e..dcf3dad9f96 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -20,7 +20,7 @@ namespace DB [[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}, bool async_load_databases = false); /// Converts `system` database from Ordinary to Atomic (if needed) -void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system_startup_tasks); +void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & load_system_metadata_tasks); /// Converts all databases (except system) from Ordinary to Atomic if convert_ordinary_to_atomic flag exists /// Waits for `load_metadata` task before conversions From 2283728aca6de26898f970c6109dc868cded7822 Mon Sep 17 00:00:00 2001 From: ortyomka Date: Fri, 25 Oct 2024 08:36:46 +0000 Subject: [PATCH 421/816] trigger ci with new settings From f70fa8b75c2899a977292bc97bb98b9736b096af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Thu, 24 Oct 2024 17:09:00 +0100 Subject: [PATCH 422/816] Add a test --- .../00463_long_sessions_in_http_interface.reference | 5 +++++ .../00463_long_sessions_in_http_interface.sh | 11 +++++++++++ 2 files changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference index a14d334a483..031ad768aae 100644 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference @@ -26,3 +26,8 @@ HelloWorld A session cannot be used by concurrent connections: 1 1 +A session successfully closes when timeout first expires with refcount != 1 and another session is created in between +45 +45 +1 +1 diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh index 86902fca4aa..d2451d0b3d8 100755 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh @@ -85,3 +85,14 @@ done ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT 1" | grep -c -F 'SESSION_IS_LOCKED' ${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; wait + +echo "A session successfully closes when timeout first expires with refcount != 1 and another session is created in between" +# Here we do not want an infinite loop - because we want this mechanism to be reliable in all cases +# So it's better to give it enough time to complete even in constrained environments +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE x (n UInt64) AS SELECT number FROM numbers(10)" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "SELECT sum(n + sleep(3)) FROM x" # This query ensures timeout expires with refcount > 1 +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE y (n UInt64) AS SELECT number FROM numbers(10)" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "SELECT sum(n) FROM y" +sleep 15 +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' \ No newline at end of file From 31490438d95f514e8ff285b80345c55872b2b485 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 25 Oct 2024 11:09:03 +0000 Subject: [PATCH 423/816] Corrected smoe ifdef issues --- .../registerStorageObjectStorage.cpp | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 823556470b0..b0122de3bf7 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -201,20 +201,6 @@ void registerStorageIceberg(StorageFactory & factory) .source_access_type = AccessType::AZURE, }); #endif - factory.registerStorage( - "IcebergLocal", - [&](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); - - return createStorageObjectStorage(args, configuration, args.getLocalContext()); - }, - { - .supports_settings = false, - .supports_schema_inference = true, - .source_access_type = AccessType::FILE, - }); #if USE_HDFS factory.registerStorage( "IcebergHDFS", @@ -231,10 +217,26 @@ void registerStorageIceberg(StorageFactory & factory) .source_access_type = AccessType::HDFS, }); #endif + factory.registerStorage( + "IcebergLocal", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, + }); } #endif + +#if USE_AWS_S3 #if USE_PARQUET void registerStorageDeltaLake(StorageFactory & factory) { @@ -272,4 +274,5 @@ void registerStorageHudi(StorageFactory & factory) .source_access_type = AccessType::S3, }); } +#endif } From 4277e688a545cbda1760ca1af7593e2360215017 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 13:19:49 +0200 Subject: [PATCH 424/816] enhance tests --- ...8_datetime_cast_losing_precision.reference | 7 +++++++ .../03208_datetime_cast_losing_precision.sql | 19 ++++++++++++++++++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference index 573541ac970..ef8adf8660c 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -1 +1,8 @@ 0 +0 +0 +0 +ᴺᵁᴸᴸ +0 +1 +0 diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql index 43246648934..042c9cacd2d 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql @@ -1,2 +1,19 @@ with toDateTime('2024-10-16 18:00:30') as t -SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (SELECT t); +SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (SELECT t) settings transform_null_in=0; + +with toDateTime('2024-10-16 18:00:30') as t +SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (SELECT t) settings transform_null_in=1; + +with toDateTime('1970-01-01 00:00:01') as t +SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (now(), Null) settings transform_null_in=1; + +with toDateTime('1970-01-01 00:00:01') as t +SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (now(), Null) settings transform_null_in=0; + +with toDateTime('1970-01-01 00:00:01') as t, + arrayJoin([Null, toDateTime64(t, 3) + interval 100 milliseconds]) as x +SELECT x IN (now(), Null) settings transform_null_in=0; + +with toDateTime('1970-01-01 00:00:01') as t, + arrayJoin([Null, toDateTime64(t, 3) + interval 100 milliseconds]) as x +SELECT x IN (now(), Null) settings transform_null_in=1; From c80a50b91b44160f1ea9f8e2de8ad483c1ab0abc Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Fri, 25 Oct 2024 14:38:58 +0300 Subject: [PATCH 425/816] Added a test --- .../__init__.py | 0 .../configs/remote_servers.xml | 23 ++ .../configs/zookeeper_config_with_ssl.xml | 20 ++ .../configs_secure/conf.d/remote_servers.xml | 17 ++ .../configs_secure/conf.d/ssl_conf.xml | 16 ++ .../configs_secure/first_client.crt | 19 ++ .../configs_secure/first_client.key | 28 +++ .../configs_secure/second_client.crt | 19 ++ .../configs_secure/second_client.key | 28 +++ .../configs_secure/third_client.crt | 19 ++ .../configs_secure/third_client.key | 28 +++ .../test_reload_client_certificate/test.py | 196 ++++++++++++++++++ 12 files changed, 413 insertions(+) create mode 100644 tests/integration/test_reload_client_certificate/__init__.py create mode 100644 tests/integration/test_reload_client_certificate/configs/remote_servers.xml create mode 100644 tests/integration/test_reload_client_certificate/configs/zookeeper_config_with_ssl.xml create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/conf.d/remote_servers.xml create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/conf.d/ssl_conf.xml create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/first_client.crt create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/first_client.key create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/second_client.crt create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/second_client.key create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/third_client.crt create mode 100644 tests/integration/test_reload_client_certificate/configs_secure/third_client.key create mode 100644 tests/integration/test_reload_client_certificate/test.py diff --git a/tests/integration/test_reload_client_certificate/__init__.py b/tests/integration/test_reload_client_certificate/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_reload_client_certificate/configs/remote_servers.xml b/tests/integration/test_reload_client_certificate/configs/remote_servers.xml new file mode 100644 index 00000000000..63fdcea5dab --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs/remote_servers.xml @@ -0,0 +1,23 @@ + + + + + + node1 + 9000 + + + + node2 + 9000 + + + + node3 + 9000 + + + + + + diff --git a/tests/integration/test_reload_client_certificate/configs/zookeeper_config_with_ssl.xml b/tests/integration/test_reload_client_certificate/configs/zookeeper_config_with_ssl.xml new file mode 100644 index 00000000000..dc0fe771426 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs/zookeeper_config_with_ssl.xml @@ -0,0 +1,20 @@ + + + + zoo1 + 2281 + 1 + + + zoo2 + 2281 + 1 + + + zoo3 + 2281 + 1 + + 3000 + + diff --git a/tests/integration/test_reload_client_certificate/configs_secure/conf.d/remote_servers.xml b/tests/integration/test_reload_client_certificate/configs_secure/conf.d/remote_servers.xml new file mode 100644 index 00000000000..548819a8c97 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/conf.d/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + + node1 + 9000 + + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_reload_client_certificate/configs_secure/conf.d/ssl_conf.xml b/tests/integration/test_reload_client_certificate/configs_secure/conf.d/ssl_conf.xml new file mode 100644 index 00000000000..d620bcee919 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/conf.d/ssl_conf.xml @@ -0,0 +1,16 @@ + + + + /etc/clickhouse-server/config.d/first_client.crt + /etc/clickhouse-server/config.d/first_client.key + true + true + sslv2,sslv3 + true + none + + RejectCertificateHandler + + + + diff --git a/tests/integration/test_reload_client_certificate/configs_secure/first_client.crt b/tests/integration/test_reload_client_certificate/configs_secure/first_client.crt new file mode 100644 index 00000000000..7ade2d96273 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/first_client.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow +FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI +4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T +4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU +7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj +sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg +pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC +kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw +DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j +4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr +85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C +L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD ++UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L +P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp +0g== +-----END CERTIFICATE----- diff --git a/tests/integration/test_reload_client_certificate/configs_secure/first_client.key b/tests/integration/test_reload_client_certificate/configs_secure/first_client.key new file mode 100644 index 00000000000..f0fb61ac443 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/first_client.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn +DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L +OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus +TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7 +JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F +KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6 +ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP +TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f +7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN +MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5 +212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu +plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr +/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu +Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa +CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g +1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5 +bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq +cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez +on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K +GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54 +nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU +2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh +vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa +dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr +vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1 +KCBtmIrQLqzMpnBpLNuSY+Q= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_reload_client_certificate/configs_secure/second_client.crt b/tests/integration/test_reload_client_certificate/configs_secure/second_client.crt new file mode 100644 index 00000000000..ff62438af62 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/second_client.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIDEDCCAfigAwIBAgIUEAdT/eB4tswNzGZg1V0rVP8WzJwwDQYJKoZIhvcNAQEL +BQAwGDEWMBQGA1UEAwwNbG9jYWxob3N0X25ldzAgFw0yNDEwMjQyMzE5MjJaGA8y +Mjk4MDgwOTIzMTkyMlowGDEWMBQGA1UEAwwNbG9jYWxob3N0X25ldzCCASIwDQYJ +KoZIhvcNAQEBBQADggEPADCCAQoCggEBALPbkqOpSRvKPKcNH21RlbtLTzpY0lnF +Ulx/boiQcJWS0J4TeEhUhaIXyOCvFNbQQ058Ty1KP1dSvcs6C4kQxjJMFY+vuPoR +NrnUtnj00L+aGjHifcK7PuXPU+KTrZat6SwHZtOGTAdZ26xMrHHlbnR0ZEGxh8x6 +elRtvRcOxmumKash0IoUZhJE1O1AmDQPlZO2gz3TEv5IcHsk6V2GYpmivX9BAlN/ +1Tnht3x+kqdl+hTeLVI/2hVxI7EZlORtk5WIkiFDBdiT3YUooX5NYSVQC6xz01N/ +Eji/hssgUVnTXKyMFcG6b7Lz4Ka8jEjcZ6a9ACtHRYY8J7qSL+YSSDkCAwEAAaNQ +ME4wHQYDVR0OBBYEFIzy297mApHGpsmRyuPxFfIwD9MEMB8GA1UdIwQYMBaAFIzy +297mApHGpsmRyuPxFfIwD9MEMAwGA1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQAD +ggEBAD0z8mRBdk93+HxqJdW1qZBN2g+AUc/GUaTUa8oW9baHOOvdwUacfdVXpyDo +ffdeTKfdQNs7JYMP5tWupHCrvAGK3sIzPMt7Yr06tBD720IIyPTR3J7A5RmpQNKm +2RCqfO49Pg6U8kx+bDBKNjdCGWowt31cZTlJNXk7NPewtWaGYhuskbvH8gJDtbMd +d9fOepIbzl3u+us8JHFVglBRgjy9sYjUYUT9mnTzfbpebmkdtiicJZNP1j08VZFR +lXoHiESasyzlP8DLI/PQcpL6Lh8KnIifKGEkvXVaryPT2wlEo6Kti2cY8AIJKQgl +0U1jwiNcCwjYoKIXjunOO8T8mKg= +-----END CERTIFICATE----- \ No newline at end of file diff --git a/tests/integration/test_reload_client_certificate/configs_secure/second_client.key b/tests/integration/test_reload_client_certificate/configs_secure/second_client.key new file mode 100644 index 00000000000..f0fb61ac443 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/second_client.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn +DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L +OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus +TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7 +JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F +KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6 +ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP +TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f +7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN +MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5 +212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu +plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr +/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu +Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa +CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g +1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5 +bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq +cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez +on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K +GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54 +nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU +2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh +vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa +dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr +vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1 +KCBtmIrQLqzMpnBpLNuSY+Q= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_reload_client_certificate/configs_secure/third_client.crt b/tests/integration/test_reload_client_certificate/configs_secure/third_client.crt new file mode 100644 index 00000000000..4efb8f1b7b9 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/third_client.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIDCDCCAfCgAwIBAgIUC749qXQA+HcnMauXvrmGf+Yz7KswDQYJKoZIhvcNAQEL +BQAwFDESMBAGA1UEAwwJbG9jYWxob3N0MCAXDTI0MTAyNTA4NDg1N1oYDzIyOTgw +ODEwMDg0ODU3WjAUMRIwEAYDVQQDDAlsb2NhbGhvc3QwggEiMA0GCSqGSIb3DQEB +AQUAA4IBDwAwggEKAoIBAQCz25KjqUkbyjynDR9tUZW7S086WNJZxVJcf26IkHCV +ktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3LOguJEMYyTBWPr7j6ETa51LZ49NC/ +mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdusTKxx5W50dGRBsYfMenpUbb0XDsZr +pimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7JOldhmKZor1/QQJTf9U54bd8fpKn +ZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92FKKF+TWElUAusc9NTfxI4v4bLIFFZ +01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6ki/mEkg5AgMBAAGjUDBOMB0GA1Ud +DgQWBBSM8tve5gKRxqbJkcrj8RXyMA/TBDAfBgNVHSMEGDAWgBSM8tve5gKRxqbJ +kcrj8RXyMA/TBDAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQB/QYNd +q8ub45u2tsCEr8xgON4CB2UGZD5RazY//W6kPWmLBf8fZjepF7yLjEWP6iQHWVWk +vIVmVsAnIyfOruUYQmxR4N770Tlit9PH7OqNtRzXHGV2el3Rp62mg8NneOx4SHX+ +HITyPF3Wcg7YyWCuwwGXXS2hZ20csQXZima1jVyTNRN0GDvp0xjX+o7gyANGxbxa +EnjXTc4IWbLJ/+k4I38suavXg8RToHt+1Ndp0sHoT7Fxj+mbxOcc3QVtYU/Ct1W7 +cirraodxjWkYX63zDeqteXU8JtNdJE43qFK4BVh3QTj7PhD3PFEAKcPbnJLbdTYC +ZU36rm75uOSdLXNB +-----END CERTIFICATE----- diff --git a/tests/integration/test_reload_client_certificate/configs_secure/third_client.key b/tests/integration/test_reload_client_certificate/configs_secure/third_client.key new file mode 100644 index 00000000000..f0fb61ac443 --- /dev/null +++ b/tests/integration/test_reload_client_certificate/configs_secure/third_client.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn +DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L +OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus +TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7 +JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F +KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6 +ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP +TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f +7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN +MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5 +212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu +plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr +/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu +Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa +CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g +1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5 +bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq +cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez +on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K +GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54 +nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU +2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh +vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa +dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr +vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1 +KCBtmIrQLqzMpnBpLNuSY+Q= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py new file mode 100644 index 00000000000..e12b5d4b35d --- /dev/null +++ b/tests/integration/test_reload_client_certificate/test.py @@ -0,0 +1,196 @@ +import os + +import threading + +import time + +import pytest + +from helpers.cluster import ClickHouseCluster + + +TEST_DIR = os.path.dirname(__file__) + +cluster = ClickHouseCluster( + __file__, + zookeeper_certfile=os.path.join(TEST_DIR, "configs_secure", "first_client.crt"), + zookeeper_keyfile=os.path.join(TEST_DIR, "configs_secure", "first_client.key"), +) + +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs_secure/first_client.crt", + "configs_secure/first_client.key", + "configs_secure/second_client.crt", + "configs_secure/second_client.key", + "configs_secure/third_client.crt", + "configs_secure/third_client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml", + ], + with_zookeeper_secure=True, +) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs_secure/first_client.crt", + "configs_secure/first_client.key", + "configs_secure/second_client.crt", + "configs_secure/second_client.key", + "configs_secure/third_client.crt", + "configs_secure/third_client.key", + "configs_secure/conf.d/remote_servers.xml", + "configs_secure/conf.d/ssl_conf.xml", + "configs/zookeeper_config_with_ssl.xml", + ], + with_zookeeper_secure=True, +) + +nodes = [node1, node2] + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def secure_connection_test(started_cluster): + # no asserts, connection works + node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") + node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") + + threads_number = 4 + iterations = 10 + threads = [] + + # just checking for race conditions + for _ in range(threads_number): + threads.append( + threading.Thread( + target=( + lambda: [ + node1.query( + "SELECT count() FROM system.zookeeper WHERE path = '/'" + ) + for _ in range(iterations) + ] + ) + ) + ) + + for thread in threads: + thread.start() + + for thread in threads: + thread.join() + + +def change_config_to_key(name): + """ + * Generate config with certificate/key name from args. + * Reload config. + """ + + for node in nodes: + node.exec_in_container( + [ + "bash", + "-c", + """cat > /etc/clickhouse-server/config.d/ssl_conf.xml << EOF + + + + /etc/clickhouse-server/config.d/{cur_name}_client.crt + /etc/clickhouse-server/config.d/{cur_name}_client.key + true + true + sslv2,sslv3 + true + none + + RejectCertificateHandler + + + + +EOF""".format( + cur_name=name + ), + ] + ) + + node.exec_in_container( + ["bash", "-c", f"touch /etc/clickhouse-server/config.d/ssl_conf.xml"], + ) + + +def check_reload_successful( + node, cert_name): + return node.grep_in_log(f"Reloaded certificate (/etc/clickhouse-server/config.d/{cert_name}_client.crt)") + +def check_error_handshake(node): + return node.count_in_log("Code: 210.") + +def clean_logs(): + for node in nodes: + node.exec_in_container( + [ + "bash", + "-c", + "echo -n > /var/log/clickhouse-server/clickhouse-server.log", + ] + ) + +def check_certificate_switch( + first, second +): + # Set first key + change_config_to_key(first) + + # Restart zookeeper to reload the session + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) + clean_logs() + + # Change to wrong certificate + change_config_to_key(second) + + # Time to log + time.sleep(10) + + # Check information about client certificates reloading in log + reload_successful = any(check_reload_successful(node, second) for node in nodes) + + # Restart zookeeper to reload the session and clean logs for new check + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + clean_logs() + cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) + + if second == "second": + time.sleep(10) + error_handshake = any(check_error_handshake(node) != "0\n" for node in nodes) + else: + check_connection = secure_connection_test(started_cluster) + error_handshake = any(check_error_handshake(node) == "0\n" for node in nodes) + assert reload_successful and error_handshake + + +def test_wrong_cn_cert(): + """Checking the certificate reload with an incorrect CN, the expected behavior is Code: 210.""" + check_certificate_switch("first", "second") + + +def test_correct_cn_cert(): + """Replacement with a valid certificate, the expected behavior is to restore the connection with Zookeeper.""" + check_certificate_switch("second", "third") + + + From 9febc9eb079800b3467d1ecada60469535a6cdf0 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 25 Oct 2024 13:39:30 +0200 Subject: [PATCH 426/816] Update src/Common/Scheduler/ResourceGuard.h Co-authored-by: Antonio Andelic --- src/Common/Scheduler/ResourceGuard.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 6ff22edd221..ba3532598af 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -115,7 +115,7 @@ public: dequeued_cv.notify_one(); } - // This function is executed inside scheduler thread and wakes thread issued this `request`. + // This function is executed inside scheduler thread and wakes thread that issued this `request`. // That thread will throw an exception. void failed(const std::exception_ptr & ptr) override { From 98517f2db78f50143fbecbd9fb6c30ebcbbf67c5 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 25 Oct 2024 11:45:19 +0000 Subject: [PATCH 427/816] addressing review comments --- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index c5f8ac0dff8..84923c49c62 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; } class UnifiedSchedulerNode; @@ -346,6 +347,7 @@ private: { if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) { + // If both semaphore and throttler exist we should reparent to the farthest from the root if (semaphore) reparent(branch_root, semaphore); else if (throttler) @@ -530,8 +532,7 @@ protected: // Hide all the ISchedulerNode interface methods as an implementation bool equals(ISchedulerNode *) override { - assert(false); - return false; + throw Exception(ErrorCodes::LOGICAL_ERROR, "UnifiedSchedulerNode should not be used with CustomResourceManager"); } /// Attaches an immediate child (used through `reparent()`) From 6aed1cf4e242eaa41bc6512619692b0054cb4d2b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 14:02:37 +0200 Subject: [PATCH 428/816] Fix reference. --- .../0_stateless/03208_datetime_cast_losing_precision.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference index ef8adf8660c..a22e639726e 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -2,7 +2,7 @@ 0 0 0 -ᴺᵁᴸᴸ +\N 0 1 0 From 32cfa6de6a83d3c4ed6b18f4e16310e0fb0ef4c6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Oct 2024 11:08:04 +0000 Subject: [PATCH 429/816] allow to prewarm mark cache for parts --- src/Access/Common/AccessType.h | 1 + src/Formats/MarkInCompressedFile.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 36 ++++++++++ src/Interpreters/InterpreterSystemQuery.h | 1 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 1 + .../parseIdentifierOrStringLiteral.cpp | 27 ++++++++ src/Parsers/parseIdentifierOrStringLiteral.h | 5 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 7 ++ .../MergeTree/IMergeTreeDataPartWriter.h | 6 ++ .../MergeTree/IMergedBlockOutputStream.h | 5 ++ .../MergeTree/MergeFromLogEntryTask.cpp | 4 ++ .../MergeTree/MergePlainMergeTreeTask.cpp | 7 +- src/Storages/MergeTree/MergeTask.cpp | 15 +++++ src/Storages/MergeTree/MergeTask.h | 9 +++ src/Storages/MergeTree/MergeTreeData.cpp | 49 ++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 3 + .../MergeTree/MergeTreeDataPartCompact.cpp | 26 ++++++++ .../MergeTree/MergeTreeDataPartCompact.h | 2 + .../MergeTree/MergeTreeDataPartWide.cpp | 41 ++++++++++++ .../MergeTree/MergeTreeDataPartWide.h | 2 + .../MergeTreeDataPartWriterCompact.cpp | 23 +++++-- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 17 ++++- .../MergeTree/MergeTreeDataPartWriterWide.h | 3 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 18 +---- .../MergeTree/MergeTreeDataWriter.cpp | 13 +++- .../MergeTree/MergeTreeIOSettings.cpp | 2 + src/Storages/MergeTree/MergeTreeIOSettings.h | 4 +- .../MergeTree/MergeTreeMarksLoader.cpp | 29 +++++++++ src/Storages/MergeTree/MergeTreeMarksLoader.h | 9 +++ src/Storages/MergeTree/MergeTreeSettings.cpp | 2 + src/Storages/MergeTree/MergeTreeSink.cpp | 9 +++ .../MergeTree/MergedBlockOutputStream.cpp | 2 + .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 5 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + src/Storages/MergeTree/MutateTask.cpp | 2 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 21 ++++++ src/Storages/StorageMergeTree.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 9 +++ .../01271_show_privileges.reference | 1 + ...03254_prewarm_mark_cache_columns.reference | 6 ++ .../03254_prewarm_mark_cache_columns.sql | 30 +++++++++ .../03254_prewarm_mark_cache_rmt.reference | 16 +++++ .../03254_prewarm_mark_cache_rmt.sql | 65 +++++++++++++++++++ 48 files changed, 518 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/03254_prewarm_mark_cache_columns.reference create mode 100644 tests/queries/0_stateless/03254_prewarm_mark_cache_columns.sql create mode 100644 tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.reference create mode 100644 tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.sql diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..777fbff1095 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -159,6 +159,7 @@ enum class AccessType : uint8_t M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_PREWARM_MARK_CACHE, "SYSTEM PREWARM MARK, PREWARM MARK CACHE, PREWARM MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MMAP_CACHE, "SYSTEM DROP MMAP, DROP MMAP CACHE, DROP MMAP", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/src/Formats/MarkInCompressedFile.h b/src/Formats/MarkInCompressedFile.h index 06ed1476410..e1bcda61b39 100644 --- a/src/Formats/MarkInCompressedFile.h +++ b/src/Formats/MarkInCompressedFile.h @@ -119,4 +119,6 @@ private: std::tuple lookUpMark(size_t idx) const; }; +using PlainMarksByName = std::unordered_map>; + } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 8aa1bda1d1c..b80eab324bd 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -89,6 +89,9 @@ namespace CurrentMetrics extern const Metric RestartReplicaThreads; extern const Metric RestartReplicaThreadsActive; extern const Metric RestartReplicaThreadsScheduled; + extern const Metric MergeTreePartsLoaderThreads; + extern const Metric MergeTreePartsLoaderThreadsActive; + extern const Metric MergeTreePartsLoaderThreadsScheduled; } namespace DB @@ -97,6 +100,7 @@ namespace Setting { extern const SettingsSeconds lock_acquire_timeout; extern const SettingsSeconds receive_timeout; + extern const SettingsMaxThreads max_threads; } namespace ServerSetting @@ -359,6 +363,11 @@ BlockIO InterpreterSystemQuery::execute() HTTPConnectionPools::instance().dropCache(); break; } + case Type::PREWARM_MARK_CACHE: + { + prewarmMarkCache(); + break; + } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context->clearMarkCache(); @@ -1298,6 +1307,28 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks() return tasks; } +void InterpreterSystemQuery::prewarmMarkCache() +{ + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE); + + if (table_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for prewarming marks cache"); + + auto table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto * merge_tree = dynamic_cast(table_ptr.get()); + + if (!merge_tree) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM MARK CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); + + ThreadPool pool( + CurrentMetrics::MergeTreePartsLoaderThreads, + CurrentMetrics::MergeTreePartsLoaderThreadsActive, + CurrentMetrics::MergeTreePartsLoaderThreadsScheduled, + getContext()->getSettingsRef()[Setting::max_threads]); + + merge_tree->prewarmMarkCache(pool); +} + AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() const { @@ -1499,6 +1530,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_WAIT_LOADING_PARTS, query.getDatabase(), query.getTable()); break; } + case Type::PREWARM_MARK_CACHE: + { + required_access.emplace_back(AccessType::SYSTEM_PREWARM_MARK_CACHE, query.getDatabase(), query.getTable()); + break; + } case Type::SYNC_DATABASE_REPLICA: { required_access.emplace_back(AccessType::SYSTEM_SYNC_DATABASE_REPLICA, query.getDatabase()); diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 3d667fcaef0..7d6aca8178e 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -82,6 +82,7 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); + void prewarmMarkCache(); }; diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index b5e5e0f208d..d76d33ce708 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -191,6 +191,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::SYNC_REPLICA: case Type::WAIT_LOADING_PARTS: case Type::FLUSH_DISTRIBUTED: + case Type::PREWARM_MARK_CACHE: { if (table) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index d9f5b425182..d9ee4d8aa22 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -23,6 +23,7 @@ public: SUSPEND, DROP_DNS_CACHE, DROP_CONNECTIONS_CACHE, + PREWARM_MARK_CACHE, DROP_MARK_CACHE, DROP_UNCOMPRESSED_CACHE, DROP_INDEX_MARK_CACHE, diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index af84dd10bfa..453ae0b5032 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -276,6 +276,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::RESTART_REPLICA: case Type::SYNC_REPLICA: case Type::WAIT_LOADING_PARTS: + case Type::PREWARM_MARK_CACHE: { if (!parseQueryWithOnCluster(res, pos, expected)) return false; diff --git a/src/Parsers/parseIdentifierOrStringLiteral.cpp b/src/Parsers/parseIdentifierOrStringLiteral.cpp index bb93145772a..71fe071ec03 100644 --- a/src/Parsers/parseIdentifierOrStringLiteral.cpp +++ b/src/Parsers/parseIdentifierOrStringLiteral.cpp @@ -6,11 +6,24 @@ #include #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; +} + +namespace Setting +{ + extern const SettingsUInt64 max_query_size; + extern const SettingsUInt64 max_parser_depth; + extern const SettingsUInt64 max_parser_backtracks; +} + bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, String & result) { return IParserBase::wrapParseImpl(pos, [&] @@ -54,4 +67,18 @@ bool parseIdentifiersOrStringLiterals(IParser::Pos & pos, Expected & expected, S return true; } +std::vector parseIdentifiersOrStringLiterals(const String & str, const Settings & settings) +{ + Tokens tokens(str.data(), str.data() + str.size(), settings[Setting::max_query_size]); + IParser::Pos pos(tokens, static_cast(settings[Setting::max_parser_depth]), static_cast(settings[Setting::max_parser_backtracks])); + + Expected expected; + std::vector res; + + if (!parseIdentifiersOrStringLiterals(pos, expected, res)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string ('{}') into vector of identifiers", str); + + return res; +} + } diff --git a/src/Parsers/parseIdentifierOrStringLiteral.h b/src/Parsers/parseIdentifierOrStringLiteral.h index b450ce8f2f0..867962d1a57 100644 --- a/src/Parsers/parseIdentifierOrStringLiteral.h +++ b/src/Parsers/parseIdentifierOrStringLiteral.h @@ -7,6 +7,8 @@ namespace DB { +struct Settings; + /** Parses a name of an object which could be written in the following forms: * name / `name` / "name" (identifier) or 'name'. * Note that empty strings are not allowed. @@ -16,4 +18,7 @@ bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, Str /** Parse a list of identifiers or string literals. */ bool parseIdentifiersOrStringLiterals(IParser::Pos & pos, Expected & expected, Strings & result); +/** Parse a list of identifiers or string literals into vector of strings. */ +std::vector parseIdentifiersOrStringLiterals(const String & str, const Settings & settings); + } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 378832d32a1..b41a1d840e1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -180,6 +180,9 @@ public: void loadRowsCountFileForUnexpectedPart(); + /// Loads marks and saves them into mark cache for specified columns. + virtual void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const = 0; + String getMarksFileExtension() const { return index_granularity_info.mark_type.getFileExtension(); } /// Generate the new name for this part according to `new_part_info` and min/max dates from the old name. diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 3d6366f9217..dbfdbbdea88 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -91,6 +91,13 @@ Columns IMergeTreeDataPartWriter::releaseIndexColumns() return result; } +PlainMarksByName IMergeTreeDataPartWriter::releaseCachedMarks() +{ + PlainMarksByName res; + std::swap(cached_marks, res); + return res; +} + SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const { auto it = serializations.find(column_name); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index eb51a1b2922..b8ac14b1750 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -46,6 +47,9 @@ public: virtual void finish(bool sync) = 0; Columns releaseIndexColumns(); + + PlainMarksByName releaseCachedMarks(); + const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } protected: @@ -69,6 +73,8 @@ protected: MutableDataPartStoragePtr data_part_storage; MutableColumns index_columns; MergeTreeIndexGranularity index_granularity; + /// Marks that will be saved to cache on finish. + PlainMarksByName cached_marks; }; using MergeTreeDataPartWriterPtr = std::unique_ptr; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index cfcfb177e05..a901b03c115 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -34,6 +34,11 @@ public: return writer->getIndexGranularity(); } + PlainMarksByName releaseCachedMarks() + { + return writer->releaseCachedMarks(); + } + protected: /// Remove all columns marked expired in data_part. Also, clears checksums diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index fa6640409e5..d7e807c689f 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -371,6 +371,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWriter write_part_log) { part = merge_task->getFuture().get(); + auto cached_marks = merge_task->releaseCachedMarks(); storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); /// Why we reset task here? Because it holds shared pointer to part and tryRemovePartImmediately will @@ -444,6 +445,9 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + addMarksToCache(*part, cached_marks, mark_cache); + write_part_log({}); StorageReplicatedMergeTree::incrementMergedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index f7b52d2216d..6aca58faf47 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -152,6 +152,12 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + auto marks = merge_task->releaseCachedMarks(); + addMarksToCache(*new_part, marks, mark_cache); + } + write_part_log({}); StorageMergeTree::incrementMergedPartsProfileEvent(new_part->getType()); transfer_profile_counters_to_initial_query(); @@ -163,7 +169,6 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); } - } ContextMutablePtr MergePlainMergeTreeTask::createTaskContext() const diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e3ace824115..193622d7b87 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -93,6 +93,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_columns_to_activate; extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_rows_to_activate; extern const MergeTreeSettingsBool vertical_merge_remote_filesystem_prefetch; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -546,6 +547,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const } } + bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); + global_ctx->to = std::make_shared( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -555,6 +558,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const ctx->compression_codec, global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, + save_marks_in_cache, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); @@ -1085,6 +1089,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->executor = std::make_unique(ctx->column_parts_pipeline); NamesAndTypesList columns_list = {*ctx->it_name_and_type}; + bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); + ctx->column_to = std::make_unique( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -1093,6 +1099,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, + save_marks_in_cache, global_ctx->to->getIndexGranularity()); ctx->column_elems_written = 0; @@ -1130,6 +1137,10 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns); global_ctx->checksums_gathered_columns.add(std::move(changed_checksums)); + auto cached_marks = ctx->column_to->releaseCachedMarks(); + for (auto & [name, marks] : cached_marks) + global_ctx->cached_marks.emplace(name, std::move(marks)); + ctx->delayed_streams.emplace_back(std::move(ctx->column_to)); while (ctx->delayed_streams.size() > ctx->max_delayed_streams) @@ -1276,6 +1287,10 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const else global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns); + auto cached_marks = global_ctx->to->releaseCachedMarks(); + for (auto & [name, marks] : cached_marks) + global_ctx->cached_marks.emplace(name, std::move(marks)); + global_ctx->new_data_part->getDataPartStorage().precommitTransaction(); global_ctx->promise.set_value(global_ctx->new_data_part); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 5a4fb1ec0b8..53792165987 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -132,6 +133,13 @@ public: return nullptr; } + PlainMarksByName releaseCachedMarks() const + { + PlainMarksByName res; + std::swap(global_ctx->cached_marks, res); + return res; + } + bool execute(); private: @@ -209,6 +217,7 @@ private: std::promise promise{}; IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; + PlainMarksByName cached_marks; MergeTreeTransactionPtr txn; bool need_prefix; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8611681a976..b89d23fb4f0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -229,6 +229,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString storage_policy; extern const MergeTreeSettingsFloat zero_copy_concurrent_part_removal_max_postpone_ratio; extern const MergeTreeSettingsUInt64 zero_copy_concurrent_part_removal_max_split_times; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -2335,6 +2336,54 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() } } +void MergeTreeData::prewarmMarkCache(ThreadPool & pool) +{ + if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) + return; + + auto * mark_cache = getContext()->getMarkCache().get(); + if (!mark_cache) + return; + + auto metadata_snaphost = getInMemoryMetadataPtr(); + auto column_names = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); + + if (column_names.empty()) + return; + + Stopwatch watch; + LOG_TRACE(log, "Prewarming mark cache"); + + auto data_parts = getDataPartsVectorForInternalUsage(); + + /// Prewarm mark cache firstly for the most fresh parts according + /// to time columns in partition key (if exists) and by modification time. + + auto to_tuple = [](const auto & part) + { + return std::make_tuple(part->getMinMaxDate().second, part->getMinMaxTime().second, part->modification_time); + }; + + std::sort(data_parts.begin(), data_parts.end(), [&to_tuple](const auto & lhs, const auto & rhs) + { + return to_tuple(lhs) > to_tuple(rhs); + }); + + ThreadPoolCallbackRunnerLocal runner(pool, "PrewarmMarks"); + + for (const auto & part : data_parts) + { + if (mark_cache->sizeInBytes() >= mark_cache->maxSizeInBytes() * 0.95) + break; + + runner([&] { part->loadMarksToCache(column_names, mark_cache); }); + } + + runner.waitForAllToFinishAndRethrowFirstError(); + watch.stop(); + LOG_TRACE(log, "Prewarmed mark cache in {} seconds", watch.elapsedSeconds()); +} + /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. /// (Only files on the first level of nesting are considered). diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7a9730e8627..a32106f76bb 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -506,6 +506,9 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks, std::optional> expected_parts); + /// Prewarm mark cache for the most recent data parts. + void prewarmMarkCache(ThreadPool & pool); + String getLogName() const { return log.loadName(); } Int64 getMaxBlockNumber() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fd46b3b9540..22f3c379398 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -136,6 +136,32 @@ void MergeTreeDataPartCompact::loadIndexGranularity() loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), getDataPartStorage()); } +void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const +{ + if (column_names.empty() || !mark_cache) + return; + + auto context = storage.getContext(); + auto read_settings = context->getReadSettings(); + auto * load_marks_threadpool = read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + auto info_for_read = std::make_shared(shared_from_this(), std::make_shared()); + + LOG_TEST(getLogger("MergeTreeDataPartCompact"), "Loading marks into mark cache for columns {} of part {}", toString(column_names), name); + + MergeTreeMarksLoader loader( + info_for_read, + mark_cache, + index_granularity_info.getMarksFilePath(DATA_FILE_NAME), + index_granularity.getMarksCount(), + index_granularity_info, + /*save_marks_in_cache=*/ true, + read_settings, + load_marks_threadpool, + columns.size()); + + loader.loadMarks(); +} + bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { if (!getColumnPosition(column.getNameInStorage())) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 9512485c54e..8e279571578 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -54,6 +54,8 @@ public: std::optional getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } + void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + ~MergeTreeDataPartCompact() override; protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 9bbf0ad9739..d6f213463f2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -182,6 +182,47 @@ void MergeTreeDataPartWide::loadIndexGranularity() loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), *any_column_filename); } +void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const +{ + if (column_names.empty() || !mark_cache) + return; + + std::vector> loaders; + + auto context = storage.getContext(); + auto read_settings = context->getReadSettings(); + auto * load_marks_threadpool = read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + auto info_for_read = std::make_shared(shared_from_this(), std::make_shared()); + + LOG_TEST(getLogger("MergeTreeDataPartWide"), "Loading marks into mark cache for columns {} of part {}", toString(column_names), name); + + for (const auto & column_name : column_names) + { + auto serialization = getSerialization(column_name); + serialization->enumerateStreams([&](const auto & subpath) + { + auto stream_name = getStreamNameForColumn(column_name, subpath, checksums); + if (!stream_name) + return; + + loaders.emplace_back(std::make_unique( + info_for_read, + mark_cache, + index_granularity_info.getMarksFilePath(*stream_name), + index_granularity.getMarksCount(), + index_granularity_info, + /*save_marks_in_cache=*/ true, + read_settings, + load_marks_threadpool, + /*num_columns_in_mark=*/ 1)); + + loaders.back()->startAsyncLoad(); + }); + } + + for (auto & loader : loaders) + loader->loadMarks(); +} bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 42893f47573..022a5fb746c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -51,6 +51,8 @@ public: std::optional getColumnModificationTime(const String & column_name) const override; + void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + protected: static void loadIndexGranularityImpl( MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index a859172023f..67a2c1ee9f1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -1,5 +1,6 @@ #include #include +#include "Formats/MarkInCompressedFile.h" namespace DB { @@ -54,6 +55,11 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( marks_source_hashing = std::make_unique(*marks_compressor); } + if (settings.save_marks_in_cache) + { + cached_marks[MergeTreeDataPartCompact::DATA_FILE_NAME] = std::make_unique(); + } + for (const auto & column : columns_list) { auto compression = getCodecDescOrDefault(column.name, default_codec); @@ -255,9 +261,12 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G return &result_stream->hashing_buf; }; + MarkInCompressedFile mark{plain_hashing.count(), static_cast(0)}; + writeBinaryLittleEndian(mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(mark.offset_in_decompressed_block, marks_out); - writeBinaryLittleEndian(plain_hashing.count(), marks_out); - writeBinaryLittleEndian(static_cast(0), marks_out); + if (!cached_marks.empty()) + cached_marks.begin()->second->push_back(mark); writeColumnSingleGranule( block.getByName(name_and_type->name), getSerialization(name_and_type->name), @@ -296,11 +305,17 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(MergeTreeDataPartChecksum if (with_final_mark && data_written) { + MarkInCompressedFile mark{plain_hashing.count(), 0}; + for (size_t i = 0; i < columns_list.size(); ++i) { - writeBinaryLittleEndian(plain_hashing.count(), marks_out); - writeBinaryLittleEndian(static_cast(0), marks_out); + writeBinaryLittleEndian(mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(mark.offset_in_decompressed_block, marks_out); + + if (!cached_marks.empty()) + cached_marks.begin()->second->push_back(mark); } + writeBinaryLittleEndian(static_cast(0), marks_out); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 8d84442981e..4a760c20b58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 459ddc1ca79..433c7c21613 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include @@ -105,6 +107,12 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( indices_to_recalc_, stats_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { + if (settings.save_marks_in_cache) + { + auto columns_vec = getColumnsToPrewarmMarks(*storage_settings, columns_list); + columns_to_load_marks = NameSet(columns_vec.begin(), columns_vec.end()); + } + for (const auto & column : columns_list) { auto compression = getCodecDescOrDefault(column.name, default_codec); @@ -198,6 +206,9 @@ void MergeTreeDataPartWriterWide::addStreams( settings.marks_compress_block_size, query_write_settings); + if (columns_to_load_marks.contains(name_and_type.name)) + cached_marks.emplace(stream_name, std::make_unique()); + full_name_to_stream_name.emplace(full_stream_name, stream_name); stream_name_to_full_name.emplace(stream_name, full_stream_name); }; @@ -366,8 +377,12 @@ void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stre writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out); writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out); + if (settings.can_use_adaptive_granularity) writeBinaryLittleEndian(rows_in_mark, marks_out); + + if (auto it = cached_marks.find(stream_with_mark.stream_name); it != cached_marks.end()) + it->second->push_back(stream_with_mark.mark); } StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( @@ -742,7 +757,6 @@ void MergeTreeDataPartWriterWide::fillChecksums(MergeTreeDataPartChecksums & che fillPrimaryIndexChecksums(checksums); fillSkipIndicesChecksums(checksums); - fillStatisticsChecksums(checksums); } @@ -756,7 +770,6 @@ void MergeTreeDataPartWriterWide::finish(bool sync) finishPrimaryIndexSerialization(sync); finishSkipIndicesSerialization(sync); - finishStatisticsSerialization(sync); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index ab86ed27c7e..68f016a7421 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -136,6 +136,9 @@ private: using MarksForColumns = std::unordered_map; MarksForColumns last_non_written_marks; + /// Set of columns to put marks in cache during write. + NameSet columns_to_load_marks; + /// How many rows we have already written in the current mark. /// More than zero when incoming blocks are smaller then their granularity. size_t rows_written_in_last_mark = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d7305045a56..1b3c58000e7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -71,10 +71,7 @@ namespace Setting extern const SettingsString force_data_skipping_indices; extern const SettingsBool force_index_by_date; extern const SettingsSeconds lock_acquire_timeout; - extern const SettingsUInt64 max_parser_backtracks; - extern const SettingsUInt64 max_parser_depth; extern const SettingsInt64 max_partitions_to_read; - extern const SettingsUInt64 max_query_size; extern const SettingsUInt64 max_threads_for_indexes; extern const SettingsNonZeroUInt64 max_parallel_replicas; extern const SettingsUInt64 merge_tree_coarse_index_granularity; @@ -640,20 +637,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (use_skip_indexes && settings[Setting::force_data_skipping_indices].changed) { - const auto & indices = settings[Setting::force_data_skipping_indices].toString(); - - Strings forced_indices; - { - Tokens tokens(indices.data(), indices.data() + indices.size(), settings[Setting::max_query_size]); - IParser::Pos pos( - tokens, static_cast(settings[Setting::max_parser_depth]), static_cast(settings[Setting::max_parser_backtracks])); - Expected expected; - if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); - } + const auto & indices_str = settings[Setting::force_data_skipping_indices].toString(); + auto forced_indices = parseIdentifiersOrStringLiterals(indices_str, settings); if (forced_indices.empty()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices_str); std::unordered_set useful_indices_names; for (const auto & useful_index : skip_indexes.useful_indices) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 67fef759ed4..ac29a9244b0 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -73,6 +73,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsFloat min_free_disk_ratio_to_perform_insert; extern const MergeTreeSettingsBool optimize_row_order; extern const MergeTreeSettingsFloat ratio_of_defaults_for_sparse_serialization; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -684,6 +685,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + bool save_marks_in_cache = (*data_settings)[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -693,8 +695,9 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( statistics, compression_codec, context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, - false, - false, + /*reset_columns=*/ false, + save_marks_in_cache, + /*blocks_are_granules_size=*/ false, context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); @@ -829,6 +832,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + bool save_marks_in_cache = (*data.getSettings())[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -839,7 +843,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ColumnsStatistics{}, compression_codec, Tx::PrehistoricTID, - false, false, data.getContext()->getWriteSettings()); + /*reset_columns=*/ false, + save_marks_in_cache, + /*blocks_are_granules_size=*/ false, + data.getContext()->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); auto finalizer = out->finalizePartAsync(new_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.cpp b/src/Storages/MergeTree/MergeTreeIOSettings.cpp index 8b87c35b4e6..bacfbbd5720 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeIOSettings.cpp @@ -34,6 +34,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, bool rewrite_primary_key_, + bool save_marks_in_cache_, bool blocks_are_granules_size_) : min_compress_block_size( (*storage_settings)[MergeTreeSetting::min_compress_block_size] ? (*storage_settings)[MergeTreeSetting::min_compress_block_size] : global_settings[Setting::min_compress_block_size]) @@ -46,6 +47,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , primary_key_compress_block_size((*storage_settings)[MergeTreeSetting::primary_key_compress_block_size]) , can_use_adaptive_granularity(can_use_adaptive_granularity_) , rewrite_primary_key(rewrite_primary_key_) + , save_marks_in_cache(save_marks_in_cache_) , blocks_are_granules_size(blocks_are_granules_size_) , query_write_settings(query_write_settings_) , low_cardinality_max_dictionary_size(global_settings[Setting::low_cardinality_max_dictionary_size]) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index fcc72815d8f..f6cacc38626 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -60,7 +60,8 @@ struct MergeTreeWriterSettings const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, bool rewrite_primary_key_, - bool blocks_are_granules_size_ = false); + bool save_marks_in_cache_, + bool blocks_are_granules_size_); size_t min_compress_block_size; size_t max_compress_block_size; @@ -74,6 +75,7 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity; bool rewrite_primary_key; + bool save_marks_in_cache; bool blocks_are_granules_size; WriteSettings query_write_settings; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 168134a329f..a271af578cc 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -3,10 +3,12 @@ #include #include #include +#include #include #include #include #include +#include #include @@ -21,6 +23,11 @@ namespace ProfileEvents namespace DB { +namespace MergeTreeSetting +{ + extern const MergeTreeSettingsString columns_to_prewarm_mark_cache; +} + namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; @@ -211,6 +218,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync() if (mark_cache) { auto key = MarkCache::hash(fs::path(data_part_storage->getFullPath()) / mrk_path); + if (save_marks_in_cache) { auto callback = [this] { return loadMarksImpl(); }; @@ -249,4 +257,25 @@ std::future MergeTreeMarksLoader::loadMarksAsync() "LoadMarksThread"); } +void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & cached_marks, MarkCache * mark_cache) +{ + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + + for (const auto & [stream_name, marks] : cached_marks) + { + auto mark_path = part.index_granularity_info.getMarksFilePath(stream_name); + auto key = MarkCache::hash(fs::path(part.getDataPartStorage().getFullPath()) / mark_path); + mark_cache->set(key, std::make_shared(*marks)); + } +} + +Names getColumnsToPrewarmMarks(const MergeTreeSettings & settings, const NamesAndTypesList & columns_list) +{ + auto columns_str = settings[MergeTreeSetting::columns_to_prewarm_mark_cache].toString(); + if (columns_str.empty()) + return columns_list.getNames(); + + return parseIdentifiersOrStringLiterals(columns_str, Context::getGlobalContextInstance()->getSettingsRef()); +} + } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 2aa4474e1c5..76262f9cdf7 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -77,4 +77,13 @@ private: using MergeTreeMarksLoaderPtr = std::shared_ptr; +class IMergeTreeDataPart; +struct MergeTreeSettings; + +/// Adds computed marks for part to the marks cache. +void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & cached_marks, MarkCache * mark_cache); + +/// Returns the list of columns suitable for prewarming of mark cache according to settings. +Names getColumnsToPrewarmMarks(const MergeTreeSettings & settings, const NamesAndTypesList & columns_list); + } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 8c6aafe48f2..3d2c9c63598 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -232,6 +232,8 @@ namespace ErrorCodes DECLARE(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ DECLARE(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ DECLARE(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ + DECLARE(Bool, prewarm_mark_cache, false, "If true mark cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ + DECLARE(String, columns_to_prewarm_mark_cache, "", "List of columns to prewarm mark cache for (if enabled). Empty means all columns", 0) \ /** Projection settings. */ \ DECLARE(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ DECLARE(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 1e42f16736d..604112c26ea 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -243,6 +243,15 @@ void MergeTreeSink::finishDelayedChunk() /// Part can be deduplicated, so increment counters and add to part log only if it's really added if (added) { + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot)); StorageMergeTree::incrementInsertedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 4ee68580d3f..77c34aae30a 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -25,6 +25,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_, + bool save_marks_in_cache, bool blocks_are_granules_size, const WriteSettings & write_settings_, const MergeTreeIndexGranularity & computed_index_granularity) @@ -39,6 +40,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( storage_settings, data_part->index_granularity_info.mark_type.adaptive, /* rewrite_primary_key = */ true, + save_marks_in_cache, blocks_are_granules_size); /// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index e212fe5bb5a..060778866e0 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -24,6 +24,7 @@ public: CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_ = false, + bool save_marks_in_cache = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}, const MergeTreeIndexGranularity & computed_index_granularity = {}); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 05cd77dcd40..bed539dfe02 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -19,6 +19,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeIndices & indices_to_recalc, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_, + bool save_marks_in_cache, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) @@ -30,7 +31,9 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( data_part->storage.getContext()->getWriteSettings(), storage_settings, index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), - /* rewrite_primary_key = */ false); + /* rewrite_primary_key = */ false, + save_marks_in_cache, + /* blocks_are_granules_size = */ false); writer = createMergeTreeDataPartWriter( data_part->getType(), diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index e837a62743e..f6bf9e37a58 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -22,6 +22,7 @@ public: const MergeTreeIndices & indices_to_recalc_, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, + bool save_marks_in_cache = false, const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ee87051371c..753b0c5d2fe 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1623,6 +1623,7 @@ private: ctx->compression_codec, ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, + /*save_marks_in_cache=*/ false, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings(), computed_granularity); @@ -1851,6 +1852,7 @@ private: std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), nullptr, + /*save_marks_in_cache=*/ false, ctx->source_part->index_granularity, &ctx->source_part->index_granularity_info ); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 95469337f8a..fe20953a52f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -481,6 +481,17 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + auto * mark_cache = storage.getContext()->getMarkCache().get(); + + if (!error && mark_cache) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); StorageReplicatedMergeTree::incrementInsertedPartsProfileEvent(part->getType()); @@ -521,8 +532,18 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa { partition.temp_part.finalize(); auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num).first; + if (conflict_block_ids.empty()) { + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*partition.temp_part.part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart( storage.getContext(), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index abc66df0d8b..40cd6e01dba 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB @@ -154,6 +155,7 @@ StorageMergeTree::StorageMergeTree( loadMutations(); loadDeduplicationLog(); + prewarmMarkCache(getActivePartsLoadingThreadPool().get()); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 850623157a1..93e72f3e0bf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -103,6 +103,7 @@ #include #include +#include #include #include @@ -207,6 +208,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool use_minimalistic_checksums_in_zookeeper; extern const MergeTreeSettingsBool use_minimalistic_part_header_in_zookeeper; extern const MergeTreeSettingsMilliseconds wait_for_unique_parts_send_before_shutdown_ms; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace FailPoints @@ -507,6 +509,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } loadDataParts(skip_sanity_checks, expected_parts_on_this_replica); + prewarmMarkCache(getActivePartsLoadingThreadPool().get()); if (LoadingStrictnessLevel::ATTACH <= mode) { @@ -5079,6 +5082,12 @@ bool StorageReplicatedMergeTree::fetchPart( ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } + if ((*getSettings())[MergeTreeSetting::prewarm_mark_cache]) + { + auto column_names = getColumnsToPrewarmMarks(*getSettings(), part->getColumns()); + part->loadMarksToCache(column_names, getContext()->getMarkCache().get()); + } + write_part_log({}); } else diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..2dbd9f088bf 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -108,6 +108,7 @@ TABLE ENGINE ['TABLE ENGINE'] TABLE_ENGINE ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE +SYSTEM PREWARM MARK CACHE ['SYSTEM PREWARM MARK','PREWARM MARK CACHE','PREWARM MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MMAP CACHE ['SYSTEM DROP MMAP','DROP MMAP CACHE','DROP MMAP'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.reference b/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.reference new file mode 100644 index 00000000000..e3b4928b2f4 --- /dev/null +++ b/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.reference @@ -0,0 +1,6 @@ +1 +1 +1 +4 +4 +4 diff --git a/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.sql b/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.sql new file mode 100644 index 00000000000..4d04cee55d0 --- /dev/null +++ b/tests/queries/0_stateless/03254_prewarm_mark_cache_columns.sql @@ -0,0 +1,30 @@ +-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_prewarm_columns; + +CREATE TABLE t_prewarm_columns (a UInt64, b UInt64, c UInt64, d UInt64) +ENGINE = MergeTree ORDER BY a +SETTINGS min_bytes_for_wide_part = 0, prewarm_mark_cache = 1, columns_to_prewarm_mark_cache = 'a,c'; + +INSERT INTO t_prewarm_columns VALUES (1, 1, 1, 1); + +SELECT count() FROM t_prewarm_columns WHERE NOT ignore(*); + +SYSTEM DROP MARK CACHE; +DETACH TABLE t_prewarm_columns; +ATTACH TABLE t_prewarm_columns; + +SELECT count() FROM t_prewarm_columns WHERE NOT ignore(*); + +SYSTEM DROP MARK CACHE; +SYSTEM PREWARM MARK CACHE t_prewarm_columns; + +SELECT count() FROM t_prewarm_columns WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['LoadedMarksCount'] FROM system.query_log +WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_columns%' +ORDER BY event_time_microseconds; + +DROP TABLE t_prewarm_columns; diff --git a/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.reference b/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.reference new file mode 100644 index 00000000000..f1bdbd462be --- /dev/null +++ b/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.reference @@ -0,0 +1,16 @@ +20000 +20000 +40000 +40000 +40000 +40000 +40000 +40000 +0 +0 +0 +0 +0 +0 +1 +0 diff --git a/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.sql b/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.sql new file mode 100644 index 00000000000..97d18185115 --- /dev/null +++ b/tests/queries/0_stateless/03254_prewarm_mark_cache_rmt.sql @@ -0,0 +1,65 @@ +-- Tags: no-parallel, no-shared-merge-tree + +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; +DROP TABLE IF EXISTS t_prewarm_cache_rmt_2; + +CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03254_prewarm_mark_cache_smt/t_prewarm_cache', '1') +ORDER BY a SETTINGS prewarm_mark_cache = 1; + +CREATE TABLE t_prewarm_cache_rmt_2 (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03254_prewarm_mark_cache_smt/t_prewarm_cache', '2') +ORDER BY a SETTINGS prewarm_mark_cache = 1; + +SYSTEM DROP MARK CACHE; + +SYSTEM STOP FETCHES t_prewarm_cache_rmt_2; + +-- Check that prewarm works on insert. +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(20000); +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE NOT ignore(*); + +-- Check that prewarm works on fetch. +SYSTEM DROP MARK CACHE; +SYSTEM START FETCHES t_prewarm_cache_rmt_2; +SYSTEM SYNC REPLICA t_prewarm_cache_rmt_2; +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE NOT ignore(*); + +-- Check that prewarm works on merge. +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(20000); +OPTIMIZE TABLE t_prewarm_cache_rmt_1 FINAL; + +SYSTEM SYNC REPLICA t_prewarm_cache_rmt_2; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE NOT ignore(*); +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE NOT ignore(*); + +-- Check that prewarm works on restart. +SYSTEM DROP MARK CACHE; + +DETACH TABLE t_prewarm_cache_rmt_1; +DETACH TABLE t_prewarm_cache_rmt_2; + +ATTACH TABLE t_prewarm_cache_rmt_1; +ATTACH TABLE t_prewarm_cache_rmt_2; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE NOT ignore(*); +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE NOT ignore(*); + +SYSTEM DROP MARK CACHE; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE NOT ignore(*); + +--- Check that system query works. +SYSTEM PREWARM MARK CACHE t_prewarm_cache_rmt_1; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['LoadedMarksCount'] > 0 FROM system.query_log +WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache%' +ORDER BY event_time_microseconds; + +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; +DROP TABLE IF EXISTS t_prewarm_cache_rmt_2; From d1ac93ba170015335b7d95bb0a82aa243d82cf44 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 14:30:18 +0200 Subject: [PATCH 430/816] Small doc improvement. --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index b1d0de21046..5738a0e5761 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1977,9 +1977,9 @@ The default is `false`. ## async_load_system_database {#async_load_system_database} -Asynchronous loading of system tables. Helpful if there is high amount of log tables and parts in system database. Independent of `async_load_databases` setting. +Asynchronous loading of system tables. Helpful if there is a high amount of log tables and parts in the `system` database. Independent of the `async_load_databases` setting. -If `true` all system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.asynchronous_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a system table, that is not yet loaded, will wait for exactly this table to be started up. The table that is waited for by at least one query will be loaded with higher priority. Also consider setting a limit `max_waiting_queries` for the total number of waiting queries. +If set to `true`, all system databases with `Ordinary`, `Atomic`, and `Replicated` engines will be loaded asynchronously after the ClickHouse server starts. See `system.asynchronous_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a system table, that is not yet loaded, will wait for exactly this table to be started up. The table that is waited for by at least one query will be loaded with higher priority. Also consider setting the `max_waiting_queries` setting to limit the total number of waiting queries. If `false`, system database loads before server start. From ce3b2a00015e9d420fe35af673e2ef27199df389 Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Fri, 25 Oct 2024 15:33:23 +0300 Subject: [PATCH 431/816] Fix a test style --- .../test_reload_client_certificate/test.py | 35 +++++-------------- 1 file changed, 9 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index e12b5d4b35d..50ca243adac 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -1,14 +1,10 @@ import os - +import pytest import threading - import time -import pytest - from helpers.cluster import ClickHouseCluster - TEST_DIR = os.path.dirname(__file__) cluster = ClickHouseCluster( @@ -32,6 +28,7 @@ node1 = cluster.add_instance( ], with_zookeeper_secure=True, ) + node2 = cluster.add_instance( "node2", main_configs=[ @@ -55,13 +52,11 @@ def started_cluster(): try: cluster.start() yield cluster - finally: cluster.shutdown() - def secure_connection_test(started_cluster): - # no asserts, connection works + # No asserts, connection works node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") @@ -69,7 +64,7 @@ def secure_connection_test(started_cluster): iterations = 10 threads = [] - # just checking for race conditions + # Just checking for race conditions for _ in range(threads_number): threads.append( threading.Thread( @@ -90,13 +85,11 @@ def secure_connection_test(started_cluster): for thread in threads: thread.join() - def change_config_to_key(name): """ * Generate config with certificate/key name from args. * Reload config. """ - for node in nodes: node.exec_in_container( [ @@ -119,9 +112,7 @@ def change_config_to_key(name): -EOF""".format( - cur_name=name - ), +EOF""".format(cur_name=name), ] ) @@ -129,9 +120,7 @@ EOF""".format( ["bash", "-c", f"touch /etc/clickhouse-server/config.d/ssl_conf.xml"], ) - -def check_reload_successful( - node, cert_name): +def check_reload_successful(node, cert_name): return node.grep_in_log(f"Reloaded certificate (/etc/clickhouse-server/config.d/{cert_name}_client.crt)") def check_error_handshake(node): @@ -147,9 +136,7 @@ def clean_logs(): ] ) -def check_certificate_switch( - first, second -): +def check_certificate_switch(first, second): # Set first key change_config_to_key(first) @@ -180,17 +167,13 @@ def check_certificate_switch( else: check_connection = secure_connection_test(started_cluster) error_handshake = any(check_error_handshake(node) == "0\n" for node in nodes) - assert reload_successful and error_handshake + assert reload_successful and error_handshake def test_wrong_cn_cert(): """Checking the certificate reload with an incorrect CN, the expected behavior is Code: 210.""" check_certificate_switch("first", "second") - def test_correct_cn_cert(): """Replacement with a valid certificate, the expected behavior is to restore the connection with Zookeeper.""" - check_certificate_switch("second", "third") - - - + check_certificate_switch("second", "third") \ No newline at end of file From 2da1926338bd0f7347ef6f2ac3768984597c895f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 25 Oct 2024 09:37:22 -0300 Subject: [PATCH 432/816] tests --- ...03254_parquet_bool_native_reader.reference | 20 +++++++++++++++++ .../03254_parquet_bool_native_reader.sh | 21 ++++++++++++++++++ .../0_stateless/data_parquet/nullbool.parquet | Bin 0 -> 508 bytes 3 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03254_parquet_bool_native_reader.reference create mode 100755 tests/queries/0_stateless/03254_parquet_bool_native_reader.sh create mode 100644 tests/queries/0_stateless/data_parquet/nullbool.parquet diff --git a/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference b/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference new file mode 100644 index 00000000000..0c7e55ad234 --- /dev/null +++ b/tests/queries/0_stateless/03254_parquet_bool_native_reader.reference @@ -0,0 +1,20 @@ +0 false +1 \N +2 false +3 \N +4 false +5 \N +6 false +7 \N +8 true +9 \N +0 false +1 \N +2 false +3 \N +4 false +5 \N +6 false +7 \N +8 true +9 \N diff --git a/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh b/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh new file mode 100755 index 00000000000..c28523b3c54 --- /dev/null +++ b/tests/queries/0_stateless/03254_parquet_bool_native_reader.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-ubsan, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +WORKING_DIR="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}" + +mkdir -p "${WORKING_DIR}" + +DATA_FILE="${CUR_DIR}/data_parquet/nullbool.parquet" + +DATA_FILE_USER_PATH="${WORKING_DIR}/nullbool.parquet" + +cp ${DATA_FILE} ${DATA_FILE_USER_PATH} + +${CLICKHOUSE_CLIENT} --query="select id, bool from file('${DATA_FILE_USER_PATH}', Parquet) order by id SETTINGS input_format_parquet_use_native_reader=false;" +${CLICKHOUSE_CLIENT} --query="select id, bool from file('${DATA_FILE_USER_PATH}', Parquet) order by id SETTINGS input_format_parquet_use_native_reader=true;" diff --git a/tests/queries/0_stateless/data_parquet/nullbool.parquet b/tests/queries/0_stateless/data_parquet/nullbool.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d9b365bbe75bcd69ccee52c5eddffd3aa17b62cb GIT binary patch literal 508 zcmZ8e%}T>S5T1>-#va5%cE~~wy)?8?OJizlBow@fry}?OsadelSWT@zm!8Cz@Z<}4 z_TWi;1>eEDM?w5HxtZN>Co|v7>^JQA@Fb8V$5=m456@ekbl}?3rs5MgEnp3(0P8>% z*Z@q*CV>2HtnaVa&&{&DYRXN?`l;AfbxXOXmSBo}OED>Su&E#g7$GFWH52x0HBuhi za>b^|<3}M!<`)_9k)7Qy&dzm~$O-~Ya;<3!2~EqbOy=_$p@FA5zU?8qNwzY)M3h&& z^6j{kQ0if76@p3+H(?U=q_|y*BYv@@!yiH(kpCz^t39fsWpPu{bi^YtG32{~xYjD{ zzfNWz;&)3j{|eb7eiQ8YHjHD&bL+SH^jhcLY@X^__ae!(yP@xr>~f~bJ-$rxtEIl) z$@0dH&KJ}9MI-12{cg}`O_tMH+K7fa)zM%-91i=vUK*ssNQXf*7=(jK5~ezgcHvEu Qw(599*mQ(f9pl~q08?vBH2?qr literal 0 HcmV?d00001 From ac135cd33c5ba1e67634ec3af9d3cf58e875ef77 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 25 Oct 2024 12:37:48 +0000 Subject: [PATCH 433/816] address review comments --- .../Workload/WorkloadEntityDiskStorage.cpp | 69 ++++++++----------- .../Workload/WorkloadEntityKeeperStorage.cpp | 2 +- .../Workload/WorkloadEntityStorageBase.cpp | 16 ++--- 3 files changed, 36 insertions(+), 51 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 209d6f06100..2dd37809b12 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -47,6 +47,10 @@ namespace ErrorCodes namespace { + static constexpr std::string_view workload_prefix = "workload_"; + static constexpr std::string_view resource_prefix = "resource_"; + static constexpr std::string_view sql_suffix = ".sql"; + /// Converts a path to an absolute path and append it with a separator. String makeDirectoryPathCanonical(const String & directory_path) { @@ -86,34 +90,22 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, String entity_create_query; readStringUntilEOF(entity_create_query, in); + auto parse = [&] (auto parser) + { + return parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); + }; + switch (entity_type) { - case WorkloadEntityType::Workload: - { - ParserCreateWorkloadQuery parser; - ASTPtr ast = parseQuery( - parser, - entity_create_query.data(), - entity_create_query.data() + entity_create_query.size(), - "", - 0, - global_context->getSettingsRef()[Setting::max_parser_depth], - global_context->getSettingsRef()[Setting::max_parser_backtracks]); - return ast; - } - case WorkloadEntityType::Resource: - { - ParserCreateResourceQuery parser; - ASTPtr ast = parseQuery( - parser, - entity_create_query.data(), - entity_create_query.data() + entity_create_query.size(), - "", - 0, - global_context->getSettingsRef()[Setting::max_parser_depth], - global_context->getSettingsRef()[Setting::max_parser_backtracks]); - return ast; - } + case WorkloadEntityType::Workload: return parse(ParserCreateWorkloadQuery()); + case WorkloadEntityType::Resource: return parse(ParserCreateResourceQuery()); case WorkloadEntityType::MAX: return nullptr; } } @@ -152,11 +144,11 @@ void WorkloadEntityDiskStorage::loadEntitiesImpl() const String & file_name = it.name(); - if (startsWith(file_name, "workload_") && endsWith(file_name, ".sql")) + if (file_name.starts_with(workload_prefix) && file_name.ends_with(sql_suffix)) { - size_t prefix_length = strlen("workload_"); - size_t suffix_length = strlen(".sql"); - String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + String name = unescapeForFileName(file_name.substr( + workload_prefix.size(), + file_name.size() - workload_prefix.size() - sql_suffix.size())); if (name.empty()) continue; @@ -166,11 +158,11 @@ void WorkloadEntityDiskStorage::loadEntitiesImpl() entities_name_and_queries.emplace_back(name, ast); } - if (startsWith(file_name, "resource_") && endsWith(file_name, ".sql")) + if (file_name.starts_with(resource_prefix) && file_name.ends_with(sql_suffix)) { - size_t prefix_length = strlen("resource_"); - size_t suffix_length = strlen(".sql"); - String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + String name = unescapeForFileName(file_name.substr( + resource_prefix.size(), + file_name.size() - resource_prefix.size() - sql_suffix.size())); if (name.empty()) continue; @@ -219,17 +211,14 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::storeEntit return OperationResult::Failed; } - WriteBufferFromOwnString create_statement_buf; - formatAST(*create_entity_query, create_statement_buf, false); - writeChar('\n', create_statement_buf); - String create_statement = create_statement_buf.str(); String temp_file_path = file_path + ".tmp"; try { - WriteBufferFromFile out(temp_file_path, create_statement.size()); - writeString(create_statement, out); + WriteBufferFromFile out(temp_file_path); + formatAST(*create_entity_query, out, false); + writeChar('\n', out); out.next(); if (settings[Setting::fsync_metadata]) out.sync(); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index 95af88d5f77..4b60a7ec57e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -41,7 +41,7 @@ WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); if (zookeeper_path.back() == '/') - zookeeper_path.resize(zookeeper_path.size() - 1); + zookeeper_path.pop_back(); /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. if (zookeeper_path.front() != '/') diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index a42252b1b8e..edeab7f6a7d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -288,15 +288,11 @@ WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const { - std::lock_guard lock(mutex); - - auto it = entities.find(entity_name); - if (it == entities.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The workload entity name '{}' is not saved", - entity_name); - - return it->second; + if (auto result = tryGet(entity_name)) + return result; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The workload entity name '{}' is not saved", + entity_name); } ASTPtr WorkloadEntityStorageBase::tryGet(const String & entity_name) const @@ -513,7 +509,7 @@ scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChange std::vector current_state; { - std::unique_lock lock{mutex}; + std::lock_guard lock{mutex}; current_state = orderEntities(entities); std::lock_guard lock2{handlers->mutex}; From ca040906c3bca0e283fc5df57451d4d0805336b3 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 25 Oct 2024 13:37:12 +0000 Subject: [PATCH 434/816] Fix some ifdef issues --- .../DataLakes/DataLakeConfiguration.h | 8 +++--- .../registerStorageObjectStorage.cpp | 10 +++---- src/Storages/registerStorages.cpp | 3 +- .../TableFunctionObjectStorage.cpp | 28 +++++++++---------- 4 files changed, 24 insertions(+), 25 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 69968dff942..866ef24aa91 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -84,15 +84,15 @@ private: }; #if USE_AVRO -# if USE_AWS_S3 +#if USE_AWS_S3 using StorageS3IcebergConfiguration = DataLakeConfiguration; # endif -# if USE_AZURE_BLOB_STORAGE +#if USE_AZURE_BLOB_STORAGE using StorageAzureIcebergConfiguration = DataLakeConfiguration; # endif -# if USE_HDFS +#if USE_HDFS using StorageHDFSIcebergConfiguration = DataLakeConfiguration; # endif @@ -100,7 +100,7 @@ using StorageLocalIcebergConfiguration = DataLakeConfiguration; # endif #endif diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index b0122de3bf7..cb1826b2976 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -11,8 +11,6 @@ namespace DB { -#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -65,8 +63,6 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } -#endif - #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { @@ -236,10 +232,10 @@ void registerStorageIceberg(StorageFactory & factory) #endif -#if USE_AWS_S3 #if USE_PARQUET void registerStorageDeltaLake(StorageFactory & factory) { +#if USE_AWS_S3 factory.registerStorage( "DeltaLake", [&](const StorageFactory::Arguments & args) @@ -254,11 +250,13 @@ void registerStorageDeltaLake(StorageFactory & factory) .supports_schema_inference = true, .source_access_type = AccessType::S3, }); +#endif } #endif void registerStorageHudi(StorageFactory & factory) { +#if USE_AWS_S3 factory.registerStorage( "Hudi", [&](const StorageFactory::Arguments & args) @@ -273,6 +271,6 @@ void registerStorageHudi(StorageFactory & factory) .supports_schema_inference = true, .source_access_type = AccessType::S3, }); -} #endif } +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index cfd406ccbe2..4eb90955a6c 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -41,10 +41,11 @@ void registerStorageS3Queue(StorageFactory & factory); #if USE_PARQUET void registerStorageDeltaLake(StorageFactory & factory); #endif +#endif + #if USE_AVRO void registerStorageIceberg(StorageFactory & factory); #endif -#endif #if USE_AZURE_BLOB_STORAGE void registerStorageAzureQueue(StorageFactory & factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 509ef92e8b2..66c90b15c0b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -228,7 +228,7 @@ template class TableFunctionObjectStorage( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", @@ -242,23 +242,23 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) .categories{"DataLake"}}, .allow_readonly = false}); -# endif -# if USE_AZURE_BLOB_STORAGE +#endif +#if USE_AZURE_BLOB_STORAGE factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); -# endif -# if USE_HDFS +#endif +#if USE_HDFS factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on HDFS virtual filesystem.)", .examples{{"icebergHDFS", "SELECT * FROM icebergHDFS(url)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); -# endif +#endif factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", @@ -268,29 +268,31 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) } #endif -#if USE_AWS_S3 -# if USE_PARQUET +#if USE_PARQUET void registerTableFunctionDeltaLake(TableFunctionFactory & factory) { +#if USE_AWS_S3 factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the DeltaLake table stored on object store.)", .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); +#endif } -# endif +#endif void registerTableFunctionHudi(TableFunctionFactory & factory) { +#if USE_AWS_S3 factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Hudi table stored on object store.)", .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); -} #endif +} void registerDataLakeTableFunctions(TableFunctionFactory & factory) { @@ -298,11 +300,9 @@ void registerDataLakeTableFunctions(TableFunctionFactory & factory) #if USE_AVRO registerTableFunctionIceberg(factory); #endif -#if USE_AWS_S3 -# if USE_PARQUET +#if USE_PARQUET registerTableFunctionDeltaLake(factory); -# endif - registerTableFunctionHudi(factory); #endif + registerTableFunctionHudi(factory); } } From 3704af3d51b0b5318aedcfb5598b3c80357fc9c3 Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Fri, 25 Oct 2024 16:52:07 +0300 Subject: [PATCH 435/816] Fix a test style 2 --- .../test_reload_client_certificate/test.py | 53 ++++++++++++------- 1 file changed, 34 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index 50ca243adac..cc9d4614ad8 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -1,8 +1,9 @@ import os -import pytest import threading import time +import pytest + from helpers.cluster import ClickHouseCluster TEST_DIR = os.path.dirname(__file__) @@ -47,6 +48,7 @@ node2 = cluster.add_instance( nodes = [node1, node2] + @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: @@ -55,8 +57,10 @@ def started_cluster(): finally: cluster.shutdown() + def secure_connection_test(started_cluster): # No asserts, connection works + node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") @@ -65,30 +69,26 @@ def secure_connection_test(started_cluster): threads = [] # Just checking for race conditions + for _ in range(threads_number): threads.append( threading.Thread( - target=( - lambda: [ - node1.query( - "SELECT count() FROM system.zookeeper WHERE path = '/'" - ) - for _ in range(iterations) - ] - ) + target=lambda: [ + node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") + for _ in range(iterations) + ] ) ) - for thread in threads: thread.start() - for thread in threads: thread.join() + def change_config_to_key(name): """ - * Generate config with certificate/key name from args. - * Reload config. + Generate config with certificate/key name from args. + Reload config. """ for node in nodes: node.exec_in_container( @@ -112,20 +112,27 @@ def change_config_to_key(name): -EOF""".format(cur_name=name), +EOF""".format( + cur_name=name + ), ] ) node.exec_in_container( - ["bash", "-c", f"touch /etc/clickhouse-server/config.d/ssl_conf.xml"], + ["bash", "-c", "touch /etc/clickhouse-server/config.d/ssl_conf.xml"], ) + def check_reload_successful(node, cert_name): - return node.grep_in_log(f"Reloaded certificate (/etc/clickhouse-server/config.d/{cert_name}_client.crt)") + return node.grep_in_log( + f"Reloaded certificate (/etc/clickhouse-server/config.d/{cert_name}_client.crt)" + ) + def check_error_handshake(node): return node.count_in_log("Code: 210.") + def clean_logs(): for node in nodes: node.exec_in_container( @@ -136,26 +143,33 @@ def clean_logs(): ] ) + def check_certificate_switch(first, second): # Set first key + change_config_to_key(first) # Restart zookeeper to reload the session + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) clean_logs() # Change to wrong certificate + change_config_to_key(second) # Time to log + time.sleep(10) # Check information about client certificates reloading in log + reload_successful = any(check_reload_successful(node, second) for node in nodes) # Restart zookeeper to reload the session and clean logs for new check + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) clean_logs() @@ -165,15 +179,16 @@ def check_certificate_switch(first, second): time.sleep(10) error_handshake = any(check_error_handshake(node) != "0\n" for node in nodes) else: - check_connection = secure_connection_test(started_cluster) + secure_connection_test(started_cluster) error_handshake = any(check_error_handshake(node) == "0\n" for node in nodes) - assert reload_successful and error_handshake + def test_wrong_cn_cert(): """Checking the certificate reload with an incorrect CN, the expected behavior is Code: 210.""" check_certificate_switch("first", "second") + def test_correct_cn_cert(): """Replacement with a valid certificate, the expected behavior is to restore the connection with Zookeeper.""" - check_certificate_switch("second", "third") \ No newline at end of file + check_certificate_switch("second", "third") From aeb2e2b1d84d35bc99d543f70109f1571459cd45 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 25 Oct 2024 16:00:41 +0200 Subject: [PATCH 436/816] Sync changes to ProtocolServerAdapter --- programs/server/Server.cpp | 2 +- src/Server/ProtocolServerAdapter.cpp | 8 ++++++-- src/Server/ProtocolServerAdapter.h | 17 +++++++++++++++-- 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 15585ac8d57..804e6716179 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2999,7 +2999,7 @@ void Server::updateServers( for (auto * server : all_servers) { - if (!server->isStopping()) + if (server->supportsRuntimeReconfiguration() && !server->isStopping()) { std::string port_name = server->getPortName(); bool has_host = false; diff --git a/src/Server/ProtocolServerAdapter.cpp b/src/Server/ProtocolServerAdapter.cpp index 6b723bc8d87..3abf5733c52 100644 --- a/src/Server/ProtocolServerAdapter.cpp +++ b/src/Server/ProtocolServerAdapter.cpp @@ -30,11 +30,13 @@ ProtocolServerAdapter::ProtocolServerAdapter( const std::string & listen_host_, const char * port_name_, const std::string & description_, - std::unique_ptr tcp_server_) + std::unique_ptr tcp_server_, + bool supports_runtime_reconfiguration_) : listen_host(listen_host_) , port_name(port_name_) , description(description_) , impl(std::make_unique(std::move(tcp_server_))) + , supports_runtime_reconfiguration(supports_runtime_reconfiguration_) { } @@ -66,11 +68,13 @@ ProtocolServerAdapter::ProtocolServerAdapter( const std::string & listen_host_, const char * port_name_, const std::string & description_, - std::unique_ptr grpc_server_) + std::unique_ptr grpc_server_, + bool supports_runtime_reconfiguration_) : listen_host(listen_host_) , port_name(port_name_) , description(description_) , impl(std::make_unique(std::move(grpc_server_))) + , supports_runtime_reconfiguration(supports_runtime_reconfiguration_) { } #endif diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index 4a0b0cae8e7..132a9b93c1b 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -21,10 +21,20 @@ class ProtocolServerAdapter public: ProtocolServerAdapter(ProtocolServerAdapter && src) = default; ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default; - ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr tcp_server_); + ProtocolServerAdapter( + const std::string & listen_host_, + const char * port_name_, + const std::string & description_, + std::unique_ptr tcp_server_, + bool supports_runtime_reconfiguration_ = true); #if USE_GRPC - ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr grpc_server_); + ProtocolServerAdapter( + const std::string & listen_host_, + const char * port_name_, + const std::string & description_, + std::unique_ptr grpc_server_, + bool supports_runtime_reconfiguration_ = true); #endif /// Starts the server. A new thread will be created that waits for and accepts incoming connections. @@ -46,6 +56,8 @@ public: /// Returns the port this server is listening to. UInt16 portNumber() const { return impl->portNumber(); } + bool supportsRuntimeReconfiguration() const { return supports_runtime_reconfiguration; } + const std::string & getListenHost() const { return listen_host; } const std::string & getPortName() const { return port_name; } @@ -72,6 +84,7 @@ private: std::string port_name; std::string description; std::unique_ptr impl; + bool supports_runtime_reconfiguration = true; }; } From 54a00e875160d87a3793c60d281bd321a5826aec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Oct 2024 14:01:55 +0000 Subject: [PATCH 437/816] fix optimization of replacing algorithm --- .../Merges/Algorithms/ReplacingSortedAlgorithm.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 0ce626b1dc9..b22f1271687 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -132,15 +132,15 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() /// and current chunk has no duplicates (we assume that parts with non-zero level have no duplicates) /// We want to insert current cursor chunk directly in merged data. - size_t source_num = current->order; - auto current_chunk = std::move(*sources[source_num].chunk); - size_t chunk_num_rows = current_chunk.getNumRows(); - /// First if merged_data is not empty we need to flush it. /// We will get into the same condition on next merge call. if (merged_data->mergedRows() != 0) return Status(merged_data->pull()); + size_t source_num = current->order; + auto current_chunk = std::move(*sources[source_num].chunk); + size_t chunk_num_rows = current_chunk.getNumRows(); + /// We will get the next block from the corresponding source, if there is one. queue.removeTop(); From 82a57e36678d7c1cd8b0897706e2158848f9a0e8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 25 Oct 2024 16:05:28 +0200 Subject: [PATCH 438/816] Uniform documentation of setting --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7f015a53908..f7e87a36833 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -111,7 +111,7 @@ static std::initializer_list Date: Fri, 25 Oct 2024 16:21:35 +0200 Subject: [PATCH 439/816] Extend test --- .../integration/test_storage_s3_queue/test.py | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 647a54ff95a..4e7c459e1ed 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -2188,6 +2188,14 @@ def test_alter_settings(started_cluster): f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" ) + assert '"tracked_files_ttl_sec":10000' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"tracked_files_limit":50' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + node1.restart_clickhouse() assert '"processing_threads_num":5' in node1.query( @@ -2202,9 +2210,17 @@ def test_alter_settings(started_cluster): f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" ) + assert '"tracked_files_ttl_sec":10000' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"tracked_files_limit":50' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + node1.query( f""" - ALTER TABLE r.{table_name} RESET SETTING after_processing + ALTER TABLE r.{table_name} RESET SETTING after_processing, tracked_file_ttl_sec """ ) @@ -2220,5 +2236,25 @@ def test_alter_settings(started_cluster): f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" ) + assert '"tracked_files_ttl_sec":0' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + node1.restart_clickhouse() assert expected_rows == get_count() + + assert '"processing_threads_num":5' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"loading_retries":10' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"after_processing":"keep"' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + + assert '"tracked_files_ttl_sec":0' in node1.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) From 72c4526e32e1a9935f5a7a63e95a0e0aff7fa032 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Fri, 25 Oct 2024 15:02:31 +0100 Subject: [PATCH 440/816] Implement review suggestions --- src/Interpreters/Session.cpp | 25 +++++++++---------- ..._long_sessions_in_http_interface.reference | 5 ---- .../00463_long_sessions_in_http_interface.sh | 11 -------- ..._expire_in_use_in_http_interface.reference | 5 ++++ ...session_expire_in_use_in_http_interface.sh | 18 +++++++++++++ 5 files changed, 35 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference create mode 100755 tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 57748e79fdf..1faf6418128 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -140,10 +140,11 @@ public: if (session->close_time_bucket != std::chrono::steady_clock::time_point{}) { - auto & bucket_sessions = close_time_buckets[session->close_time_bucket]; - bucket_sessions.erase(std::ranges::remove(bucket_sessions, key).begin(), bucket_sessions.end()); + auto bucket_it = close_time_buckets.find(session->close_time_bucket); + auto & bucket_sessions = bucket_it->second; + bucket_sessions.erase(key); if (bucket_sessions.empty()) - close_time_buckets.erase(session->close_time_bucket); + close_time_buckets.erase(bucket_it); session->close_time_bucket = std::chrono::steady_clock::time_point{}; } @@ -193,8 +194,8 @@ private: using Container = std::unordered_map, SessionKeyHash>; Container sessions; - // Ordered map of close times for sessions, groupped by the next multiple of close_interval - using CloseTimes = std::map>; + // Ordered map of close times for sessions, grouped by the next multiple of close_interval + using CloseTimes = std::map>; CloseTimes close_time_buckets; constexpr static std::chrono::steady_clock::duration close_interval = std::chrono::milliseconds(1000); @@ -210,10 +211,10 @@ private: const auto close_time_bucket = session_close_time + bucket_padding; session.close_time_bucket = close_time_bucket; - auto it = close_time_buckets.insert(std::make_pair(close_time_bucket, std::vector{})); - it.first->second.push_back(session.key); + auto it = close_time_buckets.insert(std::make_pair(close_time_bucket, std::set{})); + it.first->second.insert(session.key); - LOG_TRACE(log, "Schedule closing session with session_id: {}, user_id: {}", + LOG_TEST(log, "Schedule closing session with session_id: {}, user_id: {}", session.key.second, session.key.first); } @@ -233,9 +234,9 @@ private: { const auto now = std::chrono::steady_clock::now(); - while (!close_time_buckets.empty()) + for (auto bucket_it = close_time_buckets.begin(); bucket_it != close_time_buckets.end(); bucket_it = close_time_buckets.erase(bucket_it)) { - const auto & [time_bucket, session_keys] = *close_time_buckets.begin(); + const auto & [time_bucket, session_keys] = *bucket_it; if (time_bucket > now) break; @@ -250,7 +251,7 @@ private: if (session.use_count() != 1) { - LOG_TRACE(log, "Delay closing session with session_id: {}, user_id: {}, refcount: {}", + LOG_TEST(log, "Delay closing session with session_id: {}, user_id: {}, refcount: {}", key.second, key.first, session.use_count()); session->timeout = std::chrono::steady_clock::duration{0}; @@ -262,8 +263,6 @@ private: sessions.erase(session_it); } - - close_time_buckets.erase(close_time_buckets.begin()); } } diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference index 031ad768aae..a14d334a483 100644 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.reference @@ -26,8 +26,3 @@ HelloWorld A session cannot be used by concurrent connections: 1 1 -A session successfully closes when timeout first expires with refcount != 1 and another session is created in between -45 -45 -1 -1 diff --git a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh index d2451d0b3d8..86902fca4aa 100755 --- a/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh +++ b/tests/queries/0_stateless/00463_long_sessions_in_http_interface.sh @@ -85,14 +85,3 @@ done ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT 1" | grep -c -F 'SESSION_IS_LOCKED' ${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '${CLICKHOUSE_DATABASE}_9' SYNC FORMAT Null"; wait - -echo "A session successfully closes when timeout first expires with refcount != 1 and another session is created in between" -# Here we do not want an infinite loop - because we want this mechanism to be reliable in all cases -# So it's better to give it enough time to complete even in constrained environments -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE x (n UInt64) AS SELECT number FROM numbers(10)" -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "SELECT sum(n + sleep(3)) FROM x" # This query ensures timeout expires with refcount > 1 -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE y (n UInt64) AS SELECT number FROM numbers(10)" -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "SELECT sum(n) FROM y" -sleep 15 -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' \ No newline at end of file diff --git a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference new file mode 100644 index 00000000000..4c9a93358e2 --- /dev/null +++ b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference @@ -0,0 +1,5 @@ +A session successfully closes when timeout first expires with refcount != 1 and another session is created in between +45 +45 +1 +1 diff --git a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh new file mode 100755 index 00000000000..37f7279a932 --- /dev/null +++ b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel +# shellcheck disable=SC2015 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "A session successfully closes when timeout first expires with refcount != 1 and another session is created in between" +# Here we do not want an infinite loop - because we want this mechanism to be reliable in all cases +# So it's better to give it enough time to complete even in constrained environments +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE x (n UInt64) AS SELECT number FROM numbers(10)" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "SELECT sum(n + sleep(3)) FROM x" # This query ensures timeout expires with refcount > 1 +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE y (n UInt64) AS SELECT number FROM numbers(10)" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "SELECT sum(n) FROM y" +sleep 15 +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' From 813d715589e4aaa711068014e09a9e234b54c2aa Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 17:13:08 +0200 Subject: [PATCH 441/816] [Intermediate commit] --- src/Interpreters/Set.cpp | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 42a92bc7809..25e6c79af49 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -280,7 +280,7 @@ void Set::checkIsCreated() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built."); } -ColumnPtr checkDateTimePrecision(const ColumnWithTypeAndName & column_to_cast) +ColumnUInt8::Ptr checkDateTimePrecision(const ColumnWithTypeAndName & column_to_cast) { // Handle nullable columns const ColumnNullable * original_nullable_column = typeid_cast(column_to_cast.column.get()); @@ -320,7 +320,7 @@ ColumnPtr checkDateTimePrecision(const ColumnWithTypeAndName & column_to_cast) return precision_null_map_column; } -ColumnPtr mergeNullMaps(const ColumnPtr & null_map_column1, const ColumnPtr & null_map_column2) +ColumnPtr mergeNullMaps(const ColumnPtr & null_map_column1, const ColumnUInt8::Ptr & null_map_column2) { if (!null_map_column1) return null_map_column2; @@ -328,7 +328,7 @@ ColumnPtr mergeNullMaps(const ColumnPtr & null_map_column1, const ColumnPtr & nu return null_map_column1; const auto & null_map1 = assert_cast(*null_map_column1).getData(); - const auto & null_map2 = assert_cast(*null_map_column2).getData(); + const auto & null_map2 = (*null_map_column2).getData(); size_t size = null_map1.size(); if (size != null_map2.size()) @@ -406,7 +406,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co // If the original column is DateTime64, check for sub-second precision if (isDateTime64(column_to_cast.column->getDataType())) { - ColumnPtr filtered_null_map_column = checkDateTimePrecision(column_to_cast); + ColumnUInt8::Ptr filtered_null_map_column = checkDateTimePrecision(column_to_cast); // Extract existing null map and nested column from the result const ColumnNullable * result_nullable_column = typeid_cast(result.get()); @@ -418,16 +418,16 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co ? result_nullable_column->getNullMapColumnPtr() : nullptr; - ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); - - result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); - if (transform_null_in) { - ColumnRawPtrs key_cols{result.get()}; - null_map_holder = extractNestedColumnsAndNullMap(key_cols, null_map); + null_map_holder = filtered_null_map_column; + null_map = &filtered_null_map_column->getData(); + } + else + { + ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); - result = nested_result_column->getPtr(); /// The result is considered not nullable in HashMethodOneNumber + result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); } } From 2c10b5df6d3fdf99616c7c5484ae39d59bd1ec73 Mon Sep 17 00:00:00 2001 From: ortyomka Date: Fri, 25 Oct 2024 15:55:26 +0000 Subject: [PATCH 442/816] trigger ci From 9c5423523e475fa181a75541d5b24ab50e815c33 Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Fri, 25 Oct 2024 19:28:05 +0300 Subject: [PATCH 443/816] The test has been changed --- .../test_reload_client_certificate/test.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index cc9d4614ad8..18191a12581 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -65,7 +65,7 @@ def secure_connection_test(started_cluster): node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") threads_number = 4 - iterations = 10 + iterations = 4 threads = [] # Just checking for race conditions @@ -156,7 +156,7 @@ def check_certificate_switch(first, second): cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) clean_logs() - # Change to wrong certificate + # Change certificate change_config_to_key(second) @@ -176,12 +176,15 @@ def check_certificate_switch(first, second): cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) if second == "second": - time.sleep(10) - error_handshake = any(check_error_handshake(node) != "0\n" for node in nodes) + try: + secure_connection_test(started_cluster) + assert False + except: + assert True else: secure_connection_test(started_cluster) error_handshake = any(check_error_handshake(node) == "0\n" for node in nodes) - assert reload_successful and error_handshake + assert reload_successful and error_handshake def test_wrong_cn_cert(): From 4a8842be3dc1739e789f2ad238463610dc1460b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 18:34:20 +0200 Subject: [PATCH 444/816] Fix a typo --- src/Interpreters/AggregationCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 43c80d361d1..7df49153051 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -88,7 +88,7 @@ void fillFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const S out.resize_fill(num_rows); /// Note: here we violate strict aliasing. - /// It should be ok as log as we do not reffer to any value from `out` before filling. + /// It should be ok as log as we do not refer to any value from `out` before filling. const char * source = static_cast(column)->getRawDataBegin(); T * dest = reinterpret_cast(reinterpret_cast(out.data()) + offset); fillFixedBatch(num_rows, reinterpret_cast(source), dest); /// NOLINT(bugprone-sizeof-expression) From a2edc4691e3dfa44e3ff29fcfeda18940fcde757 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 18:34:56 +0200 Subject: [PATCH 445/816] Add merging of nullmap for multiple dt64 + add appropriate tests. --- src/Interpreters/Set.cpp | 12 +++---- ...8_datetime_cast_losing_precision.reference | 4 ++- .../03208_datetime_cast_losing_precision.sql | 36 +++++++++++++++---- 3 files changed, 39 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 25e6c79af49..f6880973743 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -387,9 +387,6 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co { ColumnPtr result; - null_map = ConstNullMapPtr(); - null_map_holder = nullptr; - const auto & column_before_cast = columns.at(i); ColumnWithTypeAndName column_to_cast = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name}; @@ -420,13 +417,16 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co if (transform_null_in) { - null_map_holder = filtered_null_map_column; - null_map = &filtered_null_map_column->getData(); + if (!null_map_holder) + null_map_holder = filtered_null_map_column; + else + null_map_holder = mergeNullMaps(null_map_holder, filtered_null_map_column); + + null_map = &assert_cast(*null_map_holder).getData(); } else { ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); - result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); } } diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference index a22e639726e..a5d609400ff 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -2,7 +2,9 @@ 0 0 0 -\N +ᴺᵁᴸᴸ 0 1 0 +0 +0 diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql index 042c9cacd2d..2e2c7009c2e 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.sql @@ -1,19 +1,43 @@ -with toDateTime('2024-10-16 18:00:30') as t +WITH toDateTime('2024-10-16 18:00:30') as t SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (SELECT t) settings transform_null_in=0; -with toDateTime('2024-10-16 18:00:30') as t +WITH toDateTime('2024-10-16 18:00:30') as t SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (SELECT t) settings transform_null_in=1; -with toDateTime('1970-01-01 00:00:01') as t +WITH toDateTime('1970-01-01 00:00:01') as t SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (now(), Null) settings transform_null_in=1; -with toDateTime('1970-01-01 00:00:01') as t +WITH toDateTime('1970-01-01 00:00:01') as t SELECT toDateTime64(t, 3) + interval 100 milliseconds IN (now(), Null) settings transform_null_in=0; -with toDateTime('1970-01-01 00:00:01') as t, +WITH toDateTime('1970-01-01 00:00:01') as t, arrayJoin([Null, toDateTime64(t, 3) + interval 100 milliseconds]) as x SELECT x IN (now(), Null) settings transform_null_in=0; -with toDateTime('1970-01-01 00:00:01') as t, +WITH toDateTime('1970-01-01 00:00:01') as t, arrayJoin([Null, toDateTime64(t, 3) + interval 100 milliseconds]) as x SELECT x IN (now(), Null) settings transform_null_in=1; + +WITH toDateTime('2024-10-16 18:00:30') as t +SELECT ( + SELECT + toDateTime64(t, 3) + interval 100 milliseconds, + toDateTime64(t, 3) + interval 101 milliseconds +) +IN ( + SELECT + t, + t +) SETTINGS transform_null_in=0; + +WITH toDateTime('2024-10-16 18:00:30') as t +SELECT ( + SELECT + toDateTime64(t, 3) + interval 100 milliseconds, + toDateTime64(t, 3) + interval 101 milliseconds +) +IN ( + SELECT + t, + t +) SETTINGS transform_null_in=1; From c19b06188c3700884341acd5615a88cae2de9551 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 25 Oct 2024 18:48:33 +0200 Subject: [PATCH 446/816] Fix test reference. --- .../0_stateless/03208_datetime_cast_losing_precision.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference index a5d609400ff..664ea35f7f6 100644 --- a/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference +++ b/tests/queries/0_stateless/03208_datetime_cast_losing_precision.reference @@ -2,7 +2,7 @@ 0 0 0 -ᴺᵁᴸᴸ +\N 0 1 0 From d88ae625f36905a46e0e6efae6963614af3efae7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 19:04:00 +0200 Subject: [PATCH 447/816] Update src/Interpreters/AggregationCommon.h Co-authored-by: Konstantin Bogdanov --- src/Interpreters/AggregationCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 7df49153051..8a81f4d4614 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -88,7 +88,7 @@ void fillFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const S out.resize_fill(num_rows); /// Note: here we violate strict aliasing. - /// It should be ok as log as we do not refer to any value from `out` before filling. + /// It should be ok as long as we do not refer to any value from `out` before filling. const char * source = static_cast(column)->getRawDataBegin(); T * dest = reinterpret_cast(reinterpret_cast(out.data()) + offset); fillFixedBatch(num_rows, reinterpret_cast(source), dest); /// NOLINT(bugprone-sizeof-expression) From 2206da6e66ebd5a22f4986ef3819836a1aa2a0f2 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 25 Oct 2024 17:13:11 +0000 Subject: [PATCH 448/816] fix race in unittest: use-after-scope --- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 10 +++++++++- .../Nodes/tests/gtest_io_resource_manager.cpp | 6 ++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 3fcbea55ee1..28a070a11a6 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -372,9 +372,17 @@ struct ResourceTestManager : public ResourceTestBase {} ~ResourceTestManager() + { + wait(); + } + + void wait() { for (auto & thread : threads) - thread.join(); + { + if (thread.joinable()) + thread.join(); + } } void update(const String & xml) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 15cd6436c47..51c2b69c705 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -256,6 +256,8 @@ TEST(SchedulerIOResourceManager, Fairness) ClassifierPtr c = t.manager->acquire("leader"); ResourceLink link = c->get("res1"); t.blockResource(link); + + t.wait(); // Wait for threads to finish before destructing locals } TEST(SchedulerIOResourceManager, DropNotEmptyQueue) @@ -289,6 +291,8 @@ TEST(SchedulerIOResourceManager, DropNotEmptyQueue) sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child t.query("CREATE WORKLOAD leaf IN intermediate"); sync_after_drop.arrive_and_wait(); + + t.wait(); // Wait for threads to finish before destructing locals } TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) @@ -326,4 +330,6 @@ TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child t.query("CREATE WORKLOAD leaf IN intermediate"); sync_after_drop.arrive_and_wait(); + + t.wait(); // Wait for threads to finish before destructing locals } From a7b23292f962eada087b2b7518c231b57ca71493 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Fri, 25 Oct 2024 17:58:43 +0000 Subject: [PATCH 449/816] add staleness to sql --- src/Analyzer/QueryTreeBuilder.cpp | 2 ++ src/Analyzer/Resolve/QueryAnalyzer.cpp | 43 ++++++++++++++++++++++-- src/Analyzer/Resolve/QueryAnalyzer.h | 3 +- src/Analyzer/SortNode.cpp | 8 +++++ src/Analyzer/SortNode.h | 21 +++++++++++- src/Parsers/ASTOrderByElement.cpp | 5 +++ src/Parsers/ASTOrderByElement.h | 3 ++ src/Parsers/CommonParsers.h | 1 + src/Parsers/ExpressionElementParsers.cpp | 6 ++++ src/Planner/Planner.cpp | 3 ++ src/Planner/PlannerActionsVisitor.cpp | 3 ++ src/Planner/PlannerSorting.cpp | 24 +++++++++++-- 12 files changed, 115 insertions(+), 7 deletions(-) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 39c59d27e2c..d3c88d39213 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -498,6 +498,8 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express sort_node->getFillTo() = buildExpression(order_by_element.getFillTo(), context); if (order_by_element.getFillStep()) sort_node->getFillStep() = buildExpression(order_by_element.getFillStep(), context); + if (order_by_element.getFillStaleness()) + sort_node->getFillStaleness() = buildExpression(order_by_element.getFillStaleness(), context); list_node->getNodes().push_back(std::move(sort_node)); } diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 381edee607d..ab29373f5fb 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -432,8 +432,13 @@ ProjectionName QueryAnalyzer::calculateWindowProjectionName(const QueryTreeNodeP return buffer.str(); } -ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeNodePtr & sort_column_node, const ProjectionName & sort_expression_projection_name, - const ProjectionName & fill_from_expression_projection_name, const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name) +ProjectionName QueryAnalyzer::calculateSortColumnProjectionName( + const QueryTreeNodePtr & sort_column_node, + const ProjectionName & sort_expression_projection_name, + const ProjectionName & fill_from_expression_projection_name, + const ProjectionName & fill_to_expression_projection_name, + const ProjectionName & fill_step_expression_projection_name, + const ProjectionName & fill_staleness_expression_projection_name) { auto & sort_node_typed = sort_column_node->as(); @@ -463,6 +468,9 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN if (sort_node_typed.hasFillStep()) sort_column_projection_name_buffer << " STEP " << fill_step_expression_projection_name; + + if (sort_node_typed.hasFillStaleness()) + sort_column_projection_name_buffer << " STALENESS " << fill_staleness_expression_projection_name; } return sort_column_projection_name_buffer.str(); @@ -3993,6 +4001,7 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_ ProjectionNames fill_from_expression_projection_names; ProjectionNames fill_to_expression_projection_names; ProjectionNames fill_step_expression_projection_names; + ProjectionNames fill_staleness_expression_projection_names; auto & sort_node_list_typed = sort_node_list->as(); for (auto & node : sort_node_list_typed.getNodes()) @@ -4083,11 +4092,38 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_ fill_step_expression_projection_names_size); } + if (sort_node.hasFillStaleness()) + { + fill_staleness_expression_projection_names = resolveExpressionNode(sort_node.getFillStaleness(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + const auto * constant_node = sort_node.getFillStaleness()->as(); + if (!constant_node) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Sort FILL STALENESS expression must be constant with numeric or interval type. Actual {}. In scope {}", + sort_node.getFillStaleness()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + bool is_number = isColumnedAsNumber(constant_node->getResultType()); + bool is_interval = WhichDataType(constant_node->getResultType()).isInterval(); + if (!is_number && !is_interval) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Sort FILL STALENESS expression must be constant with numeric or interval type. Actual {}. In scope {}", + sort_node.getFillStaleness()->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + size_t fill_staleness_expression_projection_names_size = fill_staleness_expression_projection_names.size(); + if (fill_staleness_expression_projection_names_size != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Sort FILL STALENESS expression expected 1 projection name. Actual {}", + fill_staleness_expression_projection_names_size); + } + auto sort_column_projection_name = calculateSortColumnProjectionName(node, sort_expression_projection_names[0], fill_from_expression_projection_names.empty() ? "" : fill_from_expression_projection_names.front(), fill_to_expression_projection_names.empty() ? "" : fill_to_expression_projection_names.front(), - fill_step_expression_projection_names.empty() ? "" : fill_step_expression_projection_names.front()); + fill_step_expression_projection_names.empty() ? "" : fill_step_expression_projection_names.front(), + fill_staleness_expression_projection_names.empty() ? "" : fill_staleness_expression_projection_names.front()); result_projection_names.push_back(std::move(sort_column_projection_name)); @@ -4095,6 +4131,7 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_ fill_from_expression_projection_names.clear(); fill_to_expression_projection_names.clear(); fill_step_expression_projection_names.clear(); + fill_staleness_expression_projection_names.clear(); } return result_projection_names; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.h b/src/Analyzer/Resolve/QueryAnalyzer.h index 0d4309843e6..d24bede561e 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.h +++ b/src/Analyzer/Resolve/QueryAnalyzer.h @@ -140,7 +140,8 @@ private: const ProjectionName & sort_expression_projection_name, const ProjectionName & fill_from_expression_projection_name, const ProjectionName & fill_to_expression_projection_name, - const ProjectionName & fill_step_expression_projection_name); + const ProjectionName & fill_step_expression_projection_name, + const ProjectionName & fill_staleness_expression_projection_name); QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); diff --git a/src/Analyzer/SortNode.cpp b/src/Analyzer/SortNode.cpp index e891046626a..42c010e4784 100644 --- a/src/Analyzer/SortNode.cpp +++ b/src/Analyzer/SortNode.cpp @@ -69,6 +69,12 @@ void SortNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, si buffer << '\n' << std::string(indent + 2, ' ') << "FILL STEP\n"; getFillStep()->dumpTreeImpl(buffer, format_state, indent + 4); } + + if (hasFillStaleness()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "FILL STALENESS\n"; + getFillStaleness()->dumpTreeImpl(buffer, format_state, indent + 4); + } } bool SortNode::isEqualImpl(const IQueryTreeNode & rhs, CompareOptions) const @@ -132,6 +138,8 @@ ASTPtr SortNode::toASTImpl(const ConvertToASTOptions & options) const result->setFillTo(getFillTo()->toAST(options)); if (hasFillStep()) result->setFillStep(getFillStep()->toAST(options)); + if (hasFillStaleness()) + result->setFillStaleness(getFillStaleness()->toAST(options)); return result; } diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index 0ebdde61912..d9086dc9ed7 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -105,6 +105,24 @@ public: return children[fill_step_child_index]; } + /// Returns true if sort node has fill step, false otherwise + bool hasFillStaleness() const + { + return children[fill_staleness_child_index] != nullptr; + } + + /// Get fill step + const QueryTreeNodePtr & getFillStaleness() const + { + return children[fill_staleness_child_index]; + } + + /// Get fill step + QueryTreeNodePtr & getFillStaleness() + { + return children[fill_staleness_child_index]; + } + /// Get collator const std::shared_ptr & getCollator() const { @@ -144,7 +162,8 @@ private: static constexpr size_t fill_from_child_index = 1; static constexpr size_t fill_to_child_index = 2; static constexpr size_t fill_step_child_index = 3; - static constexpr size_t children_size = fill_step_child_index + 1; + static constexpr size_t fill_staleness_child_index = 4; + static constexpr size_t children_size = fill_staleness_child_index + 1; SortDirection sort_direction = SortDirection::ASCENDING; std::optional nulls_sort_direction; diff --git a/src/Parsers/ASTOrderByElement.cpp b/src/Parsers/ASTOrderByElement.cpp index 09193a8b5e1..d87c296d398 100644 --- a/src/Parsers/ASTOrderByElement.cpp +++ b/src/Parsers/ASTOrderByElement.cpp @@ -54,6 +54,11 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : ""); fill_step->formatImpl(settings, state, frame); } + if (auto fill_staleness = getFillStaleness()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " STALENESS " << (settings.hilite ? hilite_none : ""); + fill_staleness->formatImpl(settings, state, frame); + } } } diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 6edf84d7bde..4dc35dac217 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -18,6 +18,7 @@ private: FILL_FROM, FILL_TO, FILL_STEP, + FILL_STALENESS, }; public: @@ -32,12 +33,14 @@ public: void setFillFrom(ASTPtr node) { setChild(Child::FILL_FROM, node); } void setFillTo(ASTPtr node) { setChild(Child::FILL_TO, node); } void setFillStep(ASTPtr node) { setChild(Child::FILL_STEP, node); } + void setFillStaleness(ASTPtr node) { setChild(Child::FILL_STALENESS, node); } /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ ASTPtr getCollation() const { return getChild(Child::COLLATION); } ASTPtr getFillFrom() const { return getChild(Child::FILL_FROM); } ASTPtr getFillTo() const { return getChild(Child::FILL_TO); } ASTPtr getFillStep() const { return getChild(Child::FILL_STEP); } + ASTPtr getFillStaleness() const { return getChild(Child::FILL_STALENESS); } String getID(char) const override { return "OrderByElement"; } diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 8ea9fb12b86..c10e4879214 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -541,6 +541,7 @@ namespace DB MR_MACROS(YY, "YY") \ MR_MACROS(YYYY, "YYYY") \ MR_MACROS(ZKPATH, "ZKPATH") \ + MR_MACROS(STALENESS, "STALENESS") \ /// The list of keywords where underscore is intentional #define APPLY_FOR_PARSER_KEYWORDS_WITH_UNDERSCORES(MR_MACROS) \ diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 31efcb16f02..ad062d27a37 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -2178,6 +2178,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword from(Keyword::FROM); ParserKeyword to(Keyword::TO); ParserKeyword step(Keyword::STEP); + ParserKeyword staleness(Keyword::STALENESS); ParserStringLiteral collate_locale_parser; ParserExpressionWithOptionalAlias exp_parser(false); @@ -2219,6 +2220,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; + ASTPtr fill_staleness; if (with_fill.ignore(pos, expected)) { has_with_fill = true; @@ -2230,6 +2232,9 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (step.ignore(pos, expected) && !exp_parser.parse(pos, fill_step, expected)) return false; + + if (staleness.ignore(pos, expected) && !exp_parser.parse(pos, fill_staleness, expected)) + return false; } auto elem = std::make_shared(); @@ -2244,6 +2249,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect elem->setFillFrom(fill_from); elem->setFillTo(fill_to); elem->setFillStep(fill_step); + elem->setFillStaleness(fill_staleness); node = elem; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 8d3c75fdabb..f1c752aecd0 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -847,6 +847,9 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } + if (interpolate_description) + LOG_DEBUG(getLogger("addWithFillStepIfNeeded"), "InterpolateDescription: {}", interpolate_description->actions.dumpDAG()); + const auto & query_context = planner_context->getQueryContext(); const Settings & settings = query_context->getSettingsRef(); auto filling_step = std::make_unique( diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index aea304e0ecc..aa233109fa9 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -391,6 +391,9 @@ public: if (sort_node.hasFillStep()) buffer << " STEP " << calculateActionNodeName(sort_node.getFillStep()); + + if (sort_node.hasFillStaleness()) + buffer << " STALENESS " << calculateActionNodeName(sort_node.getFillStaleness()); } if (i + 1 != order_by_nodes_size) diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index af51afdef13..0a33e2f0828 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -43,7 +43,7 @@ std::pair extractWithFillValue(const QueryTreeNodePtr & node return result; } -std::pair> extractWithFillStepValue(const QueryTreeNodePtr & node) +std::pair> extractWithFillValueWithIntervalKind(const QueryTreeNodePtr & node) { const auto & constant_node = node->as(); @@ -77,7 +77,7 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) if (sort_node.hasFillStep()) { - auto extract_result = extractWithFillStepValue(sort_node.getFillStep()); + auto extract_result = extractWithFillValueWithIntervalKind(sort_node.getFillStep()); fill_column_description.fill_step = std::move(extract_result.first); fill_column_description.step_kind = std::move(extract_result.second); } @@ -87,10 +87,30 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) fill_column_description.fill_step = Field(direction_value); } + if (sort_node.getFillStaleness()) + { + auto extract_result = extractWithFillValueWithIntervalKind(sort_node.getFillStaleness()); + fill_column_description.fill_staleness = std::move(extract_result.first); + fill_column_description.staleness_kind = std::move(extract_result.second); + } + + /////////////////////////////////// + if (applyVisitor(FieldVisitorAccurateEquals(), fill_column_description.fill_step, Field{0})) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL STEP value cannot be zero"); + if (sort_node.hasFillStaleness()) + { + if (sort_node.hasFillFrom()) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STALENESS cannot be used together with WITH FILL FROM"); + + if (applyVisitor(FieldVisitorAccurateLessOrEqual(), fill_column_description.fill_staleness, Field{0})) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STALENESS value cannot be less or equal zero"); + } + if (sort_node.getSortDirection() == SortDirection::ASCENDING) { if (applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_step, Field{0})) From 7d6beb55877936d73545fd742bdc33f1012cb26a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 20:00:24 +0200 Subject: [PATCH 450/816] Add a lot of minor things from the private repository --- src/Access/AccessControl.h | 2 ++ src/Access/Authentication.cpp | 1 + src/Access/Common/AccessType.h | 1 + src/Access/RoleCache.h | 4 ++++ src/Access/tests/gtest_access_rights_ops.cpp | 3 ++- src/Core/ServerUUID.cpp | 5 +++++ src/Core/ServerUUID.h | 3 +++ src/Core/UUID.h | 3 +++ src/Databases/enableAllExperimentalSettings.cpp | 2 ++ src/Dictionaries/ExecutablePoolDictionarySource.cpp | 7 +++++++ src/Dictionaries/RedisDictionarySource.cpp | 1 - src/Dictionaries/XDBCDictionarySource.cpp | 6 ++++++ src/Disks/DiskEncrypted.h | 2 ++ src/Disks/DiskEncryptedTransaction.cpp | 1 - src/Disks/DiskType.h | 2 ++ src/Disks/IDisk.h | 3 ++- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorageMetadata.h | 2 ++ src/Disks/ObjectStorages/MetadataStorageFromDisk.h | 2 ++ src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 + src/IO/ReadBufferFromPocoSocket.cpp | 5 +++++ src/IO/ReadBufferFromPocoSocket.h | 2 ++ src/IO/S3Common.cpp | 1 - src/Interpreters/ActionLocksManager.cpp | 2 ++ src/Interpreters/BlobStorageLog.cpp | 1 + src/Interpreters/Cache/LRUFileCachePriority.h | 4 +++- src/Interpreters/Cache/SLRUFileCachePriority.h | 5 +++++ src/Interpreters/DatabaseCatalog.h | 6 ++++++ src/Interpreters/InterpreterSystemQuery.h | 3 +++ src/Interpreters/MutationsInterpreter.h | 5 ++++- src/Interpreters/Session.h | 1 - src/Interpreters/Squashing.cpp | 1 + src/Interpreters/executeDDLQueryOnCluster.cpp | 7 +++++++ src/Parsers/CommonParsers.h | 1 + src/Parsers/IAST.cpp | 1 + src/Server/CloudPlacementInfo.cpp | 3 +++ src/Storages/MergeTree/FutureMergedMutatedPart.h | 1 + src/Storages/MergeTree/IMergeTreeReader.h | 1 + src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 ++ src/Storages/MergeTree/MergeProjectionPartsTask.cpp | 3 +++ src/Storages/MergeTree/MergeTreeDataFormatVersion.h | 4 ++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 2 ++ src/Storages/MergeTree/MergeTreeDataPartType.h | 1 + src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp | 8 ++++++++ src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h | 1 + src/Storages/MergeTree/MergeTreeMutationStatus.cpp | 4 ++-- src/Storages/MergeTree/MergeTreePartInfo.h | 7 +++++++ src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 4 +++- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.h | 1 + src/Storages/StorageGenerateRandom.cpp | 3 +++ src/Storages/TableZnodeInfo.h | 2 ++ 54 files changed, 135 insertions(+), 18 deletions(-) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index a91686433ec..a342c5300bf 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -9,6 +9,8 @@ #include +#include "config.h" + namespace Poco { diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 8d5d04a4ed2..1d69a659cd6 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -12,6 +12,7 @@ #include "config.h" + namespace DB { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..383e7f70420 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -193,6 +193,7 @@ enum class AccessType : uint8_t M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ M(SYSTEM_REPLICATION_QUEUES, "SYSTEM STOP REPLICATION QUEUES, SYSTEM START REPLICATION QUEUES, STOP REPLICATION QUEUES, START REPLICATION QUEUES", TABLE, SYSTEM) \ M(SYSTEM_VIRTUAL_PARTS_UPDATE, "SYSTEM STOP VIRTUAL PARTS UPDATE, SYSTEM START VIRTUAL PARTS UPDATE, STOP VIRTUAL PARTS UPDATE, START VIRTUAL PARTS UPDATE", TABLE, SYSTEM) \ + M(SYSTEM_REDUCE_BLOCKING_PARTS, "SYSTEM STOP REDUCE BLOCKING PARTS, SYSTEM START REDUCE BLOCKING PARTS, STOP REDUCE BLOCKING PARTS, START REDUCE BLOCKING PARTS", TABLE, SYSTEM) \ M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \ M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_REPLICA_READINESS, "SYSTEM REPLICA READY, SYSTEM REPLICA UNREADY", GLOBAL, SYSTEM) \ diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index 75d1fd32685..b707a05346f 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -22,6 +22,10 @@ public: const std::vector & current_roles, const std::vector & current_roles_with_admin_option); + std::shared_ptr getEnabledRoles( + boost::container::flat_set current_roles, + boost::container::flat_set current_roles_with_admin_option); + private: using SubscriptionsOnRoles = std::vector>; diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 902fc949840..41567905a10 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -284,7 +284,8 @@ TEST(AccessRights, Union) "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " - "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " + "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, " + "SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM REDUCE BLOCKING PARTS, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index 251b407e673..5b17017e7f4 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -68,6 +68,11 @@ UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log) } } +void ServerUUID::set(UUID & uuid) +{ + server_uuid = uuid; +} + void ServerUUID::setRandomForUnitTests() { server_uuid = UUIDHelpers::generateV4(); diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 9c7f7d32acc..26711bfbfaa 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -20,6 +20,9 @@ public: /// Loads server UUID from file or creates new one. Should be called on daemon startup. static void load(const fs::path & server_uuid_file, Poco::Logger * log); + /// Sets specific server UUID. + static void set(UUID & uuid); + static void setRandomForUnitTests(); }; diff --git a/src/Core/UUID.h b/src/Core/UUID.h index 2bdefe9d3fc..1b8a075f0d2 100644 --- a/src/Core/UUID.h +++ b/src/Core/UUID.h @@ -64,6 +64,9 @@ namespace UUIDHelpers /// Generate random UUID. UUID generateV4(); + /// Generate UUID from hash of a string. + UUID makeUUIDv4FromHash(const String & string); + constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1; constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0; diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index d1b3b776370..6efbc429fd8 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -43,6 +43,8 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("enable_zstd_qat_codec", 1); context->setSetting("allow_create_index_without_type", 1); context->setSetting("allow_experimental_s3queue", 1); + + /// clickhouse-private settings context->setSetting("allow_experimental_shared_set_join", 1); } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 403ce540e76..602fde0e0d7 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -26,6 +26,9 @@ namespace DB namespace Setting { extern const SettingsSeconds max_execution_time; + + /// Cloud only + extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -33,6 +36,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; extern const int UNSUPPORTED_METHOD; + extern const int SUPPORT_IS_DISABLED; } ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( @@ -192,6 +196,9 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { + if (global_context->getSettingsRef()[Setting::cloud_mode]) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `executable pool` is disabled"); + if (dict_struct.has_expressions) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable_pool` does not support attribute expressions"); diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 17ed515ca9a..26d9ebae1b8 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -29,7 +29,6 @@ namespace DB ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { - auto redis_config_prefix = config_prefix + ".redis"; auto host = config.getString(redis_config_prefix + ".host"); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index ebb50f79497..4e64db5831d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -28,6 +28,9 @@ namespace Setting { extern const SettingsSeconds http_receive_timeout; extern const SettingsBool odbc_bridge_use_connection_pooling; + + /// Cloud only + extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -242,6 +245,9 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) ContextPtr global_context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { + + if (global_context->getSettingsRef()[Setting::cloud_mode]) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `odbc` is disabled"); #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( global_context, diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index caba4184a73..95d9554b909 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -313,6 +313,8 @@ public: return std::make_shared(*this); } + /// Need to overwrite explicetly because this disk change + /// a lot of "delegate" methods. return createEncryptedTransaction(); } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 2660051e1d3..a528564fd1e 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -1,6 +1,5 @@ #include - #if USE_SSL #include #include diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 347e2c1cfe3..bf7ef3d30eb 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -27,9 +27,11 @@ enum class MetadataStorageType : uint8_t { None, Local, + Keeper, Plain, PlainRewritable, StaticWeb, + Memory, }; MetadataStorageType metadataTypeFromString(const String & type); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 59f58a816e9..692020c86a6 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -497,7 +497,7 @@ public: protected: - friend class DiskDecorator; + friend class DiskReadOnlyWrapper; const String name; @@ -580,6 +580,7 @@ inline String directoryPath(const String & path) return fs::path(path).parent_path() / ""; } + } template <> diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 7055a7018ce..8e4ec6f3dfb 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size) { /// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task. - if (!settings.enable_filesystem_cache) + if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache) return settings.remote_fs_buffer_size; /// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file. diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 4f45f5b7ddf..456b3a4778d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -56,6 +56,8 @@ public: void deserialize(ReadBuffer & buf); void deserializeFromString(const std::string & data); + /// This method was deleted from public fork recently by Azat + void createFromSingleObject(ObjectStorageKey object_key, size_t bytes_size, size_t ref_count_, bool is_read_only_); void serialize(WriteBuffer & buf, bool sync) const; std::string serializeToString() const; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 5d56580a57b..922990bfdb7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -71,6 +71,8 @@ public: DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; + + bool isReadOnly() const override { return disk->isReadOnly(); } }; class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index cd36429d0a2..ece2608bea5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index bbf9f96404f..93562e7bfed 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -146,4 +146,9 @@ bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const return res; } +void ReadBufferFromPocoSocketBase::setReceiveTimeout(size_t receive_timeout_microseconds) +{ + socket.setReceiveTimeout(Poco::Timespan(receive_timeout_microseconds, 0)); +} + } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 912388adaac..2a0c0213302 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -34,6 +34,8 @@ public: ssize_t socketReceiveBytesImpl(char * ptr, size_t size); + void setReceiveTimeout(size_t receive_timeout_microseconds); + private: AsyncCallback async_callback; std::string socket_description; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index af5e0339a9f..214927684b3 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -49,7 +49,6 @@ bool S3Exception::isRetryableError() const } } - namespace DB::ErrorCodes { extern const int S3_ERROR; diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 28803a94c80..da6e9d473da 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -20,6 +20,8 @@ namespace ActionLocks extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; extern const StorageActionBlockType ViewRefresh = 10; + extern const StorageActionBlockType VirtualPartsUpdate = 11; + extern const StorageActionBlockType ReduceBlockingParts = 12; } diff --git a/src/Interpreters/BlobStorageLog.cpp b/src/Interpreters/BlobStorageLog.cpp index f20ac9165ac..601005626e1 100644 --- a/src/Interpreters/BlobStorageLog.cpp +++ b/src/Interpreters/BlobStorageLog.cpp @@ -96,6 +96,7 @@ void BlobStorageLog::prepareTable() std::unique_lock lock{prepare_mutex}; const auto & relative_data_path = merge_tree_table->getRelativeDataPath(); prefix_to_ignore = normalizePath(relative_data_path); + LOG_DEBUG(log, "Will ignore blobs with prefix {}", prefix_to_ignore); } } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 0ca62b19d37..58f64b6e28d 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -12,7 +12,7 @@ namespace DB /// Based on the LRU algorithm implementation, the record with the lowest priority is stored at /// the head of the queue, and the record with the highest priority is stored at the tail. -class LRUFileCachePriority final : public IFileCachePriority +class LRUFileCachePriority : public IFileCachePriority { protected: struct State @@ -85,6 +85,8 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; + FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, state); } + private: class LRUIterator; using LRUQueue = std::list; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 23bc8c0908b..5649a12aff9 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -72,7 +72,12 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; + FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, size_ratio, probationary_queue.state, protected_queue.state); } + private: + using LRUIterator = LRUFileCachePriority::LRUIterator; + using LRUQueue = std::list; + double size_ratio; LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 83a302f117d..308d1b33e8b 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -266,6 +266,10 @@ public: void triggerReloadDisksTask(const Strings & new_added_disks); + void stopReplicatedDDLQueries(); + void startReplicatedDDLQueries(); + bool canPerformReplicatedDDLQueries() const; + private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -361,6 +365,8 @@ private: std::mutex reload_disks_mutex; std::set disks_to_reload; static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5; + + std::atomic replicated_ddl_queries_enabled = false; }; diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 3d667fcaef0..82d55125927 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -82,6 +82,9 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); + + void stopReplicatedDDLQueries(); + void startReplicatedDDLQueries(); }; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 8601558b788..84f6746ec58 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -40,7 +40,6 @@ class MutationsInterpreter { private: struct Stage; - public: struct Settings { @@ -112,6 +111,10 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } + /// Returns a chain of actions that can be + /// applied to block to execute mutation commands. + std::vector getMutationActions() const; + /// Internal class which represents a data part for MergeTree /// or just storage for other storages. /// The main idea is to create a dedicated reading from MergeTree part. diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index ab4bc53b6f1..0a20dd896a9 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -98,7 +98,6 @@ public: /// Closes and removes session void closeSession(const String & session_id); - private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 8122800f882..02d1ae528ac 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -19,6 +19,7 @@ Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_bloc , min_block_size_bytes(min_block_size_bytes_) , header(header_) { + LOG_TEST(getLogger("Squashing"), "header columns {}", header.columns()); } Chunk Squashing::flush() diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index d7d9da2a367..c5d58a873fb 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -14,6 +14,7 @@ #include #include #include +#include "Parsers/ASTSystemQuery.h" #include #include #include @@ -93,6 +94,12 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, if (!context->getSettingsRef()[Setting::allow_distributed_ddl]) throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Distributed DDL queries are prohibited for the user"); + bool is_system_query = dynamic_cast(query_ptr.get()) != nullptr; + bool replicated_ddl_queries_enabled = DatabaseCatalog::instance().canPerformReplicatedDDLQueries(); + + if (!is_system_query && !replicated_ddl_queries_enabled) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); + if (const auto * query_alter = query_ptr->as()) { for (const auto & command : query_alter->command_list->children) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 8ea9fb12b86..83b7eb71d64 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -99,6 +99,7 @@ namespace DB MR_MACROS(COMPRESSION, "COMPRESSION") \ MR_MACROS(CONST, "CONST") \ MR_MACROS(CONSTRAINT, "CONSTRAINT") \ + MR_MACROS(CONNECTIONS, "CONNECTIONS") \ MR_MACROS(CREATE_POLICY, "CREATE POLICY") \ MR_MACROS(CREATE_PROFILE, "CREATE PROFILE") \ MR_MACROS(CREATE_QUOTA, "CREATE QUOTA") \ diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 2b581f20e3b..0b1dff556f6 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -9,6 +9,7 @@ #include #include #include + #include namespace DB diff --git a/src/Server/CloudPlacementInfo.cpp b/src/Server/CloudPlacementInfo.cpp index d8810bb30de..08b4e2132ad 100644 --- a/src/Server/CloudPlacementInfo.cpp +++ b/src/Server/CloudPlacementInfo.cpp @@ -53,6 +53,9 @@ PlacementInfo & PlacementInfo::instance() void PlacementInfo::initialize(const Poco::Util::AbstractConfiguration & config) try { + if (initialized) + return; + if (!config.has(DB::PlacementInfo::PLACEMENT_CONFIG_PREFIX)) { availability_zone = ""; diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index 09fb7b01678..ca607bb4e33 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -22,6 +22,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartFormat part_format; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + std::vector blocking_parts_to_remove; MergeType merge_type = MergeType::Regular; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index d799ce57b40..c68617d3995 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -18,6 +18,7 @@ public: using ValueSizeMap = std::map; using VirtualFields = std::unordered_map; using DeserializeBinaryBulkStateMap = std::map; + using FileStreams = std::map>; IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index fa6640409e5..859d6f58f40 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -372,6 +372,8 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite { part = merge_task->getFuture().get(); + part->is_prewarmed = true; + storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); /// Why we reset task here? Because it holds shared pointer to part and tryRemovePartImmediately will /// not able to remove the part and will throw an exception (because someone holds the pointer). diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp index 4e1bb2f11a7..34cd925a8c6 100644 --- a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -83,6 +83,9 @@ bool MergeProjectionPartsTask::executeStep() ".tmp_proj"); next_level_parts.push_back(executeHere(tmp_part_merge_task)); + /// FIXME (alesapin) we should use some temporary storage for this, + /// not commit each subprojection part + next_level_parts.back()->getDataPartStorage().commitTransaction(); next_level_parts.back()->is_temp = true; } diff --git a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h index 0a84f08ea71..a61938a993c 100644 --- a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h +++ b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h @@ -8,7 +8,7 @@ namespace DB STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion) -const MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; -const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 71fcb93f369..6d209b9f931 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -106,9 +106,11 @@ public: PreformattedMessage & out_disable_reason, bool dry_run = false); + /// Actually the most fresh partition with biggest modification_time String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; /// Useful to quickly get a list of partitions that contain parts that we may want to merge + /// The result is limited by top_number_of_partitions_to_consider_for_merge PartitionIdsHint getPartitionsThatMayBeMerged( size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.h b/src/Storages/MergeTree/MergeTreeDataPartType.h index 8177809d41e..a59ccc2fab1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartType.h +++ b/src/Storages/MergeTree/MergeTreeDataPartType.h @@ -45,6 +45,7 @@ public: enum Value { Full, + Packed, Unknown, }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 58a67fc4ba2..388737915ab 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -179,8 +179,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - if (!getDataPartStorage().exists()) - getDataPartStorage().createDirectories(); + /// We don't need to check if it exists or not, createDirectories doesn't throw + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 2af7abc17f9..9211ab51ad5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -108,6 +108,14 @@ std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesyste return {}; } +MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( + MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_) + : mark_type(mark_type_) + , fixed_index_granularity(index_granularity_) + , index_granularity_bytes(index_granularity_bytes_) +{ +} + MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MergeTreeDataPartType type_) : MergeTreeIndexGranularityInfo(storage, {storage.canUseAdaptiveGranularity(), (*storage.getSettings())[MergeTreeSetting::compress_marks], type_.getValue()}) { diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index 87445c99ade..b302d6b1a4b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -49,6 +49,7 @@ public: MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MarkType mark_type_); MergeTreeIndexGranularityInfo(MergeTreeDataPartType type_, bool is_adaptive_, size_t index_granularity_, size_t index_granularity_bytes_); + MergeTreeIndexGranularityInfo(MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_); void changeGranularityIfRequired(const IDataPartStorage & data_part_storage); diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp index 6553054774e..e0214d6a79d 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp @@ -26,11 +26,11 @@ void checkMutationStatus(std::optional & status, const throw Exception( ErrorCodes::UNFINISHED, "Exception happened during execution of mutation{} '{}' with part '{}' reason: '{}'. This error maybe retryable or not. " - "In case of unretryable error, mutation can be killed with KILL MUTATION query", + "In case of unretryable error, mutation can be killed with KILL MUTATION query \n\n{}\n", mutation_ids.size() > 1 ? "s" : "", boost::algorithm::join(mutation_ids, ", "), status->latest_failed_part, - status->latest_fail_reason); + status->latest_fail_reason, StackTrace().toString()); } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index f128722b03b..28b043fcf20 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -46,6 +46,13 @@ struct MergeTreePartInfo < std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); } + bool operator>(const MergeTreePartInfo & rhs) const + { + return std::forward_as_tuple(partition_id, min_block, max_block, level, mutation) + > std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); + } + + bool operator==(const MergeTreePartInfo & rhs) const { return !(*this != rhs); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 7acc8cd88b4..13ce14e02ec 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -35,7 +35,7 @@ struct PrewhereExprStep bool remove_filter_column = false; bool need_filter = false; - /// Some PREWHERE steps should be executed without conversions. + /// Some PREWHERE steps should be executed without conversions (e.g. early mutation steps) /// A step without alter conversion cannot be executed after step with alter conversions. bool perform_alter_conversions = false; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 95469337f8a..4de52213869 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -690,6 +690,8 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: /// /// metadata_snapshot->check(part->getColumns()); + part->is_prewarmed = true; + auto block_id_path = getBlockIdPath(storage.zookeeper_path, block_id); CommitRetryContext retry_context; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2a1ddf32431..34e699bcef7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -135,7 +135,6 @@ bool isRetryableException(std::exception_ptr exception_ptr) } } - static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, const IDataPartStorage & data_part_storage, @@ -422,6 +421,7 @@ IMergeTreeDataPart::Checksums checkDataPart( } ReadSettings read_settings; + read_settings.read_through_distributed_cache = false; read_settings.enable_filesystem_cache = false; read_settings.enable_filesystem_cache_log = false; read_settings.enable_filesystem_read_prefetches_log = false; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index f08765367fa..2e7433dc7b8 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -5,6 +5,7 @@ #if USE_AWS_S3 #include #include +#include namespace DB { diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 23ee7a18b53..23df9bfa1c7 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -150,6 +150,7 @@ size_t estimateValueSize( } } +} ColumnPtr fillColumnWithRandomData( const DataTypePtr type, @@ -539,6 +540,8 @@ ColumnPtr fillColumnWithRandomData( } } +namespace +{ class GenerateSource : public ISource { diff --git a/src/Storages/TableZnodeInfo.h b/src/Storages/TableZnodeInfo.h index 729a88e7509..4e3ffb44056 100644 --- a/src/Storages/TableZnodeInfo.h +++ b/src/Storages/TableZnodeInfo.h @@ -17,6 +17,8 @@ struct StorageID; class ASTCreateQuery; class Context; using ContextPtr = std::shared_ptr; +class IDatabase; +using DatabasePtr = std::shared_ptr; /// Helper for replicated tables that use zookeeper for coordination among replicas. /// Handles things like: From e85b75b2e9e2b55036cf76573a46b57562aa92b6 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 25 Oct 2024 19:03:37 +0100 Subject: [PATCH 451/816] document extra credentials s3 --- .../engines/table-engines/integrations/s3.md | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index fb759b948a5..876dcc2c094 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -331,6 +331,26 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', NOSIGN, 'CSVWithNames'); ``` +## Using S3 credentials (ClickHouse Cloud) + +For non-public buckets, users can pass an `aws_access_key_id` and `aws_secret_access_key` to the function. For example: + +```sql +SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv', '', '','TSVWithNames') +``` + +This is appropriate for one-off accesses or in cases where credentials can easily be rotated. However, this is not recommended as a long-term solution for repeated access or where credentials are sensitive. In this case, we recommend users rely on role-based access. + +Role-based access for S3 in ClickHouse Cloud is documented [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role). + +Once configured, a roleARN can be passed to the s3 function via an `extra_credentials` parameter. For example: + +```sql +SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv','CSVWithNames',extra_credentials(role_arn = 'arn:aws:iam::111111111111:role/ClickHouseAccessRole-001')) +``` + +Further examples can be found [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role) + ## See also - [s3 table function](../../../sql-reference/table-functions/s3.md) From 5834da5a62dd4f5eacd0789b4dc4495601f51d83 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 25 Oct 2024 19:08:29 +0100 Subject: [PATCH 452/816] move section --- .../engines/table-engines/integrations/s3.md | 20 ------------------- docs/en/sql-reference/table-functions/s3.md | 20 ++++++++++++++++++- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 876dcc2c094..fb759b948a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -331,26 +331,6 @@ CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', NOSIGN, 'CSVWithNames'); ``` -## Using S3 credentials (ClickHouse Cloud) - -For non-public buckets, users can pass an `aws_access_key_id` and `aws_secret_access_key` to the function. For example: - -```sql -SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv', '', '','TSVWithNames') -``` - -This is appropriate for one-off accesses or in cases where credentials can easily be rotated. However, this is not recommended as a long-term solution for repeated access or where credentials are sensitive. In this case, we recommend users rely on role-based access. - -Role-based access for S3 in ClickHouse Cloud is documented [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role). - -Once configured, a roleARN can be passed to the s3 function via an `extra_credentials` parameter. For example: - -```sql -SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv','CSVWithNames',extra_credentials(role_arn = 'arn:aws:iam::111111111111:role/ClickHouseAccessRole-001')) -``` - -Further examples can be found [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role) - ## See also - [s3 table function](../../../sql-reference/table-functions/s3.md) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 181c92b92d4..88714d4f24f 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -93,7 +93,6 @@ LIMIT 5; ClickHouse also can determine the compression method of the file. For example, if the file was zipped up with a `.csv.gz` extension, ClickHouse would decompress the file automatically. ::: - ## Usage Suppose that we have several files with following URIs on S3: @@ -248,6 +247,25 @@ FROM s3( LIMIT 5; ``` +## Using S3 credentials (ClickHouse Cloud) + +For non-public buckets, users can pass an `aws_access_key_id` and `aws_secret_access_key` to the function. For example: + +```sql +SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv', '', '','TSVWithNames') +``` + +This is appropriate for one-off accesses or in cases where credentials can easily be rotated. However, this is not recommended as a long-term solution for repeated access or where credentials are sensitive. In this case, we recommend users rely on role-based access. + +Role-based access for S3 in ClickHouse Cloud is documented [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role). + +Once configured, a roleARN can be passed to the s3 function via an `extra_credentials` parameter. For example: + +```sql +SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv','CSVWithNames',extra_credentials(role_arn = 'arn:aws:iam::111111111111:role/ClickHouseAccessRole-001')) +``` + +Further examples can be found [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role) ## Working with archives From c46e3ca09f7e33957a621ec2588e591c20f32f12 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 25 Oct 2024 19:11:58 +0100 Subject: [PATCH 453/816] note for s3cluster --- docs/en/sql-reference/table-functions/s3Cluster.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index 9bf5a6b4da6..4702a2056e5 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -70,6 +70,10 @@ SELECT count(*) FROM s3Cluster( ) ``` +## Accessing private and public buckets + +Users can use the same approaches as document for the s3 function [here](/docs/en/sql-reference/table-functions/s3#accessing-public-buckets). + **See Also** - [S3 engine](../../engines/table-engines/integrations/s3.md) From 6d8c3e349e6eb8e59103fe1425f7f011ca2fe827 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 25 Oct 2024 18:59:05 +0000 Subject: [PATCH 454/816] add cleanup after test --- tests/integration/test_async_load_databases/test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index 6bd2b86478f..acd3ef7455b 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -238,3 +238,7 @@ def test_async_load_system_database(started_cluster): # Trigger async load of system database node2.restart_clickhouse() + + for i in range(id - 1): + node2.query(f"drop table if exists system.text_log_{i + 1}_test") + node2.query(f"drop table if exists system.query_log_{i + 1}_test") From c269bf69f2d14f027ee46da0aa37eea9dce1e845 Mon Sep 17 00:00:00 2001 From: ortyomka Date: Fri, 25 Oct 2024 19:29:26 +0000 Subject: [PATCH 455/816] trigger ci From 4b5ddd2d81d83f5c2e898da2bd4d49ec626ec04a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 21:35:32 +0200 Subject: [PATCH 456/816] Add various trash --- src/Databases/DatabaseReplicated.cpp | 12 +++++++++--- .../ObjectStorages/MetadataStorageFromDisk.h | 2 -- src/Interpreters/DatabaseCatalog.cpp | 15 +++++++++++++++ src/Interpreters/MutationsInterpreter.h | 4 ---- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 -- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 -- 6 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d4eaaf750cd..387667b1b42 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -85,6 +85,7 @@ namespace ErrorCodes extern const int NO_ACTIVE_REPLICAS; extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT; extern const int CANNOT_RESTORE_TABLE; + extern const int QUERY_IS_PROHIBITED; extern const int SUPPORT_IS_DISABLED; } @@ -1057,6 +1058,9 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex { waitDatabaseStarted(); + if (!DatabaseCatalog::instance().canPerformReplicatedDDLQueries()) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); + if (query_context->getCurrentTransaction() && query_context->getSettingsRef()[Setting::throw_on_unsupported_query_inside_transaction]) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported"); @@ -1237,14 +1241,16 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name)); auto query_context = Context::createCopy(getContext()); query_context->setSetting("allow_deprecated_database_ordinary", 1); - executeQuery(query, query_context, QueryFlags{.internal = true}); + query_context->setSetting("cloud_mode", false); + executeQuery(query, query_context, QueryFlags{ .internal = true }); /// But we want to avoid discarding UUID of ReplicatedMergeTree tables, because it will not work /// if zookeeper_path contains {uuid} macro. Replicated database do not recreate replicated tables on recovery, /// so it's ok to save UUID of replicated table. query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Atomic", backQuoteIfNeed(to_db_name_replicated)); query_context = Context::createCopy(getContext()); - executeQuery(query, query_context, QueryFlags{.internal = true}); + query_context->setSetting("cloud_mode", false); + executeQuery(query, query_context, QueryFlags{ .internal = true }); } size_t moved_tables = 0; @@ -1634,7 +1640,7 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl auto table = tryGetTable(table_name, getContext()); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} doesn't exist", table_name); - if (table->getName() == "MaterializedView" || table->getName() == "WindowView") + if (table->getName() == "MaterializedView" || table->getName() == "WindowView" || table->getName() == "SharedSet" || table->getName() == "SharedJoin") { /// Avoid recursive locking of metadata_mutex table->dropInnerTableIfAny(sync, local_context); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 922990bfdb7..5d56580a57b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -71,8 +71,6 @@ public: DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; - - bool isReadOnly() const override { return disk->isReadOnly(); } }; class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, private MetadataOperationsHolder diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c92602105c5..dc9ce23ddb9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1817,6 +1817,21 @@ void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) (*reload_disks_task)->schedule(); } +void DatabaseCatalog::stopReplicatedDDLQueries() +{ + replicated_ddl_queries_enabled = false; +} + +void DatabaseCatalog::startReplicatedDDLQueries() +{ + replicated_ddl_queries_enabled = true; +} + +bool DatabaseCatalog::canPerformReplicatedDDLQueries() const +{ + return replicated_ddl_queries_enabled; +} + static void maybeUnlockUUID(UUID uuid) { if (uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 84f6746ec58..901cd13cd2f 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -111,10 +111,6 @@ public: MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; } - /// Returns a chain of actions that can be - /// applied to block to execute mutation commands. - std::vector getMutationActions() const; - /// Internal class which represents a data part for MergeTree /// or just storage for other storages. /// The main idea is to create a dedicated reading from MergeTree part. diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 859d6f58f40..fa6640409e5 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -372,8 +372,6 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite { part = merge_task->getFuture().get(); - part->is_prewarmed = true; - storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); /// Why we reset task here? Because it holds shared pointer to part and tryRemovePartImmediately will /// not able to remove the part and will throw an exception (because someone holds the pointer). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4de52213869..1ba04fc460d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -690,8 +690,6 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: /// /// metadata_snapshot->check(part->getColumns()); - part->is_prewarmed = true; - auto block_id_path = getBlockIdPath(storage.zookeeper_path, block_id); CommitRetryContext retry_context; From 878b1bcd16efc635dbdd35136e74501613db7909 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 22:13:36 +0200 Subject: [PATCH 457/816] Remove bad test `test_system_replicated_fetches` --- .../__init__.py | 0 .../test_system_replicated_fetches/test.py | 138 ------------------ 2 files changed, 138 deletions(-) delete mode 100644 tests/integration/test_system_replicated_fetches/__init__.py delete mode 100644 tests/integration/test_system_replicated_fetches/test.py diff --git a/tests/integration/test_system_replicated_fetches/__init__.py b/tests/integration/test_system_replicated_fetches/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_system_replicated_fetches/test.py b/tests/integration/test_system_replicated_fetches/test.py deleted file mode 100644 index 8cb571c3c58..00000000000 --- a/tests/integration/test_system_replicated_fetches/test.py +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env python3 - - -import json -import random -import string -import time - -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=True) -node2 = cluster.add_instance("node2", with_zookeeper=True) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def get_random_string(length): - return "".join( - random.choice(string.ascii_uppercase + string.digits) for _ in range(length) - ) - - -def test_system_replicated_fetches(started_cluster): - node1.query( - "CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '1') ORDER BY tuple()" - ) - node2.query( - "CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '2') ORDER BY tuple()" - ) - - with PartitionManager() as pm: - node2.query("SYSTEM STOP FETCHES t") - node1.query( - "INSERT INTO t SELECT number, '{}' FROM numbers(10000)".format( - get_random_string(104857) - ) - ) - pm.add_network_delay(node1, 80) - node2.query("SYSTEM START FETCHES t") - fetches_result = [] - for _ in range(1000): - result = json.loads( - node2.query("SELECT * FROM system.replicated_fetches FORMAT JSON") - ) - if not result["data"]: - if fetches_result: - break - time.sleep(0.1) - else: - fetches_result.append(result["data"][0]) - print(fetches_result[-1]) - time.sleep(0.1) - - node2.query("SYSTEM SYNC REPLICA t", timeout=10) - assert node2.query("SELECT COUNT() FROM t") == "10000\n" - - for elem in fetches_result: - elem["bytes_read_compressed"] = float(elem["bytes_read_compressed"]) - elem["total_size_bytes_compressed"] = float(elem["total_size_bytes_compressed"]) - elem["progress"] = float(elem["progress"]) - elem["elapsed"] = float(elem["elapsed"]) - - assert len(fetches_result) > 0 - first_non_empty = fetches_result[0] - - assert first_non_empty["database"] == "default" - assert first_non_empty["table"] == "t" - assert first_non_empty["source_replica_hostname"] == "node1" - assert first_non_empty["source_replica_port"] == 9009 - assert first_non_empty["source_replica_path"] == "/clickhouse/test/t/replicas/1" - assert first_non_empty["interserver_scheme"] == "http" - assert first_non_empty["result_part_name"] == "all_0_0_0" - assert first_non_empty["result_part_path"].startswith("/var/lib/clickhouse/") - assert first_non_empty["result_part_path"].endswith("all_0_0_0/") - assert first_non_empty["partition_id"] == "all" - assert first_non_empty["URI"].startswith( - "http://node1:9009/?endpoint=DataPartsExchange" - ) - - for elem in fetches_result: - # FIXME https://github.com/ClickHouse/ClickHouse/issues/45435 - # assert ( - # elem["bytes_read_compressed"] <= elem["total_size_bytes_compressed"] - # ), "Bytes read ({}) more than total bytes ({}). It's a bug".format( - # elem["bytes_read_compressed"], elem["total_size_bytes_compressed"] - # ) - # assert ( - # 0.0 <= elem["progress"] <= 1.0 - # ), "Progress shouldn't less than 0 and bigger than 1, got {}".format( - # elem["progress"] - # ) - assert ( - 0.0 <= elem["elapsed"] - ), "Elapsed time must be greater than 0, got {}".format(elem["elapsed"]) - - prev_progress = first_non_empty["progress"] - for elem in fetches_result: - assert ( - elem["progress"] >= prev_progress - ), "Progress decreasing prev{}, next {}? It's a bug".format( - prev_progress, elem["progress"] - ) - prev_progress = elem["progress"] - - prev_bytes = first_non_empty["bytes_read_compressed"] - for elem in fetches_result: - assert ( - elem["bytes_read_compressed"] >= prev_bytes - ), "Bytes read decreasing prev {}, next {}? It's a bug".format( - prev_bytes, elem["bytes_read_compressed"] - ) - prev_bytes = elem["bytes_read_compressed"] - - prev_elapsed = first_non_empty["elapsed"] - for elem in fetches_result: - assert ( - elem["elapsed"] >= prev_elapsed - ), "Elapsed time decreasing prev {}, next {}? It's a bug".format( - prev_elapsed, elem["elapsed"] - ) - prev_elapsed = elem["elapsed"] - - node1.query("DROP TABLE IF EXISTS t SYNC") - node2.query("DROP TABLE IF EXISTS t SYNC") From 9bbebadef4ee3d8599946cafe2f74e1a473763fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Oct 2024 23:03:32 +0200 Subject: [PATCH 458/816] Fixup --- programs/server/Server.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 35dae614d87..c106a68f360 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2267,6 +2267,21 @@ try throw; } + bool found_stop_flag = false; + + if (has_zookeeper && global_context->getMacros()->getMacroMap().contains("replica")) + { + auto zookeeper = global_context->getZooKeeper(); + String stop_flag_path = "/clickhouse/stop_replicated_ddl_queries/{replica}"; + stop_flag_path = global_context->getMacros()->expand(stop_flag_path); + found_stop_flag = zookeeper->exists(stop_flag_path); + } + + if (found_stop_flag) + LOG_INFO(log, "Found a stop flag for replicated DDL queries. They will be disabled"); + else + DatabaseCatalog::instance().startReplicatedDDLQueries(); + LOG_DEBUG(log, "Loaded metadata."); if (has_trace_collector) From 97db57c09dd8ab2fbb7c2e1d35b118a92cfa212d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 26 Oct 2024 00:54:33 +0200 Subject: [PATCH 459/816] Fix negative check --- utils/list-licenses/list-licenses.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index c33ed3e412e..c06d61a9c43 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -97,4 +97,4 @@ do done # Special care for Rust -find "${LIBS_PATH}/rust_vendor/" -name 'Cargo.toml' | xargs grep 'license = ' | grep -v -P 'MIT|Apache|MPL' && echo "Fatal error: unrecognized licenses in the Rust code" >&2 && exit 1 +find "${LIBS_PATH}/rust_vendor/" -name 'Cargo.toml' | xargs grep 'license = ' | (grep -v -P 'MIT|Apache|MPL' && echo "Fatal error: unrecognized licenses in the Rust code" >&2 && exit 1 || true) From 8f64803c7e880837d2765319aa3539dd5b799726 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2024 02:10:16 +0200 Subject: [PATCH 460/816] Sync the test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..0e839ac6fc1 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -142,6 +142,7 @@ SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLICATION QUEUES','STOP REPLICATION QUEUES','START REPLICATION QUEUES'] TABLE SYSTEM SYSTEM VIRTUAL PARTS UPDATE ['SYSTEM STOP VIRTUAL PARTS UPDATE','SYSTEM START VIRTUAL PARTS UPDATE','STOP VIRTUAL PARTS UPDATE','START VIRTUAL PARTS UPDATE'] TABLE SYSTEM +SYSTEM REDUCE BLOCKING PARTS ['SYSTEM STOP REDUCE BLOCKING PARTS','SYSTEM START REDUCE BLOCKING PARTS','STOP REDUCE BLOCKING PARTS','START REDUCE BLOCKING PARTS'] TABLE SYSTEM SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM SYSTEM REPLICA READINESS ['SYSTEM REPLICA READY','SYSTEM REPLICA UNREADY'] GLOBAL SYSTEM From 9b0ede69642ea33a05fee7cbdf76983edf1d3fdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= <77632139+marcio-absmartly@users.noreply.github.com> Date: Fri, 25 Oct 2024 18:25:27 +0200 Subject: [PATCH 461/816] Apply suggestions from code review Co-authored-by: Antonio Andelic --- src/Interpreters/Session.cpp | 6 +++--- .../03254_session_expire_in_use_in_http_interface.reference | 4 +--- .../03254_session_expire_in_use_in_http_interface.sh | 5 +---- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 1faf6418128..c1286e9ac3e 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -195,7 +195,7 @@ private: Container sessions; // Ordered map of close times for sessions, grouped by the next multiple of close_interval - using CloseTimes = std::map>; + using CloseTimes = std::map>; CloseTimes close_time_buckets; constexpr static std::chrono::steady_clock::duration close_interval = std::chrono::milliseconds(1000); @@ -211,8 +211,8 @@ private: const auto close_time_bucket = session_close_time + bucket_padding; session.close_time_bucket = close_time_bucket; - auto it = close_time_buckets.insert(std::make_pair(close_time_bucket, std::set{})); - it.first->second.insert(session.key); + auto & bucket_sessions = close_time_buckets[close_time_bucket]; + bucket_sessions.insert(session.key); LOG_TEST(log, "Schedule closing session with session_id: {}, user_id: {}", session.key.second, session.key.first); diff --git a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference index 4c9a93358e2..02a9f40656d 100644 --- a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference +++ b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.reference @@ -1,5 +1,3 @@ -A session successfully closes when timeout first expires with refcount != 1 and another session is created in between -45 +A session successfully closes when timeout first expires with refcount != 1 45 1 -1 diff --git a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh index 37f7279a932..f1782cd645b 100755 --- a/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh +++ b/tests/queries/0_stateless/03254_session_expire_in_use_in_http_interface.sh @@ -6,13 +6,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "A session successfully closes when timeout first expires with refcount != 1 and another session is created in between" +echo "A session successfully closes when timeout first expires with refcount != 1" # Here we do not want an infinite loop - because we want this mechanism to be reliable in all cases # So it's better to give it enough time to complete even in constrained environments ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE x (n UInt64) AS SELECT number FROM numbers(10)" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_timeout=1" --data-binary "SELECT sum(n + sleep(3)) FROM x" # This query ensures timeout expires with refcount > 1 -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "CREATE TEMPORARY TABLE y (n UInt64) AS SELECT number FROM numbers(10)" -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_timeout=1" --data-binary "SELECT sum(n) FROM y" sleep 15 ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_10_2&session_check=1" --data-binary "SELECT 1" | grep -c -F 'SESSION_NOT_FOUND' From 73193f4de57cd0d4ca726b454e0f8de282197724 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Sat, 26 Oct 2024 10:17:22 +0200 Subject: [PATCH 462/816] shift to the master branch --- contrib/numactl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/numactl b/contrib/numactl index a1bebe8fe6f..ff32c618d63 160000 --- a/contrib/numactl +++ b/contrib/numactl @@ -1 +1 @@ -Subproject commit a1bebe8fe6f6efebb23168bc561d240f0f64ca4b +Subproject commit ff32c618d63ca7ac48cce366c5a04bb3563683a0 From a7ff9bfb1c1cf6e964e3da48dbc8716f6b38fa03 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 26 Oct 2024 11:01:35 +0000 Subject: [PATCH 463/816] Update autogenerated version to 24.11.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 ++--- .../StorageSystemContributors.generated.cpp | 37 +++++++++++++++++++ 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 91a7e976aaf..99141510248 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54491) +SET(VERSION_REVISION 54492) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 10) +SET(VERSION_MINOR 11) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH b12a367741812f9e5fe754d19ebae600e2a2614c) -SET(VERSION_DESCRIBE v24.10.1.1-testing) -SET(VERSION_STRING 24.10.1.1) +SET(VERSION_GITHASH c82cf25b3e5864bcc153cbe45adb8c6527e1ec6e) +SET(VERSION_DESCRIBE v24.11.1.1-testing) +SET(VERSION_STRING 24.11.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 5d4598c25dc..69c46fa055c 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -121,6 +121,7 @@ const char * auto_contributors[] { "Aliaksandr Shylau", "Aliaksei Khatskevich", "Alina Terekhova", + "Alsu Giliazova", "Amesaru", "Amila Welihinda", "Amir Vaza", @@ -225,6 +226,8 @@ const char * auto_contributors[] { "BSD_Conqueror", "BSWaterB", "Babacar Diassé", + "Baitur", + "Baitur Ulukbekov", "Bakhtiyor Ruziev", "BanyRule", "Barum Rho", @@ -301,6 +304,8 @@ const char * auto_contributors[] { "Dale McDiarmid", "Dale Mcdiarmid", "Dalitso Banda", + "Damian Kula", + "DamianMaslanka5", "Dan Roscigno", "Dan Wu", "DanRoscigno", @@ -312,6 +317,7 @@ const char * auto_contributors[] { "Daniel Kutenin", "Daniel Pozo Escalona", "Daniel Qin", + "Daniil Gentili", "Daniil Ivanik", "Daniil Rubin", "Danila Kutenin", @@ -324,6 +330,7 @@ const char * auto_contributors[] { "DarkWanderer", "Darío", "Dave Lahn", + "David Tsukernik", "Davit Vardanyan", "Denis Burlaka", "Denis Glazachev", @@ -335,6 +342,8 @@ const char * auto_contributors[] { "Derek Chia", "Derek Perkins", "Dergousov", + "Dergousov Maxim", + "Diana Carroll", "Diego Nieto", "Diego Nieto (lesandie)", "DimaAmega", @@ -414,6 +423,7 @@ const char * auto_contributors[] { "Fabian Stäber", "Fabiano Francesconi", "Fadi Hadzh", + "Faizan Patel", "Fan()", "Fangyuan Deng", "FawnD2", @@ -501,6 +511,7 @@ const char * auto_contributors[] { "HowePa", "HuFuwang", "Hui Wang", + "Hung Duong", "ILya Limarenko", "Ignat Loskutov", "Igor", @@ -579,6 +590,7 @@ const char * auto_contributors[] { "Jiebin Sun", "Jihyuk Bok", "Jiyoung Yoo", + "Jiří Kozlovský", "Joanna Hulboj", "Jochen Schalanda", "Joe Lynch", @@ -615,6 +627,7 @@ const char * auto_contributors[] { "János Benjamin Antal", "Kang Liu", "Karl Pietrzak", + "Kaushik Iska", "Keiji Yoshida", "Ken Chen", "Ken MacInnis", @@ -647,6 +660,7 @@ const char * auto_contributors[] { "Konstantin Podshumok", "Konstantin Rudenskii", "Konstantin Smirnov", + "Konstantin Vedernikov", "Korenevskiy Denis", "Korviakov Andrey", "Kostiantyn Storozhuk", @@ -685,6 +699,7 @@ const char * auto_contributors[] { "Li Yin", "Linh Giang", "Lino Uruñuela", + "Lionel Palacin", "Lirikl", "Liu Cong", "LiuCong", @@ -728,6 +743,7 @@ const char * auto_contributors[] { "Marek Vavruša", "Marek Vavruša", "Mariano Benítez Mulet", + "Mariia Khristenko", "Marina Fathouat", "Mark Andreev", "Mark Frost", @@ -816,6 +832,7 @@ const char * auto_contributors[] { "Mikhail Surin", "Mikhail f. Shiryaev", "MikhailBurdukov", + "Miki Matsumoto", "MikuSugar", "Milad Arabi", "Mingliang Pan", @@ -892,6 +909,7 @@ const char * auto_contributors[] { "Okada Haruki", "Oleg Ershov", "Oleg Favstov", + "Oleg Galizin", "Oleg Komarov", "Oleg Matrokhin", "Oleg Obleukhov", @@ -917,6 +935,7 @@ const char * auto_contributors[] { "Palash Goel", "PapaToemmsn", "Paramtamtam", + "Patrick Druley", "Patrick Zippenfenig", "Paul Loyd", "Pavel", @@ -1031,6 +1050,7 @@ const char * auto_contributors[] { "Sariel", "Sasha Sheikin", "Saulius Valatka", + "SayeedKhan21", "Sean Haynes", "Sean Lafferty", "Selfuppen", @@ -1071,8 +1091,11 @@ const char * auto_contributors[] { "Shane Andrade", "Shanfeng Pang", "Shani Elharrar", + "Sharath K S", "Shaun Struwig", "Sherry Wang", + "Shichao", + "Shichao Jin", "Shoh Jahon", "Shri Bodas", "Shuai li", @@ -1221,6 +1244,7 @@ const char * auto_contributors[] { "Vladimir Ch", "Vladimir Chebotarev", "Vladimir Chebotaryov", + "Vladimir Cherkasov", "Vladimir Galunshchikov", "Vladimir Golovchenko", "Vladimir Goncharov", @@ -1231,6 +1255,7 @@ const char * auto_contributors[] { "Vladimir Makarov", "Vladimir Mihailenco", "Vladimir Smirnov", + "Vladimir Valerianov", "Vladimir Varankin", "Vladislav Rassokhin", "Vladislav Smirnov", @@ -1296,6 +1321,7 @@ const char * auto_contributors[] { "Yury Karpovich", "Yury Stankevich", "Yusuke Tanaka", + "Z.H.", "Zach Naimon", "Zawa-II", "Zheng Miao", @@ -1329,6 +1355,7 @@ const char * auto_contributors[] { "akuzm", "alekar", "alekseik1", + "aleksey", "alekseygolub", "alesapin", "alex filatov", @@ -1342,6 +1369,7 @@ const char * auto_contributors[] { "alexeypavlenko", "alfredlu", "allegrinisante", + "alsu", "amesaru", "amoschen", "amudong", @@ -1593,6 +1621,7 @@ const char * auto_contributors[] { "joe09@foxmail.com", "joelynch", "johanngan", + "johnnyfish", "johnnymatthews", "josh-hildred", "jsc0218", @@ -1626,6 +1655,8 @@ const char * auto_contributors[] { "kshvakov", "kssenii", "kst-morozov", + "kurikuQwQ", + "kylhuk", "l", "l1tsolaiki", "laimuxi", @@ -1683,6 +1714,7 @@ const char * auto_contributors[] { "luc1ph3r", "lulichao", "luocongkai", + "lwz9103", "lzydmxy", "m-ves", "m4xxx1m", @@ -1840,6 +1872,7 @@ const char * auto_contributors[] { "sarielwxm", "satanson", "save-my-heart", + "scanhex12", "sdk2", "selfuppen", "serebrserg", @@ -1850,6 +1883,7 @@ const char * auto_contributors[] { "sfod", "shabroo", "shangshujie", + "sharathks118", "shedx", "shiyer7474", "shuai-xu", @@ -1858,6 +1892,7 @@ const char * auto_contributors[] { "sichenzhao", "simon-says", "simpleton", + "singhksandeep25", "siyuan", "skyoct", "slu", @@ -1874,6 +1909,7 @@ const char * auto_contributors[] { "stavrolia", "stepenhu", "su-houzhen", + "sum12", "sundy", "sundy-li", "sundyli", @@ -1906,6 +1942,7 @@ const char * auto_contributors[] { "tomtana", "topvisor", "tpanetti", + "tuanpach", "turbo jason", "tyrionhuang", "ubuntu", From 92b13c5dc74d3384a1aa836f475260f914e40af5 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 24 Oct 2024 11:17:00 +0000 Subject: [PATCH 464/816] CI: Build Job with praktika --- ci/docker/fasttest/Dockerfile | 4 + ci/jobs/build_clickhouse.py | 102 ++++++++++++++++++ ci/jobs/check_style.py | 16 +-- ci/jobs/fast_test.py | 14 ++- ci/praktika/_environment.py | 5 +- ci/praktika/_settings.py | 6 +- ci/praktika/digest.py | 18 +++- ci/praktika/hook_html.py | 53 +++++++++- ci/praktika/json.html | 190 +++++++++++++++++++--------------- ci/praktika/runner.py | 12 +-- ci/praktika/utils.py | 4 +- ci/praktika/yaml_generator.py | 3 +- ci/settings/definitions.py | 45 ++++---- ci/settings/settings.py | 2 +- ci/workflows/pull_request.py | 52 +++++++++- 15 files changed, 378 insertions(+), 148 deletions(-) create mode 100644 ci/jobs/build_clickhouse.py diff --git a/ci/docker/fasttest/Dockerfile b/ci/docker/fasttest/Dockerfile index 02595ad0d0a..66e48b163b8 100644 --- a/ci/docker/fasttest/Dockerfile +++ b/ci/docker/fasttest/Dockerfile @@ -33,6 +33,8 @@ RUN apt-get update \ # moreutils - provides ts fo FT # expect, bzip2 - requried by FT # bsdmainutils - provides hexdump for FT +# nasm - nasm copiler for one of submodules, required from normal build +# yasm - asssembler for libhdfs3, required from normal build RUN apt-get update \ && apt-get install \ @@ -53,6 +55,8 @@ RUN apt-get update \ pv \ jq \ bzip2 \ + nasm \ + yasm \ --yes --no-install-recommends \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* diff --git a/ci/jobs/build_clickhouse.py b/ci/jobs/build_clickhouse.py new file mode 100644 index 00000000000..21ed8091608 --- /dev/null +++ b/ci/jobs/build_clickhouse.py @@ -0,0 +1,102 @@ +import argparse + +from praktika.result import Result +from praktika.settings import Settings +from praktika.utils import MetaClasses, Shell, Utils + + +class JobStages(metaclass=MetaClasses.WithIter): + CHECKOUT_SUBMODULES = "checkout" + CMAKE = "cmake" + BUILD = "build" + + +def parse_args(): + parser = argparse.ArgumentParser(description="ClickHouse Build Job") + parser.add_argument("BUILD_TYPE", help="Type: ") + parser.add_argument("--param", help="Optional custom job start stage", default=None) + return parser.parse_args() + + +def main(): + + args = parse_args() + + stop_watch = Utils.Stopwatch() + + stages = list(JobStages) + stage = args.param or JobStages.CHECKOUT_SUBMODULES + if stage: + assert stage in JobStages, f"--param must be one of [{list(JobStages)}]" + print(f"Job will start from stage [{stage}]") + while stage in stages: + stages.pop(0) + stages.insert(0, stage) + + cmake_build_type = "Release" + sanitizer = "" + + if "debug" in args.BUILD_TYPE.lower(): + print("Build type set: debug") + cmake_build_type = "Debug" + + if "asan" in args.BUILD_TYPE.lower(): + print("Sanitizer set: address") + sanitizer = "address" + + # if Environment.is_local_run(): + # build_cache_type = "disabled" + # else: + build_cache_type = "sccache" + + current_directory = Utils.cwd() + build_dir = f"{Settings.TEMP_DIR}/build" + + res = True + results = [] + + if res and JobStages.CHECKOUT_SUBMODULES in stages: + Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}") + results.append( + Result.create_from_command_execution( + name="Checkout Submodules", + command=f"git submodule sync --recursive && git submodule init && git submodule update --depth 1 --recursive --jobs {min([Utils.cpu_count(), 20])}", + ) + ) + res = results[-1].is_ok() + + if res and JobStages.CMAKE in stages: + results.append( + Result.create_from_command_execution( + name="Cmake configuration", + command=f"cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE={cmake_build_type} \ + -DSANITIZE={sanitizer} -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DENABLE_TESTS=0 \ + -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr \ + -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON \ + -DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 -DCOMPILER_CACHE={build_cache_type} -DENABLE_TESTS=1 \ + -DENABLE_BUILD_PROFILING=1 {current_directory}", + workdir=build_dir, + with_log=True, + ) + ) + res = results[-1].is_ok() + + if res and JobStages.BUILD in stages: + Shell.check("sccache --show-stats") + results.append( + Result.create_from_command_execution( + name="Build ClickHouse", + command="ninja clickhouse-bundle clickhouse-odbc-bridge clickhouse-library-bridge", + workdir=build_dir, + with_log=True, + ) + ) + Shell.check("sccache --show-stats") + Shell.check(f"ls -l {build_dir}/programs/") + res = results[-1].is_ok() + + Result.create_from(results=results, stopwatch=stop_watch).finish_job_accordingly() + + +if __name__ == "__main__": + main() diff --git a/ci/jobs/check_style.py b/ci/jobs/check_style.py index 1b1b0bf689b..f9cdc76302d 100644 --- a/ci/jobs/check_style.py +++ b/ci/jobs/check_style.py @@ -68,7 +68,7 @@ def check_duplicate_includes(file_path): def check_whitespaces(file_paths): for file in file_paths: exit_code, out, err = Shell.get_res_stdout_stderr( - f'./ci_v2/jobs/scripts/check_style/double_whitespaces.pl "{file}"', + f'./ci/jobs/scripts/check_style/double_whitespaces.pl "{file}"', verbose=False, ) if out or err: @@ -174,7 +174,7 @@ def check_broken_links(path, exclude_paths): def check_cpp_code(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check_cpp.sh" + "./ci/jobs/scripts/check_style/check_cpp.sh" ) if err: out += err @@ -183,7 +183,7 @@ def check_cpp_code(): def check_repo_submodules(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check_submodules.sh" + "./ci/jobs/scripts/check_style/check_submodules.sh" ) if err: out += err @@ -192,7 +192,7 @@ def check_repo_submodules(): def check_other(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/checks_to_refactor.sh" + "./ci/jobs/scripts/check_style/checks_to_refactor.sh" ) if err: out += err @@ -201,7 +201,7 @@ def check_other(): def check_codespell(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check_typos.sh" + "./ci/jobs/scripts/check_style/check_typos.sh" ) if err: out += err @@ -210,7 +210,7 @@ def check_codespell(): def check_aspell(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check_aspell.sh" + "./ci/jobs/scripts/check_style/check_aspell.sh" ) if err: out += err @@ -219,7 +219,7 @@ def check_aspell(): def check_mypy(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check-mypy" + "./ci/jobs/scripts/check_style/check-mypy" ) if err: out += err @@ -228,7 +228,7 @@ def check_mypy(): def check_pylint(): res, out, err = Shell.get_res_stdout_stderr( - "./ci_v2/jobs/scripts/check_style/check-pylint" + "./ci/jobs/scripts/check_style/check-pylint" ) if err: out += err diff --git a/ci/jobs/fast_test.py b/ci/jobs/fast_test.py index b82c17aa42c..1dcd65b6ed2 100644 --- a/ci/jobs/fast_test.py +++ b/ci/jobs/fast_test.py @@ -1,12 +1,13 @@ +import argparse import threading from pathlib import Path -from ci_v2.jobs.scripts.functional_tests_results import FTResultsProcessor -from praktika.environment import Environment from praktika.result import Result from praktika.settings import Settings from praktika.utils import MetaClasses, Shell, Utils +from ci.jobs.scripts.functional_tests_results import FTResultsProcessor + class ClickHouseProc: def __init__(self): @@ -208,11 +209,18 @@ class JobStages(metaclass=MetaClasses.WithIter): TEST = "test" +def parse_args(): + parser = argparse.ArgumentParser(description="ClickHouse Fast Test Job") + parser.add_argument("--param", help="Optional custom job start stage", default=None) + return parser.parse_args() + + def main(): + args = parse_args() stop_watch = Utils.Stopwatch() stages = list(JobStages) - stage = Environment.LOCAL_RUN_PARAM or JobStages.CHECKOUT_SUBMODULES + stage = args.param or JobStages.CHECKOUT_SUBMODULES if stage: assert stage in JobStages, f"--param must be one of [{list(JobStages)}]" print(f"Job will start from stage [{stage}]") diff --git a/ci/praktika/_environment.py b/ci/praktika/_environment.py index ca84def1d29..ce9c6f5b486 100644 --- a/ci/praktika/_environment.py +++ b/ci/praktika/_environment.py @@ -29,9 +29,9 @@ class _Environment(MetaClasses.Serializable): INSTANCE_TYPE: str INSTANCE_ID: str INSTANCE_LIFE_CYCLE: str + LOCAL_RUN: bool = False PARAMETER: Any = None REPORT_INFO: List[str] = dataclasses.field(default_factory=list) - LOCAL_RUN_PARAM: str = "" name = "environment" @classmethod @@ -185,6 +185,9 @@ class _Environment(MetaClasses.Serializable): REPORT_URL = f"https://{path}/{Path(Settings.HTML_PAGE_FILE).name}?PR={self.PR_NUMBER}&sha={self.SHA}&name_0={urllib.parse.quote(self.WORKFLOW_NAME, safe='')}&name_1={urllib.parse.quote(self.JOB_NAME, safe='')}" return REPORT_URL + def is_local_run(self): + return self.LOCAL_RUN + def _to_object(data): if isinstance(data, dict): diff --git a/ci/praktika/_settings.py b/ci/praktika/_settings.py index bfd7ba6c1be..3052d8ef877 100644 --- a/ci/praktika/_settings.py +++ b/ci/praktika/_settings.py @@ -8,11 +8,7 @@ class _Settings: ###################################### # Pipeline generation settings # ###################################### - if Path("./ci_v2").is_dir(): - # TODO: hack for CH, remove - CI_PATH = "./ci_v2" - else: - CI_PATH = "./ci" + CI_PATH = "./ci" WORKFLOW_PATH_PREFIX: str = "./.github/workflows" WORKFLOWS_DIRECTORY: str = f"{CI_PATH}/workflows" SETTINGS_DIRECTORY: str = f"{CI_PATH}/settings" diff --git a/ci/praktika/digest.py b/ci/praktika/digest.py index 44317d5249e..d505e7e7206 100644 --- a/ci/praktika/digest.py +++ b/ci/praktika/digest.py @@ -1,6 +1,8 @@ import dataclasses import hashlib +import os from hashlib import md5 +from pathlib import Path from typing import List from praktika import Job @@ -37,7 +39,7 @@ class Digest: sorted=True, ) - print(f"calc digest: hash_key [{cache_key}], include [{included_files}] files") + print(f"calc digest for job [{job_config.name}]: hash_key [{cache_key}], include [{len(included_files)}] files") # Sort files to ensure consistent hash calculation included_files.sort() @@ -91,10 +93,16 @@ class Digest: @staticmethod def _calc_file_digest(file_path, hash_md5): - # Calculate MD5 hash - with open(file_path, "rb") as f: + # Resolve file path if it's a symbolic link + resolved_path = file_path + if Path(file_path).is_symlink(): + resolved_path = os.path.realpath(file_path) + if not Path(resolved_path).is_file(): + print(f"WARNING: No valid file resolved by link {file_path} -> {resolved_path} - skipping digest calculation") + return hash_md5.hexdigest()[:Settings.CACHE_DIGEST_LEN] + + with open(resolved_path, "rb") as f: for chunk in iter(lambda: f.read(4096), b""): hash_md5.update(chunk) - res = hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN] - return res + return hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN] diff --git a/ci/praktika/hook_html.py b/ci/praktika/hook_html.py index c998e817fe7..f4bd4435511 100644 --- a/ci/praktika/hook_html.py +++ b/ci/praktika/hook_html.py @@ -1,5 +1,8 @@ +import dataclasses +import json import urllib.parse from pathlib import Path +from typing import List from praktika._environment import _Environment from praktika.gh import GH @@ -8,12 +11,50 @@ from praktika.result import Result, ResultInfo from praktika.runtime import RunConfig from praktika.s3 import S3 from praktika.settings import Settings -from praktika.utils import Utils +from praktika.utils import Shell, Utils + + +@dataclasses.dataclass +class GitCommit: + date: str + message: str + sha: str + + @staticmethod + def from_json(json_data: str) -> List["GitCommit"]: + commits = [] + try: + data = json.loads(json_data) + + commits = [ + GitCommit( + message=commit["messageHeadline"], + sha=commit["oid"], + date=commit["committedDate"], + ) + for commit in data.get("commits", []) + ] + except Exception as e: + print( + f"ERROR: Failed to deserialize commit's data: [{json_data}], ex: [{e}]" + ) + + return commits class HtmlRunnerHooks: @classmethod def configure(cls, _workflow): + + def _get_pr_commits(pr_number): + res = [] + if not pr_number: + return res + output = Shell.get_output(f"gh pr view {pr_number} --json commits") + if output: + res = GitCommit.from_json(output) + return res + # generate pending Results for all jobs in the workflow if _workflow.enable_cache: skip_jobs = RunConfig.from_fs(_workflow.name).cache_success @@ -62,10 +103,14 @@ class HtmlRunnerHooks: or_update_comment_with_substring=f"Workflow [", ) if not (res1 or res2): - print( - "ERROR: Failed to set both GH commit status and PR comment with Workflow Status, cannot proceed" + Utils.raise_with_error( + "Failed to set both GH commit status and PR comment with Workflow Status, cannot proceed" ) - raise + + if env.PR_NUMBER: + commits = _get_pr_commits(env.PR_NUMBER) + # TODO: upload commits data to s3 to visualise it on a report page + print(commits) @classmethod def pre_run(cls, _workflow, _job): diff --git a/ci/praktika/json.html b/ci/praktika/json.html index fe7b65a5ec5..2f8c3e45d0b 100644 --- a/ci/praktika/json.html +++ b/ci/praktika/json.html @@ -24,13 +24,15 @@ margin: 0; display: flex; flex-direction: column; - font-family: monospace, sans-serif; + font-family: 'IBM Plex Mono Condensed', monospace, sans-serif; + --header-background-color: #f4f4f4; } body.night-theme { --background-color: #1F1F1C; --text-color: #fff; --tile-background: black; + --header-background-color: #1F1F1C; } #info-container { @@ -50,27 +52,41 @@ background-color: var(--tile-background); padding: 20px; box-sizing: border-box; - text-align: left; font-size: 18px; + margin: 0; + } + + #status-container a { + color: #007bff; + text-decoration: underline; font-weight: bold; - margin: 0; /* Remove margin */ - } - - #status-container button { - display: block; /* Stack buttons vertically */ - width: 100%; /* Full width of container */ - padding: 10px; - margin-bottom: 10px; /* Space between buttons */ - background-color: #4CAF50; /* Green background color */ - color: white; - border: none; - border-radius: 5px; - font-size: 16px; cursor: pointer; + display: inline-block; + margin-top: 5px; + margin-left: 20px; + padding: 2px 0; + font-size: 0.8em; } - #status-container button:hover { - background-color: #45a049; /* Darker green on hover */ + #status-container a:hover { + color: #0056b3; + text-decoration: none; + } + + .key-value-pair { + display: flex; /* Enable Flexbox for alignment */ + justify-content: space-between; /* Distribute space between key and value */ + margin-bottom: 20px; /* Add space between each pair */ + } + + .json-key { + font-weight: bold; + } + + .json-value { + font-weight: normal; + font-family: 'Source Code Pro', monospace, sans-serif; + letter-spacing: -0.5px; } #result-container { @@ -203,7 +219,7 @@ } th { - background-color: #f4f4f4; + background-color: var(--header-background-color); } .status-success { @@ -240,23 +256,6 @@ color: grey; font-weight: bold; } - - .json-key { - font-weight: bold; - margin-top: 10px; - } - - .json-value { - margin-left: 20px; - } - - .json-value a { - color: #007bff; - } - - .json-value a:hover { - text-decoration: underline; - } @@ -286,7 +285,6 @@ // Attach the toggle function to the click event of the icon document.getElementById('theme-toggle').addEventListener('click', toggleTheme); - // Function to format timestamp to "DD-mmm-YYYY HH:MM:SS.MM" function formatTimestamp(timestamp, showDate = true) { const date = new Date(timestamp * 1000); const day = String(date.getDate()).padStart(2, '0'); @@ -304,6 +302,38 @@ : `${hours}:${minutes}:${seconds}`; } + function formatDuration(durationInSeconds, detailed = false) { + // Check if the duration is empty, null, or not a number + if (!durationInSeconds || isNaN(durationInSeconds)) { + return ''; + } + + // Ensure duration is a floating-point number + const duration = parseFloat(durationInSeconds); + + if (detailed) { + // Format in the detailed format with hours, minutes, and seconds + const hours = Math.floor(duration / 3600); + const minutes = Math.floor((duration % 3600) / 60); + const seconds = Math.floor(duration % 60); + + const formattedHours = hours > 0 ? `${hours}h ` : ''; + const formattedMinutes = minutes > 0 ? `${minutes}m ` : ''; + const formattedSeconds = `${String(seconds).padStart(2, '0')}s`; + + return `${formattedHours}${formattedMinutes}${formattedSeconds}`.trim(); + } else { + // Format in the default format with seconds and milliseconds + const seconds = Math.floor(duration); + const milliseconds = Math.floor((duration % 1) * 1000); + + const formattedSeconds = String(seconds); + const formattedMilliseconds = String(milliseconds).padStart(3, '0'); + + return `${formattedSeconds}.${formattedMilliseconds}`; + } + } + // Function to determine status class based on value function getStatusClass(status) { const lowerStatus = status.toLowerCase(); @@ -316,32 +346,13 @@ return 'status-other'; } - // Function to format duration from seconds to "HH:MM:SS" - function formatDuration(durationInSeconds) { - // Check if the duration is empty, null, or not a number - if (!durationInSeconds || isNaN(durationInSeconds)) { - return ''; - } - - // Ensure duration is a floating-point number - const duration = parseFloat(durationInSeconds); - - // Calculate seconds and milliseconds - const seconds = Math.floor(duration); // Whole seconds - const milliseconds = Math.floor((duration % 1) * 1000); // Convert fraction to milliseconds - - // Format seconds and milliseconds with leading zeros where needed - const formattedSeconds = String(seconds); - const formattedMilliseconds = String(milliseconds).padStart(3, '0'); - - // Return the formatted duration as seconds.milliseconds - return `${formattedSeconds}.${formattedMilliseconds}`; - } - function addKeyValueToStatus(key, value) { const statusContainer = document.getElementById('status-container'); + let keyValuePair = document.createElement('div'); + keyValuePair.className = 'key-value-pair'; + const keyElement = document.createElement('div'); keyElement.className = 'json-key'; keyElement.textContent = key + ':'; @@ -350,8 +361,9 @@ valueElement.className = 'json-value'; valueElement.textContent = value; - statusContainer.appendChild(keyElement); - statusContainer.appendChild(valueElement); + keyValuePair.appendChild(keyElement) + keyValuePair.appendChild(valueElement) + statusContainer.appendChild(keyValuePair); } function addFileButtonToStatus(key, links) { @@ -364,64 +376,68 @@ const keyElement = document.createElement('div'); keyElement.className = 'json-key'; - keyElement.textContent = key + ':'; + keyElement.textContent = columnSymbols[key] + ':' || key; statusContainer.appendChild(keyElement); if (Array.isArray(links) && links.length > 0) { links.forEach(link => { - // const a = document.createElement('a'); - // a.href = link; - // a.textContent = link.split('/').pop(); - // a.target = '_blank'; - // statusContainer.appendChild(a); - const button = document.createElement('button'); - button.textContent = link.split('/').pop(); - button.addEventListener('click', function () { - window.location.href = link; - }); - statusContainer.appendChild(button); + const textLink = document.createElement('a'); + textLink.href = link; + textLink.textContent = link.split('/').pop(); + textLink.target = '_blank'; + statusContainer.appendChild(textLink); + statusContainer.appendChild(document.createElement('br')); }); } } function addStatusToStatus(status, start_time, duration) { - const statusContainer = document.getElementById('status-container'); + const statusContainer = document.getElementById('status-container') + let keyValuePair = document.createElement('div'); + keyValuePair.className = 'key-value-pair'; let keyElement = document.createElement('div'); let valueElement = document.createElement('div'); keyElement.className = 'json-key'; valueElement.className = 'json-value'; - keyElement.textContent = 'status:'; + keyElement.textContent = columnSymbols['status'] + ':' || 'status:'; valueElement.classList.add('status-value'); valueElement.classList.add(getStatusClass(status)); valueElement.textContent = status; - statusContainer.appendChild(keyElement); - statusContainer.appendChild(valueElement); + keyValuePair.appendChild(keyElement); + keyValuePair.appendChild(valueElement); + statusContainer.appendChild(keyValuePair); + keyValuePair = document.createElement('div'); + keyValuePair.className = 'key-value-pair'; keyElement = document.createElement('div'); valueElement = document.createElement('div'); keyElement.className = 'json-key'; valueElement.className = 'json-value'; - keyElement.textContent = 'start_time:'; + keyElement.textContent = columnSymbols['start_time'] + ':' || 'start_time:'; valueElement.textContent = formatTimestamp(start_time); - statusContainer.appendChild(keyElement); - statusContainer.appendChild(valueElement); + keyValuePair.appendChild(keyElement); + keyValuePair.appendChild(valueElement); + statusContainer.appendChild(keyValuePair); + keyValuePair = document.createElement('div'); + keyValuePair.className = 'key-value-pair'; keyElement = document.createElement('div'); valueElement = document.createElement('div'); keyElement.className = 'json-key'; valueElement.className = 'json-value'; - keyElement.textContent = 'duration:'; + keyElement.textContent = columnSymbols['duration'] + ':' || 'duration:'; if (duration === null) { // Set initial value to 0 and add a unique ID or data attribute to identify the duration element valueElement.textContent = '00:00:00'; valueElement.setAttribute('id', 'duration-value'); } else { // Format the duration if it's a valid number - valueElement.textContent = formatDuration(duration); + valueElement.textContent = formatDuration(duration, true); } - statusContainer.appendChild(keyElement); - statusContainer.appendChild(valueElement); + keyValuePair.appendChild(keyElement); + keyValuePair.appendChild(valueElement); + statusContainer.appendChild(keyValuePair); } function navigatePath(jsonObj, nameArray) { @@ -470,11 +486,12 @@ const columns = ['name', 'status', 'start_time', 'duration', 'info']; const columnSymbols = { - name: '👤', + name: '📂', status: '✔️', start_time: '🕒', duration: '⏳', - info: '⚠️' + info: 'ℹ️', + files: '📄' }; function createResultsTable(results, nest_level) { @@ -626,6 +643,7 @@ footerRight.appendChild(a); }); } + addStatusToStatus(targetData.status, targetData.start_time, targetData.duration) // Handle links @@ -639,7 +657,7 @@ const intervalId = setInterval(() => { duration++; - durationElement.textContent = formatDuration(duration); + durationElement.textContent = formatDuration(duration, true); }, 1000); } diff --git a/ci/praktika/runner.py b/ci/praktika/runner.py index 15e759397ec..797a799a74d 100644 --- a/ci/praktika/runner.py +++ b/ci/praktika/runner.py @@ -42,6 +42,7 @@ class Runner: INSTANCE_ID="", INSTANCE_TYPE="", INSTANCE_LIFE_CYCLE="", + LOCAL_RUN=True, ).dump() workflow_config = RunConfig( name=workflow.name, @@ -76,9 +77,6 @@ class Runner: os.environ[key] = value print(f"Set environment variable {key}.") - # TODO: remove - os.environ["PYTHONPATH"] = os.getcwd() - print("Read GH Environment") env = _Environment.from_env() env.JOB_NAME = job.name @@ -132,9 +130,7 @@ class Runner: f"Custom param for local tests must be of type str, got [{type(param)}]" ) env = _Environment.get() - env.LOCAL_RUN_PARAM = param env.dump() - print(f"Custom param for local tests [{param}] dumped into Environment") if job.run_in_docker and not no_docker: # TODO: add support for any image, including not from ci config (e.g. ubuntu:latest) @@ -142,9 +138,13 @@ class Runner: job.run_in_docker ] docker = docker or f"{job.run_in_docker}:{docker_tag}" - cmd = f"docker run --rm --user \"$(id -u):$(id -g)\" -e PYTHONPATH='{Settings.DOCKER_WD}' --volume ./:{Settings.DOCKER_WD} --volume {Settings.TEMP_DIR}:{Settings.TEMP_DIR} --workdir={Settings.DOCKER_WD} {docker} {job.command}" + cmd = f"docker run --rm --user \"$(id -u):$(id -g)\" -e PYTHONPATH='{Settings.DOCKER_WD}:{Settings.DOCKER_WD}/ci' --volume ./:{Settings.DOCKER_WD} --volume {Settings.TEMP_DIR}:{Settings.TEMP_DIR} --workdir={Settings.DOCKER_WD} {docker} {job.command}" else: cmd = job.command + + if param: + print(f"Custom --param [{param}] will be passed to job's script") + cmd += f" --param {param}" print(f"--- Run command [{cmd}]") with TeePopen(cmd, timeout=job.timeout) as process: diff --git a/ci/praktika/utils.py b/ci/praktika/utils.py index 1983ce274a3..b96c78e4fa7 100644 --- a/ci/praktika/utils.py +++ b/ci/praktika/utils.py @@ -348,9 +348,9 @@ class Utils: return multiprocessing.cpu_count() @staticmethod - def raise_with_error(error_message, stdout="", stderr=""): + def raise_with_error(error_message, stdout="", stderr="", ex=None): Utils.print_formatted_error(error_message, stdout, stderr) - raise + raise ex or RuntimeError() @staticmethod def timestamp(): diff --git a/ci/praktika/yaml_generator.py b/ci/praktika/yaml_generator.py index 9c61b5e2f79..00c469fec0c 100644 --- a/ci/praktika/yaml_generator.py +++ b/ci/praktika/yaml_generator.py @@ -83,8 +83,8 @@ jobs: {JOB_ADDONS} - name: Prepare env script run: | - export PYTHONPATH=.:$PYTHONPATH cat > {ENV_SETUP_SCRIPT} << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. {SETUP_ENVS} cat > {WORKFLOW_CONFIG_FILE} << 'EOF' ${{{{ needs.{WORKFLOW_CONFIG_JOB_NAME}.outputs.data }}}} @@ -100,6 +100,7 @@ jobs: - name: Run id: run run: | + . /tmp/praktika_setup_env.sh set -o pipefail {PYTHON} -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& tee {RUN_LOG} {UPLOADS_GITHUB}\ diff --git a/ci/settings/definitions.py b/ci/settings/definitions.py index 4e6a7f213f0..176e865e6f3 100644 --- a/ci/settings/definitions.py +++ b/ci/settings/definitions.py @@ -30,133 +30,133 @@ SECRETS = [ DOCKERS = [ # Docker.Config( # name="clickhouse/binary-builder", - # path="./ci_v2/docker/packager/binary-builder", + # path="./ci/docker/packager/binary-builder", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), # Docker.Config( # name="clickhouse/cctools", - # path="./ci_v2/docker/packager/cctools", + # path="./ci/docker/packager/cctools", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), # Docker.Config( # name="clickhouse/test-old-centos", - # path="./ci_v2/docker/test/compatibility/centos", + # path="./ci/docker/test/compatibility/centos", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), # Docker.Config( # name="clickhouse/test-old-ubuntu", - # path="./ci_v2/docker/test/compatibility/ubuntu", + # path="./ci/docker/test/compatibility/ubuntu", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), # Docker.Config( # name="clickhouse/test-util", - # path="./ci_v2/docker/test/util", + # path="./ci/docker/test/util", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), # Docker.Config( # name="clickhouse/integration-test", - # path="./ci_v2/docker/test/integration/base", + # path="./ci/docker/test/integration/base", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/fuzzer", - # path="./ci_v2/docker/test/fuzzer", + # path="./ci/docker/test/fuzzer", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/performance-comparison", - # path="./ci_v2/docker/test/performance-comparison", + # path="./ci/docker/test/performance-comparison", # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), Docker.Config( name="clickhouse/fasttest", - path="./ci_v2/docker/fasttest", + path="./ci/docker/fasttest", platforms=Docker.Platforms.arm_amd, depends_on=[], ), # Docker.Config( # name="clickhouse/test-base", - # path="./ci_v2/docker/test/base", + # path="./ci/docker/test/base", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-util"], # ), # Docker.Config( # name="clickhouse/clickbench", - # path="./ci_v2/docker/test/clickbench", + # path="./ci/docker/test/clickbench", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/keeper-jepsen-test", - # path="./ci_v2/docker/test/keeper-jepsen", + # path="./ci/docker/test/keeper-jepsen", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/server-jepsen-test", - # path="./ci_v2/docker/test/server-jepsen", + # path="./ci/docker/test/server-jepsen", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/sqllogic-test", - # path="./ci_v2/docker/test/sqllogic", + # path="./ci/docker/test/sqllogic", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/sqltest", - # path="./ci_v2/docker/test/sqltest", + # path="./ci/docker/test/sqltest", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/stateless-test", - # path="./ci_v2/docker/test/stateless", + # path="./ci/docker/test/stateless", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/stateful-test", - # path="./ci_v2/docker/test/stateful", + # path="./ci/docker/test/stateful", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/stateless-test"], # ), # Docker.Config( # name="clickhouse/stress-test", - # path="./ci_v2/docker/test/stress", + # path="./ci/docker/test/stress", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/stateful-test"], # ), # Docker.Config( # name="clickhouse/unit-test", - # path="./ci_v2/docker/test/unit", + # path="./ci/docker/test/unit", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), # Docker.Config( # name="clickhouse/integration-tests-runner", - # path="./ci_v2/docker/test/integration/runner", + # path="./ci/docker/test/integration/runner", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), Docker.Config( name="clickhouse/style-test", - path="./ci_v2/docker/style-test", + path="./ci/docker/style-test", platforms=Docker.Platforms.arm_amd, depends_on=[], ), # Docker.Config( # name="clickhouse/docs-builder", - # path="./ci_v2/docker/docs/builder", + # path="./ci/docker/docs/builder", # platforms=Docker.Platforms.arm_amd, # depends_on=["clickhouse/test-base"], # ), @@ -230,3 +230,4 @@ DOCKERS = [ class JobNames: STYLE_CHECK = "Style Check" FAST_TEST = "Fast test" + BUILD_AMD_DEBUG = "Build amd64 debug" diff --git a/ci/settings/settings.py b/ci/settings/settings.py index 153aab93506..8d5e7bc3c87 100644 --- a/ci/settings/settings.py +++ b/ci/settings/settings.py @@ -1,4 +1,4 @@ -from ci_v2.settings.definitions import ( +from ci.settings.definitions import ( S3_BUCKET_HTTP_ENDPOINT, S3_BUCKET_NAME, RunnerLabels, diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 0e96329788b..74129177efb 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -1,26 +1,62 @@ from typing import List -from ci_v2.settings.definitions import ( +from praktika import Artifact, Job, Workflow +from praktika.settings import Settings + +from ci.settings.definitions import ( BASE_BRANCH, DOCKERS, SECRETS, JobNames, RunnerLabels, ) -from praktika import Job, Workflow + + +class ArtifactNames: + ch_debug_binary = "clickhouse_debug_binary" + style_check_job = Job.Config( name=JobNames.STYLE_CHECK, runs_on=[RunnerLabels.CI_SERVICES], - command="python3 ./ci_v2/jobs/check_style.py", + command="python3 ./ci/jobs/check_style.py", run_in_docker="clickhouse/style-test", ) fast_test_job = Job.Config( name=JobNames.FAST_TEST, runs_on=[RunnerLabels.BUILDER], - command="python3 ./ci_v2/jobs/fast_test.py", + command="python3 ./ci/jobs/fast_test.py", run_in_docker="clickhouse/fasttest", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/fast_test.py", + "./tests/queries/0_stateless/", + "./src", + ], + ), +) + +job_build_amd_debug = Job.Config( + name=JobNames.BUILD_AMD_DEBUG, + runs_on=[RunnerLabels.BUILDER], + command="python3 ./ci/jobs/build_clickhouse.py amd_debug", + run_in_docker="clickhouse/fasttest", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./src", + "./contrib/", + "./CMakeLists.txt", + "./PreLoad.cmake", + "./cmake", + "./base", + "./programs", + "./docker/packager/packager", + "./rust", + "./tests/ci/version_helper.py", + ], + ), + provides=[ArtifactNames.ch_debug_binary], ) workflow = Workflow.Config( @@ -30,6 +66,14 @@ workflow = Workflow.Config( jobs=[ style_check_job, fast_test_job, + job_build_amd_debug, + ], + artifacts=[ + Artifact.Config( + name=ArtifactNames.ch_debug_binary, + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + ) ], dockers=DOCKERS, secrets=SECRETS, From 0f25890d3b715e0d056327b1b5a5f00a2b27bf5b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 26 Oct 2024 14:21:40 +0000 Subject: [PATCH 465/816] Automatic style fix --- ci/praktika/digest.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/ci/praktika/digest.py b/ci/praktika/digest.py index d505e7e7206..93b62b13dc0 100644 --- a/ci/praktika/digest.py +++ b/ci/praktika/digest.py @@ -39,7 +39,9 @@ class Digest: sorted=True, ) - print(f"calc digest for job [{job_config.name}]: hash_key [{cache_key}], include [{len(included_files)}] files") + print( + f"calc digest for job [{job_config.name}]: hash_key [{cache_key}], include [{len(included_files)}] files" + ) # Sort files to ensure consistent hash calculation included_files.sort() @@ -98,8 +100,10 @@ class Digest: if Path(file_path).is_symlink(): resolved_path = os.path.realpath(file_path) if not Path(resolved_path).is_file(): - print(f"WARNING: No valid file resolved by link {file_path} -> {resolved_path} - skipping digest calculation") - return hash_md5.hexdigest()[:Settings.CACHE_DIGEST_LEN] + print( + f"WARNING: No valid file resolved by link {file_path} -> {resolved_path} - skipping digest calculation" + ) + return hash_md5.hexdigest()[: Settings.CACHE_DIGEST_LEN] with open(resolved_path, "rb") as f: for chunk in iter(lambda: f.read(4096), b""): From b3ebe0dc26890ef0977f86a40ce20b70c8d8efc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2024 20:55:43 +0200 Subject: [PATCH 466/816] Fixup of TrivialMergeSelector --- src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp index b0071f9f7c4..cd1fa7b01cd 100644 --- a/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp +++ b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp @@ -78,7 +78,7 @@ TrivialMergeSelector::PartsRange TrivialMergeSelector::select( ++right; - if (partition[right].level < partition[left].level) + if (right < partition.size() && partition[right].level < partition[left].level) left = right; } From d6b38a9eaee7a9749c0880b1b8db64a109b183a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2024 20:59:26 +0200 Subject: [PATCH 467/816] Revert "Miscellaneous" --- programs/server/Server.cpp | 15 --------------- src/Access/AccessControl.h | 2 -- src/Access/Authentication.cpp | 1 - src/Access/Common/AccessType.h | 1 - src/Access/RoleCache.h | 4 ---- src/Access/tests/gtest_access_rights_ops.cpp | 3 +-- src/Core/ServerUUID.cpp | 5 ----- src/Core/ServerUUID.h | 3 --- src/Core/UUID.h | 3 --- src/Databases/DatabaseReplicated.cpp | 12 +++--------- src/Databases/enableAllExperimentalSettings.cpp | 2 -- .../ExecutablePoolDictionarySource.cpp | 7 ------- src/Dictionaries/RedisDictionarySource.cpp | 1 + src/Dictionaries/XDBCDictionarySource.cpp | 6 ------ src/Disks/DiskEncrypted.h | 2 -- src/Disks/DiskEncryptedTransaction.cpp | 1 + src/Disks/DiskType.h | 2 -- src/Disks/IDisk.h | 3 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../ObjectStorages/DiskObjectStorageMetadata.h | 2 -- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 - src/IO/ReadBufferFromPocoSocket.cpp | 5 ----- src/IO/ReadBufferFromPocoSocket.h | 2 -- src/IO/S3Common.cpp | 1 + src/Interpreters/ActionLocksManager.cpp | 2 -- src/Interpreters/BlobStorageLog.cpp | 1 - src/Interpreters/Cache/LRUFileCachePriority.h | 4 +--- src/Interpreters/Cache/SLRUFileCachePriority.h | 5 ----- src/Interpreters/DatabaseCatalog.cpp | 15 --------------- src/Interpreters/DatabaseCatalog.h | 6 ------ src/Interpreters/InterpreterSystemQuery.h | 3 --- src/Interpreters/MutationsInterpreter.h | 1 + src/Interpreters/Session.h | 1 + src/Interpreters/Squashing.cpp | 1 - src/Interpreters/executeDDLQueryOnCluster.cpp | 7 ------- src/Parsers/CommonParsers.h | 1 - src/Parsers/IAST.cpp | 1 - src/Server/CloudPlacementInfo.cpp | 3 --- src/Storages/MergeTree/FutureMergedMutatedPart.h | 1 - src/Storages/MergeTree/IMergeTreeReader.h | 1 - .../MergeTree/MergeProjectionPartsTask.cpp | 3 --- .../MergeTree/MergeTreeDataFormatVersion.h | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 2 -- src/Storages/MergeTree/MergeTreeDataPartType.h | 1 - .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 4 ++-- .../MergeTree/MergeTreeIndexGranularityInfo.cpp | 8 -------- .../MergeTree/MergeTreeIndexGranularityInfo.h | 1 - .../MergeTree/MergeTreeMutationStatus.cpp | 4 ++-- src/Storages/MergeTree/MergeTreePartInfo.h | 7 ------- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.h | 1 - src/Storages/StorageGenerateRandom.cpp | 3 --- src/Storages/TableZnodeInfo.h | 2 -- .../0_stateless/01271_show_privileges.reference | 1 - 56 files changed, 21 insertions(+), 165 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c106a68f360..35dae614d87 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2267,21 +2267,6 @@ try throw; } - bool found_stop_flag = false; - - if (has_zookeeper && global_context->getMacros()->getMacroMap().contains("replica")) - { - auto zookeeper = global_context->getZooKeeper(); - String stop_flag_path = "/clickhouse/stop_replicated_ddl_queries/{replica}"; - stop_flag_path = global_context->getMacros()->expand(stop_flag_path); - found_stop_flag = zookeeper->exists(stop_flag_path); - } - - if (found_stop_flag) - LOG_INFO(log, "Found a stop flag for replicated DDL queries. They will be disabled"); - else - DatabaseCatalog::instance().startReplicatedDDLQueries(); - LOG_DEBUG(log, "Loaded metadata."); if (has_trace_collector) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index a342c5300bf..a91686433ec 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -9,8 +9,6 @@ #include -#include "config.h" - namespace Poco { diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 1d69a659cd6..8d5d04a4ed2 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -12,7 +12,6 @@ #include "config.h" - namespace DB { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 383e7f70420..e9f24a8c685 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -193,7 +193,6 @@ enum class AccessType : uint8_t M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ M(SYSTEM_REPLICATION_QUEUES, "SYSTEM STOP REPLICATION QUEUES, SYSTEM START REPLICATION QUEUES, STOP REPLICATION QUEUES, START REPLICATION QUEUES", TABLE, SYSTEM) \ M(SYSTEM_VIRTUAL_PARTS_UPDATE, "SYSTEM STOP VIRTUAL PARTS UPDATE, SYSTEM START VIRTUAL PARTS UPDATE, STOP VIRTUAL PARTS UPDATE, START VIRTUAL PARTS UPDATE", TABLE, SYSTEM) \ - M(SYSTEM_REDUCE_BLOCKING_PARTS, "SYSTEM STOP REDUCE BLOCKING PARTS, SYSTEM START REDUCE BLOCKING PARTS, STOP REDUCE BLOCKING PARTS, START REDUCE BLOCKING PARTS", TABLE, SYSTEM) \ M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \ M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_REPLICA_READINESS, "SYSTEM REPLICA READY, SYSTEM REPLICA UNREADY", GLOBAL, SYSTEM) \ diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index b707a05346f..75d1fd32685 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -22,10 +22,6 @@ public: const std::vector & current_roles, const std::vector & current_roles_with_admin_option); - std::shared_ptr getEnabledRoles( - boost::container::flat_set current_roles, - boost::container::flat_set current_roles_with_admin_option); - private: using SubscriptionsOnRoles = std::vector>; diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 41567905a10..902fc949840 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -284,8 +284,7 @@ TEST(AccessRights, Union) "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " - "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, " - "SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM REDUCE BLOCKING PARTS, " + "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index 5b17017e7f4..251b407e673 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -68,11 +68,6 @@ UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log) } } -void ServerUUID::set(UUID & uuid) -{ - server_uuid = uuid; -} - void ServerUUID::setRandomForUnitTests() { server_uuid = UUIDHelpers::generateV4(); diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 26711bfbfaa..9c7f7d32acc 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -20,9 +20,6 @@ public: /// Loads server UUID from file or creates new one. Should be called on daemon startup. static void load(const fs::path & server_uuid_file, Poco::Logger * log); - /// Sets specific server UUID. - static void set(UUID & uuid); - static void setRandomForUnitTests(); }; diff --git a/src/Core/UUID.h b/src/Core/UUID.h index 1b8a075f0d2..2bdefe9d3fc 100644 --- a/src/Core/UUID.h +++ b/src/Core/UUID.h @@ -64,9 +64,6 @@ namespace UUIDHelpers /// Generate random UUID. UUID generateV4(); - /// Generate UUID from hash of a string. - UUID makeUUIDv4FromHash(const String & string); - constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1; constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 387667b1b42..d4eaaf750cd 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -85,7 +85,6 @@ namespace ErrorCodes extern const int NO_ACTIVE_REPLICAS; extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT; extern const int CANNOT_RESTORE_TABLE; - extern const int QUERY_IS_PROHIBITED; extern const int SUPPORT_IS_DISABLED; } @@ -1058,9 +1057,6 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex { waitDatabaseStarted(); - if (!DatabaseCatalog::instance().canPerformReplicatedDDLQueries()) - throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); - if (query_context->getCurrentTransaction() && query_context->getSettingsRef()[Setting::throw_on_unsupported_query_inside_transaction]) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported"); @@ -1241,16 +1237,14 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name)); auto query_context = Context::createCopy(getContext()); query_context->setSetting("allow_deprecated_database_ordinary", 1); - query_context->setSetting("cloud_mode", false); - executeQuery(query, query_context, QueryFlags{ .internal = true }); + executeQuery(query, query_context, QueryFlags{.internal = true}); /// But we want to avoid discarding UUID of ReplicatedMergeTree tables, because it will not work /// if zookeeper_path contains {uuid} macro. Replicated database do not recreate replicated tables on recovery, /// so it's ok to save UUID of replicated table. query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Atomic", backQuoteIfNeed(to_db_name_replicated)); query_context = Context::createCopy(getContext()); - query_context->setSetting("cloud_mode", false); - executeQuery(query, query_context, QueryFlags{ .internal = true }); + executeQuery(query, query_context, QueryFlags{.internal = true}); } size_t moved_tables = 0; @@ -1640,7 +1634,7 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl auto table = tryGetTable(table_name, getContext()); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} doesn't exist", table_name); - if (table->getName() == "MaterializedView" || table->getName() == "WindowView" || table->getName() == "SharedSet" || table->getName() == "SharedJoin") + if (table->getName() == "MaterializedView" || table->getName() == "WindowView") { /// Avoid recursive locking of metadata_mutex table->dropInnerTableIfAny(sync, local_context); diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index 6efbc429fd8..d1b3b776370 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -43,8 +43,6 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("enable_zstd_qat_codec", 1); context->setSetting("allow_create_index_without_type", 1); context->setSetting("allow_experimental_s3queue", 1); - - /// clickhouse-private settings context->setSetting("allow_experimental_shared_set_join", 1); } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 602fde0e0d7..403ce540e76 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -26,9 +26,6 @@ namespace DB namespace Setting { extern const SettingsSeconds max_execution_time; - - /// Cloud only - extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -36,7 +33,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; extern const int UNSUPPORTED_METHOD; - extern const int SUPPORT_IS_DISABLED; } ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( @@ -196,9 +192,6 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { - if (global_context->getSettingsRef()[Setting::cloud_mode]) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `executable pool` is disabled"); - if (dict_struct.has_expressions) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable_pool` does not support attribute expressions"); diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 26d9ebae1b8..17ed515ca9a 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -29,6 +29,7 @@ namespace DB ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { + auto redis_config_prefix = config_prefix + ".redis"; auto host = config.getString(redis_config_prefix + ".host"); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 4e64db5831d..ebb50f79497 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -28,9 +28,6 @@ namespace Setting { extern const SettingsSeconds http_receive_timeout; extern const SettingsBool odbc_bridge_use_connection_pooling; - - /// Cloud only - extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -245,9 +242,6 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) ContextPtr global_context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { - - if (global_context->getSettingsRef()[Setting::cloud_mode]) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `odbc` is disabled"); #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( global_context, diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 95d9554b909..caba4184a73 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -313,8 +313,6 @@ public: return std::make_shared(*this); } - /// Need to overwrite explicetly because this disk change - /// a lot of "delegate" methods. return createEncryptedTransaction(); } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index a528564fd1e..2660051e1d3 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -1,5 +1,6 @@ #include + #if USE_SSL #include #include diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index bf7ef3d30eb..347e2c1cfe3 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -27,11 +27,9 @@ enum class MetadataStorageType : uint8_t { None, Local, - Keeper, Plain, PlainRewritable, StaticWeb, - Memory, }; MetadataStorageType metadataTypeFromString(const String & type); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 692020c86a6..59f58a816e9 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -497,7 +497,7 @@ public: protected: - friend class DiskReadOnlyWrapper; + friend class DiskDecorator; const String name; @@ -580,7 +580,6 @@ inline String directoryPath(const String & path) return fs::path(path).parent_path() / ""; } - } template <> diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8e4ec6f3dfb..7055a7018ce 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size) { /// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task. - if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache) + if (!settings.enable_filesystem_cache) return settings.remote_fs_buffer_size; /// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file. diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 456b3a4778d..4f45f5b7ddf 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -56,8 +56,6 @@ public: void deserialize(ReadBuffer & buf); void deserializeFromString(const std::string & data); - /// This method was deleted from public fork recently by Azat - void createFromSingleObject(ObjectStorageKey object_key, size_t bytes_size, size_t ref_count_, bool is_read_only_); void serialize(WriteBuffer & buf, bool sync) const; std::string serializeToString() const; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 47ef97401f2..cd099be2f7f 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 93562e7bfed..bbf9f96404f 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -146,9 +146,4 @@ bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const return res; } -void ReadBufferFromPocoSocketBase::setReceiveTimeout(size_t receive_timeout_microseconds) -{ - socket.setReceiveTimeout(Poco::Timespan(receive_timeout_microseconds, 0)); -} - } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 2a0c0213302..912388adaac 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -34,8 +34,6 @@ public: ssize_t socketReceiveBytesImpl(char * ptr, size_t size); - void setReceiveTimeout(size_t receive_timeout_microseconds); - private: AsyncCallback async_callback; std::string socket_description; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 5c1ee6ccc78..e8b81b51d6a 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -48,6 +48,7 @@ bool S3Exception::isRetryableError() const } } + namespace DB::ErrorCodes { extern const int S3_ERROR; diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index da6e9d473da..28803a94c80 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -20,8 +20,6 @@ namespace ActionLocks extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; extern const StorageActionBlockType ViewRefresh = 10; - extern const StorageActionBlockType VirtualPartsUpdate = 11; - extern const StorageActionBlockType ReduceBlockingParts = 12; } diff --git a/src/Interpreters/BlobStorageLog.cpp b/src/Interpreters/BlobStorageLog.cpp index 601005626e1..f20ac9165ac 100644 --- a/src/Interpreters/BlobStorageLog.cpp +++ b/src/Interpreters/BlobStorageLog.cpp @@ -96,7 +96,6 @@ void BlobStorageLog::prepareTable() std::unique_lock lock{prepare_mutex}; const auto & relative_data_path = merge_tree_table->getRelativeDataPath(); prefix_to_ignore = normalizePath(relative_data_path); - LOG_DEBUG(log, "Will ignore blobs with prefix {}", prefix_to_ignore); } } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 58f64b6e28d..0ca62b19d37 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -12,7 +12,7 @@ namespace DB /// Based on the LRU algorithm implementation, the record with the lowest priority is stored at /// the head of the queue, and the record with the highest priority is stored at the tail. -class LRUFileCachePriority : public IFileCachePriority +class LRUFileCachePriority final : public IFileCachePriority { protected: struct State @@ -85,8 +85,6 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; - FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, state); } - private: class LRUIterator; using LRUQueue = std::list; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 5649a12aff9..23bc8c0908b 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -72,12 +72,7 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; - FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, size_ratio, probationary_queue.state, protected_queue.state); } - private: - using LRUIterator = LRUFileCachePriority::LRUIterator; - using LRUQueue = std::list; - double size_ratio; LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index dc9ce23ddb9..c92602105c5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1817,21 +1817,6 @@ void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) (*reload_disks_task)->schedule(); } -void DatabaseCatalog::stopReplicatedDDLQueries() -{ - replicated_ddl_queries_enabled = false; -} - -void DatabaseCatalog::startReplicatedDDLQueries() -{ - replicated_ddl_queries_enabled = true; -} - -bool DatabaseCatalog::canPerformReplicatedDDLQueries() const -{ - return replicated_ddl_queries_enabled; -} - static void maybeUnlockUUID(UUID uuid) { if (uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 308d1b33e8b..83a302f117d 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -266,10 +266,6 @@ public: void triggerReloadDisksTask(const Strings & new_added_disks); - void stopReplicatedDDLQueries(); - void startReplicatedDDLQueries(); - bool canPerformReplicatedDDLQueries() const; - private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -365,8 +361,6 @@ private: std::mutex reload_disks_mutex; std::set disks_to_reload; static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5; - - std::atomic replicated_ddl_queries_enabled = false; }; diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 82d55125927..3d667fcaef0 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -82,9 +82,6 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); - - void stopReplicatedDDLQueries(); - void startReplicatedDDLQueries(); }; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 901cd13cd2f..8601558b788 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -40,6 +40,7 @@ class MutationsInterpreter { private: struct Stage; + public: struct Settings { diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 0a20dd896a9..ab4bc53b6f1 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -98,6 +98,7 @@ public: /// Closes and removes session void closeSession(const String & session_id); + private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 02d1ae528ac..8122800f882 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -19,7 +19,6 @@ Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_bloc , min_block_size_bytes(min_block_size_bytes_) , header(header_) { - LOG_TEST(getLogger("Squashing"), "header columns {}", header.columns()); } Chunk Squashing::flush() diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c5d58a873fb..d7d9da2a367 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -14,7 +14,6 @@ #include #include #include -#include "Parsers/ASTSystemQuery.h" #include #include #include @@ -94,12 +93,6 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, if (!context->getSettingsRef()[Setting::allow_distributed_ddl]) throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Distributed DDL queries are prohibited for the user"); - bool is_system_query = dynamic_cast(query_ptr.get()) != nullptr; - bool replicated_ddl_queries_enabled = DatabaseCatalog::instance().canPerformReplicatedDDLQueries(); - - if (!is_system_query && !replicated_ddl_queries_enabled) - throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); - if (const auto * query_alter = query_ptr->as()) { for (const auto & command : query_alter->command_list->children) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 83b7eb71d64..8ea9fb12b86 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -99,7 +99,6 @@ namespace DB MR_MACROS(COMPRESSION, "COMPRESSION") \ MR_MACROS(CONST, "CONST") \ MR_MACROS(CONSTRAINT, "CONSTRAINT") \ - MR_MACROS(CONNECTIONS, "CONNECTIONS") \ MR_MACROS(CREATE_POLICY, "CREATE POLICY") \ MR_MACROS(CREATE_PROFILE, "CREATE PROFILE") \ MR_MACROS(CREATE_QUOTA, "CREATE QUOTA") \ diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0b1dff556f6..2b581f20e3b 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -9,7 +9,6 @@ #include #include #include - #include namespace DB diff --git a/src/Server/CloudPlacementInfo.cpp b/src/Server/CloudPlacementInfo.cpp index 08b4e2132ad..d8810bb30de 100644 --- a/src/Server/CloudPlacementInfo.cpp +++ b/src/Server/CloudPlacementInfo.cpp @@ -53,9 +53,6 @@ PlacementInfo & PlacementInfo::instance() void PlacementInfo::initialize(const Poco::Util::AbstractConfiguration & config) try { - if (initialized) - return; - if (!config.has(DB::PlacementInfo::PLACEMENT_CONFIG_PREFIX)) { availability_zone = ""; diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index ca607bb4e33..09fb7b01678 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -22,7 +22,6 @@ struct FutureMergedMutatedPart MergeTreeDataPartFormat part_format; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; - std::vector blocking_parts_to_remove; MergeType merge_type = MergeType::Regular; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index c68617d3995..d799ce57b40 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -18,7 +18,6 @@ public: using ValueSizeMap = std::map; using VirtualFields = std::unordered_map; using DeserializeBinaryBulkStateMap = std::map; - using FileStreams = std::map>; IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp index 34cd925a8c6..4e1bb2f11a7 100644 --- a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -83,9 +83,6 @@ bool MergeProjectionPartsTask::executeStep() ".tmp_proj"); next_level_parts.push_back(executeHere(tmp_part_merge_task)); - /// FIXME (alesapin) we should use some temporary storage for this, - /// not commit each subprojection part - next_level_parts.back()->getDataPartStorage().commitTransaction(); next_level_parts.back()->is_temp = true; } diff --git a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h index a61938a993c..0a84f08ea71 100644 --- a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h +++ b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h @@ -8,7 +8,7 @@ namespace DB STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion) -static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; -static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; +const MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; +const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 6d209b9f931..71fcb93f369 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -106,11 +106,9 @@ public: PreformattedMessage & out_disable_reason, bool dry_run = false); - /// Actually the most fresh partition with biggest modification_time String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; /// Useful to quickly get a list of partitions that contain parts that we may want to merge - /// The result is limited by top_number_of_partitions_to_consider_for_merge PartitionIdsHint getPartitionsThatMayBeMerged( size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.h b/src/Storages/MergeTree/MergeTreeDataPartType.h index a59ccc2fab1..8177809d41e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartType.h +++ b/src/Storages/MergeTree/MergeTreeDataPartType.h @@ -45,7 +45,6 @@ public: enum Value { Full, - Packed, Unknown, }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 388737915ab..58a67fc4ba2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -179,8 +179,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - /// We don't need to check if it exists or not, createDirectories doesn't throw - getDataPartStorage().createDirectories(); + if (!getDataPartStorage().exists()) + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 9211ab51ad5..2af7abc17f9 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -108,14 +108,6 @@ std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesyste return {}; } -MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( - MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_) - : mark_type(mark_type_) - , fixed_index_granularity(index_granularity_) - , index_granularity_bytes(index_granularity_bytes_) -{ -} - MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MergeTreeDataPartType type_) : MergeTreeIndexGranularityInfo(storage, {storage.canUseAdaptiveGranularity(), (*storage.getSettings())[MergeTreeSetting::compress_marks], type_.getValue()}) { diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index b302d6b1a4b..87445c99ade 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -49,7 +49,6 @@ public: MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MarkType mark_type_); MergeTreeIndexGranularityInfo(MergeTreeDataPartType type_, bool is_adaptive_, size_t index_granularity_, size_t index_granularity_bytes_); - MergeTreeIndexGranularityInfo(MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_); void changeGranularityIfRequired(const IDataPartStorage & data_part_storage); diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp index e0214d6a79d..6553054774e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp @@ -26,11 +26,11 @@ void checkMutationStatus(std::optional & status, const throw Exception( ErrorCodes::UNFINISHED, "Exception happened during execution of mutation{} '{}' with part '{}' reason: '{}'. This error maybe retryable or not. " - "In case of unretryable error, mutation can be killed with KILL MUTATION query \n\n{}\n", + "In case of unretryable error, mutation can be killed with KILL MUTATION query", mutation_ids.size() > 1 ? "s" : "", boost::algorithm::join(mutation_ids, ", "), status->latest_failed_part, - status->latest_fail_reason, StackTrace().toString()); + status->latest_fail_reason); } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 28b043fcf20..f128722b03b 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -46,13 +46,6 @@ struct MergeTreePartInfo < std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); } - bool operator>(const MergeTreePartInfo & rhs) const - { - return std::forward_as_tuple(partition_id, min_block, max_block, level, mutation) - > std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); - } - - bool operator==(const MergeTreePartInfo & rhs) const { return !(*this != rhs); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 13ce14e02ec..7acc8cd88b4 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -35,7 +35,7 @@ struct PrewhereExprStep bool remove_filter_column = false; bool need_filter = false; - /// Some PREWHERE steps should be executed without conversions (e.g. early mutation steps) + /// Some PREWHERE steps should be executed without conversions. /// A step without alter conversion cannot be executed after step with alter conversions. bool perform_alter_conversions = false; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 1ba04fc460d..95469337f8a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 34e699bcef7..2a1ddf32431 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -135,6 +135,7 @@ bool isRetryableException(std::exception_ptr exception_ptr) } } + static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, const IDataPartStorage & data_part_storage, @@ -421,7 +422,6 @@ IMergeTreeDataPart::Checksums checkDataPart( } ReadSettings read_settings; - read_settings.read_through_distributed_cache = false; read_settings.enable_filesystem_cache = false; read_settings.enable_filesystem_cache_log = false; read_settings.enable_filesystem_read_prefetches_log = false; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 87f2be1bf3e..57918ffd493 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -5,7 +5,6 @@ #if USE_AWS_S3 #include #include -#include namespace DB { diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 23df9bfa1c7..23ee7a18b53 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -150,7 +150,6 @@ size_t estimateValueSize( } } -} ColumnPtr fillColumnWithRandomData( const DataTypePtr type, @@ -540,8 +539,6 @@ ColumnPtr fillColumnWithRandomData( } } -namespace -{ class GenerateSource : public ISource { diff --git a/src/Storages/TableZnodeInfo.h b/src/Storages/TableZnodeInfo.h index 4e3ffb44056..729a88e7509 100644 --- a/src/Storages/TableZnodeInfo.h +++ b/src/Storages/TableZnodeInfo.h @@ -17,8 +17,6 @@ struct StorageID; class ASTCreateQuery; class Context; using ContextPtr = std::shared_ptr; -class IDatabase; -using DatabasePtr = std::shared_ptr; /// Helper for replicated tables that use zookeeper for coordination among replicas. /// Handles things like: diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 0e839ac6fc1..17554f5c8a5 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -142,7 +142,6 @@ SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLICATION QUEUES','STOP REPLICATION QUEUES','START REPLICATION QUEUES'] TABLE SYSTEM SYSTEM VIRTUAL PARTS UPDATE ['SYSTEM STOP VIRTUAL PARTS UPDATE','SYSTEM START VIRTUAL PARTS UPDATE','STOP VIRTUAL PARTS UPDATE','START VIRTUAL PARTS UPDATE'] TABLE SYSTEM -SYSTEM REDUCE BLOCKING PARTS ['SYSTEM STOP REDUCE BLOCKING PARTS','SYSTEM START REDUCE BLOCKING PARTS','STOP REDUCE BLOCKING PARTS','START REDUCE BLOCKING PARTS'] TABLE SYSTEM SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM SYSTEM REPLICA READINESS ['SYSTEM REPLICA READY','SYSTEM REPLICA UNREADY'] GLOBAL SYSTEM From 10ce7c54f8bfce5a4cab49e5b741cfc29a8c3f03 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2024 20:59:45 +0200 Subject: [PATCH 468/816] Revert "Revert "Miscellaneous"" --- programs/server/Server.cpp | 15 +++++++++++++++ src/Access/AccessControl.h | 2 ++ src/Access/Authentication.cpp | 1 + src/Access/Common/AccessType.h | 1 + src/Access/RoleCache.h | 4 ++++ src/Access/tests/gtest_access_rights_ops.cpp | 3 ++- src/Core/ServerUUID.cpp | 5 +++++ src/Core/ServerUUID.h | 3 +++ src/Core/UUID.h | 3 +++ src/Databases/DatabaseReplicated.cpp | 12 +++++++++--- src/Databases/enableAllExperimentalSettings.cpp | 2 ++ .../ExecutablePoolDictionarySource.cpp | 7 +++++++ src/Dictionaries/RedisDictionarySource.cpp | 1 - src/Dictionaries/XDBCDictionarySource.cpp | 6 ++++++ src/Disks/DiskEncrypted.h | 2 ++ src/Disks/DiskEncryptedTransaction.cpp | 1 - src/Disks/DiskType.h | 2 ++ src/Disks/IDisk.h | 3 ++- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- .../ObjectStorages/DiskObjectStorageMetadata.h | 2 ++ src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 + src/IO/ReadBufferFromPocoSocket.cpp | 5 +++++ src/IO/ReadBufferFromPocoSocket.h | 2 ++ src/IO/S3Common.cpp | 1 - src/Interpreters/ActionLocksManager.cpp | 2 ++ src/Interpreters/BlobStorageLog.cpp | 1 + src/Interpreters/Cache/LRUFileCachePriority.h | 4 +++- src/Interpreters/Cache/SLRUFileCachePriority.h | 5 +++++ src/Interpreters/DatabaseCatalog.cpp | 15 +++++++++++++++ src/Interpreters/DatabaseCatalog.h | 6 ++++++ src/Interpreters/InterpreterSystemQuery.h | 3 +++ src/Interpreters/MutationsInterpreter.h | 1 - src/Interpreters/Session.h | 1 - src/Interpreters/Squashing.cpp | 1 + src/Interpreters/executeDDLQueryOnCluster.cpp | 7 +++++++ src/Parsers/CommonParsers.h | 1 + src/Parsers/IAST.cpp | 1 + src/Server/CloudPlacementInfo.cpp | 3 +++ src/Storages/MergeTree/FutureMergedMutatedPart.h | 1 + src/Storages/MergeTree/IMergeTreeReader.h | 1 + .../MergeTree/MergeProjectionPartsTask.cpp | 3 +++ .../MergeTree/MergeTreeDataFormatVersion.h | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 2 ++ src/Storages/MergeTree/MergeTreeDataPartType.h | 1 + .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 4 ++-- .../MergeTree/MergeTreeIndexGranularityInfo.cpp | 8 ++++++++ .../MergeTree/MergeTreeIndexGranularityInfo.h | 1 + .../MergeTree/MergeTreeMutationStatus.cpp | 4 ++-- src/Storages/MergeTree/MergeTreePartInfo.h | 7 +++++++ src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.h | 1 + src/Storages/StorageGenerateRandom.cpp | 3 +++ src/Storages/TableZnodeInfo.h | 2 ++ .../0_stateless/01271_show_privileges.reference | 1 + 56 files changed, 165 insertions(+), 21 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 35dae614d87..c106a68f360 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2267,6 +2267,21 @@ try throw; } + bool found_stop_flag = false; + + if (has_zookeeper && global_context->getMacros()->getMacroMap().contains("replica")) + { + auto zookeeper = global_context->getZooKeeper(); + String stop_flag_path = "/clickhouse/stop_replicated_ddl_queries/{replica}"; + stop_flag_path = global_context->getMacros()->expand(stop_flag_path); + found_stop_flag = zookeeper->exists(stop_flag_path); + } + + if (found_stop_flag) + LOG_INFO(log, "Found a stop flag for replicated DDL queries. They will be disabled"); + else + DatabaseCatalog::instance().startReplicatedDDLQueries(); + LOG_DEBUG(log, "Loaded metadata."); if (has_trace_collector) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index a91686433ec..a342c5300bf 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -9,6 +9,8 @@ #include +#include "config.h" + namespace Poco { diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 8d5d04a4ed2..1d69a659cd6 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -12,6 +12,7 @@ #include "config.h" + namespace DB { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..383e7f70420 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -193,6 +193,7 @@ enum class AccessType : uint8_t M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ M(SYSTEM_REPLICATION_QUEUES, "SYSTEM STOP REPLICATION QUEUES, SYSTEM START REPLICATION QUEUES, STOP REPLICATION QUEUES, START REPLICATION QUEUES", TABLE, SYSTEM) \ M(SYSTEM_VIRTUAL_PARTS_UPDATE, "SYSTEM STOP VIRTUAL PARTS UPDATE, SYSTEM START VIRTUAL PARTS UPDATE, STOP VIRTUAL PARTS UPDATE, START VIRTUAL PARTS UPDATE", TABLE, SYSTEM) \ + M(SYSTEM_REDUCE_BLOCKING_PARTS, "SYSTEM STOP REDUCE BLOCKING PARTS, SYSTEM START REDUCE BLOCKING PARTS, STOP REDUCE BLOCKING PARTS, START REDUCE BLOCKING PARTS", TABLE, SYSTEM) \ M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \ M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_REPLICA_READINESS, "SYSTEM REPLICA READY, SYSTEM REPLICA UNREADY", GLOBAL, SYSTEM) \ diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index 75d1fd32685..b707a05346f 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -22,6 +22,10 @@ public: const std::vector & current_roles, const std::vector & current_roles_with_admin_option); + std::shared_ptr getEnabledRoles( + boost::container::flat_set current_roles, + boost::container::flat_set current_roles_with_admin_option); + private: using SubscriptionsOnRoles = std::vector>; diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 902fc949840..41567905a10 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -284,7 +284,8 @@ TEST(AccessRights, Union) "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " - "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " + "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, " + "SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM REDUCE BLOCKING PARTS, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " diff --git a/src/Core/ServerUUID.cpp b/src/Core/ServerUUID.cpp index 251b407e673..5b17017e7f4 100644 --- a/src/Core/ServerUUID.cpp +++ b/src/Core/ServerUUID.cpp @@ -68,6 +68,11 @@ UUID loadServerUUID(const fs::path & server_uuid_file, Poco::Logger * log) } } +void ServerUUID::set(UUID & uuid) +{ + server_uuid = uuid; +} + void ServerUUID::setRandomForUnitTests() { server_uuid = UUIDHelpers::generateV4(); diff --git a/src/Core/ServerUUID.h b/src/Core/ServerUUID.h index 9c7f7d32acc..26711bfbfaa 100644 --- a/src/Core/ServerUUID.h +++ b/src/Core/ServerUUID.h @@ -20,6 +20,9 @@ public: /// Loads server UUID from file or creates new one. Should be called on daemon startup. static void load(const fs::path & server_uuid_file, Poco::Logger * log); + /// Sets specific server UUID. + static void set(UUID & uuid); + static void setRandomForUnitTests(); }; diff --git a/src/Core/UUID.h b/src/Core/UUID.h index 2bdefe9d3fc..1b8a075f0d2 100644 --- a/src/Core/UUID.h +++ b/src/Core/UUID.h @@ -64,6 +64,9 @@ namespace UUIDHelpers /// Generate random UUID. UUID generateV4(); + /// Generate UUID from hash of a string. + UUID makeUUIDv4FromHash(const String & string); + constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1; constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d4eaaf750cd..387667b1b42 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -85,6 +85,7 @@ namespace ErrorCodes extern const int NO_ACTIVE_REPLICAS; extern const int CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT; extern const int CANNOT_RESTORE_TABLE; + extern const int QUERY_IS_PROHIBITED; extern const int SUPPORT_IS_DISABLED; } @@ -1057,6 +1058,9 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex { waitDatabaseStarted(); + if (!DatabaseCatalog::instance().canPerformReplicatedDDLQueries()) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); + if (query_context->getCurrentTransaction() && query_context->getSettingsRef()[Setting::throw_on_unsupported_query_inside_transaction]) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Distributed DDL queries inside transactions are not supported"); @@ -1237,14 +1241,16 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name)); auto query_context = Context::createCopy(getContext()); query_context->setSetting("allow_deprecated_database_ordinary", 1); - executeQuery(query, query_context, QueryFlags{.internal = true}); + query_context->setSetting("cloud_mode", false); + executeQuery(query, query_context, QueryFlags{ .internal = true }); /// But we want to avoid discarding UUID of ReplicatedMergeTree tables, because it will not work /// if zookeeper_path contains {uuid} macro. Replicated database do not recreate replicated tables on recovery, /// so it's ok to save UUID of replicated table. query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Atomic", backQuoteIfNeed(to_db_name_replicated)); query_context = Context::createCopy(getContext()); - executeQuery(query, query_context, QueryFlags{.internal = true}); + query_context->setSetting("cloud_mode", false); + executeQuery(query, query_context, QueryFlags{ .internal = true }); } size_t moved_tables = 0; @@ -1634,7 +1640,7 @@ void DatabaseReplicated::dropTable(ContextPtr local_context, const String & tabl auto table = tryGetTable(table_name, getContext()); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} doesn't exist", table_name); - if (table->getName() == "MaterializedView" || table->getName() == "WindowView") + if (table->getName() == "MaterializedView" || table->getName() == "WindowView" || table->getName() == "SharedSet" || table->getName() == "SharedJoin") { /// Avoid recursive locking of metadata_mutex table->dropInnerTableIfAny(sync, local_context); diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index d1b3b776370..6efbc429fd8 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -43,6 +43,8 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("enable_zstd_qat_codec", 1); context->setSetting("allow_create_index_without_type", 1); context->setSetting("allow_experimental_s3queue", 1); + + /// clickhouse-private settings context->setSetting("allow_experimental_shared_set_join", 1); } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 403ce540e76..602fde0e0d7 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -26,6 +26,9 @@ namespace DB namespace Setting { extern const SettingsSeconds max_execution_time; + + /// Cloud only + extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -33,6 +36,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; extern const int UNSUPPORTED_METHOD; + extern const int SUPPORT_IS_DISABLED; } ExecutablePoolDictionarySource::ExecutablePoolDictionarySource( @@ -192,6 +196,9 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { + if (global_context->getSettingsRef()[Setting::cloud_mode]) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `executable pool` is disabled"); + if (dict_struct.has_expressions) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable_pool` does not support attribute expressions"); diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 17ed515ca9a..26d9ebae1b8 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -29,7 +29,6 @@ namespace DB ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { - auto redis_config_prefix = config_prefix + ".redis"; auto host = config.getString(redis_config_prefix + ".host"); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index ebb50f79497..4e64db5831d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -28,6 +28,9 @@ namespace Setting { extern const SettingsSeconds http_receive_timeout; extern const SettingsBool odbc_bridge_use_connection_pooling; + + /// Cloud only + extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -242,6 +245,9 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) ContextPtr global_context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { + + if (global_context->getSettingsRef()[Setting::cloud_mode]) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `odbc` is disabled"); #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( global_context, diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index caba4184a73..95d9554b909 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -313,6 +313,8 @@ public: return std::make_shared(*this); } + /// Need to overwrite explicetly because this disk change + /// a lot of "delegate" methods. return createEncryptedTransaction(); } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 2660051e1d3..a528564fd1e 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -1,6 +1,5 @@ #include - #if USE_SSL #include #include diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 347e2c1cfe3..bf7ef3d30eb 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -27,9 +27,11 @@ enum class MetadataStorageType : uint8_t { None, Local, + Keeper, Plain, PlainRewritable, StaticWeb, + Memory, }; MetadataStorageType metadataTypeFromString(const String & type); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 59f58a816e9..692020c86a6 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -497,7 +497,7 @@ public: protected: - friend class DiskDecorator; + friend class DiskReadOnlyWrapper; const String name; @@ -580,6 +580,7 @@ inline String directoryPath(const String & path) return fs::path(path).parent_path() / ""; } + } template <> diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 7055a7018ce..8e4ec6f3dfb 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size) { /// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task. - if (!settings.enable_filesystem_cache) + if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache) return settings.remote_fs_buffer_size; /// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file. diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index 4f45f5b7ddf..456b3a4778d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -56,6 +56,8 @@ public: void deserialize(ReadBuffer & buf); void deserializeFromString(const std::string & data); + /// This method was deleted from public fork recently by Azat + void createFromSingleObject(ObjectStorageKey object_key, size_t bytes_size, size_t ref_count_, bool is_read_only_); void serialize(WriteBuffer & buf, bool sync) const; std::string serializeToString() const; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index cd099be2f7f..47ef97401f2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index bbf9f96404f..93562e7bfed 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -146,4 +146,9 @@ bool ReadBufferFromPocoSocketBase::poll(size_t timeout_microseconds) const return res; } +void ReadBufferFromPocoSocketBase::setReceiveTimeout(size_t receive_timeout_microseconds) +{ + socket.setReceiveTimeout(Poco::Timespan(receive_timeout_microseconds, 0)); +} + } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 912388adaac..2a0c0213302 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -34,6 +34,8 @@ public: ssize_t socketReceiveBytesImpl(char * ptr, size_t size); + void setReceiveTimeout(size_t receive_timeout_microseconds); + private: AsyncCallback async_callback; std::string socket_description; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index e8b81b51d6a..5c1ee6ccc78 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -48,7 +48,6 @@ bool S3Exception::isRetryableError() const } } - namespace DB::ErrorCodes { extern const int S3_ERROR; diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 28803a94c80..da6e9d473da 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -20,6 +20,8 @@ namespace ActionLocks extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; extern const StorageActionBlockType ViewRefresh = 10; + extern const StorageActionBlockType VirtualPartsUpdate = 11; + extern const StorageActionBlockType ReduceBlockingParts = 12; } diff --git a/src/Interpreters/BlobStorageLog.cpp b/src/Interpreters/BlobStorageLog.cpp index f20ac9165ac..601005626e1 100644 --- a/src/Interpreters/BlobStorageLog.cpp +++ b/src/Interpreters/BlobStorageLog.cpp @@ -96,6 +96,7 @@ void BlobStorageLog::prepareTable() std::unique_lock lock{prepare_mutex}; const auto & relative_data_path = merge_tree_table->getRelativeDataPath(); prefix_to_ignore = normalizePath(relative_data_path); + LOG_DEBUG(log, "Will ignore blobs with prefix {}", prefix_to_ignore); } } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 0ca62b19d37..58f64b6e28d 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -12,7 +12,7 @@ namespace DB /// Based on the LRU algorithm implementation, the record with the lowest priority is stored at /// the head of the queue, and the record with the highest priority is stored at the tail. -class LRUFileCachePriority final : public IFileCachePriority +class LRUFileCachePriority : public IFileCachePriority { protected: struct State @@ -85,6 +85,8 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; + FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, state); } + private: class LRUIterator; using LRUQueue = std::list; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 23bc8c0908b..5649a12aff9 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -72,7 +72,12 @@ public: bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) override; + FileCachePriorityPtr copy() const { return std::make_unique(max_size, max_elements, size_ratio, probationary_queue.state, protected_queue.state); } + private: + using LRUIterator = LRUFileCachePriority::LRUIterator; + using LRUQueue = std::list; + double size_ratio; LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c92602105c5..dc9ce23ddb9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1817,6 +1817,21 @@ void DatabaseCatalog::triggerReloadDisksTask(const Strings & new_added_disks) (*reload_disks_task)->schedule(); } +void DatabaseCatalog::stopReplicatedDDLQueries() +{ + replicated_ddl_queries_enabled = false; +} + +void DatabaseCatalog::startReplicatedDDLQueries() +{ + replicated_ddl_queries_enabled = true; +} + +bool DatabaseCatalog::canPerformReplicatedDDLQueries() const +{ + return replicated_ddl_queries_enabled; +} + static void maybeUnlockUUID(UUID uuid) { if (uuid == UUIDHelpers::Nil) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 83a302f117d..308d1b33e8b 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -266,6 +266,10 @@ public: void triggerReloadDisksTask(const Strings & new_added_disks); + void stopReplicatedDDLQueries(); + void startReplicatedDDLQueries(); + bool canPerformReplicatedDDLQueries() const; + private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -361,6 +365,8 @@ private: std::mutex reload_disks_mutex; std::set disks_to_reload; static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5; + + std::atomic replicated_ddl_queries_enabled = false; }; diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 3d667fcaef0..82d55125927 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -82,6 +82,9 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); + + void stopReplicatedDDLQueries(); + void startReplicatedDDLQueries(); }; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 8601558b788..901cd13cd2f 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -40,7 +40,6 @@ class MutationsInterpreter { private: struct Stage; - public: struct Settings { diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index ab4bc53b6f1..0a20dd896a9 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -98,7 +98,6 @@ public: /// Closes and removes session void closeSession(const String & session_id); - private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 8122800f882..02d1ae528ac 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -19,6 +19,7 @@ Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_bloc , min_block_size_bytes(min_block_size_bytes_) , header(header_) { + LOG_TEST(getLogger("Squashing"), "header columns {}", header.columns()); } Chunk Squashing::flush() diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index d7d9da2a367..c5d58a873fb 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -14,6 +14,7 @@ #include #include #include +#include "Parsers/ASTSystemQuery.h" #include #include #include @@ -93,6 +94,12 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, if (!context->getSettingsRef()[Setting::allow_distributed_ddl]) throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Distributed DDL queries are prohibited for the user"); + bool is_system_query = dynamic_cast(query_ptr.get()) != nullptr; + bool replicated_ddl_queries_enabled = DatabaseCatalog::instance().canPerformReplicatedDDLQueries(); + + if (!is_system_query && !replicated_ddl_queries_enabled) + throw Exception(ErrorCodes::QUERY_IS_PROHIBITED, "Replicated DDL queries are disabled"); + if (const auto * query_alter = query_ptr->as()) { for (const auto & command : query_alter->command_list->children) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 8ea9fb12b86..83b7eb71d64 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -99,6 +99,7 @@ namespace DB MR_MACROS(COMPRESSION, "COMPRESSION") \ MR_MACROS(CONST, "CONST") \ MR_MACROS(CONSTRAINT, "CONSTRAINT") \ + MR_MACROS(CONNECTIONS, "CONNECTIONS") \ MR_MACROS(CREATE_POLICY, "CREATE POLICY") \ MR_MACROS(CREATE_PROFILE, "CREATE PROFILE") \ MR_MACROS(CREATE_QUOTA, "CREATE QUOTA") \ diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 2b581f20e3b..0b1dff556f6 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -9,6 +9,7 @@ #include #include #include + #include namespace DB diff --git a/src/Server/CloudPlacementInfo.cpp b/src/Server/CloudPlacementInfo.cpp index d8810bb30de..08b4e2132ad 100644 --- a/src/Server/CloudPlacementInfo.cpp +++ b/src/Server/CloudPlacementInfo.cpp @@ -53,6 +53,9 @@ PlacementInfo & PlacementInfo::instance() void PlacementInfo::initialize(const Poco::Util::AbstractConfiguration & config) try { + if (initialized) + return; + if (!config.has(DB::PlacementInfo::PLACEMENT_CONFIG_PREFIX)) { availability_zone = ""; diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index 09fb7b01678..ca607bb4e33 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -22,6 +22,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartFormat part_format; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + std::vector blocking_parts_to_remove; MergeType merge_type = MergeType::Regular; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index d799ce57b40..c68617d3995 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -18,6 +18,7 @@ public: using ValueSizeMap = std::map; using VirtualFields = std::unordered_map; using DeserializeBinaryBulkStateMap = std::map; + using FileStreams = std::map>; IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp index 4e1bb2f11a7..34cd925a8c6 100644 --- a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -83,6 +83,9 @@ bool MergeProjectionPartsTask::executeStep() ".tmp_proj"); next_level_parts.push_back(executeHere(tmp_part_merge_task)); + /// FIXME (alesapin) we should use some temporary storage for this, + /// not commit each subprojection part + next_level_parts.back()->getDataPartStorage().commitTransaction(); next_level_parts.back()->is_temp = true; } diff --git a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h index 0a84f08ea71..a61938a993c 100644 --- a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h +++ b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h @@ -8,7 +8,7 @@ namespace DB STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion) -const MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; -const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 71fcb93f369..6d209b9f931 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -106,9 +106,11 @@ public: PreformattedMessage & out_disable_reason, bool dry_run = false); + /// Actually the most fresh partition with biggest modification_time String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; /// Useful to quickly get a list of partitions that contain parts that we may want to merge + /// The result is limited by top_number_of_partitions_to_consider_for_merge PartitionIdsHint getPartitionsThatMayBeMerged( size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.h b/src/Storages/MergeTree/MergeTreeDataPartType.h index 8177809d41e..a59ccc2fab1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartType.h +++ b/src/Storages/MergeTree/MergeTreeDataPartType.h @@ -45,6 +45,7 @@ public: enum Value { Full, + Packed, Unknown, }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 58a67fc4ba2..388737915ab 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -179,8 +179,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - if (!getDataPartStorage().exists()) - getDataPartStorage().createDirectories(); + /// We don't need to check if it exists or not, createDirectories doesn't throw + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 2af7abc17f9..9211ab51ad5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -108,6 +108,14 @@ std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesyste return {}; } +MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( + MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_) + : mark_type(mark_type_) + , fixed_index_granularity(index_granularity_) + , index_granularity_bytes(index_granularity_bytes_) +{ +} + MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MergeTreeDataPartType type_) : MergeTreeIndexGranularityInfo(storage, {storage.canUseAdaptiveGranularity(), (*storage.getSettings())[MergeTreeSetting::compress_marks], type_.getValue()}) { diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index 87445c99ade..b302d6b1a4b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -49,6 +49,7 @@ public: MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MarkType mark_type_); MergeTreeIndexGranularityInfo(MergeTreeDataPartType type_, bool is_adaptive_, size_t index_granularity_, size_t index_granularity_bytes_); + MergeTreeIndexGranularityInfo(MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_); void changeGranularityIfRequired(const IDataPartStorage & data_part_storage); diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp index 6553054774e..e0214d6a79d 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp @@ -26,11 +26,11 @@ void checkMutationStatus(std::optional & status, const throw Exception( ErrorCodes::UNFINISHED, "Exception happened during execution of mutation{} '{}' with part '{}' reason: '{}'. This error maybe retryable or not. " - "In case of unretryable error, mutation can be killed with KILL MUTATION query", + "In case of unretryable error, mutation can be killed with KILL MUTATION query \n\n{}\n", mutation_ids.size() > 1 ? "s" : "", boost::algorithm::join(mutation_ids, ", "), status->latest_failed_part, - status->latest_fail_reason); + status->latest_fail_reason, StackTrace().toString()); } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index f128722b03b..28b043fcf20 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -46,6 +46,13 @@ struct MergeTreePartInfo < std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); } + bool operator>(const MergeTreePartInfo & rhs) const + { + return std::forward_as_tuple(partition_id, min_block, max_block, level, mutation) + > std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); + } + + bool operator==(const MergeTreePartInfo & rhs) const { return !(*this != rhs); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 7acc8cd88b4..13ce14e02ec 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -35,7 +35,7 @@ struct PrewhereExprStep bool remove_filter_column = false; bool need_filter = false; - /// Some PREWHERE steps should be executed without conversions. + /// Some PREWHERE steps should be executed without conversions (e.g. early mutation steps) /// A step without alter conversion cannot be executed after step with alter conversions. bool perform_alter_conversions = false; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 95469337f8a..1ba04fc460d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2a1ddf32431..34e699bcef7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -135,7 +135,6 @@ bool isRetryableException(std::exception_ptr exception_ptr) } } - static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, const IDataPartStorage & data_part_storage, @@ -422,6 +421,7 @@ IMergeTreeDataPart::Checksums checkDataPart( } ReadSettings read_settings; + read_settings.read_through_distributed_cache = false; read_settings.enable_filesystem_cache = false; read_settings.enable_filesystem_cache_log = false; read_settings.enable_filesystem_read_prefetches_log = false; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 57918ffd493..87f2be1bf3e 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -5,6 +5,7 @@ #if USE_AWS_S3 #include #include +#include namespace DB { diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 23ee7a18b53..23df9bfa1c7 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -150,6 +150,7 @@ size_t estimateValueSize( } } +} ColumnPtr fillColumnWithRandomData( const DataTypePtr type, @@ -539,6 +540,8 @@ ColumnPtr fillColumnWithRandomData( } } +namespace +{ class GenerateSource : public ISource { diff --git a/src/Storages/TableZnodeInfo.h b/src/Storages/TableZnodeInfo.h index 729a88e7509..4e3ffb44056 100644 --- a/src/Storages/TableZnodeInfo.h +++ b/src/Storages/TableZnodeInfo.h @@ -17,6 +17,8 @@ struct StorageID; class ASTCreateQuery; class Context; using ContextPtr = std::shared_ptr; +class IDatabase; +using DatabasePtr = std::shared_ptr; /// Helper for replicated tables that use zookeeper for coordination among replicas. /// Handles things like: diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..0e839ac6fc1 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -142,6 +142,7 @@ SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLICATION QUEUES','STOP REPLICATION QUEUES','START REPLICATION QUEUES'] TABLE SYSTEM SYSTEM VIRTUAL PARTS UPDATE ['SYSTEM STOP VIRTUAL PARTS UPDATE','SYSTEM START VIRTUAL PARTS UPDATE','STOP VIRTUAL PARTS UPDATE','START VIRTUAL PARTS UPDATE'] TABLE SYSTEM +SYSTEM REDUCE BLOCKING PARTS ['SYSTEM STOP REDUCE BLOCKING PARTS','SYSTEM START REDUCE BLOCKING PARTS','STOP REDUCE BLOCKING PARTS','START REDUCE BLOCKING PARTS'] TABLE SYSTEM SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM SYSTEM REPLICA READINESS ['SYSTEM REPLICA READY','SYSTEM REPLICA UNREADY'] GLOBAL SYSTEM From d46b0963e942972427a75486a4265676b23a9293 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2024 23:54:14 +0200 Subject: [PATCH 469/816] Do not list detached parts from readonly and write-once disks --- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8611681a976..384fad3effc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6371,6 +6371,12 @@ DetachedPartsInfo MergeTreeData::getDetachedParts() const for (const auto & disk : getDisks()) { + /// While it is possible to have detached parts on readonly/write-once disks + /// if they were produces on another machine, where it wasn't readonly, + /// to avoid wasting resources for slow disks, avoid trying to enumerate them. + if (disk->isReadOnly() || disk->isWriteOnce()) + continue; + String detached_path = fs::path(relative_data_path) / DETACHED_DIR_NAME; /// Note: we don't care about TOCTOU issue here. From a74386134b08e06c1f89a5344bee94d2f2e50baf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 00:06:20 +0200 Subject: [PATCH 470/816] Do not calculate heavy asynchronous metrics by default --- programs/server/Server.cpp | 2 ++ src/Core/ServerSettings.cpp | 1 + src/Interpreters/ServerAsynchronousMetrics.cpp | 9 ++++++--- src/Interpreters/ServerAsynchronousMetrics.h | 2 ++ 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c106a68f360..79cd198e17b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -168,6 +168,7 @@ namespace ServerSetting { extern const ServerSettingsUInt32 asynchronous_heavy_metrics_update_period_s; extern const ServerSettingsUInt32 asynchronous_metrics_update_period_s; + extern const ServerSettingsBool asynchronous_metrics_enable_heavy_metrics; extern const ServerSettingsBool async_insert_queue_flush_on_shutdown; extern const ServerSettingsUInt64 async_insert_threads; extern const ServerSettingsBool async_load_databases; @@ -1060,6 +1061,7 @@ try ServerAsynchronousMetrics async_metrics( global_context, server_settings[ServerSetting::asynchronous_metrics_update_period_s], + server_settings[ServerSetting::asynchronous_metrics_enable_heavy_metrics], server_settings[ServerSetting::asynchronous_heavy_metrics_update_period_s], [&]() -> std::vector { diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 7c2cb49a2ba..4fd139752ff 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -58,6 +58,7 @@ namespace DB DECLARE(Double, cannot_allocate_thread_fault_injection_probability, 0, "For testing purposes.", 0) \ DECLARE(Int32, max_connections, 1024, "Max server connections.", 0) \ DECLARE(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ + DECLARE(Bool, asynchronous_metrics_enable_heavy_metrics, false, "Enable the calculation of heavy asynchronous metrics.", 0) \ DECLARE(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating heavy asynchronous metrics.", 0) \ DECLARE(String, default_database, "default", "Default database name.", 0) \ DECLARE(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 079029695c9..46a811822c2 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -54,12 +54,14 @@ void calculateMaxAndSum(Max & max, Sum & sum, T x) ServerAsynchronousMetrics::ServerAsynchronousMetrics( ContextPtr global_context_, unsigned update_period_seconds, + bool update_heavy_metrics_, unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_, bool update_jemalloc_epoch_, bool update_rss_) : WithContext(global_context_) , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) + , update_heavy_metrics(update_heavy_metrics_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check @@ -412,7 +414,8 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } #endif - updateHeavyMetricsIfNeeded(current_time, update_time, force_update, first_run, new_values); + if (update_heavy_metrics) + updateHeavyMetricsIfNeeded(current_time, update_time, force_update, first_run, new_values); } void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) @@ -459,10 +462,10 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats() void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) { const auto time_since_previous_update = current_time - heavy_metric_previous_update_time; - const bool update_heavy_metrics = (time_since_previous_update >= heavy_metric_update_period) || force_update || first_run; + const bool need_update_heavy_metrics = (time_since_previous_update >= heavy_metric_update_period) || force_update || first_run; Stopwatch watch; - if (update_heavy_metrics) + if (need_update_heavy_metrics) { heavy_metric_previous_update_time = update_time; if (first_run) diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index 5fab419a32b..691ddd429b4 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -13,6 +13,7 @@ public: ServerAsynchronousMetrics( ContextPtr global_context_, unsigned update_period_seconds, + bool update_heavy_metrics_, unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_, bool update_jemalloc_epoch_, @@ -24,6 +25,7 @@ private: void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override; void logImpl(AsynchronousMetricValues & new_values) override; + bool update_heavy_metrics; const Duration heavy_metric_update_period; TimePoint heavy_metric_previous_update_time; double heavy_update_interval = 0.; From 0feb4651d8cb0e31c2a3785dee8cf209e3dd127a Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Sun, 27 Oct 2024 01:04:41 +0000 Subject: [PATCH 471/816] proj lwd rebuild considers deleted rows --- src/Storages/ProjectionsDescription.cpp | 17 +++++++++++++++- ..._project_lwd_respects_row_exists.reference | 1 + .../03254_project_lwd_respects_row_exists.sql | 20 +++++++++++++++++++ 3 files changed, 37 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03254_project_lwd_respects_row_exists.reference create mode 100644 tests/queries/0_stateless/03254_project_lwd_respects_row_exists.sql diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 9654b4ef37a..065dcb3de30 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -294,8 +294,23 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) mut_context->setSetting("aggregate_functions_null_for_empty", Field(0)); mut_context->setSetting("transform_null_in", Field(0)); + ASTPtr query_ast_copy = nullptr; + /// Respect the _row_exists column. + if (block.findByName("_row_exists")) + { + query_ast_copy = query_ast->clone(); + auto * select_row_exists = query_ast_copy->as(); + if (!select_row_exists) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get ASTSelectQuery whening adding _row_exists = 1. It's a bug"); + + select_row_exists->setExpression( + ASTSelectQuery::Expression::WHERE, + makeASTFunction("equals", std::make_shared("_row_exists"), std::make_shared(1))); + // std::cout<(block)), SelectQueryOptions{ diff --git a/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.reference b/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.reference new file mode 100644 index 00000000000..ecc1f6c0911 --- /dev/null +++ b/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.reference @@ -0,0 +1 @@ +34 1 diff --git a/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.sql b/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.sql new file mode 100644 index 00000000000..794f74ad15f --- /dev/null +++ b/tests/queries/0_stateless/03254_project_lwd_respects_row_exists.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS users; + +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p1 (select age, count() group by age), +) ENGINE = MergeTree order by uid +SETTINGS lightweight_mutation_projection_mode = 'rebuild'; + +INSERT INTO users VALUES (1231, 'John', 33), (1232, 'Mary', 34); + +DELETE FROM users WHERE uid = 1231; + +SELECT + age, + count() +FROM users +GROUP BY age +SETTINGS optimize_use_projections = 1, force_optimize_projection = 1; From 78d104c0f358488d95e1f0641c9b5569480ea13e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 02:37:19 +0100 Subject: [PATCH 472/816] Lower log levels in S3 --- src/IO/S3/AWSLogger.cpp | 4 ++-- src/IO/S3/deleteFileFromS3.cpp | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/IO/S3/AWSLogger.cpp b/src/IO/S3/AWSLogger.cpp index a59ad83faf1..fd7dfbed99d 100644 --- a/src/IO/S3/AWSLogger.cpp +++ b/src/IO/S3/AWSLogger.cpp @@ -17,7 +17,7 @@ const char * S3_LOGGER_TAG_NAMES[][2] = { const std::pair & convertLogLevel(Aws::Utils::Logging::LogLevel log_level) { - /// We map levels to our own logger 1 to 1 except WARN+ levels. In most cases we failover such errors with retries + /// We map levels to our own logger 1 to 1 except INFO+ levels. In most cases we fail over such errors with retries /// and don't want to see them as Errors in our logs. static const std::unordered_map> mapping = { @@ -25,7 +25,7 @@ const std::pair & convertLogLevel(Aws::U {Aws::Utils::Logging::LogLevel::Fatal, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, {Aws::Utils::Logging::LogLevel::Error, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, {Aws::Utils::Logging::LogLevel::Warn, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, - {Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, + {Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG}}, {Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_TEST}}, {Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TEST}}, }; diff --git a/src/IO/S3/deleteFileFromS3.cpp b/src/IO/S3/deleteFileFromS3.cpp index 0554bb295bb..8d7b0ea2851 100644 --- a/src/IO/S3/deleteFileFromS3.cpp +++ b/src/IO/S3/deleteFileFromS3.cpp @@ -56,7 +56,7 @@ void deleteFileFromS3( if (outcome.IsSuccess()) { - LOG_INFO(log, "Object with path {} was removed from S3", key); + LOG_DEBUG(log, "Object with path {} was removed from S3", key); } else if (if_exists && S3::isNotFoundError(outcome.GetError().GetErrorType())) { @@ -173,7 +173,7 @@ void deleteFilesFromS3( if (errors.empty()) { /// All the objects were removed. - LOG_INFO(log, "Objects with paths [{}] were removed from S3", comma_separated_keys); + LOG_DEBUG(log, "Objects with paths [{}] were removed from S3", comma_separated_keys); } else { @@ -210,7 +210,7 @@ void deleteFilesFromS3( removed_keys_comma_separated += ", "; removed_keys_comma_separated += key; } - LOG_INFO(log, "Objects with paths [{}] were removed from S3", removed_keys_comma_separated); + LOG_DEBUG(log, "Objects with paths [{}] were removed from S3", removed_keys_comma_separated); } if (!not_found_keys.empty()) From 5a11b624189c44573aea9fe99fd8d1c36749b862 Mon Sep 17 00:00:00 2001 From: Shichao Jin Date: Sat, 26 Oct 2024 22:15:54 -0400 Subject: [PATCH 473/816] fix typo --- src/Storages/ProjectionsDescription.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 065dcb3de30..89a7acf8a72 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -301,7 +301,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) query_ast_copy = query_ast->clone(); auto * select_row_exists = query_ast_copy->as(); if (!select_row_exists) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get ASTSelectQuery whening adding _row_exists = 1. It's a bug"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get ASTSelectQuery when adding _row_exists = 1. It's a bug"); select_row_exists->setExpression( ASTSelectQuery::Expression::WHERE, From a62d37cacdcfd30987c2af532ba208ac1ccdfb76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 04:42:41 +0100 Subject: [PATCH 474/816] Remove system tables generate_series and generateSeries --- src/Storages/System/attachSystemTables.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 7c6dac7a608..70dcec884a6 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -126,8 +126,6 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); attachNoDescription(context, system_database, "numbers", "Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.", false, "number"); attachNoDescription(context, system_database, "numbers_mt", "Multithreaded version of `system.numbers`. Numbers order is not guaranteed.", true, "number"); - attachNoDescription(context, system_database, "generate_series", "Generates arithmetic progression of natural numbers in sorted order in a given segment with a given step", false, "generate_series"); - attachNoDescription(context, system_database, "generateSeries", "Generates arithmetic progression of natural numbers in sorted order in a given segment with a given step", false, "generate_series"); attachNoDescription(context, system_database, "zeros", "Produces unlimited number of non-materialized zeros.", false); attachNoDescription(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); From 387d980491aa3646a7d6bffbaa38325f1a4ca8f6 Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Sun, 27 Oct 2024 18:36:05 +0300 Subject: [PATCH 475/816] Trig CI --- tests/integration/test_reload_client_certificate/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index 18191a12581..a245953f1e2 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -145,7 +145,7 @@ def clean_logs(): def check_certificate_switch(first, second): - # Set first key + # Set first certificate change_config_to_key(first) From b5213da47cc6b83d46ecce077db0406ccd1386b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 19:21:08 +0100 Subject: [PATCH 476/816] Sync integration test with private --- .../test_grant_and_revoke/test_with_table_engine_grant.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 18d7c6bc3ee..63e7a2ae400 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -192,7 +192,7 @@ def test_grant_all_on_table(): instance.query("GRANT ALL ON test.table TO B", user="A") assert ( instance.query("SHOW GRANTS FOR B") - == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, SYSTEM UNLOAD PRIMARY KEY, dictGet ON test.`table` TO B\n" + == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, SYSTEM REDUCE BLOCKING PARTS, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, SYSTEM UNLOAD PRIMARY KEY, dictGet ON test.`table` TO B\n" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") assert instance.query("SHOW GRANTS FOR B") == "" From 4e57a9262384796b8d178984d95a81d456e29d2c Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Sun, 27 Oct 2024 22:26:04 +0300 Subject: [PATCH 477/816] Change a test 2 --- tests/integration/test_reload_client_certificate/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index a245953f1e2..3a775cad890 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -1,3 +1,6 @@ +#!/usr/bin/env python3 +# Tags: no-parallel + import os import threading import time From 5592d6893c838d3841f5d0f6ca583087a2c2164f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 22:01:16 +0100 Subject: [PATCH 478/816] Unique symbols in the `system.coverage_log` --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index a39f96867be..67a1b65fc99 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -25,7 +25,7 @@ EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> d # coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate) EXTRA_COLUMNS_COVERAGE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), " -EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), coverage)::Array(LowCardinality(String)) AS symbols" +EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayUniq(arrayMap(x -> demangle(addressToSymbol(x)), coverage))::Array(LowCardinality(String)) AS symbols" function __set_connection_args From 9dc47ca5cd45e0ac0799a3dfc49604f84f8396f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 22:02:08 +0100 Subject: [PATCH 479/816] Unique symbols in the `system.coverage_log` --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 67a1b65fc99..12f1cc4d357 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -25,7 +25,7 @@ EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> d # coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate) EXTRA_COLUMNS_COVERAGE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), " -EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayUniq(arrayMap(x -> demangle(addressToSymbol(x)), coverage))::Array(LowCardinality(String)) AS symbols" +EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayDistinct(arrayMap(x -> demangle(addressToSymbol(x)), coverage))::Array(LowCardinality(String)) AS symbols" function __set_connection_args From cf4730411a5204b88cbf438b884f283b966be110 Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Mon, 28 Oct 2024 03:27:26 +0300 Subject: [PATCH 480/816] Trying to pass the tests --- tests/integration/test_reload_client_certificate/test.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index 3a775cad890..a245953f1e2 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -1,6 +1,3 @@ -#!/usr/bin/env python3 -# Tags: no-parallel - import os import threading import time From 9327ce0265f28fcc50d4afbd9498035188eab90a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Oct 2024 04:25:28 +0100 Subject: [PATCH 481/816] Fix error in Replicated database --- programs/server/Server.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c106a68f360..dcf5b32d6b7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2271,10 +2271,19 @@ try if (has_zookeeper && global_context->getMacros()->getMacroMap().contains("replica")) { - auto zookeeper = global_context->getZooKeeper(); - String stop_flag_path = "/clickhouse/stop_replicated_ddl_queries/{replica}"; - stop_flag_path = global_context->getMacros()->expand(stop_flag_path); - found_stop_flag = zookeeper->exists(stop_flag_path); + try + { + auto zookeeper = global_context->getZooKeeper(); + String stop_flag_path = "/clickhouse/stop_replicated_ddl_queries/{replica}"; + stop_flag_path = global_context->getMacros()->expand(stop_flag_path); + found_stop_flag = zookeeper->exists(stop_flag_path); + } + catch (const Coordination::Exception & e) + { + if (e.code != Coordination::Error::ZCONNECTIONLOSS) + throw; + tryLogCurrentException(log); + } } if (found_stop_flag) From 9fe8315943865de9eec8d5badbc91cbca834caed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Oct 2024 06:36:26 +0100 Subject: [PATCH 482/816] Fix test --- .../configs/asynchronous_metrics_update_period_s.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml index 0a56d734805..fe19b730059 100644 --- a/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml +++ b/tests/integration/test_detached_parts_metrics/configs/asynchronous_metrics_update_period_s.xml @@ -1,4 +1,5 @@ 1 + 1 1 From 845a9d5a2c0173837052fd22aef420858abdae45 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 25 Oct 2024 12:00:56 +0000 Subject: [PATCH 483/816] add test 03257_client_history_max_entries --- .../03257_client_history_max_entries.py | 38 +++++++++++++++++++ ...03257_client_history_max_entries.reference | 2 + tests/queries/0_stateless/helpers/client.py | 6 ++- 3 files changed, 44 insertions(+), 2 deletions(-) create mode 100755 tests/queries/0_stateless/03257_client_history_max_entries.py create mode 100644 tests/queries/0_stateless/03257_client_history_max_entries.reference diff --git a/tests/queries/0_stateless/03257_client_history_max_entries.py b/tests/queries/0_stateless/03257_client_history_max_entries.py new file mode 100755 index 00000000000..8ba402138ed --- /dev/null +++ b/tests/queries/0_stateless/03257_client_history_max_entries.py @@ -0,0 +1,38 @@ +#!/usr/bin/env python3 +# Tags: no-parallel, no-fasttest + +import os +import signal +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from client import client, end_of_block, prompt + +log = None +# uncomment the line below for debugging +# log=sys.stdout + +TMP_FILE = os.path.join( + os.environ.get("CLICKHOUSE_TMP", "/tmp"), + os.path.basename(os.path.abspath(__file__)) + ".hist", +) + +with client( + name="client1>", + log=log, + extra_options={"history_file": TMP_FILE, "history_max_entries": 2}, +) as client: + client.expect(prompt) + client.send("SELECT 1") + client.expect(prompt) + client.send("SELECT 2") + client.expect(prompt) + client.send("SELECT 3") + client.expect(prompt) + +with open(TMP_FILE, "r") as f: + for line in f: + if not line.startswith("###"): + print(line, end="") diff --git a/tests/queries/0_stateless/03257_client_history_max_entries.reference b/tests/queries/0_stateless/03257_client_history_max_entries.reference new file mode 100644 index 00000000000..4c85f1227c6 --- /dev/null +++ b/tests/queries/0_stateless/03257_client_history_max_entries.reference @@ -0,0 +1,2 @@ +SELECT 2 +SELECT 3 diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index b721931e46d..844a2da1026 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -13,10 +13,12 @@ end_of_block = r".*\r\n.*\r\n" class client(object): - def __init__(self, command=None, name="", log=None): + def __init__(self, command=None, name="", log=None, extra_options=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - options = "--enable-progress-table-toggle=0" + extra_options = extra_options or {} + extra_options["enable-progress-table-toggle"] = 0 + options = " ".join(f"--{k}={v}" for k, v in extra_options.items()) command = ( os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client " + options ) From 1c9ac878914515aad4f97ee04f9fed99cae47e68 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Oct 2024 07:37:30 +0000 Subject: [PATCH 484/816] Revert "add test 03257_client_history_max_entries" This reverts commit 845a9d5a2c0173837052fd22aef420858abdae45. --- .../03257_client_history_max_entries.py | 38 ------------------- ...03257_client_history_max_entries.reference | 2 - tests/queries/0_stateless/helpers/client.py | 6 +-- 3 files changed, 2 insertions(+), 44 deletions(-) delete mode 100755 tests/queries/0_stateless/03257_client_history_max_entries.py delete mode 100644 tests/queries/0_stateless/03257_client_history_max_entries.reference diff --git a/tests/queries/0_stateless/03257_client_history_max_entries.py b/tests/queries/0_stateless/03257_client_history_max_entries.py deleted file mode 100755 index 8ba402138ed..00000000000 --- a/tests/queries/0_stateless/03257_client_history_max_entries.py +++ /dev/null @@ -1,38 +0,0 @@ -#!/usr/bin/env python3 -# Tags: no-parallel, no-fasttest - -import os -import signal -import sys - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -from client import client, end_of_block, prompt - -log = None -# uncomment the line below for debugging -# log=sys.stdout - -TMP_FILE = os.path.join( - os.environ.get("CLICKHOUSE_TMP", "/tmp"), - os.path.basename(os.path.abspath(__file__)) + ".hist", -) - -with client( - name="client1>", - log=log, - extra_options={"history_file": TMP_FILE, "history_max_entries": 2}, -) as client: - client.expect(prompt) - client.send("SELECT 1") - client.expect(prompt) - client.send("SELECT 2") - client.expect(prompt) - client.send("SELECT 3") - client.expect(prompt) - -with open(TMP_FILE, "r") as f: - for line in f: - if not line.startswith("###"): - print(line, end="") diff --git a/tests/queries/0_stateless/03257_client_history_max_entries.reference b/tests/queries/0_stateless/03257_client_history_max_entries.reference deleted file mode 100644 index 4c85f1227c6..00000000000 --- a/tests/queries/0_stateless/03257_client_history_max_entries.reference +++ /dev/null @@ -1,2 +0,0 @@ -SELECT 2 -SELECT 3 diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 844a2da1026..b721931e46d 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -13,12 +13,10 @@ end_of_block = r".*\r\n.*\r\n" class client(object): - def __init__(self, command=None, name="", log=None, extra_options=None): + def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - extra_options = extra_options or {} - extra_options["enable-progress-table-toggle"] = 0 - options = " ".join(f"--{k}={v}" for k, v in extra_options.items()) + options = "--enable-progress-table-toggle=0" command = ( os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client " + options ) From c2ce618ec9c163e312fee8146d1ce885b056baa6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C3=96rjan=20Fors?= Date: Mon, 28 Oct 2024 09:19:10 +0000 Subject: [PATCH 485/816] Increase auxv vector to support higher values This makes the server run on Oracle Linux UEK 6.10+. --- base/glibc-compatibility/musl/getauxval.c | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index ec2cce1e4aa..cc0cdf25b03 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -25,9 +25,10 @@ // We don't have libc struct available here. // Compute aux vector manually (from /proc/self/auxv). // -// Right now there is only 51 AT_* constants, -// so 64 should be enough until this implementation will be replaced with musl. -static unsigned long __auxv_procfs[64]; +// Right now there are 51 AT_* constants. Custom kernels have been encountered +// making use of up to 71. 128 should be enough until this implementation is +// replaced with musl. +static unsigned long __auxv_procfs[128]; static unsigned long __auxv_secure = 0; // Common static unsigned long * __auxv_environ = NULL; From 93494b08f594530e913ecf59507b24569704640f Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 28 Oct 2024 10:01:40 +0000 Subject: [PATCH 486/816] fix spelling --- docs/en/sql-reference/table-functions/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 88714d4f24f..df4e10425a5 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -259,7 +259,7 @@ This is appropriate for one-off accesses or in cases where credentials can easil Role-based access for S3 in ClickHouse Cloud is documented [here](/docs/en/cloud/security/secure-s3#access-your-s3-bucket-with-the-clickhouseaccess-role). -Once configured, a roleARN can be passed to the s3 function via an `extra_credentials` parameter. For example: +Once configured, a `roleARN` can be passed to the s3 function via an `extra_credentials` parameter. For example: ```sql SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/mta/*.tsv','CSVWithNames',extra_credentials(role_arn = 'arn:aws:iam::111111111111:role/ClickHouseAccessRole-001')) From 9eb683494670c75932d45926eb553e4d46c947cf Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 28 Oct 2024 11:38:23 +0100 Subject: [PATCH 487/816] Update error message for JSONAsObject format --- src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 1985c7433c8..06557db9aa2 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -172,7 +172,7 @@ JSONAsObjectRowInputFormat::JSONAsObjectRowInputFormat( const auto & type = header_.getByPosition(0).type; if (!isObject(type) && !isObjectDeprecated(type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Input format JSONAsObject is only suitable for tables with a single column of type Object/JSON but the column type is {}", + "Input format JSONAsObject is only suitable for tables with a single column of type JSON but the column type is {}", type->getName()); } @@ -193,8 +193,8 @@ JSONAsObjectExternalSchemaReader::JSONAsObjectExternalSchemaReader(const FormatS if (!settings.json.allow_deprecated_object_type && !settings.json.allow_json_type) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Cannot infer the data structure in JSONAsObject format because experimental Object/JSON type is not allowed. Set setting " - "allow_experimental_object_type = 1 or allow_experimental_json_type=1 in order to allow it"); + "Cannot infer the data structure in JSONAsObject format because experimental JSON type is not allowed. Set setting " + "allow_experimental_json_type = 1 in order to allow it"); } void registerInputFormatJSONAsString(FormatFactory & factory) From de880d0f8ba39258f4e6080cfb8927d2243dcce9 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 10:28:02 +0000 Subject: [PATCH 488/816] Fix race condition in system.query_metric_log If a scheduled task was pending to lock the query_mutex, it could do so between the lock.unlock() and lock.lock() of finishQuery which is used to break the exec_mutex deadlock. In that case, a task scheduled running after the task finished could attempt to do a new collection, which is wrong. Let's avoid that altogether, and also add some defensive programming to ensure this does not happen in the future. --- src/Interpreters/QueryMetricLog.cpp | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index fea2024d3e4..33eeac592f4 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +}; + static auto logger = getLogger("QueryMetricLog"); ColumnsDescription QueryMetricLogElement::getColumnsDescription() @@ -137,7 +142,7 @@ void QueryMetricLog::finishQuery(const String & query_id, QueryStatusInfoPtr que /// deactivating the task, which happens automatically on its destructor. Thus, we cannot /// deactivate/destroy the task while it's running. Now, the task locks `queries_mutex` to /// prevent concurrent edition of the queries. In short, the mutex order is: exec_mutex -> - /// queries_mutex. Thus, to prevent a deadblock we need to make sure that we always lock them in + /// queries_mutex. So, to prevent a deadblock we need to make sure that we always lock them in /// that order. { /// Take ownership of the task so that we can destroy it in this scope after unlocking `queries_lock`. @@ -162,7 +167,7 @@ std::optional QueryMetricLog::createLogMetricElement(cons auto query_status_it = queries.find(query_id); /// The query might have finished while the scheduled task is running. - if (query_status_it == queries.end()) + if (query_status_it == queries.end() || !query_status_it->second.task) return {}; QueryMetricLogElement elem; @@ -178,8 +183,16 @@ std::optional QueryMetricLog::createLogMetricElement(cons for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto & new_value = (*(query_info.profile_counters))[i]; - elem.profile_events[i] = new_value - query_status.last_profile_events[i]; - query_status.last_profile_events[i] = new_value; + auto & prev_value = query_status.last_profile_events[i]; + + /// Profile event count is monotonically increasing. + if (new_value < prev_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", + ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); + + elem.profile_events[i] = new_value - prev_value; + prev_value = new_value; } } else From bdde9da3ca59f4aa8f7270f796374dae83d036cf Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Mon, 28 Oct 2024 13:39:40 +0300 Subject: [PATCH 489/816] Trig Ci --- tests/integration/test_reload_client_certificate/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index a245953f1e2..cb091d92ea6 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -164,7 +164,7 @@ def check_certificate_switch(first, second): time.sleep(10) - # Check information about client certificates reloading in log + # Check information about client certificates reloading in log Clickhouse reload_successful = any(check_reload_successful(node, second) for node in nodes) From d725ae8e3cf8a5195763887fb3122a771f0fa17e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 11:52:39 +0100 Subject: [PATCH 490/816] Initial changelog for 24.11 --- CHANGELOG.md | 387 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 387 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6c0d21a4698..2703d5b6ee0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v24.10, 2024-10-31](#2410)**
**[ClickHouse release v24.9, 2024-09-26](#249)**
**[ClickHouse release v24.8 LTS, 2024-08-20](#248)**
**[ClickHouse release v24.7, 2024-07-30](#247)**
@@ -12,6 +13,392 @@ # 2024 Changelog +### ClickHouse release 24.10, 2024-10-31 + +#### Backward Incompatible Change +* Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#68614](https://github.com/ClickHouse/ClickHouse/pull/68614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). +* Allow empty needle in functions replaceRegexp*, like https://github.com/ClickHouse/ClickHouse/pull/69918. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). +* Reordering of filter conditions from `[PRE]WHERE` clause is now allowed by default. It could be disabled by setting `allow_reorder_prewhere_conditions` to `false`. [#70657](https://github.com/ClickHouse/ClickHouse/pull/70657) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `optimize_functions_to_subcolumns` optimization (previously could lead to `Invalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)` error), by preserving `LowCardinality` type in `mapKeys`/`mapValues`. [#70716](https://github.com/ClickHouse/ClickHouse/pull/70716) ([Azat Khuzhin](https://github.com/azat)). +* Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB, remove support for deprecated old protocol, support for connection by URI, support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). +* A new `--progress-table` option in clickhouse-client prints a table with metrics changing during query execution; a new `--enable-progress-table-toggle` is associated with the `--progress-table` option, and toggles the rendering of the progress table by pressing the control key (Space). [#63689](https://github.com/ClickHouse/ClickHouse/pull/63689) ([Maria Khristenko](https://github.com/mariaKhr)). +* This allows to grant access to the wildcard prefixes. `GRANT SELECT ON db.table_pefix_* TO user`. [#65311](https://github.com/ClickHouse/ClickHouse/pull/65311) ([pufit](https://github.com/pufit)). +* Add system.query_metric_log which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). +* A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support --copy mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). +* Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). +* Support aggreate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. BTW, it is for spark compatiability in Apache Gluten. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). +* Support function arrayElementOrNull. It returns null if array index is out of range or map key not found. [#69646](https://github.com/ClickHouse/ClickHouse/pull/69646) ([李扬](https://github.com/taiyang-li)). +* Allows users to specify regular expressions through new `message_regexp` and `message_regexp_negative` fields in the `config.xml` file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. [#69657](https://github.com/ClickHouse/ClickHouse/pull/69657) ([Peter Nguyen](https://github.com/petern48)). +* Support Dynamic type in most functions by executing them on internal types inside Dynamic. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). +* Re-added `RIPEMD160` function, which computes the RIPEMD-160 cryptographic hash of a string. Example: `SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))` returns `37F332F68DB77BD9D7EDD4969571AD671CF9DD3B`. [#70087](https://github.com/ClickHouse/ClickHouse/pull/70087) ([Dergousov Maxim](https://github.com/m7kss1)). +* Allow to cache read files for object storage table engines and data lakes using hash from ETag + file path as cache key. [#70135](https://github.com/ClickHouse/ClickHouse/pull/70135) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support reading Iceberg tables on HDFS. [#70268](https://github.com/ClickHouse/ClickHouse/pull/70268) ([flynn](https://github.com/ucasfl)). +* Allow to read/write JSON type as binary string in RowBinary format under settings `input_format_binary_read_json_as_string/output_format_binary_write_json_as_string`. [#70288](https://github.com/ClickHouse/ClickHouse/pull/70288) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow to serialize/deserialize JSON column as single String column in Native format. For output use setting `output_format_native_write_json_as_string`. For input, use serialization version `1` before the column data. [#70312](https://github.com/ClickHouse/ClickHouse/pull/70312) ([Pavel Kruglov](https://github.com/Avogar)). +* Supports standard CTE, `with insert`, as previously only supports `insert ... with ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). + +#### Performance Improvement +* Support minmax index for `pointInPolygon`. [#62085](https://github.com/ClickHouse/ClickHouse/pull/62085) ([JackyWoo](https://github.com/JackyWoo)). +* Add support for parquet bloom filters. [#62966](https://github.com/ClickHouse/ClickHouse/pull/62966) ([Arthur Passos](https://github.com/arthurpassos)). +* Lock-free parts rename to avoid INSERT affect SELECT (due to parts lock) (under normal circumstances with `fsync_part_directory`, QPS of SELECT with INSERT in parallel, increased 2x, under heavy load the effect is even bigger). Note, this only includes `ReplicatedMergeTree` for now. [#64955](https://github.com/ClickHouse/ClickHouse/pull/64955) ([Azat Khuzhin](https://github.com/azat)). +* Respect `ttl_only_drop_parts` on `materialize ttl`; only read necessary columns to recalculate TTL and drop parts by replacing them with an empty one. [#65488](https://github.com/ClickHouse/ClickHouse/pull/65488) ([Andrey Zvonov](https://github.com/zvonand)). +* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Optimized thread creation in the ThreadPool to minimize lock contention. Thread creation is now performed outside of the critical section to avoid delays in job scheduling and thread management under high load conditions. This leads to a much more responsive ClickHouse under heavy concurrent load. [#68694](https://github.com/ClickHouse/ClickHouse/pull/68694) ([filimonov](https://github.com/filimonov)). +* Enable reading LowCardinality string columns from ORC. [#69481](https://github.com/ClickHouse/ClickHouse/pull/69481) ([李扬](https://github.com/taiyang-li)). +* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). +* Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. [#69862](https://github.com/ClickHouse/ClickHouse/pull/69862) ([LiuNeng](https://github.com/liuneng1994)). +* Improved performance of parsing formats with high number of missed values (e.g. `JSONEachRow`). [#69875](https://github.com/ClickHouse/ClickHouse/pull/69875) ([Anton Popov](https://github.com/CurtizJ)). +* Use `LowCardinality` for `ProfileEvents` in system logs such as `part_log`, `query_views_log`, `filesystem_cache_log`. [#70152](https://github.com/ClickHouse/ClickHouse/pull/70152) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve performance of FromUnixTimestamp/ToUnixTimestamp functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). + +#### Improvement +* Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Fixed [#57616](https://github.com/ClickHouse/ClickHouse/issues/57616) this problem occurs because all positive number arguments are automatically identified as `uint64` type, leading to an inability to match int type data in `summapfiltered`. the issue of non-matching is indeed confusing, as the `uint64` parameters are not specified by the user. additionally, if the arguments are `[1,2,3,toint8(-3)]`, due to the `getleastsupertype()`, these parameters will be uniformly treated as `int` type, causing `'1,2,3'` to also fail in matching the `uint` type data in `summapfiltered`. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). +* `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). +* Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). +* Symbolic links for tables in the `data/database_name/` directory are created for the actual paths to the table's data, depending on the storage policy, instead of the `store/...` directory on the default disk. [#61777](https://github.com/ClickHouse/ClickHouse/pull/61777) ([Kirill](https://github.com/kirillgarbar)). +* Apply configuration updates in global context object. It fixes issues like [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308). [#62944](https://github.com/ClickHouse/ClickHouse/pull/62944) ([Amos Bird](https://github.com/amosbird)). +* Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `ReadSettings` not using user set values, because defaults were only used. [#65625](https://github.com/ClickHouse/ClickHouse/pull/65625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* While parsing an Enum field from JSON, a string containing an integer will be interpreted as the corresponding Enum element. This closes [#65119](https://github.com/ClickHouse/ClickHouse/issues/65119). [#66801](https://github.com/ClickHouse/ClickHouse/pull/66801) ([scanhex12](https://github.com/scanhex12)). +* Allow `TRIM` -ing `LEADING` or `TRAILING` empty string as a no-op. Closes [#67792](https://github.com/ClickHouse/ClickHouse/issues/67792). [#68455](https://github.com/ClickHouse/ClickHouse/pull/68455) ([Peter Nguyen](https://github.com/petern48)). +* Support creating a table with a query: `CREATE TABLE ... CLONE AS ...`. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of the `MergeTree` family Closes [#65015](https://github.com/ClickHouse/ClickHouse/issues/65015). [#69091](https://github.com/ClickHouse/ClickHouse/pull/69091) ([tuanpach](https://github.com/tuanpach)). +* In Gluten ClickHouse, Spark's timestamp type is mapped to ClickHouse's datetime64(6) type. When casting timestamp '2012-01-01 00:11:22' as a string, Spark returns '2012-01-01 00:11:22', while Gluten ClickHouse returns '2012-01-01 00:11:22.000000'. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). +* Always use the new analyzer to calculate constant expressions when `enable_analyzer` is set to `true`. Support calculation of `executable()` table function arguments without using `SELECT` query for constant expression. [#69292](https://github.com/ClickHouse/ClickHouse/pull/69292) ([Dmitry Novik](https://github.com/novikd)). +* Add `enable_secure_identifiers` to disallow insecure identifiers. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). +* Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior of the show create query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}`, or it can cause ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/69346 Point 4 described there will work now as well:. [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). +* Implement generic SerDe between Avro Union and ClickHouse Variant type. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). +* 1. CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses. Now it is supported only for the MergeTree family of table engines. 2. For example, the follow SQL statements will trigger exception in the past, but this PR fixes it: if the destination table do not provide an `ORDER BY` or `PRIMARY KEY` expression in the table definition, we will copy that from source table. [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). +* Added user-level settings `min_free_disk_bytes_to_throw_insert` and `min_free_disk_ratio_to_throw_insert` to prevent insertions on disks that are almost full. [#69755](https://github.com/ClickHouse/ClickHouse/pull/69755) ([Marco Vilas Boas](https://github.com/marco-vb)). +* If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add new column readonly_duration to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Change the join to sort settings type to unsigned int. [#69886](https://github.com/ClickHouse/ClickHouse/pull/69886) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support 64-bit XID in Keeper. It can be enabled with `use_xid_64` config. [#69908](https://github.com/ClickHouse/ClickHouse/pull/69908) ([Antonio Andelic](https://github.com/antonio2368)). +* New function getSettingOrDefault() added to return the default value and avoid exception if a custom setting is not found in the current profile. [#69917](https://github.com/ClickHouse/ClickHouse/pull/69917) ([Shankar](https://github.com/shiyer7474)). +* Enhance OpenTelemetry span logging to include query settings. [#70011](https://github.com/ClickHouse/ClickHouse/pull/70011) ([sharathks118](https://github.com/sharathks118)). +* Add info to higher-order array functions if lambda result type is unexpected. [#70093](https://github.com/ClickHouse/ClickHouse/pull/70093) ([ttanay](https://github.com/ttanay)). +* Keeper improvement: less blocking during cluster changes. [#70275](https://github.com/ClickHouse/ClickHouse/pull/70275) ([Antonio Andelic](https://github.com/antonio2368)). +* Embedded documentation for settings will be strictly more detailed and complete than the documentation on the website. This is the first step before making the website documentation always auto-generated from the source code. This has long-standing implications: - it will be guaranteed to have every setting; - there is no chance of having default values obsolete; - we can generate this documentation for each ClickHouse version; - the documentation can be displayed by the server itself even without Internet access. Generate the docs on the website from the source code. [#70289](https://github.com/ClickHouse/ClickHouse/pull/70289) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `WITH IMPLICIT` and `FINAL` keywords to the `SHOW GRANTS` command. Fix a minor bug with implicit grants: [#70094](https://github.com/ClickHouse/ClickHouse/issues/70094). [#70293](https://github.com/ClickHouse/ClickHouse/pull/70293) ([pufit](https://github.com/pufit)). +* Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. [#70299](https://github.com/ClickHouse/ClickHouse/pull/70299) ([Antonio Andelic](https://github.com/antonio2368)). +* Respect `compatibility` for MergeTree settings. The `compatibility` value is taken from the `default` profile on server startup, and default MergeTree settings are changed accordingly. Further changes of the `compatibility` setting do not affect MergeTree settings. [#70322](https://github.com/ClickHouse/ClickHouse/pull/70322) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Clickhouse-client realtime metrics follow-up: restore cursor when ctrl-c cancels query; immediately stop intercepting keystrokes when the query is canceled; display the metrics table if `--progress-table` is on, and toggling is disabled. [#70423](https://github.com/ClickHouse/ClickHouse/pull/70423) ([Julia Kartseva](https://github.com/jkartseva)). +* Command-line arguments for Bool settings are set to true when no value is provided for the argument (e.g. `clickhouse-client --optimize_aggregation_in_order --query "SELECT 1"`). [#70459](https://github.com/ClickHouse/ClickHouse/pull/70459) ([davidtsuk](https://github.com/davidtsuk)). +* Avoid spamming the logs with large HTTP response bodies in case of errors during inter-server communication. [#70487](https://github.com/ClickHouse/ClickHouse/pull/70487) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Added a new setting `max_parts_to_move` to control the maximum number of parts that can be moved at once. [#70520](https://github.com/ClickHouse/ClickHouse/pull/70520) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Limit the frequency of certain log messages. [#70601](https://github.com/ClickHouse/ClickHouse/pull/70601) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Don't do validation when synchronizing user_directories from keeper. [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). +* Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the `merge_selector_use_blurry_base` MergeTree-level setting. [#70645](https://github.com/ClickHouse/ClickHouse/pull/70645) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* `CHECK TABLE` with `PART` qualifier was incorrectly formatted in the client. [#70660](https://github.com/ClickHouse/ClickHouse/pull/70660) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support write column index and offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). +* Support parse `DateTime64` for microseond and timezone in joda syntax. [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). +* Changed an approach to figure out if a cloud storage supports [batch delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) or not. [#70786](https://github.com/ClickHouse/ClickHouse/pull/70786) ([Vitaly Baranov](https://github.com/vitlibar)). +* Support for Parquet page V2 on native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). +* Add an HTML page for visualizing merges. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A check if table has both `storage_policy` and `disk` set after alter query is added. A check if a new storage policy is compatible with an old one when using `disk` setting is added. [#70839](https://github.com/ClickHouse/ClickHouse/pull/70839) ([Kirill](https://github.com/kirillgarbar)). +* Add system.s3_queue_settings and system.azure_queue_settings. [#70841](https://github.com/ClickHouse/ClickHouse/pull/70841) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Functions `base58Encode` and `base58Decode` now accept arguments of type `FixedString`. Example: `SELECT base58Encode(toFixedString('plaintext', 9));`. [#70846](https://github.com/ClickHouse/ClickHouse/pull/70846) ([Faizan Patel](https://github.com/faizan2786)). +* Add the `partition` column to every entry type of the part log. Previously, it was set only for some entries. This closes [#70819](https://github.com/ClickHouse/ClickHouse/issues/70819). [#70848](https://github.com/ClickHouse/ClickHouse/pull/70848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add merge start and mutate start events into `system.part_log` which helps with merges analysis and visualization. [#70850](https://github.com/ClickHouse/ClickHouse/pull/70850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). +* Add a profile event about the number of merged source parts. It allows the monitoring of the fanout of the merge tree in production. [#70908](https://github.com/ClickHouse/ClickHouse/pull/70908) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). +* Background downloads to filesystem cache was enabled back. [#70929](https://github.com/ClickHouse/ClickHouse/pull/70929) ([Nikita Taranov](https://github.com/nickitat)). +* Add a new merge selector algorithm, named `Trivial`, for professional usage only. It is worse than the `Simple` merge selector. [#70969](https://github.com/ClickHouse/ClickHouse/pull/70969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix toHour-like conversion functions' monotonicity when optional time zone argument is passed. [#60264](https://github.com/ClickHouse/ClickHouse/pull/60264) ([Amos Bird](https://github.com/amosbird)). +* Relax `supportsPrewhere` check for StorageMerge. This fixes [#61064](https://github.com/ClickHouse/ClickHouse/issues/61064). It was hardened unnecessarily in [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#61091](https://github.com/ClickHouse/ClickHouse/pull/61091) ([Amos Bird](https://github.com/amosbird)). +* Fix `use_concurrency_control` setting handling for proper `concurrent_threads_soft_limit_num` limit enforcing. This enables concurrency control by default because previously it was broken. [#61473](https://github.com/ClickHouse/ClickHouse/pull/61473) ([Sergei Trifonov](https://github.com/serxa)). +* Fix incorrect JOIN ON section optimization in case of `IS NULL` check under any other function (like `NOT`) that may lead to wrong results. Closes [#67915](https://github.com/ClickHouse/ClickHouse/issues/67915). [#68049](https://github.com/ClickHouse/ClickHouse/pull/68049) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Prevent `ALTER` queries that would make the `CREATE` query of tables invalid. [#68574](https://github.com/ClickHouse/ClickHouse/pull/68574) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix inconsistent AST formatting for `negate` (`-`) and `NOT` functions with tuples and arrays. [#68600](https://github.com/ClickHouse/ClickHouse/pull/68600) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix insertion of incomplete type into Dynamic during deserialization. It could lead to `Parameter out of bound` errors. [#69291](https://github.com/ClickHouse/ClickHouse/pull/69291) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix inf loop after `restore replica` in the replicated merge tree with zero copy. [#69293](https://github.com/ClickHouse/ClickHouse/pull/69293) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Return back default value of `processing_threads_num` as number of cpu cores in storage `S3Queue`. [#69384](https://github.com/ClickHouse/ClickHouse/pull/69384) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns ( fixes [#41971](https://github.com/ClickHouse/ClickHouse/issues/41971) ). [#69556](https://github.com/ClickHouse/ClickHouse/pull/69556) ([Eliot Hautefeuille](https://github.com/hileef)). +* Fix vrash during insertion into FixedString column in PostgreSQL engine. [#69584](https://github.com/ClickHouse/ClickHouse/pull/69584) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix crash when executing `create view t as (with recursive 42 as ttt select ttt);`. [#69676](https://github.com/ClickHouse/ClickHouse/pull/69676) ([Han Fei](https://github.com/hanfei1991)). +* Added `strict_once` mode to aggregate function `windowFunnel` to avoid counting one event several times in case it matches multiple conditions, close [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835). [#69738](https://github.com/ClickHouse/ClickHouse/pull/69738) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed `maxMapState` throwing 'Bad get' if value type is DateTime64. [#69787](https://github.com/ClickHouse/ClickHouse/pull/69787) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix `getSubcolumn` with `LowCardinality` columns by overriding `useDefaultImplementationForLowCardinalityColumns` to return `true`. [#69831](https://github.com/ClickHouse/ClickHouse/pull/69831) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Fix permanent blocked distributed sends if DROP of distributed table fails. [#69843](https://github.com/ClickHouse/ClickHouse/pull/69843) ([Azat Khuzhin](https://github.com/azat)). +* Fix non-cancellable queries containing WITH FILL with NaN keys. This closes [#69261](https://github.com/ClickHouse/ClickHouse/issues/69261). [#69845](https://github.com/ClickHouse/ClickHouse/pull/69845) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix analyzer default with old compatibility value. [#69895](https://github.com/ClickHouse/ClickHouse/pull/69895) ([Raúl Marín](https://github.com/Algunenano)). +* Don't check dependencies during CREATE OR REPLACE VIEW during DROP of old table. Previously CREATE OR REPLACE query failed when there are dependent tables of the recreated view. [#69907](https://github.com/ClickHouse/ClickHouse/pull/69907) ([Pavel Kruglov](https://github.com/Avogar)). +* Implement missing decimal cases for `zeroField`. Fixes [#69730](https://github.com/ClickHouse/ClickHouse/issues/69730). [#69978](https://github.com/ClickHouse/ClickHouse/pull/69978) ([Arthur Passos](https://github.com/arthurpassos)). +* Now SQL security will work with parameterized views correctly. [#69984](https://github.com/ClickHouse/ClickHouse/pull/69984) ([pufit](https://github.com/pufit)). +* Closes [#69752](https://github.com/ClickHouse/ClickHouse/issues/69752). [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). +* Fixed a bug when the timezone could change the result of the query with a `Date` or `Date32` arguments. [#70036](https://github.com/ClickHouse/ClickHouse/pull/70036) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixes `Block structure mismatch` for queries with nested views and `WHERE` condition. Fixes [#66209](https://github.com/ClickHouse/ClickHouse/issues/66209). [#70054](https://github.com/ClickHouse/ClickHouse/pull/70054) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid reusing columns among different named tuples when evaluating `tuple` functions. This fixes [#70022](https://github.com/ClickHouse/ClickHouse/issues/70022). [#70103](https://github.com/ClickHouse/ClickHouse/pull/70103) ([Amos Bird](https://github.com/amosbird)). +* Fix wrong LOGICAL_ERROR when replacing literals in ranges. [#70122](https://github.com/ClickHouse/ClickHouse/pull/70122) ([Pablo Marcos](https://github.com/pamarcos)). +* Check for Nullable(Nothing) type during ALTER TABLE MODIFY COLUMN/QUERY to prevent tables with such data type. [#70123](https://github.com/ClickHouse/ClickHouse/pull/70123) ([Pavel Kruglov](https://github.com/Avogar)). +* Proper error message for illegal query `JOIN ... ON *` , close [#68650](https://github.com/ClickHouse/ClickHouse/issues/68650). [#70124](https://github.com/ClickHouse/ClickHouse/pull/70124) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix wrong result with skipping index. [#70127](https://github.com/ClickHouse/ClickHouse/pull/70127) ([Raúl Marín](https://github.com/Algunenano)). +* Fix data race in ColumnObject/ColumnTuple decompress method that could lead to heap use after free. [#70137](https://github.com/ClickHouse/ClickHouse/pull/70137) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix possible hung in ALTER COLUMN with Dynamic type. [#70144](https://github.com/ClickHouse/ClickHouse/pull/70144) ([Pavel Kruglov](https://github.com/Avogar)). +* Now ClickHouse will consider more errors as retriable and will not mark data parts as broken in case of such errors. [#70145](https://github.com/ClickHouse/ClickHouse/pull/70145) ([alesapin](https://github.com/alesapin)). +* Use correct `max_types` parameter during Dynamic type creation for JSON subcolumn. [#70147](https://github.com/ClickHouse/ClickHouse/pull/70147) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix the password being displayed in `system.query_log` for users with bcrypt password authentication method. [#70148](https://github.com/ClickHouse/ClickHouse/pull/70148) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix event counter for native interface (InterfaceNativeSendBytes). [#70153](https://github.com/ClickHouse/ClickHouse/pull/70153) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible crash in JSON column. [#70172](https://github.com/ClickHouse/ClickHouse/pull/70172) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix multiple issues with arrayMin and arrayMax. [#70207](https://github.com/ClickHouse/ClickHouse/pull/70207) ([Raúl Marín](https://github.com/Algunenano)). +* Respect setting allow_simdjson in JSON type parser. [#70218](https://github.com/ClickHouse/ClickHouse/pull/70218) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix server segfault on creating a materialized view with two selects and an `INTERSECT`, e.g. `CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);`. [#70264](https://github.com/ClickHouse/ClickHouse/pull/70264) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Don't modify global settings with startup scripts. Previously, changing a setting in a startup script would change it globally. [#70310](https://github.com/ClickHouse/ClickHouse/pull/70310) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ALTER of Dynamic type with reducing max_types parameter that could lead to server crash. [#70328](https://github.com/ClickHouse/ClickHouse/pull/70328) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix crash when using WITH FILL incorrectly. [#70338](https://github.com/ClickHouse/ClickHouse/pull/70338) ([Raúl Marín](https://github.com/Algunenano)). +* Fix possible use-after-free in `SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf`. [#70358](https://github.com/ClickHouse/ClickHouse/pull/70358) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash during GROUP BY JSON sub-object subcolumn. [#70374](https://github.com/ClickHouse/ClickHouse/pull/70374) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't prefetch parts for vertical merges if part has no rows. [#70452](https://github.com/ClickHouse/ClickHouse/pull/70452) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix crash in WHERE with lambda functions. [#70464](https://github.com/ClickHouse/ClickHouse/pull/70464) ([Raúl Marín](https://github.com/Algunenano)). +* Fix table creation with `CREATE ... AS table_function()` with database `Replicated` and unavailable table function source on secondary replica. [#70511](https://github.com/ClickHouse/ClickHouse/pull/70511) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Ignore all output on async insert with `wait_for_async_insert=1`. Closes [#62644](https://github.com/ClickHouse/ClickHouse/issues/62644). [#70530](https://github.com/ClickHouse/ClickHouse/pull/70530) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Ignore frozen_metadata.txt while traversing shadow directory from system.remote_data_paths. [#70590](https://github.com/ClickHouse/ClickHouse/pull/70590) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix creation of stateful window functions on misaligned memory. [#70631](https://github.com/ClickHouse/ClickHouse/pull/70631) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed rare crashes in `SELECT`-s and merges after adding a column of `Array` type with non-empty default expression. [#70695](https://github.com/ClickHouse/ClickHouse/pull/70695) ([Anton Popov](https://github.com/CurtizJ)). +* Insert into table function s3 respect query settings. [#70696](https://github.com/ClickHouse/ClickHouse/pull/70696) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix infinite recursion when infering a proto schema with skip unsupported fields enabled. [#70697](https://github.com/ClickHouse/ClickHouse/pull/70697) ([Raúl Marín](https://github.com/Algunenano)). +* Disable enable_named_columns_in_function_tuple by default. [#70833](https://github.com/ClickHouse/ClickHouse/pull/70833) ([Raúl Marín](https://github.com/Algunenano)). +* Fix S3Queue table engine setting processing_threads_num not being effective in case it was deduced from the number of cpu cores on the server. [#70837](https://github.com/ClickHouse/ClickHouse/pull/70837) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Normalize named tuple arguments in aggregation states. This fixes [#69732](https://github.com/ClickHouse/ClickHouse/issues/69732) . [#70853](https://github.com/ClickHouse/ClickHouse/pull/70853) ([Amos Bird](https://github.com/amosbird)). +* Fix a logical error due to negative zeros in the two-level hash table. This closes [#70973](https://github.com/ClickHouse/ClickHouse/issues/70973). [#70979](https://github.com/ClickHouse/ClickHouse/pull/70979) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Docker in integration tests runner is updated to latest version. It was previously pinned u until patch release 24.0.3 was out. https://github.com/moby/moby/issues/45770#issuecomment-1618255130. - HDFS image was deprecated and not running with current docker version. Switched to newer version of a derivative image based on ubuntu. - HDFS tests were hardened to allow them to run with python-repeat. [#66867](https://github.com/ClickHouse/ClickHouse/pull/66867) ([Ilya Yatsishin](https://github.com/qoega)). +* Alpine docker images now use ubuntu 22.04 as glibc donor, results in upgrade of glibc version delivered with alpine images from 2.31 to 2.35. [#69033](https://github.com/ClickHouse/ClickHouse/pull/69033) ([filimonov](https://github.com/filimonov)). +* Makes dbms independent from clickhouse_functions. [#69914](https://github.com/ClickHouse/ClickHouse/pull/69914) ([Raúl Marín](https://github.com/Algunenano)). +* Fix FreeBSD compilation of the MariaDB connector. [#70007](https://github.com/ClickHouse/ClickHouse/pull/70007) ([Raúl Marín](https://github.com/Algunenano)). +* Building on Apple Mac OS X Darwin does not produce strange warnings anymore. [#70411](https://github.com/ClickHouse/ClickHouse/pull/70411) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix building with ARCH_NATIVE CMake flag. [#70585](https://github.com/ClickHouse/ClickHouse/pull/70585) ([Daniil Gentili](https://github.com/danog)). +* The universal installer will download Musl build on Alpine Linux. Some Docker containers are using Alpine Linux, but it was not possible to install ClickHouse there with `curl https://clickhouse.com/ | sh`. [#70767](https://github.com/ClickHouse/ClickHouse/pull/70767) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "JSONCompactWithProgress query output format"'. [#69989](https://github.com/ClickHouse/ClickHouse/pull/69989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Support CREATE OR REPLACE VIEW atomically"'. [#70535](https://github.com/ClickHouse/ClickHouse/pull/70535) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Support CREATE OR REPLACE VIEW atomically""'. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Add projections size to system.projections"'. [#70858](https://github.com/ClickHouse/ClickHouse/pull/70858) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Allow writing argument of `has` or `hasAny` or `hasAll` as string values if array element type is `Enum`. [#56555](https://github.com/ClickHouse/ClickHouse/pull/56555) ([Duc Canh Le](https://github.com/canhld94)). +* Rename FileSegmentKind::Ephemeral and other changes. [#66600](https://github.com/ClickHouse/ClickHouse/pull/66600) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Closes [#67345](https://github.com/ClickHouse/ClickHouse/issues/67345). [#67346](https://github.com/ClickHouse/ClickHouse/pull/67346) ([KrJin](https://github.com/jincong8973)). +* Because it is too complicated to support. [#68410](https://github.com/ClickHouse/ClickHouse/pull/68410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix 01600_parts_states_metrics_long flakiness. [#68521](https://github.com/ClickHouse/ClickHouse/pull/68521) ([Azat Khuzhin](https://github.com/azat)). +* Reduce client start time in debug/sanitizer mode. [#68980](https://github.com/ClickHouse/ClickHouse/pull/68980) ([Raúl Marín](https://github.com/Algunenano)). +* Closes [#69038](https://github.com/ClickHouse/ClickHouse/issues/69038). [#69040](https://github.com/ClickHouse/ClickHouse/pull/69040) ([Nikolay Degterinsky](https://github.com/evillique)). +* Better exception for unsupported full_text index with non-full parts. [#69067](https://github.com/ClickHouse/ClickHouse/pull/69067) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Catch additional zk connection erros while creating table and make sure to cleanup dirs if necessary for retries. [#69093](https://github.com/ClickHouse/ClickHouse/pull/69093) ([Sumit](https://github.com/sum12)). +* Update version_date.tsv and changelog after v24.7.5.37-stable. [#69185](https://github.com/ClickHouse/ClickHouse/pull/69185) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* DOCS: Replace live view with refreshable since the former is deprecated. [#69392](https://github.com/ClickHouse/ClickHouse/pull/69392) ([Damian Kula](https://github.com/heavelock)). +* Update ORC to the current HEAD. [#69473](https://github.com/ClickHouse/ClickHouse/pull/69473) ([Nikita Taranov](https://github.com/nickitat)). +* Make a test ready for flaky check. [#69586](https://github.com/ClickHouse/ClickHouse/pull/69586) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support antlr parser to parse sql with some keywords as alias, make the behaviour same as the clickhouse-server - remove redundant `for` in the `keyword` field. [#69614](https://github.com/ClickHouse/ClickHouse/pull/69614) ([Z.H.](https://github.com/onlyacat)). +* Allow default implementations for null in function mapFromArrays for spark compatiability in apache gluten. Current change doesn't have any side effects on clickhouse in theory. [#69715](https://github.com/ClickHouse/ClickHouse/pull/69715) ([李扬](https://github.com/taiyang-li)). +* Fix exception message in AzureBlobStorage. [#69728](https://github.com/ClickHouse/ClickHouse/pull/69728) ([Pavel Kruglov](https://github.com/Avogar)). +* Add test parsing s3 URL with a bucket name including a dot. [#69743](https://github.com/ClickHouse/ClickHouse/pull/69743) ([Kaushik Iska](https://github.com/iskakaushik)). +* Make `clang-tidy` happy. [#69765](https://github.com/ClickHouse/ClickHouse/pull/69765) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Prepare to enable `clang-tidy` `readability-else-after-return`. [#69768](https://github.com/ClickHouse/ClickHouse/pull/69768) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* S3Queue: support having deprecated settings to not fail server startup. [#69769](https://github.com/ClickHouse/ClickHouse/pull/69769) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use only adaptive heuristic to choose task sizes for remote reading. [#69778](https://github.com/ClickHouse/ClickHouse/pull/69778) ([Nikita Taranov](https://github.com/nickitat)). +* Remove unused buggy code. [#69780](https://github.com/ClickHouse/ClickHouse/pull/69780) ([Raúl Marín](https://github.com/Algunenano)). +* Fix bugfix check. [#69789](https://github.com/ClickHouse/ClickHouse/pull/69789) ([Antonio Andelic](https://github.com/antonio2368)). +* Followup for [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69790](https://github.com/ClickHouse/ClickHouse/pull/69790) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update version after release. [#69816](https://github.com/ClickHouse/ClickHouse/pull/69816) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update ext-dict-functions.md. [#69819](https://github.com/ClickHouse/ClickHouse/pull/69819) ([kurikuQwQ](https://github.com/kurikuQwQ)). +* Allow cyrillic characters in generated contributor names. [#69820](https://github.com/ClickHouse/ClickHouse/pull/69820) ([Raúl Marín](https://github.com/Algunenano)). +* CI: praktika integration 1. [#69822](https://github.com/ClickHouse/ClickHouse/pull/69822) ([Max Kainov](https://github.com/maxknv)). +* Fix `test_delayed_replica_failover`. [#69826](https://github.com/ClickHouse/ClickHouse/pull/69826) ([Antonio Andelic](https://github.com/antonio2368)). +* minor change, less conflicts. [#69830](https://github.com/ClickHouse/ClickHouse/pull/69830) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Improve error message DDLWorker.cpp. [#69835](https://github.com/ClickHouse/ClickHouse/pull/69835) ([Denny Crane](https://github.com/den-crane)). +* Fix typo in description: mutation_sync -> mutations_sync. [#69838](https://github.com/ClickHouse/ClickHouse/pull/69838) ([Alexander Gololobov](https://github.com/davenger)). +* Fix changelog. [#69841](https://github.com/ClickHouse/ClickHouse/pull/69841) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#49940](https://github.com/ClickHouse/ClickHouse/issues/49940). [#69842](https://github.com/ClickHouse/ClickHouse/pull/69842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#51036](https://github.com/ClickHouse/ClickHouse/issues/51036). [#69844](https://github.com/ClickHouse/ClickHouse/pull/69844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update README.md - Update meetups. [#69849](https://github.com/ClickHouse/ClickHouse/pull/69849) ([Tanya Bragin](https://github.com/tbragin)). +* Revert [#69790](https://github.com/ClickHouse/ClickHouse/issues/69790) and [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69850](https://github.com/ClickHouse/ClickHouse/pull/69850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* See [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69851](https://github.com/ClickHouse/ClickHouse/pull/69851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#50928](https://github.com/ClickHouse/ClickHouse/issues/50928). [#69852](https://github.com/ClickHouse/ClickHouse/pull/69852) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#55981](https://github.com/ClickHouse/ClickHouse/issues/55981). [#69853](https://github.com/ClickHouse/ClickHouse/pull/69853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#56823](https://github.com/ClickHouse/ClickHouse/issues/56823). [#69854](https://github.com/ClickHouse/ClickHouse/pull/69854) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#62350](https://github.com/ClickHouse/ClickHouse/issues/62350). [#69855](https://github.com/ClickHouse/ClickHouse/pull/69855) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refactor functions and variables in statistics code. [#69860](https://github.com/ClickHouse/ClickHouse/pull/69860) ([Robert Schulze](https://github.com/rschu1ze)). +* Resubmit [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69861](https://github.com/ClickHouse/ClickHouse/pull/69861) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Improve stateless test runner. [#69864](https://github.com/ClickHouse/ClickHouse/pull/69864) ([Alexey Katsman](https://github.com/alexkats)). +* Adjust fast test time limit a bit. [#69874](https://github.com/ClickHouse/ClickHouse/pull/69874) ([Raúl Marín](https://github.com/Algunenano)). +* Add initial 24.9 CHANGELOG. [#69876](https://github.com/ClickHouse/ClickHouse/pull/69876) ([Raúl Marín](https://github.com/Algunenano)). +* Fix test `01278_random_string_utf8`. [#69878](https://github.com/ClickHouse/ClickHouse/pull/69878) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix minor fuzzer issue with experimental statistics. [#69881](https://github.com/ClickHouse/ClickHouse/pull/69881) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix linking after settings refactoring. [#69882](https://github.com/ClickHouse/ClickHouse/pull/69882) ([Robert Schulze](https://github.com/rschu1ze)). +* Add Proj Obsolete Setting. [#69883](https://github.com/ClickHouse/ClickHouse/pull/69883) ([Shichao Jin](https://github.com/jsc0218)). +* Improve remote queries startup time. [#69884](https://github.com/ClickHouse/ClickHouse/pull/69884) ([Igor Nikonov](https://github.com/devcrafter)). +* Revert "Merge pull request [#69032](https://github.com/ClickHouse/ClickHouse/issues/69032) from alexon1234/include_real_time_execution_in_http_header". [#69885](https://github.com/ClickHouse/ClickHouse/pull/69885) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A dedicated commits from https://github.com/ClickHouse/ClickHouse/pull/61473. [#69896](https://github.com/ClickHouse/ClickHouse/pull/69896) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added aliases `time_bucket`(from TimescaleDB) and `date_bin`(from PostgreSQL) for `toStartOfInterval`. [#69900](https://github.com/ClickHouse/ClickHouse/pull/69900) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* RIPE is an acronym and thus should be capital. RIPE stands for **R**ACE **I**ntegrity **P**rimitives **E**valuation and RACE stands for **R**esearch and Development in **A**dvanced **C**ommunications **T**echnologies in **E**urope. [#69901](https://github.com/ClickHouse/ClickHouse/pull/69901) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Replace error codes with error names in stateless tests. [#69906](https://github.com/ClickHouse/ClickHouse/pull/69906) ([Dmitry Novik](https://github.com/novikd)). +* Move setting to 24.10. [#69913](https://github.com/ClickHouse/ClickHouse/pull/69913) ([Raúl Marín](https://github.com/Algunenano)). +* Minor: Reduce diff between public and private repo. [#69928](https://github.com/ClickHouse/ClickHouse/pull/69928) ([Robert Schulze](https://github.com/rschu1ze)). +* Followup for [#69861](https://github.com/ClickHouse/ClickHouse/issues/69861). [#69930](https://github.com/ClickHouse/ClickHouse/pull/69930) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix test_dictionaries_all_layouts_separate_sources. [#69962](https://github.com/ClickHouse/ClickHouse/pull/69962) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix test_keeper_mntr_data_size. [#69965](https://github.com/ClickHouse/ClickHouse/pull/69965) ([Antonio Andelic](https://github.com/antonio2368)). +* This closes [#49823](https://github.com/ClickHouse/ClickHouse/issues/49823). [#69981](https://github.com/ClickHouse/ClickHouse/pull/69981) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add changelog for 24.9. [#69982](https://github.com/ClickHouse/ClickHouse/pull/69982) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#45303](https://github.com/ClickHouse/ClickHouse/issues/45303). [#69987](https://github.com/ClickHouse/ClickHouse/pull/69987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#69988](https://github.com/ClickHouse/ClickHouse/pull/69988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update README.md. [#69991](https://github.com/ClickHouse/ClickHouse/pull/69991) ([Tyler Hannan](https://github.com/tylerhannan)). +* Disable `03215_parallel_replicas_crash_after_refactoring.sql` for Azure. [#69992](https://github.com/ClickHouse/ClickHouse/pull/69992) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Update CHANGELOG.md. [#69993](https://github.com/ClickHouse/ClickHouse/pull/69993) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#70004](https://github.com/ClickHouse/ClickHouse/pull/70004) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Add RIPEMD160 function". [#70005](https://github.com/ClickHouse/ClickHouse/pull/70005) ([Robert Schulze](https://github.com/rschu1ze)). +* Update CHANGELOG.md. [#70009](https://github.com/ClickHouse/ClickHouse/pull/70009) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#70010](https://github.com/ClickHouse/ClickHouse/pull/70010) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make the pylint stricter. [#70013](https://github.com/ClickHouse/ClickHouse/pull/70013) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added a setting `restore_replace_external_dictionary_source_to_null` which enables replacing dictionary source with Null on restore for external dictionaries (useful for testing). [#70032](https://github.com/ClickHouse/ClickHouse/pull/70032) ([Alexander Tokmakov](https://github.com/tavplubix)). +* `isort` is a simple import sorter for the python to comply [pep-8](https://peps.python.org/pep-0008/#imports) requirements. It will allow to decrease conflicts during sync and beautify the code. The import block is divided into three sub-blocks: `standard library` -> `third-party libraries` -> `local imports` -> `.local imports`. Each sub-block is ordered alphabetically with sub-sub-blocks `import X` -> `from X import Y`. [#70038](https://github.com/ClickHouse/ClickHouse/pull/70038) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update version_date.tsv and changelog after v24.9.1.3278-stable. [#70049](https://github.com/ClickHouse/ClickHouse/pull/70049) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Despite the fact that we set the org-level workflow parameter `PYTHONUNBUFFERED`, it's not inherited in workflows. [#70050](https://github.com/ClickHouse/ClickHouse/pull/70050) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix ubsan issue in function sqid. [#70061](https://github.com/ClickHouse/ClickHouse/pull/70061) ([Robert Schulze](https://github.com/rschu1ze)). +* Delete a setting change. [#70071](https://github.com/ClickHouse/ClickHouse/pull/70071) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `test_distributed_ddl`. [#70075](https://github.com/ClickHouse/ClickHouse/pull/70075) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused placeholder from exception message string. [#70086](https://github.com/ClickHouse/ClickHouse/pull/70086) ([Alsu Giliazova](https://github.com/alsugiliazova)). +* Better exception message when some of the permission is missing. [#70088](https://github.com/ClickHouse/ClickHouse/pull/70088) ([pufit](https://github.com/pufit)). +* Make vector similarity indexes work with adaptive granularity. [#70101](https://github.com/ClickHouse/ClickHouse/pull/70101) ([Robert Schulze](https://github.com/rschu1ze)). +* Add missing columns `total_rows`, `data_compressed_bytes`, and `data_uncompressed_bytes` to `system.projections`. Part of https://github.com/ClickHouse/ClickHouse/pull/68901. [#70106](https://github.com/ClickHouse/ClickHouse/pull/70106) ([Jordi Villar](https://github.com/jrdi)). +* Make `00938_fix_rwlock_segfault_long` non flaky. [#70109](https://github.com/ClickHouse/ClickHouse/pull/70109) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove TODO. [#70110](https://github.com/ClickHouse/ClickHouse/pull/70110) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Change the default threshold to enable hyper threading. [#70111](https://github.com/ClickHouse/ClickHouse/pull/70111) ([Jiebin Sun](https://github.com/jiebinn)). +* Fixed [#69092](https://github.com/ClickHouse/ClickHouse/issues/69092): if `materialized_postgresql_tables_list=table1(id, code),table(id,name)` (`table1` has name that is a substring for `table`) `getTableAllowedColumns` method returns `[id, code]` for `table` before this fix. [#70114](https://github.com/ClickHouse/ClickHouse/pull/70114) ([Kruglov Kirill](https://github.com/1on)). +* Reduce log level. [#70117](https://github.com/ClickHouse/ClickHouse/pull/70117) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Rename `getNumberOfPhysicalCPUCores` and fix its decription. [#70130](https://github.com/ClickHouse/ClickHouse/pull/70130) ([Nikita Taranov](https://github.com/nickitat)). +* Adding 24.10. [#70132](https://github.com/ClickHouse/ClickHouse/pull/70132) ([Tyler Hannan](https://github.com/tylerhannan)). +* (Re?)-enable libcxx asserts for debug builds. [#70134](https://github.com/ClickHouse/ClickHouse/pull/70134) ([Robert Schulze](https://github.com/rschu1ze)). +* Refactor reading from object storage. [#70141](https://github.com/ClickHouse/ClickHouse/pull/70141) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Silence UBSAN for integer overflows in some datetime functions. [#70142](https://github.com/ClickHouse/ClickHouse/pull/70142) ([Michael Kolupaev](https://github.com/al13n321)). +* Improve pipdeptree generator for docker images. - Update requirements.txt for the integration tests runner container - Remove some small dependencies, improve `helpers/retry_decorator.py` - Upgrade docker-compose from EOL version 1 to version 2. [#70146](https://github.com/ClickHouse/ClickHouse/pull/70146) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix 'QueryPlan was not initialized' in 'loop' with empty MergeTree. [#70149](https://github.com/ClickHouse/ClickHouse/pull/70149) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove QueryPlan DataStream. [#70158](https://github.com/ClickHouse/ClickHouse/pull/70158) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_s3_queue/test.py. [#70159](https://github.com/ClickHouse/ClickHouse/pull/70159) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Small docs fix. [#70160](https://github.com/ClickHouse/ClickHouse/pull/70160) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Test: PR local plan, non-constant in source stream. [#70173](https://github.com/ClickHouse/ClickHouse/pull/70173) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix performance checks. [#70175](https://github.com/ClickHouse/ClickHouse/pull/70175) ([Antonio Andelic](https://github.com/antonio2368)). +* Simplify test 03246_range_literal_replacement_works. [#70176](https://github.com/ClickHouse/ClickHouse/pull/70176) ([Pablo Marcos](https://github.com/pamarcos)). +* Update 01079_parallel_alter_add_drop_column_zookeeper.sh. [#70196](https://github.com/ClickHouse/ClickHouse/pull/70196) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Require bugfix job for a set of labels. [#70197](https://github.com/ClickHouse/ClickHouse/pull/70197) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* CI: Praktika integration, fast test. [#70239](https://github.com/ClickHouse/ClickHouse/pull/70239) ([Max Kainov](https://github.com/maxknv)). +* Avoid `Cannot schedule a task` error when loading parts. [#70257](https://github.com/ClickHouse/ClickHouse/pull/70257) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Bump usearch to v2.15.2 and SimSIMD to v5.0.0. [#70270](https://github.com/ClickHouse/ClickHouse/pull/70270) ([Robert Schulze](https://github.com/rschu1ze)). +* Instead of balancing tests by `crc32(file_name)` we'll use `add tests to a group with a minimal number of tests`. [#70272](https://github.com/ClickHouse/ClickHouse/pull/70272) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Closes [#70263](https://github.com/ClickHouse/ClickHouse/issues/70263). [#70273](https://github.com/ClickHouse/ClickHouse/pull/70273) ([flynn](https://github.com/ucasfl)). +* Hide MergeTreeSettings implementation. [#70285](https://github.com/ClickHouse/ClickHouse/pull/70285) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Remove await feature from release branches. [#70294](https://github.com/ClickHouse/ClickHouse/pull/70294) ([Max Kainov](https://github.com/maxknv)). +* Fix `test_keeper_four_word_command`. [#70298](https://github.com/ClickHouse/ClickHouse/pull/70298) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelog after v24.9.2.42-stable. [#70301](https://github.com/ClickHouse/ClickHouse/pull/70301) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Synchronize settings with private. [#70320](https://github.com/ClickHouse/ClickHouse/pull/70320) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add Ignore Option In DeduplicateMergeProjectionMode. [#70327](https://github.com/ClickHouse/ClickHouse/pull/70327) ([Shichao Jin](https://github.com/jsc0218)). +* CI: Enable Integration Tests for backport PRs. [#70329](https://github.com/ClickHouse/ClickHouse/pull/70329) ([Max Kainov](https://github.com/maxknv)). +* There is [a failed CI job](https://s3.amazonaws.com/clickhouse-test-reports/69778/2d81c38874958bd9d54a25524173bdb1ddf2b75c/stateless_tests__release_.html) which is triggered by [03237_create_or_replace_view_atomically_with_atomic_engine](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/03237_create_or_replace_view_atomically_with_atomic_engine.sh). [#70330](https://github.com/ClickHouse/ClickHouse/pull/70330) ([tuanpach](https://github.com/tuanpach)). +* Fix flaky test `03237_insert_sparse_columns_mem`. [#70333](https://github.com/ClickHouse/ClickHouse/pull/70333) ([Anton Popov](https://github.com/CurtizJ)). +* Rename enable_secure_identifiers -> enforce_strict_identifier_format. [#70335](https://github.com/ClickHouse/ClickHouse/pull/70335) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Attempt to fix flaky RabbitMQ tests. Maybe closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#70336](https://github.com/ClickHouse/ClickHouse/pull/70336) ([filimonov](https://github.com/filimonov)). +* Don't fail the stateless check script if we can't collect minio logs. [#70350](https://github.com/ClickHouse/ClickHouse/pull/70350) ([Raúl Marín](https://github.com/Algunenano)). +* Fix tiny mistake, responsible for some of kafka test flaps. Example [report](https://s3.amazonaws.com/clickhouse-test-reports/0/3198aafac59c368993e7b5f49d95674cc1b1be18/integration_tests__release__[2_4].html). [#70352](https://github.com/ClickHouse/ClickHouse/pull/70352) ([filimonov](https://github.com/filimonov)). +* Closes [#69634](https://github.com/ClickHouse/ClickHouse/issues/69634). [#70354](https://github.com/ClickHouse/ClickHouse/pull/70354) ([pufit](https://github.com/pufit)). +* Fix 02346_fulltext_index_bug52019. [#70357](https://github.com/ClickHouse/ClickHouse/pull/70357) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Use new JSON for collecting minio logs. [#70359](https://github.com/ClickHouse/ClickHouse/pull/70359) ([Antonio Andelic](https://github.com/antonio2368)). +* Update comments in VectorSimilarityCondition (WHERE is not supported). [#70360](https://github.com/ClickHouse/ClickHouse/pull/70360) ([Azat Khuzhin](https://github.com/azat)). +* Remove 02492_clickhouse_local_context_uaf test. [#70363](https://github.com/ClickHouse/ClickHouse/pull/70363) ([Azat Khuzhin](https://github.com/azat)). +* Fix `clang-19` build issues. [#70412](https://github.com/ClickHouse/ClickHouse/pull/70412) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Ignore "Invalid multibyte data detected" error during completion. [#70422](https://github.com/ClickHouse/ClickHouse/pull/70422) ([Azat Khuzhin](https://github.com/azat)). +* Make QueryPlan explain methods const. [#70444](https://github.com/ClickHouse/ClickHouse/pull/70444) ([Alexander Gololobov](https://github.com/davenger)). +* Fix 0.1 second delay for interactive queries (due to keystroke interceptor). [#70445](https://github.com/ClickHouse/ClickHouse/pull/70445) ([Azat Khuzhin](https://github.com/azat)). +* Increase lock timeout in attempt to fix 02125_many_mutations. [#70448](https://github.com/ClickHouse/ClickHouse/pull/70448) ([Azat Khuzhin](https://github.com/azat)). +* Fix order in 03249_dynamic_alter_consistency. [#70453](https://github.com/ClickHouse/ClickHouse/pull/70453) ([Alexander Gololobov](https://github.com/davenger)). +* Fix refreshable MV in system database breaking server startup. [#70460](https://github.com/ClickHouse/ClickHouse/pull/70460) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix flaky test_refreshable_mv_in_replicated_db. [#70462](https://github.com/ClickHouse/ClickHouse/pull/70462) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelog after v24.8.5.115-lts. [#70463](https://github.com/ClickHouse/ClickHouse/pull/70463) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Decrease probability of "Server died" due to 00913_many_threads. [#70473](https://github.com/ClickHouse/ClickHouse/pull/70473) ([Azat Khuzhin](https://github.com/azat)). +* Fixes for killing leftovers in clikhouse-test. [#70474](https://github.com/ClickHouse/ClickHouse/pull/70474) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelog after v24.3.12.75-lts. [#70485](https://github.com/ClickHouse/ClickHouse/pull/70485) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Use logging instead of print. [#70505](https://github.com/ClickHouse/ClickHouse/pull/70505) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Remove slow poll() logs in keeper. [#70508](https://github.com/ClickHouse/ClickHouse/pull/70508) ([Raúl Marín](https://github.com/Algunenano)). +* Add timeouts for retry loops in test_storage_rabbitmq. It should prevent cascading failures of the whole test suite caused by deadloop in one of the test scenarios. Also added small sleeps in a 'tight' loops to make retries bit less agressive. [#70510](https://github.com/ClickHouse/ClickHouse/pull/70510) ([filimonov](https://github.com/filimonov)). +* CI: Fix for canceled Sync workflow. [#70521](https://github.com/ClickHouse/ClickHouse/pull/70521) ([Max Kainov](https://github.com/maxknv)). +* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/70412, don't know why it's ok in release build, simply changing `_` to `_1` is ok for both release and debug build. [#70532](https://github.com/ClickHouse/ClickHouse/pull/70532) ([Chang chen](https://github.com/baibaichen)). +* Refreshable materialized views are not experimental anymore. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix 24.9 setting compatibility `database_replicated_allow_explicit_uuid`. [#70565](https://github.com/ClickHouse/ClickHouse/pull/70565) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix typos. [#70588](https://github.com/ClickHouse/ClickHouse/pull/70588) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Vector search: allow to specify HNSW parameter `ef_search` at query time. [#70616](https://github.com/ClickHouse/ClickHouse/pull/70616) ([Robert Schulze](https://github.com/rschu1ze)). +* Increase max_rows_to_read limit in some tests. [#70617](https://github.com/ClickHouse/ClickHouse/pull/70617) ([Raúl Marín](https://github.com/Algunenano)). +* Reduce sync efforts with private. [#70634](https://github.com/ClickHouse/ClickHouse/pull/70634) ([Raúl Marín](https://github.com/Algunenano)). +* Fix parsing of some formats into sparse columns. [#70635](https://github.com/ClickHouse/ClickHouse/pull/70635) ([Anton Popov](https://github.com/CurtizJ)). +* Fix typos. [#70637](https://github.com/ClickHouse/ClickHouse/pull/70637) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Try fix 00180_no_seek_avoiding_when_reading_from_cache. [#70640](https://github.com/ClickHouse/ClickHouse/pull/70640) ([Kseniia Sumarokova](https://github.com/kssenii)). +* When the `PR Check` status is set, it's a valid RunConfig job failure. [#70643](https://github.com/ClickHouse/ClickHouse/pull/70643) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix timeout in materialized pg tests. [#70646](https://github.com/ClickHouse/ClickHouse/pull/70646) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Introduced MergeTree setting which allow to change merge selecting algorithm. However we still have only one algorithm and it's mostly for future experiments. [#70647](https://github.com/ClickHouse/ClickHouse/pull/70647) ([alesapin](https://github.com/alesapin)). +* Docs: Follow-up for [#70585](https://github.com/ClickHouse/ClickHouse/issues/70585). [#70654](https://github.com/ClickHouse/ClickHouse/pull/70654) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove strange file. [#70662](https://github.com/ClickHouse/ClickHouse/pull/70662) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Locally I had lots of errors like `'AllocList' does not refer to a value` around places which used `offsetof`. Changing it to `__builtin_offsetof ` helped and I didn't debug any further. [#70671](https://github.com/ClickHouse/ClickHouse/pull/70671) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Adding the report link to a test result and files' list. [#70677](https://github.com/ClickHouse/ClickHouse/pull/70677) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* materialized postgres: minor fixes. [#70710](https://github.com/ClickHouse/ClickHouse/pull/70710) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Probably fix flaky test_refreshable_mv_in_replicated_db. [#70714](https://github.com/ClickHouse/ClickHouse/pull/70714) ([Michael Kolupaev](https://github.com/al13n321)). +* Move more setting structs to pImpl. [#70739](https://github.com/ClickHouse/ClickHouse/pull/70739) ([Raúl Marín](https://github.com/Algunenano)). +* Reduce sync effort. [#70747](https://github.com/ClickHouse/ClickHouse/pull/70747) ([Raúl Marín](https://github.com/Algunenano)). +* Add s3queue settings check for cloud. [#70750](https://github.com/ClickHouse/ClickHouse/pull/70750) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix readiness/health check for OpenLDAP container. [#70755](https://github.com/ClickHouse/ClickHouse/pull/70755) ([Julian Maicher](https://github.com/jmaicher)). +* Allow update plan headers for all the steps. [#70761](https://github.com/ClickHouse/ClickHouse/pull/70761) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Autogenerate documentation for settings. [#70768](https://github.com/ClickHouse/ClickHouse/pull/70768) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Not a logical error. [#70770](https://github.com/ClickHouse/ClickHouse/pull/70770) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Aarch64 build with Asan. [#70778](https://github.com/ClickHouse/ClickHouse/pull/70778) ([Max Kainov](https://github.com/maxknv)). +* Minor fix. [#70783](https://github.com/ClickHouse/ClickHouse/pull/70783) ([Anton Popov](https://github.com/CurtizJ)). +* The docs for settings should be located in the source code. Now, the CI supports that. [#70784](https://github.com/ClickHouse/ClickHouse/pull/70784) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update style-test image. [#70785](https://github.com/ClickHouse/ClickHouse/pull/70785) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid double finalization of `WriteBuffer` in library bridge. [#70799](https://github.com/ClickHouse/ClickHouse/pull/70799) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Make Array Field serialization consistent. [#70803](https://github.com/ClickHouse/ClickHouse/pull/70803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* A follow-up for [#70785](https://github.com/ClickHouse/ClickHouse/issues/70785), [jwt](https://pypi.org/project/jwt/#history) looks very outdated, and we have issue with conflicting paths. [#70815](https://github.com/ClickHouse/ClickHouse/pull/70815) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove inneficient code. [#70816](https://github.com/ClickHouse/ClickHouse/pull/70816) ([Raúl Marín](https://github.com/Algunenano)). +* Allow large object files if OMIT_HEAVY_DEBUG_SYMBOLS = 0. [#70818](https://github.com/ClickHouse/ClickHouse/pull/70818) ([Michael Kolupaev](https://github.com/al13n321)). +* Add test with distributed queries for 15768. [#70834](https://github.com/ClickHouse/ClickHouse/pull/70834) ([Nikita Taranov](https://github.com/nickitat)). +* More setting structs to pImpl and reuse code. [#70840](https://github.com/ClickHouse/ClickHouse/pull/70840) ([Raúl Marín](https://github.com/Algunenano)). +* Update default HNSW parameter settings. [#70873](https://github.com/ClickHouse/ClickHouse/pull/70873) ([Robert Schulze](https://github.com/rschu1ze)). +* Limiting logging some lines about configs. [#70879](https://github.com/ClickHouse/ClickHouse/pull/70879) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix `limit by`, `limit with ties` for distributed and parallel replicas. [#70880](https://github.com/ClickHouse/ClickHouse/pull/70880) ([Nikita Taranov](https://github.com/nickitat)). +* Fix darwin build. [#70894](https://github.com/ClickHouse/ClickHouse/pull/70894) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add dots for consistency. [#70909](https://github.com/ClickHouse/ClickHouse/pull/70909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Logical error fix for substrings, found by fuzzer. [#70914](https://github.com/ClickHouse/ClickHouse/pull/70914) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* More setting structs to pImpl. [#70942](https://github.com/ClickHouse/ClickHouse/pull/70942) ([Raúl Marín](https://github.com/Algunenano)). +* Add logging for mock HTTP servers used in minio integration tests. [#70943](https://github.com/ClickHouse/ClickHouse/pull/70943) ([Vitaly Baranov](https://github.com/vitlibar)). +* Minor fixups of [#70011](https://github.com/ClickHouse/ClickHouse/issues/70011) and [#69918](https://github.com/ClickHouse/ClickHouse/issues/69918). [#70959](https://github.com/ClickHouse/ClickHouse/pull/70959) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Do not skip Build report and status fix. [#70965](https://github.com/ClickHouse/ClickHouse/pull/70965) ([Max Kainov](https://github.com/maxknv)). +* Fix Keeper entry serialization compatibility. [#70972](https://github.com/ClickHouse/ClickHouse/pull/70972) ([Antonio Andelic](https://github.com/antonio2368)). +* Update exception message. [#70975](https://github.com/ClickHouse/ClickHouse/pull/70975) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix `utils/c++expr` option `-b`. [#70978](https://github.com/ClickHouse/ClickHouse/pull/70978) ([Sergei Trifonov](https://github.com/serxa)). +* Fix `test_keeper_broken_logs`. [#70982](https://github.com/ClickHouse/ClickHouse/pull/70982) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix `01039_test_setting_parse`. [#70986](https://github.com/ClickHouse/ClickHouse/pull/70986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tests for languages support for Embedded Dictionaries. [#71004](https://github.com/ClickHouse/ClickHouse/pull/71004) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Required for internal test runs with the same image build in public CI. [#71008](https://github.com/ClickHouse/ClickHouse/pull/71008) ([Ilya Yatsishin](https://github.com/qoega)). +* Move remaining settings objects to pImpl and start simplification. [#71019](https://github.com/ClickHouse/ClickHouse/pull/71019) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Rearrange directories for praktika ci. [#71029](https://github.com/ClickHouse/ClickHouse/pull/71029) ([Max Kainov](https://github.com/maxknv)). +* Fix assert in RemoteSource::onAsyncJobReady(). [#71034](https://github.com/ClickHouse/ClickHouse/pull/71034) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix showing error message in ReadBufferFromS3 when retrying. Without this PR information about a retryable failure in `ReadBufferFromS3` could look like this:. [#71038](https://github.com/ClickHouse/ClickHouse/pull/71038) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `test_truncate_database`. [#71057](https://github.com/ClickHouse/ClickHouse/pull/71057) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix clickhouse-test useless 5 second delay in case of multiple threads are used. [#71069](https://github.com/ClickHouse/ClickHouse/pull/71069) ([Azat Khuzhin](https://github.com/azat)). + +#### Not for changeling + +* Reverted. [#69812](https://github.com/ClickHouse/ClickHouse/pull/69812) ([tuanpach](https://github.com/tuanpach)). + + ### ClickHouse release 24.9, 2024-09-26 #### Backward Incompatible Change From c583cbca0055812a5824653dde869472b4d775a5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 11:02:12 +0000 Subject: [PATCH 491/816] Remove wrong LOGICAL_ERROR --- src/Interpreters/QueryMetricLog.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 33eeac592f4..4b8d56d7f32 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -184,13 +184,6 @@ std::optional QueryMetricLog::createLogMetricElement(cons { const auto & new_value = (*(query_info.profile_counters))[i]; auto & prev_value = query_status.last_profile_events[i]; - - /// Profile event count is monotonically increasing. - if (new_value < prev_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", - ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); - elem.profile_events[i] = new_value - prev_value; prev_value = new_value; } From 25f5979ca3db87da83badc718cc01fb72ece9c49 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 11:15:10 +0000 Subject: [PATCH 492/816] Fix another source of race condition At first I thought of doing the bare minimum when `queries_mutex` was locked. However, scheduling the task does not lock `exec_mutex` but other ones. So, there is no deadlock in sight of scheduling within `queries_mutex`. This way, we prevent a rare case in which the task is scheduled before the query is even added to the queries list. --- src/Interpreters/QueryMetricLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 4b8d56d7f32..b978466ac48 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -115,9 +115,8 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t LOG_TRACE(logger, "Query {} finished already while this collecting task was running", query_id); }); - status.task->scheduleAfter(interval_milliseconds); - std::lock_guard lock(queries_mutex); + status.task->scheduleAfter(interval_milliseconds); queries.emplace(query_id, std::move(status)); } From b6ff82959f2ba2bc0d543dee6937075bdd7414d5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 11:24:24 +0000 Subject: [PATCH 493/816] fixup! Remove wrong LOGICAL_ERROR --- src/Interpreters/QueryMetricLog.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index b978466ac48..3b983e61dda 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -21,11 +21,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -}; - static auto logger = getLogger("QueryMetricLog"); ColumnsDescription QueryMetricLogElement::getColumnsDescription() From 0c5a5a0b0729c8cef88c98eda1f37cb263be56ec Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 28 Oct 2024 12:25:08 +0100 Subject: [PATCH 494/816] Slightly update JSON docs for better search --- docs/en/sql-reference/data-types/newjson.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 68952590eb9..7e6d4dd934f 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -5,7 +5,7 @@ sidebar_label: JSON keywords: [json, data type] --- -# JSON +# JSON Data Type Stores JavaScript Object Notation (JSON) documents in a single column. From 36d25d3cf35eec10e11c4aaf1a7248b2a09946b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 12:26:26 +0100 Subject: [PATCH 495/816] First round of cleanup --- CHANGELOG.md | 261 ++++----------------------------------------------- 1 file changed, 19 insertions(+), 242 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2703d5b6ee0..412130f58be 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,7 @@ #### Backward Incompatible Change * Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#68614](https://github.com/ClickHouse/ClickHouse/pull/68614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). -* Allow empty needle in functions replaceRegexp*, like https://github.com/ClickHouse/ClickHouse/pull/69918. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). +* Allow empty needle in functions replaceRegexp*. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). * Reordering of filter conditions from `[PRE]WHERE` clause is now allowed by default. It could be disabled by setting `allow_reorder_prewhere_conditions` to `false`. [#70657](https://github.com/ClickHouse/ClickHouse/pull/70657) ([Nikita Taranov](https://github.com/nickitat)). * Fix `optimize_functions_to_subcolumns` optimization (previously could lead to `Invalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)` error), by preserving `LowCardinality` type in `mapKeys`/`mapValues`. [#70716](https://github.com/ClickHouse/ClickHouse/pull/70716) ([Azat Khuzhin](https://github.com/azat)). * Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -31,16 +31,21 @@ * A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support --copy mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). * Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). -* Support aggreate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. BTW, it is for spark compatiability in Apache Gluten. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). +* Support aggregate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. This is for spark compatibility. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). * Support function arrayElementOrNull. It returns null if array index is out of range or map key not found. [#69646](https://github.com/ClickHouse/ClickHouse/pull/69646) ([李扬](https://github.com/taiyang-li)). * Allows users to specify regular expressions through new `message_regexp` and `message_regexp_negative` fields in the `config.xml` file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. [#69657](https://github.com/ClickHouse/ClickHouse/pull/69657) ([Peter Nguyen](https://github.com/petern48)). -* Support Dynamic type in most functions by executing them on internal types inside Dynamic. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). * Re-added `RIPEMD160` function, which computes the RIPEMD-160 cryptographic hash of a string. Example: `SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))` returns `37F332F68DB77BD9D7EDD4969571AD671CF9DD3B`. [#70087](https://github.com/ClickHouse/ClickHouse/pull/70087) ([Dergousov Maxim](https://github.com/m7kss1)). * Allow to cache read files for object storage table engines and data lakes using hash from ETag + file path as cache key. [#70135](https://github.com/ClickHouse/ClickHouse/pull/70135) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support reading Iceberg tables on HDFS. [#70268](https://github.com/ClickHouse/ClickHouse/pull/70268) ([flynn](https://github.com/ucasfl)). +* Supports standard CTE, `with insert`, as previously only supports `insert ... with ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). + +#### Experimental feature +* Refreshable materialized views are not experimental anymore. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). +* Support Dynamic type in most functions by executing them on internal types inside Dynamic. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). * Allow to read/write JSON type as binary string in RowBinary format under settings `input_format_binary_read_json_as_string/output_format_binary_write_json_as_string`. [#70288](https://github.com/ClickHouse/ClickHouse/pull/70288) ([Pavel Kruglov](https://github.com/Avogar)). * Allow to serialize/deserialize JSON column as single String column in Native format. For output use setting `output_format_native_write_json_as_string`. For input, use serialization version `1` before the column data. [#70312](https://github.com/ClickHouse/ClickHouse/pull/70312) ([Pavel Kruglov](https://github.com/Avogar)). -* Supports standard CTE, `with insert`, as previously only supports `insert ... with ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). +* Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the `merge_selector_use_blurry_base` MergeTree-level setting. [#70645](https://github.com/ClickHouse/ClickHouse/pull/70645) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). #### Performance Improvement * Support minmax index for `pointInPolygon`. [#62085](https://github.com/ClickHouse/ClickHouse/pull/62085) ([JackyWoo](https://github.com/JackyWoo)). @@ -58,23 +63,19 @@ #### Improvement * Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). -* Fixed [#57616](https://github.com/ClickHouse/ClickHouse/issues/57616) this problem occurs because all positive number arguments are automatically identified as `uint64` type, leading to an inability to match int type data in `summapfiltered`. the issue of non-matching is indeed confusing, as the `uint64` parameters are not specified by the user. additionally, if the arguments are `[1,2,3,toint8(-3)]`, due to the `getleastsupertype()`, these parameters will be uniformly treated as `int` type, causing `'1,2,3'` to also fail in matching the `uint` type data in `summapfiltered`. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). * `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). * Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). * Symbolic links for tables in the `data/database_name/` directory are created for the actual paths to the table's data, depending on the storage policy, instead of the `store/...` directory on the default disk. [#61777](https://github.com/ClickHouse/ClickHouse/pull/61777) ([Kirill](https://github.com/kirillgarbar)). -* Apply configuration updates in global context object. It fixes issues like [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308). [#62944](https://github.com/ClickHouse/ClickHouse/pull/62944) ([Amos Bird](https://github.com/amosbird)). -* Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix `ReadSettings` not using user set values, because defaults were only used. [#65625](https://github.com/ClickHouse/ClickHouse/pull/65625) ([Kseniia Sumarokova](https://github.com/kssenii)). * While parsing an Enum field from JSON, a string containing an integer will be interpreted as the corresponding Enum element. This closes [#65119](https://github.com/ClickHouse/ClickHouse/issues/65119). [#66801](https://github.com/ClickHouse/ClickHouse/pull/66801) ([scanhex12](https://github.com/scanhex12)). * Allow `TRIM` -ing `LEADING` or `TRAILING` empty string as a no-op. Closes [#67792](https://github.com/ClickHouse/ClickHouse/issues/67792). [#68455](https://github.com/ClickHouse/ClickHouse/pull/68455) ([Peter Nguyen](https://github.com/petern48)). * Support creating a table with a query: `CREATE TABLE ... CLONE AS ...`. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of the `MergeTree` family Closes [#65015](https://github.com/ClickHouse/ClickHouse/issues/65015). [#69091](https://github.com/ClickHouse/ClickHouse/pull/69091) ([tuanpach](https://github.com/tuanpach)). -* In Gluten ClickHouse, Spark's timestamp type is mapped to ClickHouse's datetime64(6) type. When casting timestamp '2012-01-01 00:11:22' as a string, Spark returns '2012-01-01 00:11:22', while Gluten ClickHouse returns '2012-01-01 00:11:22.000000'. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). +* Improve compatibility of cast(timestamp as string) with spark. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). * Always use the new analyzer to calculate constant expressions when `enable_analyzer` is set to `true`. Support calculation of `executable()` table function arguments without using `SELECT` query for constant expression. [#69292](https://github.com/ClickHouse/ClickHouse/pull/69292) ([Dmitry Novik](https://github.com/novikd)). * Add `enable_secure_identifiers` to disallow insecure identifiers. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). * Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior of the show create query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}`, or it can cause ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/69346 Point 4 described there will work now as well:. [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). +* Improve restoring of access entities' dependencies [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). * Implement generic SerDe between Avro Union and ClickHouse Variant type. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). -* 1. CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses. Now it is supported only for the MergeTree family of table engines. 2. For example, the follow SQL statements will trigger exception in the past, but this PR fixes it: if the destination table do not provide an `ORDER BY` or `PRIMARY KEY` expression in the table definition, we will copy that from source table. [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). +* CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses (MergeTree tables). [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). * Added user-level settings `min_free_disk_bytes_to_throw_insert` and `min_free_disk_ratio_to_throw_insert` to prevent insertions on disks that are almost full. [#69755](https://github.com/ClickHouse/ClickHouse/pull/69755) ([Marco Vilas Boas](https://github.com/marco-vb)). * If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add new column readonly_duration to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). @@ -94,10 +95,9 @@ * Added a new setting `max_parts_to_move` to control the maximum number of parts that can be moved at once. [#70520](https://github.com/ClickHouse/ClickHouse/pull/70520) ([Vladimir Cherkasov](https://github.com/vdimir)). * Limit the frequency of certain log messages. [#70601](https://github.com/ClickHouse/ClickHouse/pull/70601) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Don't do validation when synchronizing user_directories from keeper. [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). -* Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the `merge_selector_use_blurry_base` MergeTree-level setting. [#70645](https://github.com/ClickHouse/ClickHouse/pull/70645) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * `CHECK TABLE` with `PART` qualifier was incorrectly formatted in the client. [#70660](https://github.com/ClickHouse/ClickHouse/pull/70660) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support write column index and offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). -* Support parse `DateTime64` for microseond and timezone in joda syntax. [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support parse `DateTime64` for microsecond and timezone in joda syntax. [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). * Changed an approach to figure out if a cloud storage supports [batch delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) or not. [#70786](https://github.com/ClickHouse/ClickHouse/pull/70786) ([Vitaly Baranov](https://github.com/vitlibar)). * Support for Parquet page V2 on native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). * Add an HTML page for visualizing merges. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -111,8 +111,12 @@ * Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). * Background downloads to filesystem cache was enabled back. [#70929](https://github.com/ClickHouse/ClickHouse/pull/70929) ([Nikita Taranov](https://github.com/nickitat)). * Add a new merge selector algorithm, named `Trivial`, for professional usage only. It is worse than the `Simple` merge selector. [#70969](https://github.com/ClickHouse/ClickHouse/pull/70969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support CREATE OR REPLACE VIEW atomically. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([tuanpach](https://github.com/tuanpach)) #### Bug Fix (user-visible misbehavior in an official stable release) +* Apply configuration updates in global context object. It fixes issues like [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308). [#62944](https://github.com/ClickHouse/ClickHouse/pull/62944) ([Amos Bird](https://github.com/amosbird)). +* Fix `ReadSettings` not using user set values, because defaults were only used. [#65625](https://github.com/ClickHouse/ClickHouse/pull/65625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix type mismatch issue in sumMapFiltered when using signed arguments. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). * Fix toHour-like conversion functions' monotonicity when optional time zone argument is passed. [#60264](https://github.com/ClickHouse/ClickHouse/pull/60264) ([Amos Bird](https://github.com/amosbird)). * Relax `supportsPrewhere` check for StorageMerge. This fixes [#61064](https://github.com/ClickHouse/ClickHouse/issues/61064). It was hardened unnecessarily in [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#61091](https://github.com/ClickHouse/ClickHouse/pull/61091) ([Amos Bird](https://github.com/amosbird)). * Fix `use_concurrency_control` setting handling for proper `concurrent_threads_soft_limit_num` limit enforcing. This enables concurrency control by default because previously it was broken. [#61473](https://github.com/ClickHouse/ClickHouse/pull/61473) ([Sergei Trifonov](https://github.com/serxa)). @@ -123,7 +127,7 @@ * Fix inf loop after `restore replica` in the replicated merge tree with zero copy. [#69293](https://github.com/ClickHouse/ClickHouse/pull/69293) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Return back default value of `processing_threads_num` as number of cpu cores in storage `S3Queue`. [#69384](https://github.com/ClickHouse/ClickHouse/pull/69384) ([Kseniia Sumarokova](https://github.com/kssenii)). * Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns ( fixes [#41971](https://github.com/ClickHouse/ClickHouse/issues/41971) ). [#69556](https://github.com/ClickHouse/ClickHouse/pull/69556) ([Eliot Hautefeuille](https://github.com/hileef)). -* Fix vrash during insertion into FixedString column in PostgreSQL engine. [#69584](https://github.com/ClickHouse/ClickHouse/pull/69584) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix crash during insertion into FixedString column in PostgreSQL engine. [#69584](https://github.com/ClickHouse/ClickHouse/pull/69584) ([Pavel Kruglov](https://github.com/Avogar)). * Fix crash when executing `create view t as (with recursive 42 as ttt select ttt);`. [#69676](https://github.com/ClickHouse/ClickHouse/pull/69676) ([Han Fei](https://github.com/hanfei1991)). * Added `strict_once` mode to aggregate function `windowFunnel` to avoid counting one event several times in case it matches multiple conditions, close [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835). [#69738](https://github.com/ClickHouse/ClickHouse/pull/69738) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fixed `maxMapState` throwing 'Bad get' if value type is DateTime64. [#69787](https://github.com/ClickHouse/ClickHouse/pull/69787) ([Michael Kolupaev](https://github.com/al13n321)). @@ -134,7 +138,7 @@ * Don't check dependencies during CREATE OR REPLACE VIEW during DROP of old table. Previously CREATE OR REPLACE query failed when there are dependent tables of the recreated view. [#69907](https://github.com/ClickHouse/ClickHouse/pull/69907) ([Pavel Kruglov](https://github.com/Avogar)). * Implement missing decimal cases for `zeroField`. Fixes [#69730](https://github.com/ClickHouse/ClickHouse/issues/69730). [#69978](https://github.com/ClickHouse/ClickHouse/pull/69978) ([Arthur Passos](https://github.com/arthurpassos)). * Now SQL security will work with parameterized views correctly. [#69984](https://github.com/ClickHouse/ClickHouse/pull/69984) ([pufit](https://github.com/pufit)). -* Closes [#69752](https://github.com/ClickHouse/ClickHouse/issues/69752). [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). +* Fix parsing for definers. [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). * Fixed a bug when the timezone could change the result of the query with a `Date` or `Date32` arguments. [#70036](https://github.com/ClickHouse/ClickHouse/pull/70036) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Fixes `Block structure mismatch` for queries with nested views and `WHERE` condition. Fixes [#66209](https://github.com/ClickHouse/ClickHouse/issues/66209). [#70054](https://github.com/ClickHouse/ClickHouse/pull/70054) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Avoid reusing columns among different named tuples when evaluating `tuple` functions. This fixes [#70022](https://github.com/ClickHouse/ClickHouse/issues/70022). [#70103](https://github.com/ClickHouse/ClickHouse/pull/70103) ([Amos Bird](https://github.com/amosbird)). @@ -170,234 +174,7 @@ * Fix S3Queue table engine setting processing_threads_num not being effective in case it was deduced from the number of cpu cores on the server. [#70837](https://github.com/ClickHouse/ClickHouse/pull/70837) ([Kseniia Sumarokova](https://github.com/kssenii)). * Normalize named tuple arguments in aggregation states. This fixes [#69732](https://github.com/ClickHouse/ClickHouse/issues/69732) . [#70853](https://github.com/ClickHouse/ClickHouse/pull/70853) ([Amos Bird](https://github.com/amosbird)). * Fix a logical error due to negative zeros in the two-level hash table. This closes [#70973](https://github.com/ClickHouse/ClickHouse/issues/70973). [#70979](https://github.com/ClickHouse/ClickHouse/pull/70979) ([Alexey Milovidov](https://github.com/alexey-milovidov)). - -#### Build/Testing/Packaging Improvement -* Docker in integration tests runner is updated to latest version. It was previously pinned u until patch release 24.0.3 was out. https://github.com/moby/moby/issues/45770#issuecomment-1618255130. - HDFS image was deprecated and not running with current docker version. Switched to newer version of a derivative image based on ubuntu. - HDFS tests were hardened to allow them to run with python-repeat. [#66867](https://github.com/ClickHouse/ClickHouse/pull/66867) ([Ilya Yatsishin](https://github.com/qoega)). -* Alpine docker images now use ubuntu 22.04 as glibc donor, results in upgrade of glibc version delivered with alpine images from 2.31 to 2.35. [#69033](https://github.com/ClickHouse/ClickHouse/pull/69033) ([filimonov](https://github.com/filimonov)). -* Makes dbms independent from clickhouse_functions. [#69914](https://github.com/ClickHouse/ClickHouse/pull/69914) ([Raúl Marín](https://github.com/Algunenano)). -* Fix FreeBSD compilation of the MariaDB connector. [#70007](https://github.com/ClickHouse/ClickHouse/pull/70007) ([Raúl Marín](https://github.com/Algunenano)). -* Building on Apple Mac OS X Darwin does not produce strange warnings anymore. [#70411](https://github.com/ClickHouse/ClickHouse/pull/70411) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix building with ARCH_NATIVE CMake flag. [#70585](https://github.com/ClickHouse/ClickHouse/pull/70585) ([Daniil Gentili](https://github.com/danog)). -* The universal installer will download Musl build on Alpine Linux. Some Docker containers are using Alpine Linux, but it was not possible to install ClickHouse there with `curl https://clickhouse.com/ | sh`. [#70767](https://github.com/ClickHouse/ClickHouse/pull/70767) ([Alexey Milovidov](https://github.com/alexey-milovidov)). - -#### NO CL ENTRY - -* NO CL ENTRY: 'Revert "JSONCompactWithProgress query output format"'. [#69989](https://github.com/ClickHouse/ClickHouse/pull/69989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Support CREATE OR REPLACE VIEW atomically"'. [#70535](https://github.com/ClickHouse/ClickHouse/pull/70535) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Revert "Support CREATE OR REPLACE VIEW atomically""'. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Add projections size to system.projections"'. [#70858](https://github.com/ClickHouse/ClickHouse/pull/70858) ([Alexey Milovidov](https://github.com/alexey-milovidov)). - -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Allow writing argument of `has` or `hasAny` or `hasAll` as string values if array element type is `Enum`. [#56555](https://github.com/ClickHouse/ClickHouse/pull/56555) ([Duc Canh Le](https://github.com/canhld94)). -* Rename FileSegmentKind::Ephemeral and other changes. [#66600](https://github.com/ClickHouse/ClickHouse/pull/66600) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Closes [#67345](https://github.com/ClickHouse/ClickHouse/issues/67345). [#67346](https://github.com/ClickHouse/ClickHouse/pull/67346) ([KrJin](https://github.com/jincong8973)). -* Because it is too complicated to support. [#68410](https://github.com/ClickHouse/ClickHouse/pull/68410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix 01600_parts_states_metrics_long flakiness. [#68521](https://github.com/ClickHouse/ClickHouse/pull/68521) ([Azat Khuzhin](https://github.com/azat)). -* Reduce client start time in debug/sanitizer mode. [#68980](https://github.com/ClickHouse/ClickHouse/pull/68980) ([Raúl Marín](https://github.com/Algunenano)). -* Closes [#69038](https://github.com/ClickHouse/ClickHouse/issues/69038). [#69040](https://github.com/ClickHouse/ClickHouse/pull/69040) ([Nikolay Degterinsky](https://github.com/evillique)). -* Better exception for unsupported full_text index with non-full parts. [#69067](https://github.com/ClickHouse/ClickHouse/pull/69067) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Catch additional zk connection erros while creating table and make sure to cleanup dirs if necessary for retries. [#69093](https://github.com/ClickHouse/ClickHouse/pull/69093) ([Sumit](https://github.com/sum12)). -* Update version_date.tsv and changelog after v24.7.5.37-stable. [#69185](https://github.com/ClickHouse/ClickHouse/pull/69185) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* DOCS: Replace live view with refreshable since the former is deprecated. [#69392](https://github.com/ClickHouse/ClickHouse/pull/69392) ([Damian Kula](https://github.com/heavelock)). -* Update ORC to the current HEAD. [#69473](https://github.com/ClickHouse/ClickHouse/pull/69473) ([Nikita Taranov](https://github.com/nickitat)). -* Make a test ready for flaky check. [#69586](https://github.com/ClickHouse/ClickHouse/pull/69586) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Support antlr parser to parse sql with some keywords as alias, make the behaviour same as the clickhouse-server - remove redundant `for` in the `keyword` field. [#69614](https://github.com/ClickHouse/ClickHouse/pull/69614) ([Z.H.](https://github.com/onlyacat)). -* Allow default implementations for null in function mapFromArrays for spark compatiability in apache gluten. Current change doesn't have any side effects on clickhouse in theory. [#69715](https://github.com/ClickHouse/ClickHouse/pull/69715) ([李扬](https://github.com/taiyang-li)). -* Fix exception message in AzureBlobStorage. [#69728](https://github.com/ClickHouse/ClickHouse/pull/69728) ([Pavel Kruglov](https://github.com/Avogar)). -* Add test parsing s3 URL with a bucket name including a dot. [#69743](https://github.com/ClickHouse/ClickHouse/pull/69743) ([Kaushik Iska](https://github.com/iskakaushik)). -* Make `clang-tidy` happy. [#69765](https://github.com/ClickHouse/ClickHouse/pull/69765) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Prepare to enable `clang-tidy` `readability-else-after-return`. [#69768](https://github.com/ClickHouse/ClickHouse/pull/69768) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* S3Queue: support having deprecated settings to not fail server startup. [#69769](https://github.com/ClickHouse/ClickHouse/pull/69769) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Use only adaptive heuristic to choose task sizes for remote reading. [#69778](https://github.com/ClickHouse/ClickHouse/pull/69778) ([Nikita Taranov](https://github.com/nickitat)). -* Remove unused buggy code. [#69780](https://github.com/ClickHouse/ClickHouse/pull/69780) ([Raúl Marín](https://github.com/Algunenano)). -* Fix bugfix check. [#69789](https://github.com/ClickHouse/ClickHouse/pull/69789) ([Antonio Andelic](https://github.com/antonio2368)). -* Followup for [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69790](https://github.com/ClickHouse/ClickHouse/pull/69790) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Update version after release. [#69816](https://github.com/ClickHouse/ClickHouse/pull/69816) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update ext-dict-functions.md. [#69819](https://github.com/ClickHouse/ClickHouse/pull/69819) ([kurikuQwQ](https://github.com/kurikuQwQ)). -* Allow cyrillic characters in generated contributor names. [#69820](https://github.com/ClickHouse/ClickHouse/pull/69820) ([Raúl Marín](https://github.com/Algunenano)). -* CI: praktika integration 1. [#69822](https://github.com/ClickHouse/ClickHouse/pull/69822) ([Max Kainov](https://github.com/maxknv)). -* Fix `test_delayed_replica_failover`. [#69826](https://github.com/ClickHouse/ClickHouse/pull/69826) ([Antonio Andelic](https://github.com/antonio2368)). -* minor change, less conflicts. [#69830](https://github.com/ClickHouse/ClickHouse/pull/69830) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Improve error message DDLWorker.cpp. [#69835](https://github.com/ClickHouse/ClickHouse/pull/69835) ([Denny Crane](https://github.com/den-crane)). -* Fix typo in description: mutation_sync -> mutations_sync. [#69838](https://github.com/ClickHouse/ClickHouse/pull/69838) ([Alexander Gololobov](https://github.com/davenger)). -* Fix changelog. [#69841](https://github.com/ClickHouse/ClickHouse/pull/69841) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* This closes [#49940](https://github.com/ClickHouse/ClickHouse/issues/49940). [#69842](https://github.com/ClickHouse/ClickHouse/pull/69842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* This closes [#51036](https://github.com/ClickHouse/ClickHouse/issues/51036). [#69844](https://github.com/ClickHouse/ClickHouse/pull/69844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update README.md - Update meetups. [#69849](https://github.com/ClickHouse/ClickHouse/pull/69849) ([Tanya Bragin](https://github.com/tbragin)). -* Revert [#69790](https://github.com/ClickHouse/ClickHouse/issues/69790) and [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69850](https://github.com/ClickHouse/ClickHouse/pull/69850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* See [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69851](https://github.com/ClickHouse/ClickHouse/pull/69851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add a test for [#50928](https://github.com/ClickHouse/ClickHouse/issues/50928). [#69852](https://github.com/ClickHouse/ClickHouse/pull/69852) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add a test for [#55981](https://github.com/ClickHouse/ClickHouse/issues/55981). [#69853](https://github.com/ClickHouse/ClickHouse/pull/69853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add a test for [#56823](https://github.com/ClickHouse/ClickHouse/issues/56823). [#69854](https://github.com/ClickHouse/ClickHouse/pull/69854) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* This closes [#62350](https://github.com/ClickHouse/ClickHouse/issues/62350). [#69855](https://github.com/ClickHouse/ClickHouse/pull/69855) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Refactor functions and variables in statistics code. [#69860](https://github.com/ClickHouse/ClickHouse/pull/69860) ([Robert Schulze](https://github.com/rschu1ze)). -* Resubmit [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69861](https://github.com/ClickHouse/ClickHouse/pull/69861) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Improve stateless test runner. [#69864](https://github.com/ClickHouse/ClickHouse/pull/69864) ([Alexey Katsman](https://github.com/alexkats)). -* Adjust fast test time limit a bit. [#69874](https://github.com/ClickHouse/ClickHouse/pull/69874) ([Raúl Marín](https://github.com/Algunenano)). -* Add initial 24.9 CHANGELOG. [#69876](https://github.com/ClickHouse/ClickHouse/pull/69876) ([Raúl Marín](https://github.com/Algunenano)). -* Fix test `01278_random_string_utf8`. [#69878](https://github.com/ClickHouse/ClickHouse/pull/69878) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix minor fuzzer issue with experimental statistics. [#69881](https://github.com/ClickHouse/ClickHouse/pull/69881) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix linking after settings refactoring. [#69882](https://github.com/ClickHouse/ClickHouse/pull/69882) ([Robert Schulze](https://github.com/rschu1ze)). -* Add Proj Obsolete Setting. [#69883](https://github.com/ClickHouse/ClickHouse/pull/69883) ([Shichao Jin](https://github.com/jsc0218)). -* Improve remote queries startup time. [#69884](https://github.com/ClickHouse/ClickHouse/pull/69884) ([Igor Nikonov](https://github.com/devcrafter)). -* Revert "Merge pull request [#69032](https://github.com/ClickHouse/ClickHouse/issues/69032) from alexon1234/include_real_time_execution_in_http_header". [#69885](https://github.com/ClickHouse/ClickHouse/pull/69885) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* A dedicated commits from https://github.com/ClickHouse/ClickHouse/pull/61473. [#69896](https://github.com/ClickHouse/ClickHouse/pull/69896) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Added aliases `time_bucket`(from TimescaleDB) and `date_bin`(from PostgreSQL) for `toStartOfInterval`. [#69900](https://github.com/ClickHouse/ClickHouse/pull/69900) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* RIPE is an acronym and thus should be capital. RIPE stands for **R**ACE **I**ntegrity **P**rimitives **E**valuation and RACE stands for **R**esearch and Development in **A**dvanced **C**ommunications **T**echnologies in **E**urope. [#69901](https://github.com/ClickHouse/ClickHouse/pull/69901) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Replace error codes with error names in stateless tests. [#69906](https://github.com/ClickHouse/ClickHouse/pull/69906) ([Dmitry Novik](https://github.com/novikd)). -* Move setting to 24.10. [#69913](https://github.com/ClickHouse/ClickHouse/pull/69913) ([Raúl Marín](https://github.com/Algunenano)). -* Minor: Reduce diff between public and private repo. [#69928](https://github.com/ClickHouse/ClickHouse/pull/69928) ([Robert Schulze](https://github.com/rschu1ze)). -* Followup for [#69861](https://github.com/ClickHouse/ClickHouse/issues/69861). [#69930](https://github.com/ClickHouse/ClickHouse/pull/69930) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix test_dictionaries_all_layouts_separate_sources. [#69962](https://github.com/ClickHouse/ClickHouse/pull/69962) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix test_keeper_mntr_data_size. [#69965](https://github.com/ClickHouse/ClickHouse/pull/69965) ([Antonio Andelic](https://github.com/antonio2368)). -* This closes [#49823](https://github.com/ClickHouse/ClickHouse/issues/49823). [#69981](https://github.com/ClickHouse/ClickHouse/pull/69981) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add changelog for 24.9. [#69982](https://github.com/ClickHouse/ClickHouse/pull/69982) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add a test for [#45303](https://github.com/ClickHouse/ClickHouse/issues/45303). [#69987](https://github.com/ClickHouse/ClickHouse/pull/69987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update CHANGELOG.md. [#69988](https://github.com/ClickHouse/ClickHouse/pull/69988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update README.md. [#69991](https://github.com/ClickHouse/ClickHouse/pull/69991) ([Tyler Hannan](https://github.com/tylerhannan)). -* Disable `03215_parallel_replicas_crash_after_refactoring.sql` for Azure. [#69992](https://github.com/ClickHouse/ClickHouse/pull/69992) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Update CHANGELOG.md. [#69993](https://github.com/ClickHouse/ClickHouse/pull/69993) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update CHANGELOG.md. [#70004](https://github.com/ClickHouse/ClickHouse/pull/70004) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Revert "Add RIPEMD160 function". [#70005](https://github.com/ClickHouse/ClickHouse/pull/70005) ([Robert Schulze](https://github.com/rschu1ze)). -* Update CHANGELOG.md. [#70009](https://github.com/ClickHouse/ClickHouse/pull/70009) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update CHANGELOG.md. [#70010](https://github.com/ClickHouse/ClickHouse/pull/70010) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Make the pylint stricter. [#70013](https://github.com/ClickHouse/ClickHouse/pull/70013) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Added a setting `restore_replace_external_dictionary_source_to_null` which enables replacing dictionary source with Null on restore for external dictionaries (useful for testing). [#70032](https://github.com/ClickHouse/ClickHouse/pull/70032) ([Alexander Tokmakov](https://github.com/tavplubix)). -* `isort` is a simple import sorter for the python to comply [pep-8](https://peps.python.org/pep-0008/#imports) requirements. It will allow to decrease conflicts during sync and beautify the code. The import block is divided into three sub-blocks: `standard library` -> `third-party libraries` -> `local imports` -> `.local imports`. Each sub-block is ordered alphabetically with sub-sub-blocks `import X` -> `from X import Y`. [#70038](https://github.com/ClickHouse/ClickHouse/pull/70038) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Update version_date.tsv and changelog after v24.9.1.3278-stable. [#70049](https://github.com/ClickHouse/ClickHouse/pull/70049) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Despite the fact that we set the org-level workflow parameter `PYTHONUNBUFFERED`, it's not inherited in workflows. [#70050](https://github.com/ClickHouse/ClickHouse/pull/70050) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix ubsan issue in function sqid. [#70061](https://github.com/ClickHouse/ClickHouse/pull/70061) ([Robert Schulze](https://github.com/rschu1ze)). -* Delete a setting change. [#70071](https://github.com/ClickHouse/ClickHouse/pull/70071) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix `test_distributed_ddl`. [#70075](https://github.com/ClickHouse/ClickHouse/pull/70075) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Remove unused placeholder from exception message string. [#70086](https://github.com/ClickHouse/ClickHouse/pull/70086) ([Alsu Giliazova](https://github.com/alsugiliazova)). -* Better exception message when some of the permission is missing. [#70088](https://github.com/ClickHouse/ClickHouse/pull/70088) ([pufit](https://github.com/pufit)). -* Make vector similarity indexes work with adaptive granularity. [#70101](https://github.com/ClickHouse/ClickHouse/pull/70101) ([Robert Schulze](https://github.com/rschu1ze)). -* Add missing columns `total_rows`, `data_compressed_bytes`, and `data_uncompressed_bytes` to `system.projections`. Part of https://github.com/ClickHouse/ClickHouse/pull/68901. [#70106](https://github.com/ClickHouse/ClickHouse/pull/70106) ([Jordi Villar](https://github.com/jrdi)). -* Make `00938_fix_rwlock_segfault_long` non flaky. [#70109](https://github.com/ClickHouse/ClickHouse/pull/70109) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove TODO. [#70110](https://github.com/ClickHouse/ClickHouse/pull/70110) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Change the default threshold to enable hyper threading. [#70111](https://github.com/ClickHouse/ClickHouse/pull/70111) ([Jiebin Sun](https://github.com/jiebinn)). -* Fixed [#69092](https://github.com/ClickHouse/ClickHouse/issues/69092): if `materialized_postgresql_tables_list=table1(id, code),table(id,name)` (`table1` has name that is a substring for `table`) `getTableAllowedColumns` method returns `[id, code]` for `table` before this fix. [#70114](https://github.com/ClickHouse/ClickHouse/pull/70114) ([Kruglov Kirill](https://github.com/1on)). -* Reduce log level. [#70117](https://github.com/ClickHouse/ClickHouse/pull/70117) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Rename `getNumberOfPhysicalCPUCores` and fix its decription. [#70130](https://github.com/ClickHouse/ClickHouse/pull/70130) ([Nikita Taranov](https://github.com/nickitat)). -* Adding 24.10. [#70132](https://github.com/ClickHouse/ClickHouse/pull/70132) ([Tyler Hannan](https://github.com/tylerhannan)). -* (Re?)-enable libcxx asserts for debug builds. [#70134](https://github.com/ClickHouse/ClickHouse/pull/70134) ([Robert Schulze](https://github.com/rschu1ze)). -* Refactor reading from object storage. [#70141](https://github.com/ClickHouse/ClickHouse/pull/70141) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Silence UBSAN for integer overflows in some datetime functions. [#70142](https://github.com/ClickHouse/ClickHouse/pull/70142) ([Michael Kolupaev](https://github.com/al13n321)). -* Improve pipdeptree generator for docker images. - Update requirements.txt for the integration tests runner container - Remove some small dependencies, improve `helpers/retry_decorator.py` - Upgrade docker-compose from EOL version 1 to version 2. [#70146](https://github.com/ClickHouse/ClickHouse/pull/70146) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix 'QueryPlan was not initialized' in 'loop' with empty MergeTree. [#70149](https://github.com/ClickHouse/ClickHouse/pull/70149) ([Michael Kolupaev](https://github.com/al13n321)). -* Remove QueryPlan DataStream. [#70158](https://github.com/ClickHouse/ClickHouse/pull/70158) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Update test_storage_s3_queue/test.py. [#70159](https://github.com/ClickHouse/ClickHouse/pull/70159) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Small docs fix. [#70160](https://github.com/ClickHouse/ClickHouse/pull/70160) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Test: PR local plan, non-constant in source stream. [#70173](https://github.com/ClickHouse/ClickHouse/pull/70173) ([Igor Nikonov](https://github.com/devcrafter)). -* Fix performance checks. [#70175](https://github.com/ClickHouse/ClickHouse/pull/70175) ([Antonio Andelic](https://github.com/antonio2368)). -* Simplify test 03246_range_literal_replacement_works. [#70176](https://github.com/ClickHouse/ClickHouse/pull/70176) ([Pablo Marcos](https://github.com/pamarcos)). -* Update 01079_parallel_alter_add_drop_column_zookeeper.sh. [#70196](https://github.com/ClickHouse/ClickHouse/pull/70196) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Require bugfix job for a set of labels. [#70197](https://github.com/ClickHouse/ClickHouse/pull/70197) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* CI: Praktika integration, fast test. [#70239](https://github.com/ClickHouse/ClickHouse/pull/70239) ([Max Kainov](https://github.com/maxknv)). -* Avoid `Cannot schedule a task` error when loading parts. [#70257](https://github.com/ClickHouse/ClickHouse/pull/70257) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Bump usearch to v2.15.2 and SimSIMD to v5.0.0. [#70270](https://github.com/ClickHouse/ClickHouse/pull/70270) ([Robert Schulze](https://github.com/rschu1ze)). -* Instead of balancing tests by `crc32(file_name)` we'll use `add tests to a group with a minimal number of tests`. [#70272](https://github.com/ClickHouse/ClickHouse/pull/70272) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Closes [#70263](https://github.com/ClickHouse/ClickHouse/issues/70263). [#70273](https://github.com/ClickHouse/ClickHouse/pull/70273) ([flynn](https://github.com/ucasfl)). -* Hide MergeTreeSettings implementation. [#70285](https://github.com/ClickHouse/ClickHouse/pull/70285) ([Raúl Marín](https://github.com/Algunenano)). -* CI: Remove await feature from release branches. [#70294](https://github.com/ClickHouse/ClickHouse/pull/70294) ([Max Kainov](https://github.com/maxknv)). -* Fix `test_keeper_four_word_command`. [#70298](https://github.com/ClickHouse/ClickHouse/pull/70298) ([Antonio Andelic](https://github.com/antonio2368)). -* Update version_date.tsv and changelog after v24.9.2.42-stable. [#70301](https://github.com/ClickHouse/ClickHouse/pull/70301) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Synchronize settings with private. [#70320](https://github.com/ClickHouse/ClickHouse/pull/70320) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add Ignore Option In DeduplicateMergeProjectionMode. [#70327](https://github.com/ClickHouse/ClickHouse/pull/70327) ([Shichao Jin](https://github.com/jsc0218)). -* CI: Enable Integration Tests for backport PRs. [#70329](https://github.com/ClickHouse/ClickHouse/pull/70329) ([Max Kainov](https://github.com/maxknv)). -* There is [a failed CI job](https://s3.amazonaws.com/clickhouse-test-reports/69778/2d81c38874958bd9d54a25524173bdb1ddf2b75c/stateless_tests__release_.html) which is triggered by [03237_create_or_replace_view_atomically_with_atomic_engine](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/03237_create_or_replace_view_atomically_with_atomic_engine.sh). [#70330](https://github.com/ClickHouse/ClickHouse/pull/70330) ([tuanpach](https://github.com/tuanpach)). -* Fix flaky test `03237_insert_sparse_columns_mem`. [#70333](https://github.com/ClickHouse/ClickHouse/pull/70333) ([Anton Popov](https://github.com/CurtizJ)). -* Rename enable_secure_identifiers -> enforce_strict_identifier_format. [#70335](https://github.com/ClickHouse/ClickHouse/pull/70335) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Attempt to fix flaky RabbitMQ tests. Maybe closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#70336](https://github.com/ClickHouse/ClickHouse/pull/70336) ([filimonov](https://github.com/filimonov)). -* Don't fail the stateless check script if we can't collect minio logs. [#70350](https://github.com/ClickHouse/ClickHouse/pull/70350) ([Raúl Marín](https://github.com/Algunenano)). -* Fix tiny mistake, responsible for some of kafka test flaps. Example [report](https://s3.amazonaws.com/clickhouse-test-reports/0/3198aafac59c368993e7b5f49d95674cc1b1be18/integration_tests__release__[2_4].html). [#70352](https://github.com/ClickHouse/ClickHouse/pull/70352) ([filimonov](https://github.com/filimonov)). -* Closes [#69634](https://github.com/ClickHouse/ClickHouse/issues/69634). [#70354](https://github.com/ClickHouse/ClickHouse/pull/70354) ([pufit](https://github.com/pufit)). -* Fix 02346_fulltext_index_bug52019. [#70357](https://github.com/ClickHouse/ClickHouse/pull/70357) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Use new JSON for collecting minio logs. [#70359](https://github.com/ClickHouse/ClickHouse/pull/70359) ([Antonio Andelic](https://github.com/antonio2368)). -* Update comments in VectorSimilarityCondition (WHERE is not supported). [#70360](https://github.com/ClickHouse/ClickHouse/pull/70360) ([Azat Khuzhin](https://github.com/azat)). -* Remove 02492_clickhouse_local_context_uaf test. [#70363](https://github.com/ClickHouse/ClickHouse/pull/70363) ([Azat Khuzhin](https://github.com/azat)). -* Fix `clang-19` build issues. [#70412](https://github.com/ClickHouse/ClickHouse/pull/70412) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Ignore "Invalid multibyte data detected" error during completion. [#70422](https://github.com/ClickHouse/ClickHouse/pull/70422) ([Azat Khuzhin](https://github.com/azat)). -* Make QueryPlan explain methods const. [#70444](https://github.com/ClickHouse/ClickHouse/pull/70444) ([Alexander Gololobov](https://github.com/davenger)). -* Fix 0.1 second delay for interactive queries (due to keystroke interceptor). [#70445](https://github.com/ClickHouse/ClickHouse/pull/70445) ([Azat Khuzhin](https://github.com/azat)). -* Increase lock timeout in attempt to fix 02125_many_mutations. [#70448](https://github.com/ClickHouse/ClickHouse/pull/70448) ([Azat Khuzhin](https://github.com/azat)). -* Fix order in 03249_dynamic_alter_consistency. [#70453](https://github.com/ClickHouse/ClickHouse/pull/70453) ([Alexander Gololobov](https://github.com/davenger)). -* Fix refreshable MV in system database breaking server startup. [#70460](https://github.com/ClickHouse/ClickHouse/pull/70460) ([Michael Kolupaev](https://github.com/al13n321)). -* Fix flaky test_refreshable_mv_in_replicated_db. [#70462](https://github.com/ClickHouse/ClickHouse/pull/70462) ([Michael Kolupaev](https://github.com/al13n321)). -* Update version_date.tsv and changelog after v24.8.5.115-lts. [#70463](https://github.com/ClickHouse/ClickHouse/pull/70463) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Decrease probability of "Server died" due to 00913_many_threads. [#70473](https://github.com/ClickHouse/ClickHouse/pull/70473) ([Azat Khuzhin](https://github.com/azat)). -* Fixes for killing leftovers in clikhouse-test. [#70474](https://github.com/ClickHouse/ClickHouse/pull/70474) ([Azat Khuzhin](https://github.com/azat)). -* Update version_date.tsv and changelog after v24.3.12.75-lts. [#70485](https://github.com/ClickHouse/ClickHouse/pull/70485) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Use logging instead of print. [#70505](https://github.com/ClickHouse/ClickHouse/pull/70505) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). -* Remove slow poll() logs in keeper. [#70508](https://github.com/ClickHouse/ClickHouse/pull/70508) ([Raúl Marín](https://github.com/Algunenano)). -* Add timeouts for retry loops in test_storage_rabbitmq. It should prevent cascading failures of the whole test suite caused by deadloop in one of the test scenarios. Also added small sleeps in a 'tight' loops to make retries bit less agressive. [#70510](https://github.com/ClickHouse/ClickHouse/pull/70510) ([filimonov](https://github.com/filimonov)). -* CI: Fix for canceled Sync workflow. [#70521](https://github.com/ClickHouse/ClickHouse/pull/70521) ([Max Kainov](https://github.com/maxknv)). -* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/70412, don't know why it's ok in release build, simply changing `_` to `_1` is ok for both release and debug build. [#70532](https://github.com/ClickHouse/ClickHouse/pull/70532) ([Chang chen](https://github.com/baibaichen)). -* Refreshable materialized views are not experimental anymore. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). -* Fix 24.9 setting compatibility `database_replicated_allow_explicit_uuid`. [#70565](https://github.com/ClickHouse/ClickHouse/pull/70565) ([Nikita Fomichev](https://github.com/fm4v)). -* Fix typos. [#70588](https://github.com/ClickHouse/ClickHouse/pull/70588) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Vector search: allow to specify HNSW parameter `ef_search` at query time. [#70616](https://github.com/ClickHouse/ClickHouse/pull/70616) ([Robert Schulze](https://github.com/rschu1ze)). -* Increase max_rows_to_read limit in some tests. [#70617](https://github.com/ClickHouse/ClickHouse/pull/70617) ([Raúl Marín](https://github.com/Algunenano)). -* Reduce sync efforts with private. [#70634](https://github.com/ClickHouse/ClickHouse/pull/70634) ([Raúl Marín](https://github.com/Algunenano)). -* Fix parsing of some formats into sparse columns. [#70635](https://github.com/ClickHouse/ClickHouse/pull/70635) ([Anton Popov](https://github.com/CurtizJ)). -* Fix typos. [#70637](https://github.com/ClickHouse/ClickHouse/pull/70637) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Try fix 00180_no_seek_avoiding_when_reading_from_cache. [#70640](https://github.com/ClickHouse/ClickHouse/pull/70640) ([Kseniia Sumarokova](https://github.com/kssenii)). -* When the `PR Check` status is set, it's a valid RunConfig job failure. [#70643](https://github.com/ClickHouse/ClickHouse/pull/70643) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix timeout in materialized pg tests. [#70646](https://github.com/ClickHouse/ClickHouse/pull/70646) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Introduced MergeTree setting which allow to change merge selecting algorithm. However we still have only one algorithm and it's mostly for future experiments. [#70647](https://github.com/ClickHouse/ClickHouse/pull/70647) ([alesapin](https://github.com/alesapin)). -* Docs: Follow-up for [#70585](https://github.com/ClickHouse/ClickHouse/issues/70585). [#70654](https://github.com/ClickHouse/ClickHouse/pull/70654) ([Robert Schulze](https://github.com/rschu1ze)). -* Remove strange file. [#70662](https://github.com/ClickHouse/ClickHouse/pull/70662) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Locally I had lots of errors like `'AllocList' does not refer to a value` around places which used `offsetof`. Changing it to `__builtin_offsetof ` helped and I didn't debug any further. [#70671](https://github.com/ClickHouse/ClickHouse/pull/70671) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Adding the report link to a test result and files' list. [#70677](https://github.com/ClickHouse/ClickHouse/pull/70677) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* materialized postgres: minor fixes. [#70710](https://github.com/ClickHouse/ClickHouse/pull/70710) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Probably fix flaky test_refreshable_mv_in_replicated_db. [#70714](https://github.com/ClickHouse/ClickHouse/pull/70714) ([Michael Kolupaev](https://github.com/al13n321)). -* Move more setting structs to pImpl. [#70739](https://github.com/ClickHouse/ClickHouse/pull/70739) ([Raúl Marín](https://github.com/Algunenano)). -* Reduce sync effort. [#70747](https://github.com/ClickHouse/ClickHouse/pull/70747) ([Raúl Marín](https://github.com/Algunenano)). -* Add s3queue settings check for cloud. [#70750](https://github.com/ClickHouse/ClickHouse/pull/70750) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix readiness/health check for OpenLDAP container. [#70755](https://github.com/ClickHouse/ClickHouse/pull/70755) ([Julian Maicher](https://github.com/jmaicher)). -* Allow update plan headers for all the steps. [#70761](https://github.com/ClickHouse/ClickHouse/pull/70761) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Autogenerate documentation for settings. [#70768](https://github.com/ClickHouse/ClickHouse/pull/70768) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Not a logical error. [#70770](https://github.com/ClickHouse/ClickHouse/pull/70770) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* CI: Aarch64 build with Asan. [#70778](https://github.com/ClickHouse/ClickHouse/pull/70778) ([Max Kainov](https://github.com/maxknv)). -* Minor fix. [#70783](https://github.com/ClickHouse/ClickHouse/pull/70783) ([Anton Popov](https://github.com/CurtizJ)). -* The docs for settings should be located in the source code. Now, the CI supports that. [#70784](https://github.com/ClickHouse/ClickHouse/pull/70784) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update style-test image. [#70785](https://github.com/ClickHouse/ClickHouse/pull/70785) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Avoid double finalization of `WriteBuffer` in library bridge. [#70799](https://github.com/ClickHouse/ClickHouse/pull/70799) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Make Array Field serialization consistent. [#70803](https://github.com/ClickHouse/ClickHouse/pull/70803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* A follow-up for [#70785](https://github.com/ClickHouse/ClickHouse/issues/70785), [jwt](https://pypi.org/project/jwt/#history) looks very outdated, and we have issue with conflicting paths. [#70815](https://github.com/ClickHouse/ClickHouse/pull/70815) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Remove inneficient code. [#70816](https://github.com/ClickHouse/ClickHouse/pull/70816) ([Raúl Marín](https://github.com/Algunenano)). -* Allow large object files if OMIT_HEAVY_DEBUG_SYMBOLS = 0. [#70818](https://github.com/ClickHouse/ClickHouse/pull/70818) ([Michael Kolupaev](https://github.com/al13n321)). -* Add test with distributed queries for 15768. [#70834](https://github.com/ClickHouse/ClickHouse/pull/70834) ([Nikita Taranov](https://github.com/nickitat)). -* More setting structs to pImpl and reuse code. [#70840](https://github.com/ClickHouse/ClickHouse/pull/70840) ([Raúl Marín](https://github.com/Algunenano)). -* Update default HNSW parameter settings. [#70873](https://github.com/ClickHouse/ClickHouse/pull/70873) ([Robert Schulze](https://github.com/rschu1ze)). -* Limiting logging some lines about configs. [#70879](https://github.com/ClickHouse/ClickHouse/pull/70879) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Fix `limit by`, `limit with ties` for distributed and parallel replicas. [#70880](https://github.com/ClickHouse/ClickHouse/pull/70880) ([Nikita Taranov](https://github.com/nickitat)). -* Fix darwin build. [#70894](https://github.com/ClickHouse/ClickHouse/pull/70894) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add dots for consistency. [#70909](https://github.com/ClickHouse/ClickHouse/pull/70909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Logical error fix for substrings, found by fuzzer. [#70914](https://github.com/ClickHouse/ClickHouse/pull/70914) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* More setting structs to pImpl. [#70942](https://github.com/ClickHouse/ClickHouse/pull/70942) ([Raúl Marín](https://github.com/Algunenano)). -* Add logging for mock HTTP servers used in minio integration tests. [#70943](https://github.com/ClickHouse/ClickHouse/pull/70943) ([Vitaly Baranov](https://github.com/vitlibar)). -* Minor fixups of [#70011](https://github.com/ClickHouse/ClickHouse/issues/70011) and [#69918](https://github.com/ClickHouse/ClickHouse/issues/69918). [#70959](https://github.com/ClickHouse/ClickHouse/pull/70959) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Do not skip Build report and status fix. [#70965](https://github.com/ClickHouse/ClickHouse/pull/70965) ([Max Kainov](https://github.com/maxknv)). -* Fix Keeper entry serialization compatibility. [#70972](https://github.com/ClickHouse/ClickHouse/pull/70972) ([Antonio Andelic](https://github.com/antonio2368)). -* Update exception message. [#70975](https://github.com/ClickHouse/ClickHouse/pull/70975) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix `utils/c++expr` option `-b`. [#70978](https://github.com/ClickHouse/ClickHouse/pull/70978) ([Sergei Trifonov](https://github.com/serxa)). -* Fix `test_keeper_broken_logs`. [#70982](https://github.com/ClickHouse/ClickHouse/pull/70982) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix `01039_test_setting_parse`. [#70986](https://github.com/ClickHouse/ClickHouse/pull/70986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Tests for languages support for Embedded Dictionaries. [#71004](https://github.com/ClickHouse/ClickHouse/pull/71004) ([Max Vostrikov](https://github.com/max-vostrikov)). -* Required for internal test runs with the same image build in public CI. [#71008](https://github.com/ClickHouse/ClickHouse/pull/71008) ([Ilya Yatsishin](https://github.com/qoega)). -* Move remaining settings objects to pImpl and start simplification. [#71019](https://github.com/ClickHouse/ClickHouse/pull/71019) ([Raúl Marín](https://github.com/Algunenano)). -* CI: Rearrange directories for praktika ci. [#71029](https://github.com/ClickHouse/ClickHouse/pull/71029) ([Max Kainov](https://github.com/maxknv)). -* Fix assert in RemoteSource::onAsyncJobReady(). [#71034](https://github.com/ClickHouse/ClickHouse/pull/71034) ([Igor Nikonov](https://github.com/devcrafter)). -* Fix showing error message in ReadBufferFromS3 when retrying. Without this PR information about a retryable failure in `ReadBufferFromS3` could look like this:. [#71038](https://github.com/ClickHouse/ClickHouse/pull/71038) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix `test_truncate_database`. [#71057](https://github.com/ClickHouse/ClickHouse/pull/71057) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix clickhouse-test useless 5 second delay in case of multiple threads are used. [#71069](https://github.com/ClickHouse/ClickHouse/pull/71069) ([Azat Khuzhin](https://github.com/azat)). - -#### Not for changeling - -* Reverted. [#69812](https://github.com/ClickHouse/ClickHouse/pull/69812) ([tuanpach](https://github.com/tuanpach)). - ### ClickHouse release 24.9, 2024-09-26 From 07508cb3819a89fec7e63604e2de64ff1bd4904a Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 28 Oct 2024 11:47:01 +0000 Subject: [PATCH 496/816] Handle some problems with tests --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 3 +-- src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h | 3 +++ src/Storages/ObjectStorage/StorageObjectStorage.cpp | 3 ++- src/Storages/ObjectStorage/registerStorageObjectStorage.cpp | 3 ++- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index cd36429d0a2..4e6d0d985dd 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -500,8 +500,7 @@ void S3ObjectStorage::applyNewSettings( } auto current_settings = s3_settings.get(); - if (options.allow_client_change - && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) + if (options.allow_client_change && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) { auto new_client = getClient(uri, *modified_settings, context, for_disk_s3); client.set(std::move(new_client)); diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 866ef24aa91..18ff6d93c46 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -30,10 +30,13 @@ public: bool isDataLakeConfiguration() const override { return true; } + bool isStaticConfiguration() const override { return false; } + std::string getEngineName() const override { return DataLakeMetadata::name; } void update(ObjectStoragePtr object_storage, ContextPtr local_context) override { + BaseStorageConfiguration::update(object_storage, local_context); auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); if (current_metadata && *current_metadata == *new_metadata) return; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index a67c1628b6d..ddc6276a8a1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -87,8 +87,9 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - configuration_->update(object_storage_, context); ColumnsDescription columns{columns_}; + LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size()); + configuration->update(object_storage, context); std::string sample_path; resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index cb1826b2976..9a525b4e21a 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -27,7 +27,6 @@ static std::shared_ptr createStorageObjectStorage( StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, context, false); - // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current // session and user are ignored. @@ -251,6 +250,7 @@ void registerStorageDeltaLake(StorageFactory & factory) .source_access_type = AccessType::S3, }); #endif + UNUSED(factory); } #endif @@ -272,5 +272,6 @@ void registerStorageHudi(StorageFactory & factory) .source_access_type = AccessType::S3, }); #endif + UNUSED(factory); } } From 55c9a50100c517c7da8829fd29e5a41a30d6a4fc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Oct 2024 11:58:36 +0000 Subject: [PATCH 497/816] Revert "Make the definition of the new setting correct" This reverts commit 424bec90cbc9787a7892d631fd5f0b36dbd63d45. --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 88d39d6d393..c4274e2ae2e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -107,7 +107,7 @@ static std::initializer_list Date: Thu, 3 Oct 2024 08:30:52 +0000 Subject: [PATCH 498/816] mv fixReplicaMetadataVersionIfNeeded from attach thread to restarting thread --- .../ReplicatedMergeTreeAttachThread.cpp | 90 +----------------- .../ReplicatedMergeTreeAttachThread.h | 2 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 1 + .../ReplicatedMergeTreeRestartingThread.cpp | 92 +++++++++++++++++++ .../ReplicatedMergeTreeRestartingThread.h | 4 + tests/integration/helpers/cluster.py | 27 ++++-- .../test_fix_metadata_version/__init__.py | 0 .../configs/config.xml | 16 ++++ .../test_fix_metadata_version/test.py | 73 +++++++++++++++ 10 files changed, 206 insertions(+), 101 deletions(-) create mode 100644 tests/integration/test_fix_metadata_version/__init__.py create mode 100644 tests/integration/test_fix_metadata_version/configs/config.xml create mode 100644 tests/integration/test_fix_metadata_version/test.py diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index 22b8ccca151..c258048354e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -20,7 +21,6 @@ namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; extern const int REPLICA_STATUS_CHANGED; - extern const int LOGICAL_ERROR; } ReplicatedMergeTreeAttachThread::ReplicatedMergeTreeAttachThread(StorageReplicatedMergeTree & storage_) @@ -123,67 +123,6 @@ void ReplicatedMergeTreeAttachThread::checkHasReplicaMetadataInZooKeeper(const z } } -Int32 ReplicatedMergeTreeAttachThread::fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper) -{ - const String & zookeeper_path = storage.zookeeper_path; - const String & replica_path = storage.replica_path; - const bool replica_readonly = storage.is_readonly; - - for (size_t i = 0; i != 2; ++i) - { - String replica_metadata_version_str; - const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version_str); - if (!replica_metadata_version_exists) - return -1; - - const Int32 metadata_version = parse(replica_metadata_version_str); - - if (metadata_version != 0 || replica_readonly) - { - /// No need to fix anything - return metadata_version; - } - - Coordination::Stat stat; - zookeeper->get(fs::path(zookeeper_path) / "metadata", &stat); - if (stat.version == 0) - { - /// No need to fix anything - return metadata_version; - } - - ReplicatedMergeTreeQueue & queue = storage.queue; - queue.pullLogsToQueue(zookeeper); - if (queue.getStatus().metadata_alters_in_queue != 0) - { - LOG_DEBUG(log, "No need to update metadata_version as there are ALTER_METADATA entries in the queue"); - return metadata_version; - } - - const Coordination::Requests ops = { - zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(stat.version), 0), - zkutil::makeCheckRequest(fs::path(zookeeper_path) / "metadata", stat.version), - }; - Coordination::Responses ops_responses; - const auto code = zookeeper->tryMulti(ops, ops_responses); - if (code == Coordination::Error::ZOK) - { - LOG_DEBUG(log, "Successfully set metadata_version to {}", stat.version); - return stat.version; - } - if (code != Coordination::Error::ZBADVERSION) - { - throw zkutil::KeeperException(code); - } - } - - /// Second attempt is only possible if metadata_version != 0 or metadata.version changed during the first attempt. - /// If metadata_version != 0, on second attempt we will return the new metadata_version. - /// If metadata.version changed, on second attempt we will either get metadata_version != 0 and return the new metadata_version or we will get metadata_alters_in_queue != 0 and return 0. - /// Either way, on second attempt this method should return. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fix replica metadata_version in ZooKeeper after two attempts"); -} - void ReplicatedMergeTreeAttachThread::runImpl() { storage.setZooKeeper(); @@ -227,33 +166,6 @@ void ReplicatedMergeTreeAttachThread::runImpl() /// Just in case it was not removed earlier due to connection loss zookeeper->tryRemove(replica_path + "/flags/force_restore_data"); - const Int32 replica_metadata_version = fixReplicaMetadataVersionIfNeeded(zookeeper); - const bool replica_metadata_version_exists = replica_metadata_version != -1; - if (replica_metadata_version_exists) - { - storage.setInMemoryMetadata(metadata_snapshot->withMetadataVersion(replica_metadata_version)); - } - else - { - /// Table was created before 20.4 and was never altered, - /// let's initialize replica metadata version from global metadata version. - Coordination::Stat table_metadata_version_stat; - zookeeper->get(zookeeper_path + "/metadata", &table_metadata_version_stat); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/metadata", table_metadata_version_stat.version)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(table_metadata_version_stat.version), zkutil::CreateMode::Persistent)); - - Coordination::Responses res; - auto code = zookeeper->tryMulti(ops, res); - - if (code == Coordination::Error::ZBADVERSION) - throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Failed to initialize metadata_version " - "because table was concurrently altered, will retry"); - - zkutil::KeeperMultiException::check(code, ops, res); - } - storage.checkTableStructure(replica_path, metadata_snapshot); storage.checkParts(skip_sanity_checks); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h index bfc97442598..250a5ed34d1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h @@ -48,8 +48,6 @@ private: void runImpl(); void finalizeInitialization(); - - Int32 fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper); }; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 6b1581645f8..b1564b58a6c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -615,7 +615,7 @@ std::pair ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::Zo { std::lock_guard lock(pull_logs_to_queue_mutex); - if (reason != LOAD) + if (reason != LOAD && reason != FIX_METADATA_VERSION) { /// It's totally ok to load queue on readonly replica (that's what RestartingThread does on initialization). /// It's ok if replica became readonly due to connection loss after we got current zookeeper (in this case zookeeper must be expired). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 9d3349663e2..6ec8818b0c6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -334,6 +334,7 @@ public: UPDATE, MERGE_PREDICATE, SYNC, + FIX_METADATA_VERSION, OTHER, }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 9d3e26cdc8d..93124e634bd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -29,6 +29,8 @@ namespace MergeTreeSetting namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; + extern const int REPLICA_STATUS_CHANGED; + extern const int LOGICAL_ERROR; } namespace FailPoints @@ -207,6 +209,36 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() throw; } + const Int32 replica_metadata_version = fixReplicaMetadataVersionIfNeeded(zookeeper); + const bool replica_metadata_version_exists = replica_metadata_version != -1; + if (replica_metadata_version_exists) + { + storage.setInMemoryMetadata(storage.getInMemoryMetadataPtr()->withMetadataVersion(replica_metadata_version)); + } + else + { + /// Table was created before 20.4 and was never altered, + /// let's initialize replica metadata version from global metadata version. + + const String & zookeeper_path = storage.zookeeper_path, & replica_path = storage.replica_path; + + Coordination::Stat table_metadata_version_stat; + zookeeper->get(zookeeper_path + "/metadata", &table_metadata_version_stat); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/metadata", table_metadata_version_stat.version)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(table_metadata_version_stat.version), zkutil::CreateMode::Persistent)); + + Coordination::Responses res; + auto code = zookeeper->tryMulti(ops, res); + + if (code == Coordination::Error::ZBADVERSION) + throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Failed to initialize metadata_version " + "because table was concurrently altered, will retry"); + + zkutil::KeeperMultiException::check(code, ops, res); + } + storage.queue.removeCurrentPartsFromMutations(); storage.last_queue_update_finish_time.store(time(nullptr)); @@ -424,4 +456,64 @@ void ReplicatedMergeTreeRestartingThread::setNotReadonly() storage.readonly_start_time.store(0, std::memory_order_relaxed); } + +Int32 ReplicatedMergeTreeRestartingThread::fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper) +{ + const String & zookeeper_path = storage.zookeeper_path; + const String & replica_path = storage.replica_path; + + const size_t num_attempts = 2; + for (size_t attempt = 0; attempt != num_attempts; ++attempt) + { + String replica_metadata_version_str; + Coordination::Stat replica_stat; + const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version_str, &replica_stat); + if (!replica_metadata_version_exists) + return -1; + + const Int32 metadata_version = parse(replica_metadata_version_str); + if (metadata_version != 0) + return metadata_version; + + Coordination::Stat table_stat; + zookeeper->get(fs::path(zookeeper_path) / "metadata", &table_stat); + if (table_stat.version == 0) + return metadata_version; + + ReplicatedMergeTreeQueue & queue = storage.queue; + queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::FIX_METADATA_VERSION); + if (queue.getStatus().metadata_alters_in_queue != 0) + { + LOG_INFO(log, "Skipping updating metadata_version as there are ALTER_METADATA entries in the queue"); + return metadata_version; + } + + const Coordination::Requests ops = { + zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(table_stat.version), replica_stat.version), + zkutil::makeCheckRequest(fs::path(zookeeper_path) / "metadata", table_stat.version), + }; + Coordination::Responses ops_responses; + const Coordination::Error code = zookeeper->tryMulti(ops, ops_responses); + if (code == Coordination::Error::ZOK) + { + LOG_DEBUG(log, "Successfully set metadata_version to {}", table_stat.version); + return table_stat.version; + } + + if (code == Coordination::Error::ZBADVERSION) + { + LOG_WARNING(log, "Cannot fix metadata_version because either metadata.version or metadata_version.version changed, attempts left = {}", num_attempts - attempt - 1); + continue; + } + + throw zkutil::KeeperException(code); + } + + /// Second attempt is only possible if either metadata_version.version or metadata.version changed during the first attempt. + /// If metadata_version changed to non-zero value during the first attempt, on second attempt we will return the new metadata_version. + /// If metadata.version changed during first attempt, on second attempt we will either get metadata_version != 0 and return the new metadata_version or we will get metadata_alters_in_queue != 0 and return 0. + /// So either first or second attempt should return unless metadata_version was rewritten from 0 to 0 during the first attempt which is highly unlikely. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fix replica metadata_version in ZooKeeper after two attempts"); +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index d719505ae5e..6f450dc1d40 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -68,6 +69,9 @@ private: /// Disable readonly mode for table void setNotReadonly(); + + /// Fix replica metadata_version if needed + Int32 fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper); }; diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3c92df51ac4..1e98561b9f7 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -83,6 +83,8 @@ CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.lo # This means that this minimum need to be, at least, 1 year older than the current release CLICKHOUSE_CI_MIN_TESTED_VERSION = "23.3" +ZOOKEEPER_CONTAINERS = ("zoo1", "zoo2", "zoo3") + # to create docker-compose env file def _create_env_file(path, variables): @@ -2061,6 +2063,11 @@ class ClickHouseCluster: container_id = self.get_container_id(instance_name) return self.docker_client.api.logs(container_id).decode() + def query_zookeeper(self, query, node=ZOOKEEPER_CONTAINERS[0], nothrow=False): + cmd = f'clickhouse keeper-client -p {self.zookeeper_port} -q "{query}"' + container_id = self.get_container_id(node) + return self.exec_in_container(container_id, cmd, nothrow=nothrow, use_cli=False) + def exec_in_container( self, container_id: str, @@ -2391,16 +2398,16 @@ class ClickHouseCluster: def wait_zookeeper_secure_to_start(self, timeout=20): logging.debug("Wait ZooKeeper Secure to start") - nodes = ["zoo1", "zoo2", "zoo3"] - self.wait_zookeeper_nodes_to_start(nodes, timeout) + self.wait_zookeeper_nodes_to_start(ZOOKEEPER_CONTAINERS, timeout) def wait_zookeeper_to_start(self, timeout: float = 180) -> None: logging.debug("Wait ZooKeeper to start") - nodes = ["zoo1", "zoo2", "zoo3"] - self.wait_zookeeper_nodes_to_start(nodes, timeout) + self.wait_zookeeper_nodes_to_start(ZOOKEEPER_CONTAINERS, timeout) def wait_zookeeper_nodes_to_start( - self, nodes: List[str], timeout: float = 60 + self, + nodes: List[str], + timeout: float = 60, ) -> None: start = time.time() err = Exception("") @@ -3226,7 +3233,11 @@ class ClickHouseCluster: return zk def run_kazoo_commands_with_retries( - self, kazoo_callback, zoo_instance_name="zoo1", repeats=1, sleep_for=1 + self, + kazoo_callback, + zoo_instance_name=ZOOKEEPER_CONTAINERS[0], + repeats=1, + sleep_for=1, ): zk = self.get_kazoo_client(zoo_instance_name) logging.debug( @@ -4648,9 +4659,7 @@ class ClickHouseInstance: depends_on.append("nats1") if self.with_zookeeper: - depends_on.append("zoo1") - depends_on.append("zoo2") - depends_on.append("zoo3") + depends_on += list(ZOOKEEPER_CONTAINERS) if self.with_minio: depends_on.append("minio1") diff --git a/tests/integration/test_fix_metadata_version/__init__.py b/tests/integration/test_fix_metadata_version/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_fix_metadata_version/configs/config.xml b/tests/integration/test_fix_metadata_version/configs/config.xml new file mode 100644 index 00000000000..4662e6794e3 --- /dev/null +++ b/tests/integration/test_fix_metadata_version/configs/config.xml @@ -0,0 +1,16 @@ + + 9000 + + + + + + + + + default + + + + + diff --git a/tests/integration/test_fix_metadata_version/test.py b/tests/integration/test_fix_metadata_version/test.py new file mode 100644 index 00000000000..085872bba05 --- /dev/null +++ b/tests/integration/test_fix_metadata_version/test.py @@ -0,0 +1,73 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + stay_alive=True, + with_zookeeper=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_fix_metadata_version(start_cluster): + zookeeper_path = "/clickhouse/test_fix_metadata_version" + replica = "replica1" + replica_path = f"{zookeeper_path}/replicas/{replica}" + + def get_metadata_versions(): + table_metadata_version = int( + node.query( + f""" + SELECT version + FROM system.zookeeper + WHERE path = '{zookeeper_path}' AND name = 'metadata' + """ + ).strip() + ) + + replica_metadata_version = int( + node.query( + f""" + SELECT value + FROM system.zookeeper + WHERE path = '{replica_path}' AND name = 'metadata_version' + """ + ).strip() + ) + + return table_metadata_version, replica_metadata_version + + node.query( + f""" + DROP TABLE IF EXISTS t SYNC; + CREATE TABLE t + ( + `x` UInt32 + ) + ENGINE = ReplicatedMergeTree('{zookeeper_path}', '{replica}') + ORDER BY x + """ + ) + + node.query("ALTER TABLE t (ADD COLUMN `y` UInt32)") + + assert get_metadata_versions() == (1, 1) + + cluster.query_zookeeper(f"set '{replica_path}/metadata_version' '0'") + + assert get_metadata_versions() == (1, 0) + + node.restart_clickhouse() + + assert get_metadata_versions() == (1, 1) From ffdca3693700473bb334e5b4677012563ef73761 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 14:40:39 +0100 Subject: [PATCH 499/816] Clear backward incompatible changes --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 412130f58be..9724eb7eb61 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,8 +17,6 @@ #### Backward Incompatible Change * Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#68614](https://github.com/ClickHouse/ClickHouse/pull/68614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). -* Allow empty needle in functions replaceRegexp*. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). * Reordering of filter conditions from `[PRE]WHERE` clause is now allowed by default. It could be disabled by setting `allow_reorder_prewhere_conditions` to `false`. [#70657](https://github.com/ClickHouse/ClickHouse/pull/70657) ([Nikita Taranov](https://github.com/nickitat)). * Fix `optimize_functions_to_subcolumns` optimization (previously could lead to `Invalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)` error), by preserving `LowCardinality` type in `mapKeys`/`mapValues`. [#70716](https://github.com/ClickHouse/ClickHouse/pull/70716) ([Azat Khuzhin](https://github.com/azat)). * Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -62,6 +60,8 @@ * Improve performance of FromUnixTimestamp/ToUnixTimestamp functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). #### Improvement +* Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). +* Allow empty needle in functions replaceRegexp*. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). * Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). * `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). * Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). From d18da05f3cea480c9f947cd6094d31bf3f82ec62 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 28 Oct 2024 13:45:47 +0000 Subject: [PATCH 500/816] fix --- src/Storages/ProjectionsDescription.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 89a7acf8a72..26c3238c940 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -306,7 +306,6 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) select_row_exists->setExpression( ASTSelectQuery::Expression::WHERE, makeASTFunction("equals", std::make_shared("_row_exists"), std::make_shared(1))); - // std::cout< Date: Mon, 28 Oct 2024 13:47:45 +0000 Subject: [PATCH 501/816] Update compatibility value for hnsw_candidate_list_size_for_search --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 88d39d6d393..1906b287586 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -108,7 +108,7 @@ static std::initializer_list Date: Mon, 28 Oct 2024 14:01:37 +0000 Subject: [PATCH 502/816] add baseline --- src/Common/FieldVisitorMul.cpp | 50 ++++++ src/Common/FieldVisitorMul.h | 53 ++++++ src/Core/Field.h | 8 + src/Core/SortDescription.h | 5 +- src/Interpreters/FillingRow.cpp | 94 +++++++++-- src/Interpreters/FillingRow.h | 9 +- .../Transforms/FillingTransform.cpp | 159 +++++++++++------- 7 files changed, 306 insertions(+), 72 deletions(-) create mode 100644 src/Common/FieldVisitorMul.cpp create mode 100644 src/Common/FieldVisitorMul.h diff --git a/src/Common/FieldVisitorMul.cpp b/src/Common/FieldVisitorMul.cpp new file mode 100644 index 00000000000..36c32c40c05 --- /dev/null +++ b/src/Common/FieldVisitorMul.cpp @@ -0,0 +1,50 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +FieldVisitorMul::FieldVisitorMul(const Field & rhs_) : rhs(rhs_) {} + +// We can add all ints as unsigned regardless of their actual signedness. +bool FieldVisitorMul::operator() (Int64 & x) const { return this->operator()(reinterpret_cast(x)); } +bool FieldVisitorMul::operator() (UInt64 & x) const +{ + x *= applyVisitor(FieldVisitorConvertToNumber(), rhs); + return x != 0; +} + +bool FieldVisitorMul::operator() (Float64 & x) const { + x *= rhs.safeGet(); + return x != 0; +} + +bool FieldVisitorMul::operator() (Null &) const +{ + /// Do not add anything + return false; +} + +bool FieldVisitorMul::operator() (String &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Strings"); } +bool FieldVisitorMul::operator() (Array &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Arrays"); } +bool FieldVisitorMul::operator() (Tuple &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Tuples"); } +bool FieldVisitorMul::operator() (Map &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Maps"); } +bool FieldVisitorMul::operator() (Object &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Objects"); } +bool FieldVisitorMul::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply UUIDs"); } +bool FieldVisitorMul::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv4s"); } +bool FieldVisitorMul::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv6s"); } +bool FieldVisitorMul::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply custom type {}", x.getTypeName()); } + +bool FieldVisitorMul::operator() (AggregateFunctionStateData &) const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply AggregateFunctionStates"); +} + +bool FieldVisitorMul::operator() (bool &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Bools"); } + +} diff --git a/src/Common/FieldVisitorMul.h b/src/Common/FieldVisitorMul.h new file mode 100644 index 00000000000..5bce41f1e71 --- /dev/null +++ b/src/Common/FieldVisitorMul.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** Implements `*=` operation. + * Returns false if the result is zero. + */ +class FieldVisitorMul : public StaticVisitor +{ +private: + const Field & rhs; +public: + explicit FieldVisitorMul(const Field & rhs_); + + // We can add all ints as unsigned regardless of their actual signedness. + bool operator() (Int64 & x) const; + bool operator() (UInt64 & x) const; + bool operator() (Float64 & x) const; + bool operator() (Null &) const; + bool operator() (String &) const; + bool operator() (Array &) const; + bool operator() (Tuple &) const; + bool operator() (Map &) const; + bool operator() (Object &) const; + bool operator() (UUID &) const; + bool operator() (IPv4 &) const; + bool operator() (IPv6 &) const; + bool operator() (AggregateFunctionStateData &) const; + bool operator() (CustomType &) const; + bool operator() (bool &) const; + + template + bool operator() (DecimalField & x) const + { + x *= rhs.safeGet>(); + return x.getValue() != T(0); + } + + template + requires is_big_int_v + bool operator() (T & x) const + { + x *= applyVisitor(FieldVisitorConvertToNumber(), rhs); + return x != T(0); + } +}; + +} diff --git a/src/Core/Field.h b/src/Core/Field.h index 7b916d30646..47df5c2907e 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -185,6 +185,14 @@ public: return *this; } + const DecimalField & operator *= (const DecimalField & r) + { + if (scale != r.getScale()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Multiply different decimal fields"); + dec *= r.getValue(); + return *this; + } + const DecimalField & operator -= (const DecimalField & r) { if (scale != r.getScale()) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 5c6f3e3150a..7a7c92f3b53 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -33,9 +33,12 @@ struct FillColumnDescription DataTypePtr fill_to_type; Field fill_step; /// Default = +1 or -1 according to direction std::optional step_kind; + Field fill_staleness; /// Default = Null - should not be considered + std::optional staleness_kind; - using StepFunction = std::function; + using StepFunction = std::function; StepFunction step_func; + StepFunction staleness_step_func; }; /// Description of the sorting rule by one column. diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 21b5b04bca3..1d3eae03ddd 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -28,6 +28,7 @@ FillingRow::FillingRow(const SortDescription & sort_description_) : sort_description(sort_description_) { row.resize(sort_description.size()); + staleness_base_row.resize(sort_description.size()); } bool FillingRow::operator<(const FillingRow & other) const @@ -63,7 +64,53 @@ bool FillingRow::isNull() const return true; } -std::pair FillingRow::next(const FillingRow & to_row) +std::optional FillingRow::doJump(const FillColumnDescription& descr, size_t column_ind) +{ + Field next_value = row[column_ind]; + descr.step_func(next_value, 1); + + if (!descr.fill_to.isNull() && less(descr.fill_to, next_value, getDirection(column_ind))) + return std::nullopt; + + if (!descr.fill_staleness.isNull()) { + Field staleness_border = staleness_base_row[column_ind]; + descr.staleness_step_func(staleness_border, 1); + + if (less(next_value, staleness_border, getDirection(column_ind))) + return next_value; + else + return std::nullopt; + } + + return next_value; +} + +std::optional FillingRow::doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to) +{ + Field shifted_value = row[column_ind]; + + if (less(to, shifted_value, getDirection(column_ind))) + return std::nullopt; + + for (int32_t step_len = 1, step_no = 0; step_no < 100; ++step_no) { + Field next_value = shifted_value; + descr.step_func(next_value, step_len); + + if (less(next_value, to, getDirection(0))) + { + shifted_value = std::move(next_value); + step_len *= 2; + } + else + { + step_len /= 2; + } + } + + return shifted_value; +} + +std::pair FillingRow::next(const FillingRow & to_row, bool long_jump) { const size_t row_size = size(); size_t pos = 0; @@ -85,23 +132,43 @@ std::pair FillingRow::next(const FillingRow & to_row) if (fill_column_desc.fill_to.isNull() || row[i].isNull()) continue; - Field next_value = row[i]; - fill_column_desc.step_func(next_value); - if (less(next_value, fill_column_desc.fill_to, getDirection(i))) + auto next_value = doJump(fill_column_desc, i); + if (next_value.has_value() && !equals(next_value.value(), fill_column_desc.fill_to)) { - row[i] = next_value; + row[i] = std::move(next_value.value()); initFromDefaults(i + 1); return {true, true}; } } - auto next_value = row[pos]; - getFillDescription(pos).step_func(next_value); + auto & fill_column_desc = getFillDescription(pos); + std::optional next_value; - if (less(to_row.row[pos], next_value, getDirection(pos)) || equals(next_value, getFillDescription(pos).fill_to)) - return {false, false}; + if (long_jump) + { + next_value = doLongJump(fill_column_desc, pos, to_row[pos]); - row[pos] = next_value; + if (!next_value.has_value()) + return {false, false}; + + Field calibration_jump_value = next_value.value(); + fill_column_desc.step_func(calibration_jump_value, 1); + + if (equals(calibration_jump_value, to_row[pos])) + next_value = calibration_jump_value; + + if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) + return {false, false}; + } + else + { + next_value = doJump(fill_column_desc, pos); + + if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) + return {false, false}; + } + + row[pos] = std::move(next_value.value()); if (equals(row[pos], to_row.row[pos])) { bool is_less = false; @@ -128,6 +195,13 @@ void FillingRow::initFromDefaults(size_t from_pos) row[i] = getFillDescription(i).fill_from; } +void FillingRow::initStalenessRow(const Columns& base_row, size_t row_ind) +{ + for (size_t i = 0; i < size(); ++i) { + staleness_base_row[i] = (*base_row[i])[row_ind]; + } +} + String FillingRow::dump() const { WriteBufferFromOwnString out; diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 004b417542c..14b6034ce35 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { @@ -15,6 +15,9 @@ bool equals(const Field & lhs, const Field & rhs); */ class FillingRow { + std::optional doJump(const FillColumnDescription & descr, size_t column_ind); + std::optional doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to); + public: explicit FillingRow(const SortDescription & sort_description); @@ -22,9 +25,10 @@ public: /// Return pair of boolean /// apply - true if filling values should be inserted into result set /// value_changed - true if filling row value was changed - std::pair next(const FillingRow & to_row); + std::pair next(const FillingRow & to_row, bool long_jump); void initFromDefaults(size_t from_pos = 0); + void initStalenessRow(const Columns& base_row, size_t row_ind); Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } @@ -42,6 +46,7 @@ public: private: Row row; + Row staleness_base_row; SortDescription sort_description; }; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 95f4a674ebb..1d68f73e8c2 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -7,15 +7,17 @@ #include #include #include +#include #include #include #include +#include namespace DB { -constexpr bool debug_logging_enabled = false; +constexpr bool debug_logging_enabled = true; template void logDebug(String key, const T & value, const char * separator = " : ") @@ -60,15 +62,78 @@ static FillColumnDescription::StepFunction getStepFunction( { #define DECLARE_CASE(NAME) \ case IntervalKind::Kind::NAME: \ - return [step, scale, &date_lut](Field & field) { \ + return [step, scale, &date_lut](Field & field, Int32 jumps_count) { \ field = Add##NAME##sImpl::execute(static_cast(\ - field.safeGet()), static_cast(step), date_lut, utc_time_zone, scale); }; + field.safeGet()), static_cast(step) * jumps_count, date_lut, utc_time_zone, scale); }; FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE } } +static FillColumnDescription::StepFunction getStepFunction(const Field & step, const std::optional & step_kind, const DataTypePtr & type) +{ + WhichDataType which(type); + + if (step_kind) + { + if (which.isDate() || which.isDate32()) + { + Int64 avg_seconds = step.safeGet() * step_kind->toAvgSeconds(); + if (std::abs(avg_seconds) < 86400) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); + } + + if (which.isDate()) + return getStepFunction(step_kind.value(), step.safeGet(), DateLUT::instance()); + else if (which.isDate32()) + return getStepFunction(step_kind.value(), step.safeGet(), DateLUT::instance()); + else if (const auto * date_time = checkAndGetDataType(type.get())) + return getStepFunction(step_kind.value(), step.safeGet(), date_time->getTimeZone()); + else if (const auto * date_time64 = checkAndGetDataType(type.get())) + { + const auto & step_dec = step.safeGet &>(); + Int64 converted_step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); + static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); + + switch (step_kind.value()) // NOLINT(bugprone-switch-missing-default-case) + { +#define DECLARE_CASE(NAME) \ + case IntervalKind::Kind::NAME: \ + return [converted_step, &time_zone = date_time64->getTimeZone()](Field & field, Int32 jumps_count) \ + { \ + auto field_decimal = field.safeGet>(); \ + auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), converted_step * jumps_count, time_zone, utc_time_zone, field_decimal.getScale()); \ + field = DecimalField(res, field_decimal.getScale()); \ + }; \ + break; + + FOR_EACH_INTERVAL_KIND(DECLARE_CASE) +#undef DECLARE_CASE + } + } + else + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName()); + } + else + { + return [step](Field & field, Int32 jumps_count) + { + auto shifted_step = step; + if (jumps_count != 1) + applyVisitor(FieldVisitorMul(jumps_count), shifted_step); + + logDebug("field", field.dump()); + logDebug("step", step.dump()); + logDebug("shifted field", shifted_step.dump()); + + applyVisitor(FieldVisitorSum(shifted_step), field); + }; + } +} + static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type) { auto max_type = Field::Types::Null; @@ -125,7 +190,8 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type - || descr.fill_step.getType() > max_type) + || descr.fill_step.getType() > max_type + || descr.fill_staleness.getType() > max_type) return false; if (!descr.fill_from.isNull()) @@ -134,56 +200,11 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & descr.fill_to = convertFieldToTypeOrThrow(descr.fill_to, *to_type); if (!descr.fill_step.isNull()) descr.fill_step = convertFieldToTypeOrThrow(descr.fill_step, *to_type); + if (!descr.fill_staleness.isNull()) + descr.fill_staleness = convertFieldToTypeOrThrow(descr.fill_staleness, *to_type); - if (descr.step_kind) - { - if (which.isDate() || which.isDate32()) - { - Int64 avg_seconds = descr.fill_step.safeGet() * descr.step_kind->toAvgSeconds(); - if (std::abs(avg_seconds) < 86400) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Value of step is to low ({} seconds). Must be >= 1 day", std::abs(avg_seconds)); - } - - if (which.isDate()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); - else if (which.isDate32()) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), DateLUT::instance()); - else if (const auto * date_time = checkAndGetDataType(type.get())) - descr.step_func = getStepFunction(*descr.step_kind, descr.fill_step.safeGet(), date_time->getTimeZone()); - else if (const auto * date_time64 = checkAndGetDataType(type.get())) - { - const auto & step_dec = descr.fill_step.safeGet &>(); - Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); - static const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC"); - - switch (*descr.step_kind) // NOLINT(bugprone-switch-missing-default-case) - { -#define DECLARE_CASE(NAME) \ - case IntervalKind::Kind::NAME: \ - descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ - { \ - auto field_decimal = field.safeGet>(); \ - auto res = Add##NAME##sImpl::execute(field_decimal.getValue(), step, time_zone, utc_time_zone, field_decimal.getScale()); \ - field = DecimalField(res, field_decimal.getScale()); \ - }; \ - break; - - FOR_EACH_INTERVAL_KIND(DECLARE_CASE) -#undef DECLARE_CASE - } - } - else - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName()); - } - else - { - descr.step_func = [step = descr.fill_step](Field & field) - { - applyVisitor(FieldVisitorSum(step), field); - }; - } + descr.step_func = getStepFunction(descr.fill_step, descr.step_kind, type); + descr.staleness_step_func = getStepFunction(descr.fill_staleness, descr.staleness_kind, type); return true; } @@ -482,8 +503,8 @@ bool FillingTransform::generateSuffixIfNeeded( MutableColumnRawPtrs res_sort_prefix_columns, MutableColumnRawPtrs res_other_columns) { - logDebug("generateSuffixIfNeeded() filling_row", filling_row); - logDebug("generateSuffixIfNeeded() next_row", next_row); + logDebug("generateSuffixIfNeeded filling_row", filling_row); + logDebug("generateSuffixIfNeeded next_row", next_row); /// Determines if we should insert filling row before start generating next rows bool should_insert_first = (next_row < filling_row && !filling_row_inserted) || next_row.isNull(); @@ -492,11 +513,11 @@ bool FillingTransform::generateSuffixIfNeeded( for (size_t i = 0, size = filling_row.size(); i < size; ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; - logDebug("generateSuffixIfNeeded() next_row updated", next_row); + logDebug("generateSuffixIfNeeded next_row updated", next_row); if (filling_row >= next_row) { - logDebug("generateSuffixIfNeeded()", "no need to generate suffix"); + logDebug("generateSuffixIfNeeded", "no need to generate suffix"); return false; } @@ -516,7 +537,7 @@ bool FillingTransform::generateSuffixIfNeeded( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row); + const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/false); filling_row_changed = changed; if (!apply) break; @@ -593,6 +614,9 @@ void FillingTransform::transformRange( const auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; + logDebug("current value", current_value.dump()); + logDebug("fill from", fill_from.dump()); + if (!fill_from.isNull() && !equals(current_value, fill_from)) { filling_row.initFromDefaults(i); @@ -609,6 +633,9 @@ void FillingTransform::transformRange( } } + /// Init staleness first interval + filling_row.initStalenessRow(input_fill_columns, range_begin); + for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) { logDebug("row", row_ind); @@ -623,6 +650,9 @@ void FillingTransform::transformRange( const auto current_value = (*input_fill_columns[i])[row_ind]; const auto & fill_to = filling_row.getFillDescription(i).fill_to; + logDebug("current value", current_value.dump()); + logDebug("fill to", fill_to.dump()); + if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) next_row[i] = current_value; else @@ -643,7 +673,7 @@ void FillingTransform::transformRange( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row); + const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/false); filling_row_changed = changed; if (!apply) break; @@ -652,6 +682,14 @@ void FillingTransform::transformRange( insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } + + const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/true); + logDebug("apply", apply); + logDebug("changed", changed); + + if (changed) + filling_row_changed = true; + /// new valid filling row was generated but not inserted, will use it during suffix generation if (filling_row_changed) filling_row_inserted = false; @@ -662,6 +700,9 @@ void FillingTransform::transformRange( copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); + + /// Init next staleness interval with current row, because we have already made the long jump to it + filling_row.initStalenessRow(input_fill_columns, row_ind); } /// save sort prefix of last row in the range, it's used to generate suffix From d94435c6abd0bfdf66a01c66dc8a83282c56b971 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 28 Oct 2024 15:02:29 +0000 Subject: [PATCH 503/816] fix tidy build --- src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp | 6 +++--- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 2 +- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h | 2 +- src/Parsers/ASTCreateWorkloadQuery.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 2dd37809b12..1bff672c150 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -47,9 +47,9 @@ namespace ErrorCodes namespace { - static constexpr std::string_view workload_prefix = "workload_"; - static constexpr std::string_view resource_prefix = "resource_"; - static constexpr std::string_view sql_suffix = ".sql"; + constexpr std::string_view workload_prefix = "workload_"; + constexpr std::string_view resource_prefix = "resource_"; + constexpr std::string_view sql_suffix = ".sql"; /// Converts a path to an absolute path and append it with a separator. String makeDirectoryPathCanonical(const String & directory_path) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index edeab7f6a7d..ee3cb5bb0c8 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -722,7 +722,7 @@ std::vector WorkloadEntityStorageBase::orderEn std::vector result; std::unordered_map workloads; - for (auto & [entity_name, ast] : all_entities) + for (const auto & [entity_name, ast] : all_entities) { if (typeid_cast(ast.get())) { diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index f1ef4124e98..d57bf8201b3 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -100,7 +100,7 @@ private: /// Returns an ordered vector of `entities` std::vector orderEntities( - const std::unordered_map & all_entitites, + const std::unordered_map & all_entities, std::optional change = {}); struct Handlers diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h index 71e27295bc1..8a4cecc001e 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -39,7 +39,7 @@ public: ASTPtr clone() const override; - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } From a02a6f4e3d3323c0a4788dfefc66600b6310f90b Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 28 Oct 2024 15:09:46 +0000 Subject: [PATCH 504/816] get rid of short debug log messages --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index ee3cb5bb0c8..1b7a559698c 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -620,7 +620,7 @@ void WorkloadEntityStorageBase::applyEvent( { if (event.entity) // CREATE || CREATE OR REPLACE { - LOG_DEBUG(log, "Create or replace entity: {}", serializeAST(*event.entity)); + LOG_DEBUG(log, "Create or replace workload entity: {}", serializeAST(*event.entity)); auto * workload = typeid_cast(event.entity.get()); @@ -643,7 +643,7 @@ void WorkloadEntityStorageBase::applyEvent( auto it = entities.find(event.name); chassert(it != entities.end()); - LOG_DEBUG(log, "Drop entity: {}", event.name); + LOG_DEBUG(log, "Drop workload entity: {}", event.name); if (event.name == root_name) root_name.clear(); From de046be699582986482dff34ff4427ecf01f2bf9 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 15:13:33 +0000 Subject: [PATCH 505/816] change mul to scale --- src/Common/FieldVisitorMul.cpp | 50 ----------------- src/Common/FieldVisitorMul.h | 53 ------------------- src/Common/FieldVisitorScale.cpp | 30 +++++++++++ src/Common/FieldVisitorScale.h | 46 ++++++++++++++++ .../Transforms/FillingTransform.cpp | 4 +- 5 files changed, 78 insertions(+), 105 deletions(-) delete mode 100644 src/Common/FieldVisitorMul.cpp delete mode 100644 src/Common/FieldVisitorMul.h create mode 100644 src/Common/FieldVisitorScale.cpp create mode 100644 src/Common/FieldVisitorScale.h diff --git a/src/Common/FieldVisitorMul.cpp b/src/Common/FieldVisitorMul.cpp deleted file mode 100644 index 36c32c40c05..00000000000 --- a/src/Common/FieldVisitorMul.cpp +++ /dev/null @@ -1,50 +0,0 @@ -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -FieldVisitorMul::FieldVisitorMul(const Field & rhs_) : rhs(rhs_) {} - -// We can add all ints as unsigned regardless of their actual signedness. -bool FieldVisitorMul::operator() (Int64 & x) const { return this->operator()(reinterpret_cast(x)); } -bool FieldVisitorMul::operator() (UInt64 & x) const -{ - x *= applyVisitor(FieldVisitorConvertToNumber(), rhs); - return x != 0; -} - -bool FieldVisitorMul::operator() (Float64 & x) const { - x *= rhs.safeGet(); - return x != 0; -} - -bool FieldVisitorMul::operator() (Null &) const -{ - /// Do not add anything - return false; -} - -bool FieldVisitorMul::operator() (String &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Strings"); } -bool FieldVisitorMul::operator() (Array &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Arrays"); } -bool FieldVisitorMul::operator() (Tuple &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Tuples"); } -bool FieldVisitorMul::operator() (Map &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Maps"); } -bool FieldVisitorMul::operator() (Object &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Objects"); } -bool FieldVisitorMul::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply UUIDs"); } -bool FieldVisitorMul::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv4s"); } -bool FieldVisitorMul::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv6s"); } -bool FieldVisitorMul::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply custom type {}", x.getTypeName()); } - -bool FieldVisitorMul::operator() (AggregateFunctionStateData &) const -{ - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply AggregateFunctionStates"); -} - -bool FieldVisitorMul::operator() (bool &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Bools"); } - -} diff --git a/src/Common/FieldVisitorMul.h b/src/Common/FieldVisitorMul.h deleted file mode 100644 index 5bce41f1e71..00000000000 --- a/src/Common/FieldVisitorMul.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Implements `*=` operation. - * Returns false if the result is zero. - */ -class FieldVisitorMul : public StaticVisitor -{ -private: - const Field & rhs; -public: - explicit FieldVisitorMul(const Field & rhs_); - - // We can add all ints as unsigned regardless of their actual signedness. - bool operator() (Int64 & x) const; - bool operator() (UInt64 & x) const; - bool operator() (Float64 & x) const; - bool operator() (Null &) const; - bool operator() (String &) const; - bool operator() (Array &) const; - bool operator() (Tuple &) const; - bool operator() (Map &) const; - bool operator() (Object &) const; - bool operator() (UUID &) const; - bool operator() (IPv4 &) const; - bool operator() (IPv6 &) const; - bool operator() (AggregateFunctionStateData &) const; - bool operator() (CustomType &) const; - bool operator() (bool &) const; - - template - bool operator() (DecimalField & x) const - { - x *= rhs.safeGet>(); - return x.getValue() != T(0); - } - - template - requires is_big_int_v - bool operator() (T & x) const - { - x *= applyVisitor(FieldVisitorConvertToNumber(), rhs); - return x != T(0); - } -}; - -} diff --git a/src/Common/FieldVisitorScale.cpp b/src/Common/FieldVisitorScale.cpp new file mode 100644 index 00000000000..fdb566007c3 --- /dev/null +++ b/src/Common/FieldVisitorScale.cpp @@ -0,0 +1,30 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +FieldVisitorScale::FieldVisitorScale(Int32 rhs_) : rhs(rhs_) {} + +void FieldVisitorScale::operator() (Int64 & x) const { x *= rhs; } +void FieldVisitorScale::operator() (UInt64 & x) const { x *= rhs; } +void FieldVisitorScale::operator() (Float64 & x) const { x *= rhs; } +void FieldVisitorScale::operator() (Null &) const { /*Do not scale anything*/ } + +void FieldVisitorScale::operator() (String &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Strings"); } +void FieldVisitorScale::operator() (Array &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Arrays"); } +void FieldVisitorScale::operator() (Tuple &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Tuples"); } +void FieldVisitorScale::operator() (Map &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Maps"); } +void FieldVisitorScale::operator() (Object &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Objects"); } +void FieldVisitorScale::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply UUIDs"); } +void FieldVisitorScale::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv4s"); } +void FieldVisitorScale::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv6s"); } +void FieldVisitorScale::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply custom type {}", x.getTypeName()); } +void FieldVisitorScale::operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply AggregateFunctionStates"); } +void FieldVisitorScale::operator() (bool &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Bools"); } + +} diff --git a/src/Common/FieldVisitorScale.h b/src/Common/FieldVisitorScale.h new file mode 100644 index 00000000000..45bacdccc9c --- /dev/null +++ b/src/Common/FieldVisitorScale.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include "base/Decimal.h" +#include "base/extended_types.h" + +namespace DB +{ + +/** Implements `*=` operation by number + */ +class FieldVisitorScale : public StaticVisitor +{ +private: + Int32 rhs; + +public: + explicit FieldVisitorScale(Int32 rhs_); + + void operator() (Int64 & x) const; + void operator() (UInt64 & x) const; + void operator() (Float64 & x) const; + void operator() (Null &) const; + [[noreturn]] void operator() (String &) const; + [[noreturn]] void operator() (Array &) const; + [[noreturn]] void operator() (Tuple &) const; + [[noreturn]] void operator() (Map &) const; + [[noreturn]] void operator() (Object &) const; + [[noreturn]] void operator() (UUID &) const; + [[noreturn]] void operator() (IPv4 &) const; + [[noreturn]] void operator() (IPv6 &) const; + [[noreturn]] void operator() (AggregateFunctionStateData &) const; + [[noreturn]] void operator() (CustomType &) const; + [[noreturn]] void operator() (bool &) const; + + template + void operator() (DecimalField & x) const { x = DecimalField(x.getValue() * T(rhs), x.getScale()); } + + template + requires is_big_int_v + void operator() (T & x) const { x *= rhs; } +}; + +} diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 1d68f73e8c2..54331186302 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -123,7 +123,7 @@ static FillColumnDescription::StepFunction getStepFunction(const Field & step, c { auto shifted_step = step; if (jumps_count != 1) - applyVisitor(FieldVisitorMul(jumps_count), shifted_step); + applyVisitor(FieldVisitorScale(jumps_count), shifted_step); logDebug("field", field.dump()); logDebug("step", step.dump()); From f8c13061a743fc162fc0094ceb773292df4677f6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Oct 2024 15:20:01 +0000 Subject: [PATCH 506/816] fix optimization of replacing algorithm --- .../Algorithms/ReplacingSortedAlgorithm.cpp | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index b22f1271687..5059bc806a8 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -5,6 +5,7 @@ #include #include #include +#include "Common/Logger.h" #include namespace DB @@ -122,7 +123,25 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() return Status(current.impl->order); } - if (current.impl->isFirst() + RowRef current_row; + setRowRef(current_row, current); + + bool key_differs = selected_row.empty() || rowsHaveDifferentSortColumns(selected_row, current_row); + if (key_differs) + { + /// If there are enough rows and the last one is calculated completely + if (merged_data->hasEnoughRows()) + return Status(merged_data->pull()); + + /// Write the data for the previous primary key. + if (!selected_row.empty()) + insertRow(); + + selected_row.clear(); + } + + if (current->isFirst() + && key_differs && is_deleted_column_number == -1 /// Ignore optimization if we need to filter deleted rows. && sources_origin_merge_tree_part_level[current->order] > 0 && !skipLastRowFor(current->order) /// Ignore optimization if last row should be skipped. @@ -152,9 +171,9 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() std::iota(replace_final_data.begin(), replace_final_data.end(), 0); current_chunk.getChunkInfos().add(std::make_shared(std::move(replace_final_selection))); - Status status(merged_data->pull(), false); + Status status(std::move(current_chunk), false); status.required_source = source_num; - return Status(std::move(current_chunk), false); + return status; } merged_data->insertChunk(std::move(current_chunk), chunk_num_rows); @@ -174,23 +193,6 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() return status; } - RowRef current_row; - setRowRef(current_row, current); - - bool key_differs = selected_row.empty() || rowsHaveDifferentSortColumns(selected_row, current_row); - if (key_differs) - { - /// If there are enough rows and the last one is calculated completely - if (merged_data->hasEnoughRows()) - return Status(merged_data->pull()); - - /// Write the data for the previous primary key. - if (!selected_row.empty()) - insertRow(); - - selected_row.clear(); - } - /// Initially, skip all rows. Unskip last on insert. size_t current_pos = current_row_sources.size(); if (out_row_sources_buf) From 961fa4d5ce0d7d7494b358d927e617dc7ed19eb5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 15:21:56 +0000 Subject: [PATCH 507/816] Use the proper timestamp from where the query_info is collected --- src/Interpreters/QueryMetricLog.cpp | 10 +++++----- src/Interpreters/QueryMetricLog.h | 4 ++-- src/Interpreters/executeQuery.cpp | 14 ++++++++------ 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 3b983e61dda..1499c32f53e 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -86,11 +86,11 @@ void QueryMetricLog::shutdown() Base::shutdown(); } -void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds) +void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, UInt64 interval_milliseconds) { QueryMetricLogStatus status; status.interval_milliseconds = interval_milliseconds; - status.next_collect_time = query_start_time + std::chrono::milliseconds(interval_milliseconds); + status.next_collect_time = start_time + std::chrono::milliseconds(interval_milliseconds); auto context = getContext(); const auto & process_list = context->getProcessList(); @@ -115,7 +115,7 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t queries.emplace(query_id, std::move(status)); } -void QueryMetricLog::finishQuery(const String & query_id, QueryStatusInfoPtr query_info) +void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time, QueryStatusInfoPtr query_info) { std::unique_lock lock(queries_mutex); auto it = queries.find(query_id); @@ -127,7 +127,7 @@ void QueryMetricLog::finishQuery(const String & query_id, QueryStatusInfoPtr que if (query_info) { - auto elem = createLogMetricElement(query_id, *query_info, std::chrono::system_clock::now(), false); + auto elem = createLogMetricElement(query_id, *query_info, finish_time, false); if (elem) add(std::move(elem.value())); } @@ -187,7 +187,7 @@ std::optional QueryMetricLog::createLogMetricElement(cons elem.profile_events = query_status.last_profile_events; } - if (query_status.task && schedule_next) + if (schedule_next) { query_status.next_collect_time += std::chrono::milliseconds(query_status.interval_milliseconds); const auto wait_time = std::chrono::duration_cast(query_status.next_collect_time - std::chrono::system_clock::now()).count(); diff --git a/src/Interpreters/QueryMetricLog.h b/src/Interpreters/QueryMetricLog.h index d7642bf0ab1..37797cfce65 100644 --- a/src/Interpreters/QueryMetricLog.h +++ b/src/Interpreters/QueryMetricLog.h @@ -52,8 +52,8 @@ public: void shutdown() final; // Both startQuery and finishQuery are called from the thread that executes the query - void startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds); - void finishQuery(const String & query_id, QueryStatusInfoPtr query_info = nullptr); + void startQuery(const String & query_id, TimePoint start_time, UInt64 interval_milliseconds); + void finishQuery(const String & query_id, TimePoint finish_time, QueryStatusInfoPtr query_info = nullptr); private: std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time, bool schedule_next = true); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a8fcfff65ad..9250c069283 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -81,6 +81,7 @@ #include #include +#include #include #include @@ -460,7 +461,7 @@ QueryLogElement logQueryStart( return elem; } -void logQueryMetricLogFinish(ContextPtr context, bool internal, String query_id, QueryStatusInfoPtr info) +void logQueryMetricLogFinish(ContextPtr context, bool internal, String query_id, std::chrono::system_clock::time_point finish_time, QueryStatusInfoPtr info) { if (auto query_metric_log = context->getQueryMetricLog(); query_metric_log && !internal) { @@ -475,11 +476,11 @@ void logQueryMetricLogFinish(ContextPtr context, bool internal, String query_id, /// to query the final state in query_log. auto collect_on_finish = info->elapsed_microseconds > interval_milliseconds * 1000; auto query_info = collect_on_finish ? info : nullptr; - query_metric_log->finishQuery(query_id, query_info); + query_metric_log->finishQuery(query_id, finish_time, query_info); } else { - query_metric_log->finishQuery(query_id, nullptr); + query_metric_log->finishQuery(query_id, finish_time, nullptr); } } } @@ -503,6 +504,7 @@ void logQueryFinish( /// Update performance counters before logging to query_log CurrentThread::finalizePerformanceCounters(); + auto time_now = std::chrono::system_clock::now(); QueryStatusInfo info = process_list_elem->getInfo(true, settings[Setting::log_profile_events]); elem.type = QueryLogElementType::QUERY_FINISH; @@ -597,7 +599,7 @@ void logQueryFinish( } } - logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, std::make_shared(info)); + logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, std::make_shared(info)); } if (query_span) @@ -697,7 +699,7 @@ void logQueryException( query_span->finish(); } - logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, info); + logQueryMetricLogFinish(context, internal, elem.client_info.current_query_id, time_now, info); } void logExceptionBeforeStart( @@ -796,7 +798,7 @@ void logExceptionBeforeStart( } } - logQueryMetricLogFinish(context, false, elem.client_info.current_query_id, nullptr); + logQueryMetricLogFinish(context, false, elem.client_info.current_query_id, std::chrono::system_clock::now(), nullptr); } void validateAnalyzerSettings(ASTPtr ast, bool context_value) From 66013a2bb9e8ff567e1e2ac448fdee7cce7465a3 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 28 Oct 2024 15:38:32 +0000 Subject: [PATCH 508/816] Fix for collecting old metrics already collected --- src/Interpreters/QueryMetricLog.cpp | 40 +++++++++++++++++++++++------ src/Interpreters/QueryMetricLog.h | 3 ++- 2 files changed, 34 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 1499c32f53e..54a09efba7b 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +}; + static auto logger = getLogger("QueryMetricLog"); ColumnsDescription QueryMetricLogElement::getColumnsDescription() @@ -106,8 +111,6 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, U auto elem = createLogMetricElement(query_id, *query_info, current_time); if (elem) add(std::move(elem.value())); - else - LOG_TRACE(logger, "Query {} finished already while this collecting task was running", query_id); }); std::lock_guard lock(queries_mutex); @@ -154,37 +157,58 @@ void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time, queries.erase(query_id); } -std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time, bool schedule_next) +std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next) { LOG_DEBUG(logger, "Collecting query_metric_log for query {}. Schedule next: {}", query_id, schedule_next); - std::lock_guard lock(queries_mutex); + std::unique_lock lock(queries_mutex); auto query_status_it = queries.find(query_id); /// The query might have finished while the scheduled task is running. if (query_status_it == queries.end() || !query_status_it->second.task) + { + lock.unlock(); + LOG_TRACE(logger, "Query {} finished already while this collecting task was running", query_id); return {}; + } + + auto & query_status = query_status_it->second; + if (query_info_time < query_status.last_collect_time) + { + lock.unlock(); + LOG_TRACE(logger, "Query {} has a more recent metrics collected. Skipping this one", query_id); + return {}; + } + + query_status.last_collect_time = query_info_time; QueryMetricLogElement elem; - elem.event_time = timeInSeconds(current_time); - elem.event_time_microseconds = timeInMicroseconds(current_time); + elem.event_time = timeInSeconds(query_info_time); + elem.event_time_microseconds = timeInMicroseconds(query_info_time); elem.query_id = query_status_it->first; elem.memory_usage = query_info.memory_usage > 0 ? query_info.memory_usage : 0; elem.peak_memory_usage = query_info.peak_memory_usage > 0 ? query_info.peak_memory_usage : 0; - auto & query_status = query_status_it->second; if (query_info.profile_counters) { for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto & new_value = (*(query_info.profile_counters))[i]; auto & prev_value = query_status.last_profile_events[i]; + + /// Profile event count is monotonically increasing. + if (new_value < prev_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", + ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); + elem.profile_events[i] = new_value - prev_value; prev_value = new_value; } } else { - elem.profile_events = query_status.last_profile_events; + LOG_TRACE(logger, "Query {} has no profile counters", query_id); + elem.profile_events = std::vector(ProfileEvents::end()); } if (schedule_next) diff --git a/src/Interpreters/QueryMetricLog.h b/src/Interpreters/QueryMetricLog.h index 37797cfce65..802cee7bf26 100644 --- a/src/Interpreters/QueryMetricLog.h +++ b/src/Interpreters/QueryMetricLog.h @@ -37,6 +37,7 @@ struct QueryMetricLogElement struct QueryMetricLogStatus { UInt64 interval_milliseconds; + std::chrono::system_clock::time_point last_collect_time; std::chrono::system_clock::time_point next_collect_time; std::vector last_profile_events = std::vector(ProfileEvents::end()); BackgroundSchedulePool::TaskHolder task; @@ -56,7 +57,7 @@ public: void finishQuery(const String & query_id, TimePoint finish_time, QueryStatusInfoPtr query_info = nullptr); private: - std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time, bool schedule_next = true); + std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next = true); std::recursive_mutex queries_mutex; std::unordered_map queries; From ab46e9b1794ae864b9c19139d24058573bf9ff11 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 28 Oct 2024 16:58:30 +0100 Subject: [PATCH 509/816] Check if default DB exists after authorization --- src/Server/TCPHandler.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 921c53b6bcb..1e7a2fc3b6c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1604,6 +1604,7 @@ void TCPHandler::receiveHello() session->authenticate( SSLCertificateCredentials{user, extractSSLCertificateSubjects(secure_socket.peerCertificate())}, getClientAddress(client_info)); + DatabaseCatalog::instance().assertDatabaseExists(default_db); return; } catch (const Exception & e) @@ -1671,11 +1672,13 @@ void TCPHandler::receiveHello() auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge)); session->authenticate(cred, getClientAddress(client_info)); + DatabaseCatalog::instance().assertDatabaseExists(default_db); return; } #endif session->authenticate(user, password, getClientAddress(client_info)); + DatabaseCatalog::instance().assertDatabaseExists(default_db); } void TCPHandler::receiveAddendum() From 700f04b15e44a9242465a4ebdb89818829bec55a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 28 Oct 2024 17:02:48 +0100 Subject: [PATCH 510/816] Wrong field --- src/Server/TCPHandler.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1e7a2fc3b6c..d57238285c3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1604,7 +1604,7 @@ void TCPHandler::receiveHello() session->authenticate( SSLCertificateCredentials{user, extractSSLCertificateSubjects(secure_socket.peerCertificate())}, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_db); + DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } catch (const Exception & e) @@ -1672,13 +1672,13 @@ void TCPHandler::receiveHello() auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge)); session->authenticate(cred, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_db); + DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } #endif session->authenticate(user, password, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_db); + DatabaseCatalog::instance().assertDatabaseExists(default_database); } void TCPHandler::receiveAddendum() From 8f6439e4e91c49eff8faf10717a2d1d19f03aad3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Oct 2024 17:12:17 +0100 Subject: [PATCH 511/816] Allow to change more settings, better tests --- .../ObjectStorageQueueMetadata.cpp | 13 +- .../ObjectStorageQueueSettings.cpp | 6 +- .../ObjectStorageQueueTableMetadata.h | 19 +++ .../StorageObjectStorageQueue.cpp | 46 +++++- .../StorageObjectStorageQueue.h | 6 +- .../integration/test_storage_s3_queue/test.py | 139 +++++++++--------- 6 files changed, 145 insertions(+), 84 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 525ca1e484b..692f001dd7b 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -248,7 +248,10 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) for (const auto & change : changes) { - if (endsWith(change.name, "processing_threads_num")) + if (!ObjectStorageQueueTableMetadata::isStoredInKeeper(change.name)) + continue; + + if (change.name == "processing_threads_num") { const auto value = change.value.safeGet(); if (table_metadata.processing_threads_num == value) @@ -259,7 +262,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) } new_table_metadata.processing_threads_num = value; } - else if (endsWith(change.name, "loading_retries")) + else if (change.name == "loading_retries") { const auto value = change.value.safeGet(); if (table_metadata.loading_retries == value) @@ -270,7 +273,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) } new_table_metadata.loading_retries = value; } - else if (endsWith(change.name, "after_processing")) + else if (change.name == "after_processing") { const auto value = ObjectStorageQueueTableMetadata::actionFromString(change.value.safeGet()); if (table_metadata.after_processing == value) @@ -281,7 +284,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) } new_table_metadata.after_processing = value; } - else if (endsWith(change.name, "tracked_files_limit")) + else if (change.name == "tracked_files_limit") { const auto value = change.value.safeGet(); if (table_metadata.tracked_files_limit == value) @@ -292,7 +295,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) } new_table_metadata.tracked_files_limit = value; } - else if (endsWith(change.name, "tracked_file_ttl_sec")) + else if (change.name == "tracked_file_ttl_sec") { const auto value = change.value.safeGet(); if (table_metadata.tracked_files_ttl_sec == value) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp index 338f575721a..060f1cd2dd5 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSettings.cpp @@ -29,9 +29,9 @@ namespace ErrorCodes DECLARE(String, last_processed_path, "", "For Ordered mode. Files that have lexicographically smaller file name are considered already processed", 0) \ DECLARE(UInt64, tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ DECLARE(UInt64, tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ - DECLARE(UInt32, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ - DECLARE(UInt32, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - DECLARE(UInt32, polling_backoff_ms, 1000, "Polling backoff", 0) \ + DECLARE(UInt64, polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ + DECLARE(UInt64, polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ + DECLARE(UInt64, polling_backoff_ms, 1000, "Polling backoff", 0) \ DECLARE(UInt32, cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \ DECLARE(UInt32, cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \ DECLARE(UInt32, buckets, 0, "Number of buckets for Ordered mode parallel processing", 0) \ diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h index 3a07d4690fc..6dfc705a7b6 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h @@ -77,6 +77,25 @@ struct ObjectStorageQueueTableMetadata void checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const; + static bool isStoredInKeeper(const std::string & name) + { + static const std::unordered_set settings_names + { + "format_name", + "columns", + "mode", + "buckets", + "last_processed_path", + "after_processing", + "loading_retries", + "processing_threads_num", + "tracked_files_limit", + "tracked_file_ttl_sec", + "tracked_files_ttl_sec", + }; + return settings_names.contains(name); + } + private: void checkImmutableFieldsEquals(const ObjectStorageQueueTableMetadata & from_zk) const; }; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 5124a4a7641..200872a2f4c 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -52,9 +52,9 @@ namespace ObjectStorageQueueSetting extern const ObjectStorageQueueSettingsUInt64 max_processed_files_before_commit; extern const ObjectStorageQueueSettingsUInt64 max_processed_rows_before_commit; extern const ObjectStorageQueueSettingsUInt64 max_processing_time_sec_before_commit; - extern const ObjectStorageQueueSettingsUInt32 polling_min_timeout_ms; - extern const ObjectStorageQueueSettingsUInt32 polling_max_timeout_ms; - extern const ObjectStorageQueueSettingsUInt32 polling_backoff_ms; + extern const ObjectStorageQueueSettingsUInt64 polling_min_timeout_ms; + extern const ObjectStorageQueueSettingsUInt64 polling_max_timeout_ms; + extern const ObjectStorageQueueSettingsUInt64 polling_backoff_ms; extern const ObjectStorageQueueSettingsUInt64 processing_threads_num; extern const ObjectStorageQueueSettingsUInt32 buckets; extern const ObjectStorageQueueSettingsUInt64 tracked_file_ttl_sec; @@ -565,21 +565,33 @@ static const std::unordered_set changeable_settings_unordered_ "after_processing", "tracked_files_limit", "tracked_file_ttl_sec", + "polling_min_timeout_ms", + "polling_max_timeout_ms", + "polling_backoff_ms", /// For compatibility. "s3queue_processing_threads_num", "s3queue_loading_retries", "s3queue_after_processing", "s3queue_tracked_files_limit", "s3queue_tracked_file_ttl_sec", + "s3queue_polling_min_timeout_ms", + "s3queue_polling_max_timeout_ms", + "s3queue_polling_backoff_ms", }; static const std::unordered_set changeable_settings_ordered_mode { "loading_retries", "after_processing", + "polling_min_timeout_ms", + "polling_max_timeout_ms", + "polling_backoff_ms", /// For compatibility. "s3queue_loading_retries", "s3queue_after_processing", + "s3queue_polling_min_timeout_ms", + "s3queue_polling_max_timeout_ms", + "s3queue_polling_backoff_ms", }; static bool isSettingChangeable(const std::string & name, ObjectStorageQueueMode mode) @@ -660,6 +672,8 @@ void StorageObjectStorageQueue::alter( } SettingsChanges changed_settings; + std::set changed_settings_set; + const auto mode = getTableMetadata().getMode(); for (const auto & setting : new_settings) { @@ -679,11 +693,31 @@ void StorageObjectStorageQueue::alter( setting.name, magic_enum::enum_name(mode), getName()); } - changed_settings.push_back(setting); + SettingChange result_setting(setting); + if (result_setting.name.starts_with("s3queue_")) + result_setting.name = result_setting.name.substr(std::strlen("s3queue_")); + + changed_settings.push_back(result_setting); + + auto inserted = changed_settings_set.emplace(result_setting.name).second; + if (!inserted) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting {} is duplicated", setting.name); } + /// Alter settings which are stored in keeper. files_metadata->alterSettings(changed_settings); + /// Alter settings which are not stored in keeper. + for (const auto & change : changed_settings) + { + if (change.name == "polling_min_timeout_ms") + polling_min_timeout_ms = change.value.safeGet(); + if (change.name == "polling_max_timeout_ms") + polling_max_timeout_ms = change.value.safeGet(); + if (change.name == "polling_backoff_ms") + polling_backoff_ms = change.value.safeGet(); + } + StorageInMemoryMetadata metadata = getInMemoryMetadata(); metadata.setSettingsChanges(new_metadata.settings_changes); setInMemoryMetadata(metadata); @@ -719,8 +753,8 @@ ObjectStorageQueueSettings StorageObjectStorageQueue::getSettings() const settings[ObjectStorageQueueSetting::processing_threads_num] = table_metadata.processing_threads_num; settings[ObjectStorageQueueSetting::enable_logging_to_queue_log] = enable_logging_to_queue_log; settings[ObjectStorageQueueSetting::last_processed_path] = table_metadata.last_processed_path; - settings[ObjectStorageQueueSetting::tracked_file_ttl_sec] = 0; - settings[ObjectStorageQueueSetting::tracked_files_limit] = 0; + settings[ObjectStorageQueueSetting::tracked_file_ttl_sec] = table_metadata.tracked_files_ttl_sec; + settings[ObjectStorageQueueSetting::tracked_files_limit] = table_metadata.tracked_files_limit; settings[ObjectStorageQueueSetting::polling_min_timeout_ms] = polling_min_timeout_ms; settings[ObjectStorageQueueSetting::polling_max_timeout_ms] = polling_max_timeout_ms; settings[ObjectStorageQueueSetting::polling_backoff_ms] = polling_backoff_ms; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h index 08eb32928b3..371e409825f 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h @@ -72,9 +72,9 @@ private: const std::string engine_name; const fs::path zk_path; const bool enable_logging_to_queue_log; - const UInt32 polling_min_timeout_ms; - const UInt32 polling_max_timeout_ms; - const UInt32 polling_backoff_ms; + UInt64 polling_min_timeout_ms; + UInt64 polling_max_timeout_ms; + UInt64 polling_backoff_ms; const CommitSettings commit_settings; std::shared_ptr files_metadata; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 4e7c459e1ed..c495fc1d44f 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -2172,51 +2172,73 @@ def test_alter_settings(started_cluster): node1.query( f""" ALTER TABLE r.{table_name} - MODIFY SETTING processing_threads_num=5, loading_retries=10, after_processing='delete', tracked_files_limit=50, tracked_file_ttl_sec=10000 + MODIFY SETTING processing_threads_num=5, + loading_retries=10, + after_processing='delete', + tracked_files_limit=50, + tracked_file_ttl_sec=10000, + polling_min_timeout_ms=222, + polling_max_timeout_ms=333, + polling_backoff_ms=111 """ ) - assert '"processing_threads_num":5' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + int_settings = { + "processing_threads_num": 5, + "loading_retries": 10, + "tracked_files_ttl_sec": 10000, + "tracked_files_limit": 50, + "polling_min_timeout_ms": 222, + "polling_max_timeout_ms": 333, + "polling_backoff_ms": 111, + } + string_settings = {"after_processing": "delete"} - assert '"loading_retries":10' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + def with_keeper(setting): + return setting in { + "after_processing", + "loading_retries", + "processing_threads_num", + "tracked_files_limit", + "tracked_files_ttl_sec", + } - assert '"after_processing":"delete"' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + def check_int_settings(node, settings): + for setting, value in settings.items(): + if with_keeper(setting): + assert f'"{setting}":{value}' in node.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + if setting == "tracked_files_ttl_sec": + setting = "tracked_file_ttl_sec" + assert ( + str(value) + == node.query( + f"SELECT value FROM system.s3_queue_settings WHERE name = '{setting}' and table = '{table_name}'" + ).strip() + ) - assert '"tracked_files_ttl_sec":10000' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + def check_string_settings(node, settings): + for setting, value in settings.items(): + if with_keeper(setting): + assert f'"{setting}":"{value}"' in node.query( + f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" + ) + assert ( + str(value) + == node.query( + f"SELECT value FROM system.s3_queue_settings WHERE name = '{setting}' and table = '{table_name}'" + ).strip() + ) - assert '"tracked_files_limit":50' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + for node in [node1, node2]: + check_int_settings(node, int_settings) + check_string_settings(node, string_settings) - node1.restart_clickhouse() + node.restart_clickhouse() - assert '"processing_threads_num":5' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"loading_retries":10' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"after_processing":"delete"' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"tracked_files_ttl_sec":10000' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"tracked_files_limit":50' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + check_int_settings(node, int_settings) + check_string_settings(node, string_settings) node1.query( f""" @@ -2224,37 +2246,20 @@ def test_alter_settings(started_cluster): """ ) - assert '"processing_threads_num":5' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + int_settings = { + "processing_threads_num": 5, + "loading_retries": 10, + "tracked_files_ttl_sec": 0, + "tracked_files_limit": 50, + } + string_settings = {"after_processing": "keep"} - assert '"loading_retries":10' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + for node in [node1, node2]: + check_int_settings(node, int_settings) + check_string_settings(node, string_settings) - assert '"after_processing":"keep"' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + node.restart_clickhouse() + assert expected_rows == get_count() - assert '"tracked_files_ttl_sec":0' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - node1.restart_clickhouse() - assert expected_rows == get_count() - - assert '"processing_threads_num":5' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"loading_retries":10' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"after_processing":"keep"' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) - - assert '"tracked_files_ttl_sec":0' in node1.query( - f"SELECT * FROM system.zookeeper WHERE path = '{keeper_path}'" - ) + check_int_settings(node, int_settings) + check_string_settings(node, string_settings) From 1e7b7d4aa9ce73350703e8272516d9038c3aacc9 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 28 Oct 2024 17:16:44 +0100 Subject: [PATCH 512/816] Check for an empty string --- src/Server/TCPHandler.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d57238285c3..afca8b4ab25 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1604,7 +1604,8 @@ void TCPHandler::receiveHello() session->authenticate( SSLCertificateCredentials{user, extractSSLCertificateSubjects(secure_socket.peerCertificate())}, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_database); + if (!default_database.empty()) + DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } catch (const Exception & e) @@ -1672,13 +1673,15 @@ void TCPHandler::receiveHello() auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge)); session->authenticate(cred, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_database); + if (!default_database.empty()) + DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } #endif session->authenticate(user, password, getClientAddress(client_info)); - DatabaseCatalog::instance().assertDatabaseExists(default_database); + if (!default_database.empty()) + DatabaseCatalog::instance().assertDatabaseExists(default_database); } void TCPHandler::receiveAddendum() From aa4aa91ae87e3b565c552fff35d6fede169e5bf3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Oct 2024 16:23:19 +0000 Subject: [PATCH 513/816] better prewarm of mark cache --- src/Core/ServerSettings.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 7c2cb49a2ba..4f5542e041e 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -99,6 +99,7 @@ namespace DB DECLARE(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \ DECLARE(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ DECLARE(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \ + DECLARE(Double, mark_cache_prewarm_ratio, 0.95, "The ratio of total size of mark cache to fill during prewarm.", 0) \ DECLARE(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \ DECLARE(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \ DECLARE(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b80eab324bd..f877b74c5ff 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1309,11 +1309,11 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks() void InterpreterSystemQuery::prewarmMarkCache() { - getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE); - if (table_id.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for prewarming marks cache"); + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); + auto table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); auto * merge_tree = dynamic_cast(table_ptr.get()); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b89d23fb4f0..b3c017e98b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -154,6 +155,7 @@ namespace namespace DB { + namespace Setting { extern const SettingsBool allow_drop_detached; @@ -232,6 +234,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool prewarm_mark_cache; } +namespace ServerSetting +{ + extern const ServerSettingsDouble mark_cache_prewarm_ratio; +} + namespace ErrorCodes { extern const int NO_SUCH_DATA_PART; @@ -2370,10 +2377,11 @@ void MergeTreeData::prewarmMarkCache(ThreadPool & pool) }); ThreadPoolCallbackRunnerLocal runner(pool, "PrewarmMarks"); + double ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio]; for (const auto & part : data_parts) { - if (mark_cache->sizeInBytes() >= mark_cache->maxSizeInBytes() * 0.95) + if (mark_cache->sizeInBytes() >= mark_cache->maxSizeInBytes() * ratio_to_prewarm) break; runner([&] { part->loadMarksToCache(column_names, mark_cache); }); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 93e72f3e0bf..fc3245eafcf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5082,7 +5082,7 @@ bool StorageReplicatedMergeTree::fetchPart( ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } - if ((*getSettings())[MergeTreeSetting::prewarm_mark_cache]) + if ((*getSettings())[MergeTreeSetting::prewarm_mark_cache] && getContext()->getMarkCache()) { auto column_names = getColumnsToPrewarmMarks(*getSettings(), part->getColumns()); part->loadMarksToCache(column_names, getContext()->getMarkCache().get()); From 5e9aa01f33a2a5745e4d4a131f3d3ddbe84a5808 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:25:52 +0100 Subject: [PATCH 514/816] Introduce Setting tiers --- docs/en/operations/system-tables/settings.md | 105 ++++++++++++++++-- src/Core/BaseSettings.cpp | 9 ++ src/Core/BaseSettings.h | 59 +++++----- src/Core/ServerSettings.cpp | 2 +- src/Core/Settings.cpp | 18 +-- src/Core/SettingsObsoleteMacros.h | 4 +- src/Core/SettingsTierType.cpp | 19 ++++ src/Core/SettingsTierType.h | 26 +++++ src/Storages/MergeTree/MergeTreeSettings.cpp | 4 +- src/Storages/System/StorageSystemSettings.cpp | 10 ++ 10 files changed, 206 insertions(+), 50 deletions(-) create mode 100644 src/Core/SettingsTierType.cpp create mode 100644 src/Core/SettingsTierType.h diff --git a/docs/en/operations/system-tables/settings.md b/docs/en/operations/system-tables/settings.md index a04e095e990..1cfee0ba5f4 100644 --- a/docs/en/operations/system-tables/settings.md +++ b/docs/en/operations/system-tables/settings.md @@ -18,6 +18,11 @@ Columns: - `1` — Current user can’t change the setting. - `default` ([String](../../sql-reference/data-types/string.md)) — Setting default value. - `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete. +- `tier` ([Enum8](../../sql-reference/data-types/enum.md)) — Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their development and the expectations one might have when using them. Values: + - `'Production'` — The feature is stable, safe to use and does not have issues interacting with other **production** features. . + - `'Beta'` — The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. + - `'Experimental'` — The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. + - `'Obsolete'` — No longer supported. Either it is already removed or it will be removed in future releases. **Example** @@ -26,19 +31,99 @@ The following example shows how to get information about settings which name con ``` sql SELECT * FROM system.settings -WHERE name LIKE '%min_i%' +WHERE name LIKE '%min_insert_block_size_%' +FORMAT Vertical ``` ``` text -┌─name───────────────────────────────────────────────_─value─────_─changed─_─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────_─min──_─max──_─readonly─_─type─────────_─default───_─alias_for─_─is_obsolete─┐ -│ min_insert_block_size_rows │ 1048449 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ____ │ ____ │ 0 │ UInt64 │ 1048449 │ │ 0 │ -│ min_insert_block_size_bytes │ 268402944 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ____ │ ____ │ 0 │ UInt64 │ 268402944 │ │ 0 │ -│ min_insert_block_size_rows_for_materialized_views │ 0 │ 0 │ Like min_insert_block_size_rows, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_rows) │ ____ │ ____ │ 0 │ UInt64 │ 0 │ │ 0 │ -│ min_insert_block_size_bytes_for_materialized_views │ 0 │ 0 │ Like min_insert_block_size_bytes, but applied only during pushing to MATERIALIZED VIEW (default: min_insert_block_size_bytes) │ ____ │ ____ │ 0 │ UInt64 │ 0 │ │ 0 │ -│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ____ │ ____ │ 0 │ Milliseconds │ 1000 │ │ 0 │ -└────────────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────── -──────────────────────────────────────────────────────┴──────┴──────┴──────────┴──────────────┴───────────┴───────────┴─────────────┘ -``` +Row 1: +────── +name: min_insert_block_size_rows +value: 1048449 +changed: 0 +description: Sets the minimum number of rows in the block that can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. + +Possible values: + +- Positive integer. +- 0 — Squashing disabled. +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 1048449 +alias_for: +is_obsolete: 0 +tier: Production + +Row 2: +────── +name: min_insert_block_size_bytes +value: 268402944 +changed: 0 +description: Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. + +Possible values: + +- Positive integer. +- 0 — Squashing disabled. +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 268402944 +alias_for: +is_obsolete: 0 +tier: Production + +Row 3: +────── +name: min_insert_block_size_rows_for_materialized_views +value: 0 +changed: 0 +description: Sets the minimum number of rows in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. + +Possible values: + +- Any positive integer. +- 0 — Squashing disabled. + +**See Also** + +- [min_insert_block_size_rows](#min-insert-block-size-rows) +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 0 +alias_for: +is_obsolete: 0 +tier: Production + +Row 4: +────── +name: min_insert_block_size_bytes_for_materialized_views +value: 0 +changed: 0 +description: Sets the minimum number of bytes in the block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. + +Possible values: + +- Any positive integer. +- 0 — Squashing disabled. + +**See also** + +- [min_insert_block_size_bytes](#min-insert-block-size-bytes) +min: ᴺᵁᴸᴸ +max: ᴺᵁᴸᴸ +readonly: 0 +type: UInt64 +default: 0 +alias_for: +is_obsolete: 0 +tier: Production + ``` Using of `WHERE changed` can be useful, for example, when you want to check: diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index c535b9ce65e..7bfa581598d 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -8,6 +8,7 @@ namespace DB { namespace ErrorCodes { + extern const int INCORRECT_DATA; extern const int UNKNOWN_SETTING; } @@ -38,6 +39,14 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) return static_cast(res); } +SettingsTierType BaseSettingsHelpers::getTier(Flags flags) +{ + int8_t tier = (flags & Flags::TIER); + if (tier > SettingsTierType::OBSOLETE) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); + return SettingsTierType{tier}; +} + void BaseSettingsHelpers::throwSettingNotFound(std::string_view name) { diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 2a2e0bb334e..218460330f4 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -21,6 +22,27 @@ namespace DB class ReadBuffer; class WriteBuffer; +struct BaseSettingsHelpers +{ + [[noreturn]] static void throwSettingNotFound(std::string_view name); + static void warningSettingNotFound(std::string_view name); + + static void writeString(std::string_view str, WriteBuffer & out); + static String readString(ReadBuffer & in); + + enum Flags : UInt64 + { + IMPORTANT = 0x01, + CUSTOM = 0x02, + TIER = 0x0c, /// 0b1100 == 2 bits + /// If adding new flags, consider first if Tier might need more bits + }; + + static SettingsTierType getTier(Flags flags); + static void writeFlags(Flags flags, WriteBuffer & out); + static Flags readFlags(ReadBuffer & in); +}; + /** Template class to define collections of settings. * If you create a new setting, please also add it to ./utils/check-style/check-settings-style * for validation @@ -138,7 +160,7 @@ public: const char * getTypeName() const; const char * getDescription() const; bool isCustom() const; - bool isObsolete() const; + SettingsTierType getTier() const; bool operator==(const SettingFieldRef & other) const { return (getName() == other.getName()) && (getValue() == other.getValue()); } bool operator!=(const SettingFieldRef & other) const { return !(*this == other); } @@ -225,24 +247,6 @@ private: std::conditional_t custom_settings_map; }; -struct BaseSettingsHelpers -{ - [[noreturn]] static void throwSettingNotFound(std::string_view name); - static void warningSettingNotFound(std::string_view name); - - static void writeString(std::string_view str, WriteBuffer & out); - static String readString(ReadBuffer & in); - - enum Flags : UInt64 - { - IMPORTANT = 0x01, - CUSTOM = 0x02, - OBSOLETE = 0x04, - }; - static void writeFlags(Flags flags, WriteBuffer & out); - static Flags readFlags(ReadBuffer & in); -}; - template void BaseSettings::set(std::string_view name, const Field & value) { @@ -797,14 +801,14 @@ bool BaseSettings::SettingFieldRef::isCustom() const } template -bool BaseSettings::SettingFieldRef::isObsolete() const +SettingsTierType BaseSettings::SettingFieldRef::getTier() const { if constexpr (Traits::allow_custom_settings) { if (custom_setting) - return false; + return SettingsTierType::PRODUCTION; } - return accessor->isObsolete(index); + return accessor->getTier(index); } using AliasMap = std::unordered_map; @@ -835,8 +839,8 @@ using AliasMap = std::unordered_map; const String & getName(size_t index) const { return field_infos[index].name; } \ const char * getTypeName(size_t index) const { return field_infos[index].type; } \ const char * getDescription(size_t index) const { return field_infos[index].description; } \ - bool isImportant(size_t index) const { return field_infos[index].is_important; } \ - bool isObsolete(size_t index) const { return field_infos[index].is_obsolete; } \ + bool isImportant(size_t index) const { return field_infos[index].flags & BaseSettingsHelpers::Flags::IMPORTANT; } \ + SettingsTierType getTier(size_t index) const { return BaseSettingsHelpers::getTier(field_infos[index].flags); } \ Field castValueUtil(size_t index, const Field & value) const { return field_infos[index].cast_value_util_function(value); } \ String valueToStringUtil(size_t index, const Field & value) const { return field_infos[index].value_to_string_util_function(value); } \ Field stringToValueUtil(size_t index, const String & str) const { return field_infos[index].string_to_value_util_function(str); } \ @@ -856,8 +860,7 @@ using AliasMap = std::unordered_map; String name; \ const char * type; \ const char * description; \ - bool is_important; \ - bool is_obsolete; \ + BaseSettingsHelpers::Flags flags; \ Field (*cast_value_util_function)(const Field &); \ String (*value_to_string_util_function)(const Field &); \ Field (*string_to_value_util_function)(const String &); \ @@ -968,8 +971,8 @@ struct DefineAliases /// NOLINTNEXTLINE #define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ res.field_infos.emplace_back( \ - FieldInfo{#NAME, #TYPE, DESCRIPTION, (FLAGS) & IMPORTANT, \ - static_cast((FLAGS) & BaseSettingsHelpers::Flags::OBSOLETE), \ + FieldInfo{#NAME, #TYPE, DESCRIPTION, \ + static_cast(FLAGS), \ [](const Field & value) -> Field { return static_cast(SettingField##TYPE{value}); }, \ [](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \ [](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); }, \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 7c2cb49a2ba..326f151b12f 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -337,7 +337,7 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); res_columns[6]->insert(is_changeable ? changeable_settings_it->second.second : ChangeableWithoutRestart::No); - res_columns[7]->insert(setting.isObsolete()); + res_columns[7]->insert(setting.getTier() == SettingsTierType::OBSOLETE); } } } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0aecb7cf941..54cd3ad9a4f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1,7 +1,5 @@ -#include #include #include -#include #include #include #include @@ -40,10 +38,15 @@ namespace ErrorCodes * Note: as an alternative, we could implement settings to be completely dynamic in the form of the map: String -> Field, * but we are not going to do it, because settings are used everywhere as static struct fields. * - * `flags` can be either 0 or IMPORTANT. + * `flags` can be either 0 or IMPORTANT + a Tier (PRODUCTION | BETA | EXPERIMENTAL) * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. + * Tiers: + * EXPERIMENTAL: The feature is in active development stage. Mostly for developers or for ClickHouse enthusiasts. + * BETA: There are no known bugs problems in the functionality, but the outcome of using it together with other + * features/components is unknown and correctness is not guaranteed. + * PRODUCTION (Default): The feature is safe to use along with other features from the PRODUCTION tier. * - * When adding new or changing existing settings add them to the settings changes history in SettingsChangesHistory.h + * When adding new or changing existing settings add them to the settings changes history in SettingsChangesHistory.cpp * for tracking settings changes in different versions and for special `compatibility` settings to work correctly. */ @@ -6007,7 +6010,7 @@ void SettingsImpl::checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfi { const auto & name = setting.getName(); bool should_skip_check = name == "max_table_size_to_drop" || name == "max_partition_size_to_drop"; - if (config.has(name) && !setting.isObsolete() && !should_skip_check) + if (config.has(name) && (setting.getTier() != SettingsTierType::OBSOLETE) && !should_skip_check) { throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "A setting '{}' appeared at top level in config {}." " But it is user-level setting that should be located in users.xml inside section for specific profile." @@ -6183,7 +6186,7 @@ std::vector Settings::getChangedAndObsoleteNames() const std::vector setting_names; for (const auto & setting : impl->allChanged()) { - if (setting.isObsolete()) + if (setting.getTier() == SettingsTierType::OBSOLETE) setting_names.emplace_back(setting.getName()); } return setting_names; @@ -6232,7 +6235,8 @@ void Settings::dumpToSystemSettingsColumns(MutableColumnsAndConstraints & params res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getDefaultValueString()); - res_columns[10]->insert(setting.isObsolete()); + res_columns[10]->insert(setting.getTier() == SettingsTierType::OBSOLETE); + res_columns[11]->insert(setting.getTier()); }; const auto & settings_to_aliases = SettingsImpl::Traits::settingsToAliases(); diff --git a/src/Core/SettingsObsoleteMacros.h b/src/Core/SettingsObsoleteMacros.h index 97db1def294..c680cdc45b6 100644 --- a/src/Core/SettingsObsoleteMacros.h +++ b/src/Core/SettingsObsoleteMacros.h @@ -2,8 +2,8 @@ // clang-format off #define MAKE_OBSOLETE(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", SettingsTierType::OBSOLETE) /// NOTE: ServerSettings::loadSettingsFromConfig() should be updated to include this settings #define MAKE_DEPRECATED_BY_SERVER_CONFIG(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "User-level setting is deprecated, and it must be defined in the server configuration instead.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "User-level setting is deprecated, and it must be defined in the server configuration instead.", SettingsTierType::OBSOLETE) diff --git a/src/Core/SettingsTierType.cpp b/src/Core/SettingsTierType.cpp new file mode 100644 index 00000000000..48090f26fae --- /dev/null +++ b/src/Core/SettingsTierType.cpp @@ -0,0 +1,19 @@ +#include +#include + +namespace DB +{ + +std::shared_ptr getSettingsTierEnum() +{ + return std::make_shared( + DataTypeEnum8::Values + { + {"Production", static_cast(SettingsTierType::PRODUCTION)}, + {"Obsolete", static_cast(SettingsTierType::OBSOLETE)}, + {"Experimental", static_cast(SettingsTierType::EXPERIMENTAL)}, + {"Beta", static_cast(SettingsTierType::BETA)} + }); +} + +} diff --git a/src/Core/SettingsTierType.h b/src/Core/SettingsTierType.h new file mode 100644 index 00000000000..d8bba89bc18 --- /dev/null +++ b/src/Core/SettingsTierType.h @@ -0,0 +1,26 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +template +class DataTypeEnum; +using DataTypeEnum8 = DataTypeEnum; + +// Make it signed for compatibility with DataTypeEnum8 +enum SettingsTierType : int8_t +{ + PRODUCTION = 0b0000, + OBSOLETE = 0b0100, + EXPERIMENTAL = 0b1000, + BETA = 0b1100 +}; + +std::shared_ptr getSettingsTierEnum(); + +} diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 8c6aafe48f2..b95b3a856de 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -238,7 +238,7 @@ namespace ErrorCodes DECLARE(DeduplicateMergeProjectionMode, deduplicate_merge_projection_mode, DeduplicateMergeProjectionMode::THROW, "Whether to allow create projection for the table with non-classic MergeTree. Ignore option is purely for compatibility which might result in incorrect answer. Otherwise, if allowed, what is the action when merge, drop or rebuild.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", SettingsTierType::OBSOLETE) #define OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. */ \ @@ -648,7 +648,7 @@ void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndCo res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); - res_columns[8]->insert(setting.isObsolete()); + res_columns[8]->insert(setting.getTier() == SettingsTierType::OBSOLETE); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index 9309f10378e..debd40386a6 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -34,6 +36,14 @@ ColumnsDescription StorageSystemSettings::getColumnsDescription() {"default", std::make_shared(), "Setting default value."}, {"alias_for", std::make_shared(), "Flag that shows whether this name is an alias to another setting."}, {"is_obsolete", std::make_shared(), "Shows whether a setting is obsolete."}, + {"tier", getSettingsTierEnum(), R"( +Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their +development and the expectations one might have when using them: +* PRODUCTION: The feature is stable, safe to use and does not have issues interacting with other PRODUCTION features. +* BETA: The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. +* EXPERIMENTAL: The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. +* OBSOLETE: No longer supported. Either it is already removed or it will be removed in future releases. +)"}, }; } From 51e2a25934093cead242ce32ebf1ece38063f895 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Oct 2024 17:27:05 +0100 Subject: [PATCH 515/816] Update src/Storages/MergeTree/MergeTreeData.cpp Co-authored-by: Vladimir Cherkasov --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 384fad3effc..7c5db844815 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6372,7 +6372,7 @@ DetachedPartsInfo MergeTreeData::getDetachedParts() const for (const auto & disk : getDisks()) { /// While it is possible to have detached parts on readonly/write-once disks - /// if they were produces on another machine, where it wasn't readonly, + /// (if they were produced on another machine, where it wasn't readonly) /// to avoid wasting resources for slow disks, avoid trying to enumerate them. if (disk->isReadOnly() || disk->isWriteOnce()) continue; From 87cfc1dfd7efc5422306ebb6129e937e4f091f7b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Oct 2024 17:29:15 +0100 Subject: [PATCH 516/816] Update cluster.py --- 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 cb28cae4c99..bac783501e1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2352,7 +2352,7 @@ class ClickHouseCluster: time.sleep(0.5) raise Exception("Cannot wait PostgreSQL Java Client container") - def wait_rabbitmq_to_start(self, timeout=60): + def wait_rabbitmq_to_start(self, timeout=120): self.print_all_docker_pieces() self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) From 309f18debef94455e1d50ca08fc9dbe3baa54796 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:26:20 +0100 Subject: [PATCH 517/816] Mark some NON-PRODUCTION settings --- src/Core/BaseSettings.cpp | 2 +- src/Core/Settings.cpp | 403 +++++++++--------- src/Storages/MergeTree/MergeTreeSettings.cpp | 19 +- .../System/StorageSystemMergeTreeSettings.cpp | 11 +- 4 files changed, 227 insertions(+), 208 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 7bfa581598d..51e99262bdb 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -42,7 +42,7 @@ BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) SettingsTierType BaseSettingsHelpers::getTier(Flags flags) { int8_t tier = (flags & Flags::TIER); - if (tier > SettingsTierType::OBSOLETE) + if (tier > SettingsTierType::BETA) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); return SettingsTierType{tier}; } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 54cd3ad9a4f..4159758fe76 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5506,90 +5506,102 @@ For testing purposes. Replaces all external table functions to Null to not initi DECLARE(Bool, restore_replace_external_dictionary_source_to_null, false, R"( Replace external dictionary sources to Null on restore. Useful for testing purposes )", 0) \ - DECLARE(Bool, create_if_not_exists, false, R"( -Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. -)", 0) \ - DECLARE(Bool, enforce_strict_identifier_format, false, R"( -If enabled, only allow identifiers containing alphanumeric characters and underscores. -)", 0) \ - DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( -If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. -)", 0) \ - \ - /* ###################################### */ \ - /* ######## EXPERIMENTAL FEATURES ####### */ \ - /* ###################################### */ \ - DECLARE(Bool, allow_experimental_materialized_postgresql_table, false, R"( -Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental -)", 0) \ - DECLARE(Bool, allow_experimental_funnel_functions, false, R"( -Enable experimental functions for funnel analysis. -)", 0) \ - DECLARE(Bool, allow_experimental_nlp_functions, false, R"( -Enable experimental functions for natural language processing. -)", 0) \ - DECLARE(Bool, allow_experimental_hash_functions, false, R"( -Enable experimental hash functions -)", 0) \ - DECLARE(Bool, allow_experimental_object_type, false, R"( -Allow Object and JSON data types -)", 0) \ - DECLARE(Bool, allow_experimental_time_series_table, false, R"( -Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. + /* Parallel replicas */ \ + DECLARE(UInt64, allow_experimental_parallel_reading_from_replicas, 0, R"( +Use up to `max_parallel_replicas` the number of replicas from each shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure +)", BETA) ALIAS(enable_parallel_replicas) \ + DECLARE(NonZeroUInt64, max_parallel_replicas, 1, R"( +The maximum number of replicas for each shard when executing a query. Possible values: -- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. -- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. -)", 0) \ - DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( -Allow experimental vector similarity index -)", 0) \ - DECLARE(Bool, allow_experimental_variant_type, false, R"( -Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). -)", 0) \ - DECLARE(Bool, allow_experimental_dynamic_type, false, R"( -Allow Dynamic data type -)", 0) \ - DECLARE(Bool, allow_experimental_json_type, false, R"( -Allow JSON data type -)", 0) \ - DECLARE(Bool, allow_experimental_codecs, false, R"( -If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). -)", 0) \ - DECLARE(Bool, allow_experimental_shared_set_join, true, R"( -Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin -)", 0) \ - DECLARE(UInt64, max_limit_for_ann_queries, 1'000'000, R"( -SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes. -)", 0) \ - DECLARE(UInt64, hnsw_candidate_list_size_for_search, 256, R"( -The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. -)", 0) \ - DECLARE(Bool, throw_on_unsupported_query_inside_transaction, true, R"( -Throw exception if unsupported query is used inside transaction -)", 0) \ - DECLARE(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, R"( -Wait for committed changes to become actually visible in the latest snapshot -)", 0) \ - DECLARE(Bool, implicit_transaction, false, R"( -If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback) -)", 0) \ - DECLARE(UInt64, grace_hash_join_initial_buckets, 1, R"( -Initial number of grace hash join buckets -)", 0) \ - DECLARE(UInt64, grace_hash_join_max_buckets, 1024, R"( -Limit on the number of grace hash join buckets -)", 0) \ - DECLARE(UInt64, join_to_sort_minimum_perkey_rows, 40, R"( -The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys -)", 0) \ - DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, R"( -The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join. -)", 0) \ - DECLARE(Bool, allow_experimental_join_right_table_sorting, false, R"( -If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join. +- Positive integer. + +**Additional Info** + +This options will produce different results depending on the settings used. + +:::note +This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. +::: + +### Parallel processing using `SAMPLE` key + +A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: + +- The position of the sampling key in the partitioning key does not allow efficient range scans. +- Adding a sampling key to the table makes filtering by other columns less efficient. +- The sampling key is an expression that is expensive to calculate. +- The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. + +### Parallel processing using [parallel_replicas_custom_key](#parallel_replicas_custom_key) + +This setting is useful for any replicated table. )", 0) \ + DECLARE(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::READ_TASKS, R"( +Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_count, 0, R"( +This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing. +)", BETA) \ + DECLARE(UInt64, parallel_replica_offset, 0, R"( +This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas. +)", BETA) \ + DECLARE(String, parallel_replicas_custom_key, "", R"( +An arbitrary integer expression that can be used to split work between replicas for a specific table. +The value can be any integer expression. + +Simple expressions using primary keys are preferred. + +If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. +Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_custom_key_range_lower, 0, R"( +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. + +When used in conjunction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_custom_key_range_upper, 0, R"( +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. + +When used in conjunction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. + +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing +)", BETA) \ + DECLARE(String, cluster_for_parallel_replicas, "", R"( +Cluster for a shard in which current server is located +)", BETA) \ + DECLARE(Bool, parallel_replicas_allow_in_with_subquery, true, R"( +If true, subquery for IN will be executed on every follower replica. +)", BETA) \ + DECLARE(Float, parallel_replicas_single_task_marks_count_multiplier, 2, R"( +A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas. +)", BETA) \ + DECLARE(Bool, parallel_replicas_for_non_replicated_merge_tree, false, R"( +If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables +)", BETA) \ + DECLARE(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, R"( +Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas' +)", BETA) \ + DECLARE(Bool, parallel_replicas_prefer_local_join, true, R"( +If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN. +)", BETA) \ + DECLARE(UInt64, parallel_replicas_mark_segment_size, 0, R"( +Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384] +)", BETA) \ + DECLARE(Bool, parallel_replicas_local_plan, false, R"( +Build local plan for local replica +)", BETA) \ + \ + DECLARE(Bool, allow_experimental_analyzer, true, R"( +Allow new query analyzer. +)", IMPORTANT | BETA) ALIAS(enable_analyzer) \ + DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"( +Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`). +)", BETA) \ + \ DECLARE(Timezone, session_timezone, "", R"( Sets the implicit time zone of the current session or query. The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. @@ -5649,126 +5661,121 @@ This happens due to different parsing pipelines: **See also** - [timezone](../server-configuration-parameters/settings.md#timezone) +)", BETA) \ +DECLARE(Bool, create_if_not_exists, false, R"( +Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. +)", 0) \ + DECLARE(Bool, enforce_strict_identifier_format, false, R"( +If enabled, only allow identifiers containing alphanumeric characters and underscores. +)", 0) \ + DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( +If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. +)", 0) \ + DECLARE(Bool, implicit_select, false, R"( +Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. )", 0) \ - DECLARE(Bool, use_hive_partitioning, false, R"( -When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. -)", 0)\ \ - DECLARE(Bool, allow_statistics_optimize, false, R"( -Allows using statistics to optimize queries -)", 0) ALIAS(allow_statistic_optimize) \ - DECLARE(Bool, allow_experimental_statistics, false, R"( -Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). -)", 0) ALIAS(allow_experimental_statistic) \ \ - /* Parallel replicas */ \ - DECLARE(UInt64, allow_experimental_parallel_reading_from_replicas, 0, R"( -Use up to `max_parallel_replicas` the number of replicas from each shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure -)", 0) ALIAS(enable_parallel_replicas) \ - DECLARE(NonZeroUInt64, max_parallel_replicas, 1, R"( -The maximum number of replicas for each shard when executing a query. + /* ####################################################### */ \ + /* ########### START OF EXPERIMENTAL FEATURES ############ */ \ + /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ + /* ####################################################### */ \ + \ + DECLARE(Bool, allow_experimental_materialized_postgresql_table, false, R"( +Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_funnel_functions, false, R"( +Enable experimental functions for funnel analysis. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_nlp_functions, false, R"( +Enable experimental functions for natural language processing. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_hash_functions, false, R"( +Enable experimental hash functions +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_object_type, false, R"( +Allow Object and JSON data types +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_time_series_table, false, R"( +Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. Possible values: -- Positive integer. - -**Additional Info** - -This options will produce different results depending on the settings used. - -:::note -This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. -::: - -### Parallel processing using `SAMPLE` key - -A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: - -- The position of the sampling key in the partitioning key does not allow efficient range scans. -- Adding a sampling key to the table makes filtering by other columns less efficient. -- The sampling key is an expression that is expensive to calculate. -- The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. - -### Parallel processing using [parallel_replicas_custom_key](#parallel_replicas_custom_key) - -This setting is useful for any replicated table. -)", 0) \ - DECLARE(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::READ_TASKS, R"( -Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key. -)", 0) \ - DECLARE(UInt64, parallel_replicas_count, 0, R"( -This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing. -)", 0) \ - DECLARE(UInt64, parallel_replica_offset, 0, R"( -This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas. -)", 0) \ - DECLARE(String, parallel_replicas_custom_key, "", R"( -An arbitrary integer expression that can be used to split work between replicas for a specific table. -The value can be any integer expression. - -Simple expressions using primary keys are preferred. - -If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. -Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. -)", 0) \ - DECLARE(UInt64, parallel_replicas_custom_key_range_lower, 0, R"( -Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. - -When used in conjunction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. - -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. -)", 0) \ - DECLARE(UInt64, parallel_replicas_custom_key_range_upper, 0, R"( -Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. - -When used in conjunction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. - -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing -)", 0) \ - DECLARE(String, cluster_for_parallel_replicas, "", R"( -Cluster for a shard in which current server is located -)", 0) \ - DECLARE(Bool, parallel_replicas_allow_in_with_subquery, true, R"( -If true, subquery for IN will be executed on every follower replica. -)", 0) \ - DECLARE(Float, parallel_replicas_single_task_marks_count_multiplier, 2, R"( -A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas. -)", 0) \ - DECLARE(Bool, parallel_replicas_for_non_replicated_merge_tree, false, R"( -If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables -)", 0) \ - DECLARE(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, R"( -Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas' -)", 0) \ - DECLARE(Bool, parallel_replicas_prefer_local_join, true, R"( -If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN. -)", 0) \ - DECLARE(UInt64, parallel_replicas_mark_segment_size, 0, R"( -Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384] +- 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. +- 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. )", 0) \ + DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( +Allow experimental vector similarity index +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_variant_type, false, R"( +Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_dynamic_type, false, R"( +Allow Dynamic data type +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_json_type, false, R"( +Allow JSON data type +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_codecs, false, R"( +If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_shared_set_join, true, R"( +Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin +)", EXPERIMENTAL) \ + DECLARE(UInt64, max_limit_for_ann_queries, 1'000'000, R"( +SELECT queries with LIMIT bigger than this setting cannot use vector similarity indexes. Helps to prevent memory overflows in vector similarity indexes. +)", EXPERIMENTAL) \ + DECLARE(UInt64, hnsw_candidate_list_size_for_search, 256, R"( +The size of the dynamic candidate list when searching the vector similarity index, also known as 'ef_search'. +)", EXPERIMENTAL) \ + DECLARE(Bool, throw_on_unsupported_query_inside_transaction, true, R"( +Throw exception if unsupported query is used inside transaction +)", EXPERIMENTAL) \ + DECLARE(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, R"( +Wait for committed changes to become actually visible in the latest snapshot +)", EXPERIMENTAL) \ + DECLARE(Bool, implicit_transaction, false, R"( +If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback) +)", EXPERIMENTAL) \ + DECLARE(UInt64, grace_hash_join_initial_buckets, 1, R"( +Initial number of grace hash join buckets +)", EXPERIMENTAL) \ + DECLARE(UInt64, grace_hash_join_max_buckets, 1024, R"( +Limit on the number of grace hash join buckets +)", EXPERIMENTAL) \ + DECLARE(UInt64, join_to_sort_minimum_perkey_rows, 40, R"( +The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys +)", EXPERIMENTAL) \ + DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, R"( +The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_join_right_table_sorting, false, R"( +If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join. +)", EXPERIMENTAL) \ + DECLARE(Bool, use_hive_partitioning, false, R"( +When enabled, ClickHouse will detect Hive-style partitioning in path (`/name=value/`) in file-like table engines [File](../../engines/table-engines/special/file.md#hive-style-partitioning)/[S3](../../engines/table-engines/integrations/s3.md#hive-style-partitioning)/[URL](../../engines/table-engines/special/url.md#hive-style-partitioning)/[HDFS](../../engines/table-engines/integrations/hdfs.md#hive-style-partitioning)/[AzureBlobStorage](../../engines/table-engines/integrations/azureBlobStorage.md#hive-style-partitioning) and will allow to use partition columns as virtual columns in the query. These virtual columns will have the same names as in the partitioned path, but starting with `_`. +)", EXPERIMENTAL)\ + \ + DECLARE(Bool, allow_statistics_optimize, false, R"( +Allows using statistics to optimize queries +)", EXPERIMENTAL) ALIAS(allow_statistic_optimize) \ + DECLARE(Bool, allow_experimental_statistics, false, R"( +Allows defining columns with [statistics](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) and [manipulate statistics](../../engines/table-engines/mergetree-family/mergetree.md#column-statistics). +)", EXPERIMENTAL) ALIAS(allow_experimental_statistic) \ + \ DECLARE(Bool, allow_archive_path_syntax, true, R"( File/S3 engines/table function will parse paths with '::' as '\\ :: \\' if archive has correct extension -)", 0) \ - DECLARE(Bool, parallel_replicas_local_plan, false, R"( -Build local plan for local replica -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_inverted_index, false, R"( If it is set to true, allow to use experimental inverted index. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_full_text_index, false, R"( If it is set to true, allow to use experimental full-text index. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_join_condition, false, R"( Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y. -)", 0) \ - \ - DECLARE(Bool, allow_experimental_analyzer, true, R"( -Allow new query analyzer. -)", IMPORTANT) ALIAS(enable_analyzer) \ - DECLARE(Bool, analyzer_compatibility_join_using_top_level_identifier, false, R"( -Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`). )", 0) \ \ DECLARE(Bool, allow_experimental_live_view, false, R"( @@ -5781,43 +5788,45 @@ Possible values: )", 0) \ DECLARE(Seconds, live_view_heartbeat_interval, 15, R"( The heartbeat interval in seconds to indicate live query is alive. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(UInt64, max_live_view_insert_blocks_before_refresh, 64, R"( Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_window_view, false, R"( Enable WINDOW VIEW. Not mature enough. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, window_view_clean_interval, 60, R"( The clean interval of window view in seconds to free outdated data. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, window_view_heartbeat_interval, 15, R"( The heartbeat interval in seconds to indicate watch query is alive. -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Seconds, wait_for_window_view_fire_signal_timeout, 10, R"( Timeout for waiting for window view fire signal in event time processing -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, stop_refreshable_materialized_views_on_startup, false, R"( On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW \\ afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views. -)", 0) \ +)", EXPERIMENTAL) \ \ DECLARE(Bool, allow_experimental_database_materialized_mysql, false, R"( Allow to create database with Engine=MaterializedMySQL(...). -)", 0) \ +)", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_database_materialized_postgresql, false, R"( Allow to create database with Engine=MaterializedPostgreSQL(...). -)", 0) \ +)", EXPERIMENTAL) \ \ /** Experimental feature for moving data between shards. */ \ DECLARE(Bool, allow_experimental_query_deduplication, false, R"( Experimental data deduplication for SELECT queries based on part UUIDs -)", 0) \ - DECLARE(Bool, implicit_select, false, R"( -Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. -)", 0) - +)", EXPERIMENTAL) \ + \ + /* ####################################################### */ \ + /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ + /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ + /* ####################################################### */ \ + /* ####################################################### */ \ // End of COMMON_SETTINGS // Please add settings related to formats in Core/FormatFactorySettings.h, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index b95b3a856de..36e146f4624 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -88,7 +88,7 @@ namespace ErrorCodes DECLARE(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ DECLARE(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ DECLARE(UInt64, number_of_free_entries_in_pool_to_execute_optimize_entire_partition, 25, "When there is less than specified number of free entries in pool, do not try to execute optimize entire partition with a merge (this merge is created when set min_age_to_force_merge_seconds > 0 and min_age_to_force_merge_on_partition_only = true). This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - DECLARE(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ + DECLARE(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", EXPERIMENTAL) \ DECLARE(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ DECLARE(UInt64, number_of_mutations_to_delay, 500, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ DECLARE(UInt64, number_of_mutations_to_throw, 1000, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ @@ -214,14 +214,14 @@ namespace ErrorCodes DECLARE(Bool, enable_block_offset_column, false, "Enable persisting column _block_offset for each row.", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ - DECLARE(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ - DECLARE(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ - DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ - DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ - DECLARE(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ - DECLARE(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", 0) \ - DECLARE(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", 0) \ - DECLARE(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", 0) \ + DECLARE(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", EXPERIMENTAL) \ + DECLARE(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", EXPERIMENTAL) \ + DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", BETA) \ + DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", EXPERIMENTAL) \ + DECLARE(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", EXPERIMENTAL) \ + DECLARE(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", EXPERIMENTAL) \ + DECLARE(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", EXPERIMENTAL) \ \ /** Compress marks and primary key. */ \ DECLARE(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ @@ -649,6 +649,7 @@ void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndCo res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getTier() == SettingsTierType::OBSOLETE); + res_columns[9]->insert(setting.getTier()); } } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 35d975216f6..1da4835dba5 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -1,4 +1,5 @@ -#include +#include +#include #include #include #include @@ -30,6 +31,14 @@ ColumnsDescription SystemMergeTreeSettings::getColumnsDescription() }, {"type", std::make_shared(), "Setting type (implementation specific string value)."}, {"is_obsolete", std::make_shared(), "Shows whether a setting is obsolete."}, + {"tier", getSettingsTierEnum(), R"( +Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their +development and the expectations one might have when using them: +* PRODUCTION: The feature is stable, safe to use and does not have issues interacting with other PRODUCTION features. +* BETA: The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. +* EXPERIMENTAL: The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. +* OBSOLETE: No longer supported. Either it is already removed or it will be removed in future releases. +)"}, }; } From f43c6e8b2f0e8161ae2159a2848f6e030d9e159a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Oct 2024 17:42:18 +0100 Subject: [PATCH 518/816] randomize setting enable_vertical_final --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 100a6358dcf..9c035b7cc35 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -920,6 +920,7 @@ class SettingsRandomizer: "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), + "enable_vertical_final": lambda: random.randint(0, 1), } @staticmethod From 08d070d982ababa39f726480efc4ba76d85f365e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:46:11 +0100 Subject: [PATCH 519/816] Add basic test for setting tiers --- .../queries/0_stateless/03257_setting_tiers.reference | 10 ++++++++++ tests/queries/0_stateless/03257_setting_tiers.sql | 11 +++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/03257_setting_tiers.reference create mode 100644 tests/queries/0_stateless/03257_setting_tiers.sql diff --git a/tests/queries/0_stateless/03257_setting_tiers.reference b/tests/queries/0_stateless/03257_setting_tiers.reference new file mode 100644 index 00000000000..d3d171221e8 --- /dev/null +++ b/tests/queries/0_stateless/03257_setting_tiers.reference @@ -0,0 +1,10 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03257_setting_tiers.sql b/tests/queries/0_stateless/03257_setting_tiers.sql new file mode 100644 index 00000000000..c7ffe87a80b --- /dev/null +++ b/tests/queries/0_stateless/03257_setting_tiers.sql @@ -0,0 +1,11 @@ +SELECT count() > 0 FROM system.settings WHERE tier = 'Production'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Beta'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Experimental'; +SELECT count() > 0 FROM system.settings WHERE tier = 'Obsolete'; +SELECT count() == countIf(tier IN ['Production', 'Beta', 'Experimental', 'Obsolete']) FROM system.settings; + +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Production'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Beta'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Experimental'; +SELECT count() > 0 FROM system.merge_tree_settings WHERE tier = 'Obsolete'; +SELECT count() == countIf(tier IN ['Production', 'Beta', 'Experimental', 'Obsolete']) FROM system.merge_tree_settings; From 02381ec1f8a8250403ee7ee69908e61663b51b4b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 28 Oct 2024 17:43:43 +0100 Subject: [PATCH 520/816] Add a test --- src/Server/TCPHandler.cpp | 9 +++------ tests/queries/0_stateless/03258_nonexistent_db.reference | 2 ++ tests/queries/0_stateless/03258_nonexistent_db.sh | 7 +++++++ 3 files changed, 12 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03258_nonexistent_db.reference create mode 100755 tests/queries/0_stateless/03258_nonexistent_db.sh diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index afca8b4ab25..e7e4ae25a68 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -301,6 +301,9 @@ void TCPHandler::runImpl() { receiveHello(); + if (!default_database.empty()) + DatabaseCatalog::instance().assertDatabaseExists(default_database); + /// In interserver mode queries are executed without a session context. if (!is_interserver_mode) session->makeSessionContext(); @@ -1604,8 +1607,6 @@ void TCPHandler::receiveHello() session->authenticate( SSLCertificateCredentials{user, extractSSLCertificateSubjects(secure_socket.peerCertificate())}, getClientAddress(client_info)); - if (!default_database.empty()) - DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } catch (const Exception & e) @@ -1673,15 +1674,11 @@ void TCPHandler::receiveHello() auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge)); session->authenticate(cred, getClientAddress(client_info)); - if (!default_database.empty()) - DatabaseCatalog::instance().assertDatabaseExists(default_database); return; } #endif session->authenticate(user, password, getClientAddress(client_info)); - if (!default_database.empty()) - DatabaseCatalog::instance().assertDatabaseExists(default_database); } void TCPHandler::receiveAddendum() diff --git a/tests/queries/0_stateless/03258_nonexistent_db.reference b/tests/queries/0_stateless/03258_nonexistent_db.reference new file mode 100644 index 00000000000..825bae3beaa --- /dev/null +++ b/tests/queries/0_stateless/03258_nonexistent_db.reference @@ -0,0 +1,2 @@ +UNKNOWN_DATABASE +OK diff --git a/tests/queries/0_stateless/03258_nonexistent_db.sh b/tests/queries/0_stateless/03258_nonexistent_db.sh new file mode 100755 index 00000000000..847d692c440 --- /dev/null +++ b/tests/queries/0_stateless/03258_nonexistent_db.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +timeout 5 ${CLICKHOUSE_CLIENT_BINARY} --database "nonexistent" 2>&1 | grep -o "UNKNOWN_DATABASE" && echo "OK" || echo "FAIL" From 849ab015af2e9e644e61d7f580f62b2b8538e989 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 17:51:34 +0100 Subject: [PATCH 521/816] Update CHANGELOG.md Co-authored-by: Vladimir Cherkasov --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9724eb7eb61..c26e47a78fa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,7 +22,7 @@ * Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB, remove support for deprecated old protocol, support for connection by URI, support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). +* MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. Note that new inegration is disabled by default, to use it, please set `` to `false` in server config. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). * A new `--progress-table` option in clickhouse-client prints a table with metrics changing during query execution; a new `--enable-progress-table-toggle` is associated with the `--progress-table` option, and toggles the rendering of the progress table by pressing the control key (Space). [#63689](https://github.com/ClickHouse/ClickHouse/pull/63689) ([Maria Khristenko](https://github.com/mariaKhr)). * This allows to grant access to the wildcard prefixes. `GRANT SELECT ON db.table_pefix_* TO user`. [#65311](https://github.com/ClickHouse/ClickHouse/pull/65311) ([pufit](https://github.com/pufit)). * Add system.query_metric_log which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). From f89887de6a6d5ffa7a5e8eec20a4a2358fed4410 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 18:18:09 +0100 Subject: [PATCH 522/816] Adjust existing tests --- .../queries/0_stateless/01221_system_settings.reference | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 32a0ed11b6c..821d2e386a9 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 Timeout for sending data to the network, in seconds. If a client needs to send some data but is not able to send any bytes in this interval, the exception is thrown. If you set this setting on the client, the \'receive_timeout\' for the socket will also be set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 -storage_policy default 0 Name of storage disk policy \N \N 0 String 0 +send_timeout 300 0 Timeout for sending data to the network, in seconds. If a client needs to send some data but is not able to send any bytes in this interval, the exception is thrown. If you set this setting on the client, the \'receive_timeout\' for the socket will also be set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 Production +storage_policy default 0 Name of storage disk policy \N \N 0 String 0 Production 1 1 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index b260e2dce6c..2ea62444cff 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -342,7 +342,8 @@ CREATE TABLE system.merge_tree_settings `max` Nullable(String), `readonly` UInt8, `type` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemMergeTreeSettings COMMENT 'Contains a list of all MergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query.' @@ -932,7 +933,8 @@ CREATE TABLE system.replicated_merge_tree_settings `max` Nullable(String), `readonly` UInt8, `type` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemReplicatedMergeTreeSettings COMMENT 'Contains a list of all ReplicatedMergeTree engine specific settings, their current and default values along with descriptions. You may change any of them in SETTINGS section in CREATE query. ' @@ -1009,7 +1011,8 @@ CREATE TABLE system.settings `type` String, `default` String, `alias_for` String, - `is_obsolete` UInt8 + `is_obsolete` UInt8, + `tier` Enum8('Production' = 0, 'Obsolete' = 4, 'Experimental' = 8, 'Beta' = 12) ) ENGINE = SystemSettings COMMENT 'Contains a list of all user-level settings (which can be modified in a scope of query or session), their current and default values along with descriptions.' From 49655e71f5dc6ca87a41ef30de6bd8b2b53be354 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 28 Oct 2024 18:20:43 +0100 Subject: [PATCH 523/816] Update docs --- docs/en/operations/system-tables/merge_tree_settings.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/system-tables/merge_tree_settings.md b/docs/en/operations/system-tables/merge_tree_settings.md index 48217d63f9d..473315d3941 100644 --- a/docs/en/operations/system-tables/merge_tree_settings.md +++ b/docs/en/operations/system-tables/merge_tree_settings.md @@ -18,6 +18,11 @@ Columns: - `1` — Current user can’t change the setting. - `type` ([String](../../sql-reference/data-types/string.md)) — Setting type (implementation specific string value). - `is_obsolete` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) - Shows whether a setting is obsolete. +- `tier` ([Enum8](../../sql-reference/data-types/enum.md)) — Support level for this feature. ClickHouse features are organized in tiers, varying depending on the current status of their development and the expectations one might have when using them. Values: + - `'Production'` — The feature is stable, safe to use and does not have issues interacting with other **production** features. . + - `'Beta'` — The feature is stable and safe. The outcome of using it together with other features is unknown and correctness is not guaranteed. Testing and reports are welcome. + - `'Experimental'` — The feature is under development. Only intended for developers and ClickHouse enthusiasts. The feature might or might not work and could be removed at any time. + - `'Obsolete'` — No longer supported. Either it is already removed or it will be removed in future releases. **Example** ```sql From 8fb38750d5a7210ae2c57e19bb792cf3a8ac796d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Oct 2024 18:24:00 +0100 Subject: [PATCH 524/816] Fix bad test `02561_sorting_constants_and_distinct_crash` --- .../0_stateless/02561_sorting_constants_and_distinct_crash.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02561_sorting_constants_and_distinct_crash.sql b/tests/queries/0_stateless/02561_sorting_constants_and_distinct_crash.sql index 93a47c6736a..93c10dce52c 100644 --- a/tests/queries/0_stateless/02561_sorting_constants_and_distinct_crash.sql +++ b/tests/queries/0_stateless/02561_sorting_constants_and_distinct_crash.sql @@ -16,7 +16,8 @@ select distinct from ( select string_value from test_table -); +) +order by all; select distinct 'constant_1' as constant_value, * From 2d7de40ba70d6609f6fd79c5ef8534002803b707 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 17:24:03 +0000 Subject: [PATCH 525/816] fix sparse tables --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 54331186302..635b46de3ee 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -458,7 +458,7 @@ void FillingTransform::initColumns( non_const_columns.reserve(input_columns.size()); for (const auto & column : input_columns) - non_const_columns.push_back(column->convertToFullColumnIfConst()); + non_const_columns.push_back(column->convertToFullColumnIfConst()->convertToFullColumnIfSparse()); for (const auto & column : non_const_columns) output_columns.push_back(column->cloneEmpty()->assumeMutable()); From 1e59e2932b6ed5cb8a3da6a5eb32e60081e801cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 28 Oct 2024 17:36:32 +0000 Subject: [PATCH 526/816] Add test to verify #62308 works --- .../test_config_reload/__init__.py | 0 .../test_config_reload/configs/kafka.xml | 11 +++ tests/integration/test_config_reload/test.py | 71 +++++++++++++++++++ 3 files changed, 82 insertions(+) create mode 100644 tests/integration/test_config_reload/__init__.py create mode 100644 tests/integration/test_config_reload/configs/kafka.xml create mode 100644 tests/integration/test_config_reload/test.py diff --git a/tests/integration/test_config_reload/__init__.py b/tests/integration/test_config_reload/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_config_reload/configs/kafka.xml b/tests/integration/test_config_reload/configs/kafka.xml new file mode 100644 index 00000000000..8ac6ff89156 --- /dev/null +++ b/tests/integration/test_config_reload/configs/kafka.xml @@ -0,0 +1,11 @@ + + + consumer + + + config_test + 424242 + + + + diff --git a/tests/integration/test_config_reload/test.py b/tests/integration/test_config_reload/test.py new file mode 100644 index 00000000000..ccd4338b455 --- /dev/null +++ b/tests/integration/test_config_reload/test.py @@ -0,0 +1,71 @@ +import pytest + +from helpers.cluster import ClickHouseCluster, is_arm + +if is_arm(): + pytestmark = pytest.mark.skip + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + main_configs=["configs/kafka.xml"], + with_kafka=True, + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +DEFAULT_VALUE = "424242" +CHANGED_VALUE = "414141" + + +def check_value(value): + instance.query( + f""" + CREATE TABLE test (x Int64) ENGINE = Kafka + SETTINGS + kafka_broker_list = '{cluster.kafka_host}:{cluster.kafka_port}', + kafka_topic_list = 'config_test', + kafka_group_name = 'config_test_group', + kafka_format = 'JSON'; + """ + ) + + instance.query( + "SELECT * FROM test SETTINGS stream_like_engine_allow_direct_select=1", + ignore_error=True, + ) + + assert instance.wait_for_log_line("Consumer set property session.timeout.ms") + instance.query("DROP TABLE test SYNC") + + instance.contains_in_log(f"Consumer set property session.timeout.ms:{value}") + + +def test_system_reload_config_with_global_context(start_cluster): + # When running the this test multiple times, make sure failure of one test won't cause the failure of every subsequent tests + instance.query("DROP TABLE IF EXISTS test SYNC") + instance.replace_in_config( + "/etc/clickhouse-server/config.d/kafka.xml", CHANGED_VALUE, DEFAULT_VALUE + ) + instance.restart_clickhouse() + + check_value(DEFAULT_VALUE) + + instance.rotate_logs() + + instance.replace_in_config( + "/etc/clickhouse-server/config.d/kafka.xml", DEFAULT_VALUE, CHANGED_VALUE + ) + + instance.query("SYSTEM RELOAD CONFIG") + + check_value(CHANGED_VALUE) From 37f691bf9d1168431500c39c47432722a441a29e Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 17:42:52 +0000 Subject: [PATCH 527/816] add test --- .../03266_with_fill_staleness.reference | 28 +++++++++++++++++ .../0_stateless/03266_with_fill_staleness.sql | 31 +++++++++++++++++++ 2 files changed, 59 insertions(+) create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness.reference create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness.sql diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.reference b/tests/queries/0_stateless/03266_with_fill_staleness.reference new file mode 100644 index 00000000000..6061ecfe400 --- /dev/null +++ b/tests/queries/0_stateless/03266_with_fill_staleness.reference @@ -0,0 +1,28 @@ +add samples +regular with fill +2016-06-15 23:00:00 0 +2016-06-15 23:00:01 0 +2016-06-15 23:00:02 0 +2016-06-15 23:00:03 0 +2016-06-15 23:00:04 0 +2016-06-15 23:00:05 5 +2016-06-15 23:00:06 5 +2016-06-15 23:00:07 5 +2016-06-15 23:00:08 5 +2016-06-15 23:00:09 5 +2016-06-15 23:00:10 10 +2016-06-15 23:00:11 10 +2016-06-15 23:00:12 10 +2016-06-15 23:00:13 10 +2016-06-15 23:00:14 10 +2016-06-15 23:00:15 15 +2016-06-15 23:00:16 15 +2016-06-15 23:00:17 15 +2016-06-15 23:00:18 15 +2016-06-15 23:00:19 15 +2016-06-15 23:00:20 20 +2016-06-15 23:00:21 20 +2016-06-15 23:00:22 20 +2016-06-15 23:00:23 20 +2016-06-15 23:00:24 20 +2016-06-15 23:00:25 25 diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.sql b/tests/queries/0_stateless/03266_with_fill_staleness.sql new file mode 100644 index 00000000000..3ab9be63a08 --- /dev/null +++ b/tests/queries/0_stateless/03266_with_fill_staleness.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS with_fill_staleness; +CREATE TABLE with_fill_staleness (a DateTime, b DateTime, c UInt64) ENGINE = MergeTree ORDER BY a; + +SELECT 'add samples'; + +INSERT INTO with_fill_staleness +SELECT + toDateTime('2016-06-15 23:00:00') + number AS a, a as b, number as c +FROM numbers(30) +WHERE (number % 5) == 0; + +SELECT 'regular with fill'; +SELECT a, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL INTERPOLATE (c); + +SELECT 'staleness 1 seconds'; +SELECT a, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL STALENESS INTERVAL 1 SECOND INTERPOLATE (c); + +SELECT 'staleness 3 seconds'; +SELECT a, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL STALENESS INTERVAL 3 SECOND INTERPOLATE (c); + +SELECT 'descending order'; +SELECT a, c, 'original' as original FROM with_fill_staleness ORDER BY a DESC WITH FILL STALENESS INTERVAL -2 SECOND INTERPOLATE (c); + +SELECT 'staleness with to and step'; +SELECT a, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL TO toDateTime('2016-06-15 23:00:40') STEP 3 STALENESS INTERVAL 7 SECOND INTERPOLATE (c); + +SELECT 'staleness with another regular with fill'; +SELECT a, b, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL STALENESS INTERVAL 2 SECOND, b ASC WITH FILL FROM 0 TO 3 INTERPOLATE (c); + +SELECT 'double staleness'; +SELECT a, b, c, 'original' as original FROM with_fill_staleness ORDER BY a ASC WITH FILL STALENESS INTERVAL 2 SECOND, b ASC WITH FILL TO toDateTime('2016-06-15 23:01:00') STEP 2 STALENESS 5 INTERPOLATE (c); From 9760d39efe82339403de7a7177706c42c8d8c5a5 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 17:43:15 +0000 Subject: [PATCH 528/816] allow negative staleness for descending order --- src/Planner/PlannerSorting.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerSorting.cpp b/src/Planner/PlannerSorting.cpp index 0a33e2f0828..9476ae348c5 100644 --- a/src/Planner/PlannerSorting.cpp +++ b/src/Planner/PlannerSorting.cpp @@ -105,10 +105,6 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) if (sort_node.hasFillFrom()) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL STALENESS cannot be used together with WITH FILL FROM"); - - if (applyVisitor(FieldVisitorAccurateLessOrEqual(), fill_column_description.fill_staleness, Field{0})) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL STALENESS value cannot be less or equal zero"); } if (sort_node.getSortDirection() == SortDirection::ASCENDING) @@ -117,6 +113,10 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL STEP value cannot be negative for sorting in ascending direction"); + if (applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_staleness, Field{0})) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STALENESS value cannot be negative for sorting in ascending direction"); + if (!fill_column_description.fill_from.isNull() && !fill_column_description.fill_to.isNull() && applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_to, fill_column_description.fill_from)) { @@ -130,6 +130,10 @@ FillColumnDescription extractWithFillDescription(const SortNode & sort_node) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "WITH FILL STEP value cannot be positive for sorting in descending direction"); + if (applyVisitor(FieldVisitorAccurateLess(), Field{0}, fill_column_description.fill_staleness)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "WITH FILL STALENESS value cannot be positive for sorting in descending direction"); + if (!fill_column_description.fill_from.isNull() && !fill_column_description.fill_to.isNull() && applyVisitor(FieldVisitorAccurateLess(), fill_column_description.fill_from, fill_column_description.fill_to)) { From fc33593ff05ab3c5ca4271b79ba4eb39957fa057 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 17:45:02 +0000 Subject: [PATCH 529/816] fix style --- src/Interpreters/FillingRow.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 1d3eae03ddd..fdd3b55b66b 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -72,7 +72,8 @@ std::optional FillingRow::doJump(const FillColumnDescription& descr, size if (!descr.fill_to.isNull() && less(descr.fill_to, next_value, getDirection(column_ind))) return std::nullopt; - if (!descr.fill_staleness.isNull()) { + if (!descr.fill_staleness.isNull()) + { Field staleness_border = staleness_base_row[column_ind]; descr.staleness_step_func(staleness_border, 1); @@ -92,7 +93,8 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, if (less(to, shifted_value, getDirection(column_ind))) return std::nullopt; - for (int32_t step_len = 1, step_no = 0; step_no < 100; ++step_no) { + for (int32_t step_len = 1, step_no = 0; step_no < 100; ++step_no) + { Field next_value = shifted_value; descr.step_func(next_value, step_len); @@ -197,9 +199,8 @@ void FillingRow::initFromDefaults(size_t from_pos) void FillingRow::initStalenessRow(const Columns& base_row, size_t row_ind) { - for (size_t i = 0; i < size(); ++i) { + for (size_t i = 0; i < size(); ++i) staleness_base_row[i] = (*base_row[i])[row_ind]; - } } String FillingRow::dump() const From 4c9d865e7592985507accd7aa805647ef9335d72 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 17:45:27 +0000 Subject: [PATCH 530/816] disable debug logs --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 635b46de3ee..7f81b86697c 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -17,7 +17,7 @@ namespace DB { -constexpr bool debug_logging_enabled = true; +constexpr bool debug_logging_enabled = false; template void logDebug(String key, const T & value, const char * separator = " : ") From bef86391fa5db9c21427d153e7256c1336b55d2f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Oct 2024 18:58:14 +0100 Subject: [PATCH 531/816] `system.session_log` is quite okay --- src/Interpreters/SystemLog.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index bbdeb4567af..aafe819967f 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -298,9 +298,6 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf #undef CREATE_PUBLIC_MEMBERS /// NOLINTEND(bugprone-macro-parentheses) - if (session_log) - global_context->addWarningMessage("Table system.session_log is enabled. It's unreliable and may contain garbage. Do not use it for any kind of security monitoring."); - bool should_prepare = global_context->getServerSettings()[ServerSetting::prepare_system_log_tables_on_startup]; try { From 83844841b4f00a24a654ac7ce9f665c321b4df85 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 18:04:00 +0000 Subject: [PATCH 532/816] fix test timezone --- .../03266_with_fill_staleness.reference | 163 +++++++++++++++--- .../0_stateless/03266_with_fill_staleness.sql | 2 + 2 files changed, 139 insertions(+), 26 deletions(-) diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.reference b/tests/queries/0_stateless/03266_with_fill_staleness.reference index 6061ecfe400..6b090443359 100644 --- a/tests/queries/0_stateless/03266_with_fill_staleness.reference +++ b/tests/queries/0_stateless/03266_with_fill_staleness.reference @@ -1,28 +1,139 @@ add samples regular with fill -2016-06-15 23:00:00 0 -2016-06-15 23:00:01 0 -2016-06-15 23:00:02 0 -2016-06-15 23:00:03 0 -2016-06-15 23:00:04 0 -2016-06-15 23:00:05 5 -2016-06-15 23:00:06 5 -2016-06-15 23:00:07 5 -2016-06-15 23:00:08 5 -2016-06-15 23:00:09 5 -2016-06-15 23:00:10 10 -2016-06-15 23:00:11 10 -2016-06-15 23:00:12 10 -2016-06-15 23:00:13 10 -2016-06-15 23:00:14 10 -2016-06-15 23:00:15 15 -2016-06-15 23:00:16 15 -2016-06-15 23:00:17 15 -2016-06-15 23:00:18 15 -2016-06-15 23:00:19 15 -2016-06-15 23:00:20 20 -2016-06-15 23:00:21 20 -2016-06-15 23:00:22 20 -2016-06-15 23:00:23 20 -2016-06-15 23:00:24 20 -2016-06-15 23:00:25 25 +2016-06-15 23:00:00 0 original +2016-06-15 23:00:01 0 +2016-06-15 23:00:02 0 +2016-06-15 23:00:03 0 +2016-06-15 23:00:04 0 +2016-06-15 23:00:05 5 original +2016-06-15 23:00:06 5 +2016-06-15 23:00:07 5 +2016-06-15 23:00:08 5 +2016-06-15 23:00:09 5 +2016-06-15 23:00:10 10 original +2016-06-15 23:00:11 10 +2016-06-15 23:00:12 10 +2016-06-15 23:00:13 10 +2016-06-15 23:00:14 10 +2016-06-15 23:00:15 15 original +2016-06-15 23:00:16 15 +2016-06-15 23:00:17 15 +2016-06-15 23:00:18 15 +2016-06-15 23:00:19 15 +2016-06-15 23:00:20 20 original +2016-06-15 23:00:21 20 +2016-06-15 23:00:22 20 +2016-06-15 23:00:23 20 +2016-06-15 23:00:24 20 +2016-06-15 23:00:25 25 original +staleness 1 seconds +2016-06-15 23:00:00 0 original +2016-06-15 23:00:05 5 original +2016-06-15 23:00:10 10 original +2016-06-15 23:00:15 15 original +2016-06-15 23:00:20 20 original +2016-06-15 23:00:25 25 original +staleness 3 seconds +2016-06-15 23:00:00 0 original +2016-06-15 23:00:01 0 +2016-06-15 23:00:02 0 +2016-06-15 23:00:05 5 original +2016-06-15 23:00:06 5 +2016-06-15 23:00:07 5 +2016-06-15 23:00:10 10 original +2016-06-15 23:00:11 10 +2016-06-15 23:00:12 10 +2016-06-15 23:00:15 15 original +2016-06-15 23:00:16 15 +2016-06-15 23:00:17 15 +2016-06-15 23:00:20 20 original +2016-06-15 23:00:21 20 +2016-06-15 23:00:22 20 +2016-06-15 23:00:25 25 original +descending order +2016-06-15 23:00:25 25 original +2016-06-15 23:00:24 25 +2016-06-15 23:00:20 20 original +2016-06-15 23:00:19 20 +2016-06-15 23:00:15 15 original +2016-06-15 23:00:14 15 +2016-06-15 23:00:10 10 original +2016-06-15 23:00:09 10 +2016-06-15 23:00:05 5 original +2016-06-15 23:00:04 5 +2016-06-15 23:00:00 0 original +staleness with to and step +2016-06-15 23:00:00 0 original +2016-06-15 23:00:03 0 +2016-06-15 23:00:05 5 original +2016-06-15 23:00:06 5 +2016-06-15 23:00:09 5 +2016-06-15 23:00:10 10 original +2016-06-15 23:00:12 10 +2016-06-15 23:00:15 15 original +2016-06-15 23:00:18 15 +2016-06-15 23:00:20 20 original +2016-06-15 23:00:21 20 +2016-06-15 23:00:24 20 +2016-06-15 23:00:25 25 original +2016-06-15 23:00:27 25 +2016-06-15 23:00:30 25 +staleness with another regular with fill +2016-06-15 23:00:00 1970-01-01 01:00:00 0 +2016-06-15 23:00:00 1970-01-01 01:00:01 0 +2016-06-15 23:00:00 1970-01-01 01:00:02 0 +2016-06-15 23:00:00 2016-06-15 23:00:00 0 original +2016-06-15 23:00:01 1970-01-01 01:00:00 0 +2016-06-15 23:00:01 1970-01-01 01:00:01 0 +2016-06-15 23:00:01 1970-01-01 01:00:02 0 +2016-06-15 23:00:05 2016-06-15 23:00:05 5 original +2016-06-15 23:00:05 1970-01-01 01:00:01 5 +2016-06-15 23:00:05 1970-01-01 01:00:02 5 +2016-06-15 23:00:06 1970-01-01 01:00:00 5 +2016-06-15 23:00:06 1970-01-01 01:00:01 5 +2016-06-15 23:00:06 1970-01-01 01:00:02 5 +2016-06-15 23:00:10 2016-06-15 23:00:10 10 original +2016-06-15 23:00:10 1970-01-01 01:00:01 10 +2016-06-15 23:00:10 1970-01-01 01:00:02 10 +2016-06-15 23:00:11 1970-01-01 01:00:00 10 +2016-06-15 23:00:11 1970-01-01 01:00:01 10 +2016-06-15 23:00:11 1970-01-01 01:00:02 10 +2016-06-15 23:00:15 2016-06-15 23:00:15 15 original +2016-06-15 23:00:15 1970-01-01 01:00:01 15 +2016-06-15 23:00:15 1970-01-01 01:00:02 15 +2016-06-15 23:00:16 1970-01-01 01:00:00 15 +2016-06-15 23:00:16 1970-01-01 01:00:01 15 +2016-06-15 23:00:16 1970-01-01 01:00:02 15 +2016-06-15 23:00:20 2016-06-15 23:00:20 20 original +2016-06-15 23:00:20 1970-01-01 01:00:01 20 +2016-06-15 23:00:20 1970-01-01 01:00:02 20 +2016-06-15 23:00:21 1970-01-01 01:00:00 20 +2016-06-15 23:00:21 1970-01-01 01:00:01 20 +2016-06-15 23:00:21 1970-01-01 01:00:02 20 +2016-06-15 23:00:25 2016-06-15 23:00:25 25 original +2016-06-15 23:00:25 1970-01-01 01:00:01 25 +2016-06-15 23:00:25 1970-01-01 01:00:02 25 +double staleness +2016-06-15 23:00:00 2016-06-15 23:00:00 0 original +2016-06-15 23:00:00 2016-06-15 23:00:02 0 +2016-06-15 23:00:00 2016-06-15 23:00:04 0 +2016-06-15 23:00:01 1970-01-01 01:00:00 0 +2016-06-15 23:00:05 2016-06-15 23:00:05 5 original +2016-06-15 23:00:05 2016-06-15 23:00:07 5 +2016-06-15 23:00:05 2016-06-15 23:00:09 5 +2016-06-15 23:00:06 1970-01-01 01:00:00 5 +2016-06-15 23:00:10 2016-06-15 23:00:10 10 original +2016-06-15 23:00:10 2016-06-15 23:00:12 10 +2016-06-15 23:00:10 2016-06-15 23:00:14 10 +2016-06-15 23:00:11 1970-01-01 01:00:00 10 +2016-06-15 23:00:15 2016-06-15 23:00:15 15 original +2016-06-15 23:00:15 2016-06-15 23:00:17 15 +2016-06-15 23:00:15 2016-06-15 23:00:19 15 +2016-06-15 23:00:16 1970-01-01 01:00:00 15 +2016-06-15 23:00:20 2016-06-15 23:00:20 20 original +2016-06-15 23:00:20 2016-06-15 23:00:22 20 +2016-06-15 23:00:20 2016-06-15 23:00:24 20 +2016-06-15 23:00:21 1970-01-01 01:00:00 20 +2016-06-15 23:00:25 2016-06-15 23:00:25 25 original +2016-06-15 23:00:25 2016-06-15 23:00:27 25 +2016-06-15 23:00:25 2016-06-15 23:00:29 25 diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.sql b/tests/queries/0_stateless/03266_with_fill_staleness.sql index 3ab9be63a08..fff702ffd83 100644 --- a/tests/queries/0_stateless/03266_with_fill_staleness.sql +++ b/tests/queries/0_stateless/03266_with_fill_staleness.sql @@ -1,3 +1,5 @@ +SET session_timezone='Europe/Amsterdam'; + DROP TABLE IF EXISTS with_fill_staleness; CREATE TABLE with_fill_staleness (a DateTime, b DateTime, c UInt64) ENGINE = MergeTree ORDER BY a; From 116db8d6b795c0087dc6e6aba1386ece3a461c49 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 28 Oct 2024 19:06:03 +0100 Subject: [PATCH 533/816] Fix --- .../ObjectStorageQueue/ObjectStorageQueueMetadata.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 692f001dd7b..6aac853b011 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -258,7 +258,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) { LOG_TRACE(log, "Setting `processing_threads_num` already equals {}. " "Will do nothing", value); - return; + continue; } new_table_metadata.processing_threads_num = value; } @@ -269,7 +269,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) { LOG_TRACE(log, "Setting `loading_retries` already equals {}. " "Will do nothing", value); - return; + continue; } new_table_metadata.loading_retries = value; } @@ -280,7 +280,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) { LOG_TRACE(log, "Setting `after_processing` already equals {}. " "Will do nothing", value); - return; + continue; } new_table_metadata.after_processing = value; } @@ -291,7 +291,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) { LOG_TRACE(log, "Setting `tracked_files_limit` already equals {}. " "Will do nothing", value); - return; + continue; } new_table_metadata.tracked_files_limit = value; } @@ -302,7 +302,7 @@ void ObjectStorageQueueMetadata::alterSettings(const SettingsChanges & changes) { LOG_TRACE(log, "Setting `tracked_file_ttl_sec` already equals {}. " "Will do nothing", value); - return; + continue; } new_table_metadata.tracked_files_ttl_sec = value; } From 60f0efa67689c28bd5b155eefd3266f385822b94 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 18:08:25 +0000 Subject: [PATCH 534/816] remove debug log --- src/Planner/Planner.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index f1c752aecd0..8d3c75fdabb 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -847,9 +847,6 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, interpolate_description = std::make_shared(std::move(interpolate_actions_dag), empty_aliases); } - if (interpolate_description) - LOG_DEBUG(getLogger("addWithFillStepIfNeeded"), "InterpolateDescription: {}", interpolate_description->actions.dumpDAG()); - const auto & query_context = planner_context->getQueryContext(); const Settings & settings = query_context->getSettingsRef(); auto filling_step = std::make_unique( From 64d038c4408f500ae58a6a3cdd68e99c2901faa0 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 18:14:56 +0000 Subject: [PATCH 535/816] cleanup --- src/Analyzer/SortNode.h | 6 ++--- src/Common/FieldVisitorScale.cpp | 22 +++++++++---------- src/Common/FieldVisitorScale.h | 3 --- src/Core/Field.h | 8 ------- .../Transforms/FillingTransform.cpp | 8 ++----- 5 files changed, 16 insertions(+), 31 deletions(-) diff --git a/src/Analyzer/SortNode.h b/src/Analyzer/SortNode.h index d9086dc9ed7..6f0010abdaa 100644 --- a/src/Analyzer/SortNode.h +++ b/src/Analyzer/SortNode.h @@ -105,19 +105,19 @@ public: return children[fill_step_child_index]; } - /// Returns true if sort node has fill step, false otherwise + /// Returns true if sort node has fill staleness, false otherwise bool hasFillStaleness() const { return children[fill_staleness_child_index] != nullptr; } - /// Get fill step + /// Get fill staleness const QueryTreeNodePtr & getFillStaleness() const { return children[fill_staleness_child_index]; } - /// Get fill step + /// Get fill staleness QueryTreeNodePtr & getFillStaleness() { return children[fill_staleness_child_index]; diff --git a/src/Common/FieldVisitorScale.cpp b/src/Common/FieldVisitorScale.cpp index fdb566007c3..a6c0f6d0c5b 100644 --- a/src/Common/FieldVisitorScale.cpp +++ b/src/Common/FieldVisitorScale.cpp @@ -15,16 +15,16 @@ void FieldVisitorScale::operator() (UInt64 & x) const { x *= rhs; } void FieldVisitorScale::operator() (Float64 & x) const { x *= rhs; } void FieldVisitorScale::operator() (Null &) const { /*Do not scale anything*/ } -void FieldVisitorScale::operator() (String &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Strings"); } -void FieldVisitorScale::operator() (Array &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Arrays"); } -void FieldVisitorScale::operator() (Tuple &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Tuples"); } -void FieldVisitorScale::operator() (Map &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Maps"); } -void FieldVisitorScale::operator() (Object &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Objects"); } -void FieldVisitorScale::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply UUIDs"); } -void FieldVisitorScale::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv4s"); } -void FieldVisitorScale::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply IPv6s"); } -void FieldVisitorScale::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply custom type {}", x.getTypeName()); } -void FieldVisitorScale::operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply AggregateFunctionStates"); } -void FieldVisitorScale::operator() (bool &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot multiply Bools"); } +void FieldVisitorScale::operator() (String &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Strings"); } +void FieldVisitorScale::operator() (Array &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Arrays"); } +void FieldVisitorScale::operator() (Tuple &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Tuples"); } +void FieldVisitorScale::operator() (Map &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Maps"); } +void FieldVisitorScale::operator() (Object &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Objects"); } +void FieldVisitorScale::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale UUIDs"); } +void FieldVisitorScale::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale IPv4s"); } +void FieldVisitorScale::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale IPv6s"); } +void FieldVisitorScale::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale custom type {}", x.getTypeName()); } +void FieldVisitorScale::operator() (AggregateFunctionStateData &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale AggregateFunctionStates"); } +void FieldVisitorScale::operator() (bool &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot scale Bools"); } } diff --git a/src/Common/FieldVisitorScale.h b/src/Common/FieldVisitorScale.h index 45bacdccc9c..90d86cc53bd 100644 --- a/src/Common/FieldVisitorScale.h +++ b/src/Common/FieldVisitorScale.h @@ -1,10 +1,7 @@ #pragma once -#include #include #include -#include "base/Decimal.h" -#include "base/extended_types.h" namespace DB { diff --git a/src/Core/Field.h b/src/Core/Field.h index 47df5c2907e..7b916d30646 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -185,14 +185,6 @@ public: return *this; } - const DecimalField & operator *= (const DecimalField & r) - { - if (scale != r.getScale()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Multiply different decimal fields"); - dec *= r.getValue(); - return *this; - } - const DecimalField & operator -= (const DecimalField & r) { if (scale != r.getScale()) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 7f81b86697c..46a670394a5 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -125,10 +125,6 @@ static FillColumnDescription::StepFunction getStepFunction(const Field & step, c if (jumps_count != 1) applyVisitor(FieldVisitorScale(jumps_count), shifted_step); - logDebug("field", field.dump()); - logDebug("step", step.dump()); - logDebug("shifted field", shifted_step.dump()); - applyVisitor(FieldVisitorSum(shifted_step), field); }; } @@ -684,8 +680,8 @@ void FillingTransform::transformRange( } const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/true); - logDebug("apply", apply); - logDebug("changed", changed); + logDebug("long jump apply", apply); + logDebug("long jump changed", changed); if (changed) filling_row_changed = true; From f905c804f5b5aa0c0b14e9aaab1034fa8fbbef03 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 19:58:53 +0000 Subject: [PATCH 536/816] fix calibration jump --- src/Interpreters/FillingRow.cpp | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index fdd3b55b66b..49ee558cb20 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -153,23 +153,17 @@ std::pair FillingRow::next(const FillingRow & to_row, bool long_jump if (!next_value.has_value()) return {false, false}; - Field calibration_jump_value = next_value.value(); - fill_column_desc.step_func(calibration_jump_value, 1); - - if (equals(calibration_jump_value, to_row[pos])) - next_value = calibration_jump_value; - - if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) - return {false, false}; + /// We need value >= to_row[pos] + fill_column_desc.step_func(next_value.value(), 1); } else { next_value = doJump(fill_column_desc, pos); - - if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) - return {false, false}; } + if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) + return {false, false}; + row[pos] = std::move(next_value.value()); if (equals(row[pos], to_row.row[pos])) { From dda0c2d151363b415946519b0c0c06f70e9a5a9e Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Mon, 28 Oct 2024 23:37:27 +0300 Subject: [PATCH 537/816] The testing method has been changed --- .../test_reload_client_certificate/test.py | 117 ++++++++++++++---- 1 file changed, 94 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index cb091d92ea6..8859875b798 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager TEST_DIR = os.path.dirname(__file__) @@ -144,37 +145,107 @@ def clean_logs(): ) +def drop_secure_zk_connection(pm, node, action="DROP"): + pm._check_instance(node) + pm._add_rule( + { + "source": node.ip_address, + "destination_port": 2281, + "action": action, + } + ) + pm._add_rule( + { + "destination": node.ip_address, + "source_port": 2281, + "action": action, + } + ) + + if node.ipv6_address: + pm._add_rule( + { + "source": node.ipv6_address, + "destination_port": 2281, + "action": action, + } + ) + pm._add_rule( + { + "destination": node.ipv6_address, + "source_port": 2281, + "action": action, + } + ) + + +def restore_secure_zk_connection(pm, node, action="DROP"): + pm._check_instance(node) + pm._delete_rule( + { + "source": node.ip_address, + "destination_port": 2281, + "action": action, + } + ) + pm._delete_rule( + { + "destination": node.ip_address, + "source_port": 2281, + "action": action, + } + ) + + if node.ipv6_address: + pm._delete_rule( + { + "source": node.ipv6_address, + "destination_port": 2281, + "action": action, + } + ) + pm._delete_rule( + { + "destination": node.ipv6_address, + "source_port": 2281, + "action": action, + } + ) + + def check_certificate_switch(first, second): # Set first certificate change_config_to_key(first) - # Restart zookeeper to reload the session + # Restart zookeeper the connection - cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) + with PartitionManager() as pm: + for node in nodes: + drop_secure_zk_connection(pm, node) + for node in nodes: + restore_secure_zk_connection(pm, node) + clean_logs() + + # Change certificate + + change_config_to_key(second) + + # Time to log + + time.sleep(10) + + # Check information about client certificates reloading in log + + reload_successful = any(check_reload_successful(node, second) for node in nodes) + + # Restart zookeeper to reload the session and clean logs for new check + + for node in nodes: + drop_secure_zk_connection(pm, node) + restore_secure_zk_connection(pm, node) clean_logs() - # Change certificate - - change_config_to_key(second) - - # Time to log - - time.sleep(10) - - # Check information about client certificates reloading in log Clickhouse - - reload_successful = any(check_reload_successful(node, second) for node in nodes) - - # Restart zookeeper to reload the session and clean logs for new check - - cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - clean_logs() - cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) - if second == "second": try: secure_connection_test(started_cluster) From 6cf3da7982cf9c678388bf45e4092d778560eade Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 28 Oct 2024 20:53:39 +0000 Subject: [PATCH 538/816] better vertical final in Replacing --- .../Merges/Algorithms/ReplacingSortedAlgorithm.cpp | 13 +++---------- .../Merges/Algorithms/ReplacingSortedAlgorithm.h | 8 ++++++-- .../Transforms/SelectByIndicesTransform.h | 6 +++++- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 5059bc806a8..dbce348d1aa 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -5,8 +5,6 @@ #include #include #include -#include "Common/Logger.h" -#include namespace DB { @@ -165,13 +163,8 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() if (enable_vertical_final) { - auto replace_final_selection = ColumnUInt64::create(chunk_num_rows); - auto & replace_final_data = replace_final_selection->getData(); - - std::iota(replace_final_data.begin(), replace_final_data.end(), 0); - current_chunk.getChunkInfos().add(std::make_shared(std::move(replace_final_selection))); - - Status status(std::move(current_chunk), false); + current_chunk.getChunkInfos().add(std::make_shared()); + Status status(std::move(current_chunk)); status.required_source = source_num; return status; } @@ -188,7 +181,7 @@ IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge() out_row_sources_buf->write(row_source.data); } - Status status(merged_data->pull(), false); + Status status(merged_data->pull()); status.required_source = source_num; return status; } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index b0dd4fe4b08..ec366b900f5 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -13,8 +13,7 @@ class Logger; namespace DB { -/** Use in skipping final to keep list of indices of selected row after merging final - */ +//// Used in skipping final to keep the list of indices of selected rows after merging. struct ChunkSelectFinalIndices : public ChunkInfoCloneable { explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); @@ -24,6 +23,11 @@ struct ChunkSelectFinalIndices : public ChunkInfoCloneable +{ +}; + /** Merges several sorted inputs into one. * For each group of consecutive identical values of the primary key (the columns by which the data is sorted), * keeps row with max `version` value. diff --git a/src/Processors/Transforms/SelectByIndicesTransform.h b/src/Processors/Transforms/SelectByIndicesTransform.h index b44f5a3203e..e67d3bfde51 100644 --- a/src/Processors/Transforms/SelectByIndicesTransform.h +++ b/src/Processors/Transforms/SelectByIndicesTransform.h @@ -26,8 +26,12 @@ public: void transform(Chunk & chunk) override { size_t num_rows = chunk.getNumRows(); - auto select_final_indices_info = chunk.getChunkInfos().extract(); + auto select_all_rows_info = chunk.getChunkInfos().extract(); + if (select_all_rows_info) + return; + + auto select_final_indices_info = chunk.getChunkInfos().extract(); if (!select_final_indices_info || !select_final_indices_info->select_final_indices) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk passed to SelectByIndicesTransform without indices column"); From 5f140ea0a82601b83cc4aa253f9f794d3402fd00 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 28 Oct 2024 18:06:16 -0300 Subject: [PATCH 539/816] progress --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 7 --- .../Impl/Parquet/ParquetLeafColReader.cpp | 57 +++++++++---------- 2 files changed, 27 insertions(+), 37 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 977f2ad298b..9a79bcffad3 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -549,13 +549,6 @@ void ParquetRleDictReader::readBatch( ); } -template <> -void ParquetRleDictReader::readBatch( - MutableColumnPtr & , LazyNullMap &, UInt32) -{ - assert(false); -} - template void ParquetRleDictReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index f32d7e61062..c3c7db510ed 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -425,16 +425,29 @@ void ParquetLeafColReader::initDataReader( degradeDictionary(); } - ParquetDataBuffer parquet_buffer = [&]() + if (col_descriptor.physical_type() == parquet::Type::BOOLEAN) { - if constexpr (!std::is_same_v, TColumn>) - return ParquetDataBuffer(buffer, max_size); + if constexpr (std::is_same_v) + { + auto bit_reader = std::make_unique(buffer, max_size); + data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(bit_reader)); + } + } + else + { + ParquetDataBuffer parquet_buffer = [&]() + { + if constexpr (!std::is_same_v, TColumn>) + return ParquetDataBuffer(buffer, max_size); - auto scale = assert_cast(*base_data_type).getScale(); - return ParquetDataBuffer(buffer, max_size, scale); - }(); - data_values_reader = createPlainReader( - col_descriptor, std::move(def_level_reader), std::move(parquet_buffer)); + auto scale = assert_cast(*base_data_type).getScale(); + return ParquetDataBuffer(buffer, max_size, scale); + }(); + data_values_reader = createPlainReader( + col_descriptor, std::move(def_level_reader), std::move(parquet_buffer)); + } break; } case parquet::Encoding::RLE_DICTIONARY: @@ -463,28 +476,6 @@ void ParquetLeafColReader::initDataReader( } } -template <> -void ParquetLeafColReader::initDataReader( - parquet::Encoding::type enconding_type, - const uint8_t * buffer, - std::size_t max_size, - std::unique_ptr && def_level_reader) -{ - switch (enconding_type) - { - case parquet::Encoding::PLAIN: - { - auto bit_reader = std::make_unique(buffer, max_size); - data_values_reader = std::make_unique>(col_descriptor.max_definition_level(), - std::move(def_level_reader), - std::move(bit_reader)); - break; - } - default: - throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unknown encoding type: {}", enconding_type); - } -} - template void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) { @@ -634,6 +625,12 @@ std::unique_ptr ParquetLeafColReader::createDi }); return res; } + + if (col_descriptor.physical_type() == parquet::Type::type::BOOLEAN) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Dictionary encoding for booleans is not supported"); + } + return std::make_unique>( col_descriptor.max_definition_level(), std::move(def_level_reader), From 6772d3fe6623f73edb4509a7d6e9cbdc5e9883f9 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Mon, 28 Oct 2024 22:08:38 +0000 Subject: [PATCH 540/816] little improvement --- src/Interpreters/FillingRow.cpp | 17 ++++++++++------- src/Interpreters/FillingRow.h | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 49ee558cb20..8c5f102bcd6 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -28,7 +28,7 @@ FillingRow::FillingRow(const SortDescription & sort_description_) : sort_description(sort_description_) { row.resize(sort_description.size()); - staleness_base_row.resize(sort_description.size()); + staleness_border.resize(sort_description.size()); } bool FillingRow::operator<(const FillingRow & other) const @@ -74,10 +74,7 @@ std::optional FillingRow::doJump(const FillColumnDescription& descr, size if (!descr.fill_staleness.isNull()) { - Field staleness_border = staleness_base_row[column_ind]; - descr.staleness_step_func(staleness_border, 1); - - if (less(next_value, staleness_border, getDirection(column_ind))) + if (less(next_value, staleness_border[column_ind], getDirection(column_ind))) return next_value; else return std::nullopt; @@ -93,7 +90,7 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, if (less(to, shifted_value, getDirection(column_ind))) return std::nullopt; - for (int32_t step_len = 1, step_no = 0; step_no < 100; ++step_no) + for (int32_t step_len = 1, step_no = 0; step_no < 100 && step_len > 0; ++step_no) { Field next_value = shifted_value; descr.step_func(next_value, step_len); @@ -194,7 +191,13 @@ void FillingRow::initFromDefaults(size_t from_pos) void FillingRow::initStalenessRow(const Columns& base_row, size_t row_ind) { for (size_t i = 0; i < size(); ++i) - staleness_base_row[i] = (*base_row[i])[row_ind]; + { + staleness_border[i] = (*base_row[i])[row_ind]; + + const auto& descr = getFillDescription(i); + if (!descr.fill_staleness.isNull()) + descr.staleness_step_func(staleness_border[i], 1); + } } String FillingRow::dump() const diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 14b6034ce35..dc787173191 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -46,7 +46,7 @@ public: private: Row row; - Row staleness_base_row; + Row staleness_border; SortDescription sort_description; }; From 812fdf30f8998d24a344a396d3acc4f32ce45068 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 28 Oct 2024 22:22:14 +0000 Subject: [PATCH 541/816] fix tidy build --- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 7 +++++-- .../Scheduler/Nodes/tests/gtest_io_resource_manager.cpp | 2 +- .../Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp | 4 +++- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 28a070a11a6..5c2d859bd07 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -189,7 +189,7 @@ public: ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first ISchedulerNode * node = root_node.get(); size_t pos = 1; - while (pos < path.length()) + while (node && pos < path.length()) { size_t slash = path.find('/', pos); if (slash != String::npos) @@ -204,12 +204,15 @@ public: pos = String::npos; } } - enqueueImpl(dynamic_cast(node), costs); + if (node) + enqueueImpl(dynamic_cast(node), costs); } void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs, const String & name = {}) { ASSERT_TRUE(queue != nullptr); // not a queue + if (!queue) + return; // to make clang-analyzer-core.NonNullParamChecker happy for (ResourceCost cost : costs) queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 51c2b69c705..2bac69185d3 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -28,7 +28,7 @@ public: : WorkloadEntityStorageBase(Context::getGlobalContextInstance()) {} - virtual void loadEntities() override {} + void loadEntities() override {} void executeQuery(const String & query) { diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 159ccc616f4..b5bcc07f71a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -462,14 +462,16 @@ TEST(SchedulerUnifiedNode, ResourceGuardException) std::thread consumer([queue = all->getQueue()] { ResourceLink link{.queue = queue.get()}; + bool caught = false; try { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link); - FAIL(); } catch (...) { + caught = true; } + ASSERT_TRUE(caught); }); // This will destroy the queue and fail both requests From c1f1fe464da7f640f8347167d620b08e4ca8b710 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 28 Oct 2024 22:23:47 +0000 Subject: [PATCH 542/816] enable old tests, leak is fixed --- .../Nodes/tests/gtest_resource_class_fair.cpp | 15 +++++++-------- .../Nodes/tests/gtest_resource_class_priority.cpp | 13 ++++++------- .../Nodes/tests/gtest_throttler_constraint.cpp | 14 ++++++-------- 3 files changed, 19 insertions(+), 23 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp index 16cce309c2a..d859693eba5 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp @@ -8,18 +8,17 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerFairPolicy, Factory) +TEST(SchedulerFairPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", /* event_queue = */ nullptr, *cfg, ""); + EventQueue event_queue; + SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", &event_queue, *cfg, ""); EXPECT_TRUE(dynamic_cast(fair.get()) != nullptr); } -TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) +TEST(SchedulerFairPolicy, FairnessWeights) { ResourceTest t; @@ -43,7 +42,7 @@ TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) t.consumed("B", 20); } -TEST(DISABLED_SchedulerFairPolicy, Activation) +TEST(SchedulerFairPolicy, Activation) { ResourceTest t; @@ -79,7 +78,7 @@ TEST(DISABLED_SchedulerFairPolicy, Activation) t.consumed("B", 10); } -TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) +TEST(SchedulerFairPolicy, FairnessMaxMin) { ResourceTest t; @@ -103,7 +102,7 @@ TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) t.consumed("A", 20); } -TEST(DISABLED_SchedulerFairPolicy, HierarchicalFairness) +TEST(SchedulerFairPolicy, HierarchicalFairness) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp index d3d38aae048..ab248209635 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp @@ -8,18 +8,17 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerPriorityPolicy, Factory) +TEST(SchedulerPriorityPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", /* event_queue = */ nullptr, *cfg, ""); + EventQueue event_queue; + SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", &event_queue, *cfg, ""); EXPECT_TRUE(dynamic_cast(prio.get()) != nullptr); } -TEST(DISABLED_SchedulerPriorityPolicy, Priorities) +TEST(SchedulerPriorityPolicy, Priorities) { ResourceTest t; @@ -53,7 +52,7 @@ TEST(DISABLED_SchedulerPriorityPolicy, Priorities) t.consumed("C", 0); } -TEST(DISABLED_SchedulerPriorityPolicy, Activation) +TEST(SchedulerPriorityPolicy, Activation) { ResourceTest t; @@ -94,7 +93,7 @@ TEST(DISABLED_SchedulerPriorityPolicy, Activation) t.consumed("C", 0); } -TEST(DISABLED_SchedulerPriorityPolicy, SinglePriority) +TEST(SchedulerPriorityPolicy, SinglePriority) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 9bb1bc572b8..585bb738b27 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -10,9 +10,7 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) +TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -42,7 +40,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) t.consumed("A", 10); } -TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) +TEST(SchedulerThrottlerConstraint, Unlimited) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -59,7 +57,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) } } -TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) +TEST(SchedulerThrottlerConstraint, Pacing) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -79,7 +77,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) } } -TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) +TEST(SchedulerThrottlerConstraint, BucketFilling) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -113,7 +111,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) t.consumed("A", 3); } -TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) +TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -141,7 +139,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) } } -TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) +TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); From 0c2ca9f0fbc0f2a5923734e030d3f21c7ce094c2 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 28 Oct 2024 22:26:16 +0000 Subject: [PATCH 543/816] fix test dtor --- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 5c2d859bd07..927f87d5aa6 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -117,7 +117,8 @@ class ResourceTestClass : public ResourceTestBase public: ~ResourceTestClass() { - dequeue(); // Just to avoid any leaks of `Request` object + if (root_node) + dequeue(); // Just to avoid any leaks of `Request` object } template From 7a5a298cfd1707e423ce9d6d0973587c9d12505e Mon Sep 17 00:00:00 2001 From: alsu Date: Mon, 28 Oct 2024 23:26:25 +0100 Subject: [PATCH 544/816] add example with other timezone that requires non-obvious backticks --- docs/en/sql-reference/statements/create/user.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index ec160ea2663..d79e8103763 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -185,6 +185,7 @@ Examples: - `CREATE USER name1 VALID UNTIL '2025-01-01'` - `CREATE USER name1 VALID UNTIL '2025-01-01 12:00:00 UTC'` - `CREATE USER name1 VALID UNTIL 'infinity'` +- ```CREATE USER name1 VALID UNTIL '2025-01-01 12:00:00 `Asia/Tokyo`'``` ## GRANTEES Clause From 2ccfad77e59c64bd620d58a6b5bf4f7a36d157e7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 28 Oct 2024 22:33:20 +0000 Subject: [PATCH 545/816] Fix 02932_refreshable_materialized_views_1 flakiness --- .../02932_refreshable_materialized_views_1.reference | 1 - .../0_stateless/02932_refreshable_materialized_views_1.sh | 8 -------- 2 files changed, 9 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference index b21356db24e..3ec0d3b9ee2 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference @@ -2,7 +2,6 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x <2: refreshed> 3 1 1 <3: time difference at least> 1000 -<4: next refresh in> 2 Scheduled <4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 1 3 3 3 0 <4.5: altered> Scheduled 2050-01-01 00:00:01 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh index 739617a2986..e28d88310c6 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh @@ -50,14 +50,6 @@ done # to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. $CLICKHOUSE_CLIENT -q " select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000);" -while : -do - # Wait for status to change to Scheduled. If status = Scheduling, next_refresh_time is stale. - res="`$CLICKHOUSE_CLIENT -q "select '<4: next refresh in>', next_refresh_time-last_success_time, status from refreshes -- $LINENO"`" - echo "$res" | grep -q 'Scheduled' && break - sleep 0.5 -done -echo "$res" # Create a source table from which views will read. $CLICKHOUSE_CLIENT -q " From a26a34d456e6676b1b27935802ee9f04b29e68fb Mon Sep 17 00:00:00 2001 From: Romeo58rus Date: Tue, 29 Oct 2024 01:52:33 +0300 Subject: [PATCH 546/816] Downgrade a test to stable old method --- .../test_reload_client_certificate/test.py | 117 ++++-------------- 1 file changed, 23 insertions(+), 94 deletions(-) diff --git a/tests/integration/test_reload_client_certificate/test.py b/tests/integration/test_reload_client_certificate/test.py index 8859875b798..cb091d92ea6 100644 --- a/tests/integration/test_reload_client_certificate/test.py +++ b/tests/integration/test_reload_client_certificate/test.py @@ -5,7 +5,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager TEST_DIR = os.path.dirname(__file__) @@ -145,107 +144,37 @@ def clean_logs(): ) -def drop_secure_zk_connection(pm, node, action="DROP"): - pm._check_instance(node) - pm._add_rule( - { - "source": node.ip_address, - "destination_port": 2281, - "action": action, - } - ) - pm._add_rule( - { - "destination": node.ip_address, - "source_port": 2281, - "action": action, - } - ) - - if node.ipv6_address: - pm._add_rule( - { - "source": node.ipv6_address, - "destination_port": 2281, - "action": action, - } - ) - pm._add_rule( - { - "destination": node.ipv6_address, - "source_port": 2281, - "action": action, - } - ) - - -def restore_secure_zk_connection(pm, node, action="DROP"): - pm._check_instance(node) - pm._delete_rule( - { - "source": node.ip_address, - "destination_port": 2281, - "action": action, - } - ) - pm._delete_rule( - { - "destination": node.ip_address, - "source_port": 2281, - "action": action, - } - ) - - if node.ipv6_address: - pm._delete_rule( - { - "source": node.ipv6_address, - "destination_port": 2281, - "action": action, - } - ) - pm._delete_rule( - { - "destination": node.ipv6_address, - "source_port": 2281, - "action": action, - } - ) - - def check_certificate_switch(first, second): # Set first certificate change_config_to_key(first) - # Restart zookeeper the connection + # Restart zookeeper to reload the session - with PartitionManager() as pm: - for node in nodes: - drop_secure_zk_connection(pm, node) - for node in nodes: - restore_secure_zk_connection(pm, node) - clean_logs() - - # Change certificate - - change_config_to_key(second) - - # Time to log - - time.sleep(10) - - # Check information about client certificates reloading in log - - reload_successful = any(check_reload_successful(node, second) for node in nodes) - - # Restart zookeeper to reload the session and clean logs for new check - - for node in nodes: - drop_secure_zk_connection(pm, node) - restore_secure_zk_connection(pm, node) + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) clean_logs() + # Change certificate + + change_config_to_key(second) + + # Time to log + + time.sleep(10) + + # Check information about client certificates reloading in log Clickhouse + + reload_successful = any(check_reload_successful(node, second) for node in nodes) + + # Restart zookeeper to reload the session and clean logs for new check + + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + clean_logs() + cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"]) + if second == "second": try: secure_connection_test(started_cluster) From 30621285a260aca1faf56213994cf8f9f8fa69d5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 29 Oct 2024 11:43:12 +0800 Subject: [PATCH 547/816] Improve plan step desc of optimized trivial count --- .../QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp | 3 ++- .../0_stateless/01710_projection_pk_trivial_count.reference | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 511ae274101..dee16bfcb1a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -752,7 +752,8 @@ std::optional optimizeUseAggregateProjections(QueryPlan::Node & node, Qu Pipe pipe(std::make_shared(std::move(block_with_count))); projection_reading = std::make_unique(std::move(pipe)); - selected_projection_name = "Optimized trivial count"; + /// Use @minmax_count_projection name as it goes through the same optimization. + selected_projection_name = metadata->minmax_count_projection->name; has_ordinary_parts = reading->getAnalyzedResult() != nullptr; } else diff --git a/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference index 43316772467..546c26a232b 100644 --- a/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference +++ b/tests/queries/0_stateless/01710_projection_pk_trivial_count.reference @@ -1,3 +1,3 @@ ReadFromMergeTree (default.x) - ReadFromPreparedSource (Optimized trivial count) + ReadFromPreparedSource (_minmax_count_projection) 5 From 219cc4e5d241201d8bb4838cc440735ec5c905ea Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 29 Oct 2024 12:15:13 +0800 Subject: [PATCH 548/816] fix mismatched aggreage function name of quantileExactWeightedInterpolated --- .../AggregateFunctionQuantileExactWeighted.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp index 58b3b75b056..116b04bf4ba 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp @@ -387,7 +387,7 @@ template using FuncQuantileExactWeighted = AggregateFunctionQuantile< Value, QuantileExactWeighted, - NameQuantileExactWeighted, + std::conditional_t, true, std::conditional_t, false, @@ -396,7 +396,7 @@ template using FuncQuantilesExactWeighted = AggregateFunctionQuantile< Value, QuantileExactWeighted, - NameQuantilesExactWeighted, + std::conditional_t, true, std::conditional_t, true, From 190703b603fe8bfef6d92cc883f9e0107fdce83c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 29 Oct 2024 05:32:52 +0100 Subject: [PATCH 549/816] Close #8687 --- .../03258_multiple_array_joins.reference | 8 +++++++ .../03258_multiple_array_joins.sql | 24 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/03258_multiple_array_joins.reference create mode 100644 tests/queries/0_stateless/03258_multiple_array_joins.sql diff --git a/tests/queries/0_stateless/03258_multiple_array_joins.reference b/tests/queries/0_stateless/03258_multiple_array_joins.reference new file mode 100644 index 00000000000..4d357c8ac80 --- /dev/null +++ b/tests/queries/0_stateless/03258_multiple_array_joins.reference @@ -0,0 +1,8 @@ +1 Michel Foucault alive no +1 Michel Foucault profession philosopher +1 Thomas Aquinas alive no +1 Thomas Aquinas profession philosopher +2 Nicola Tesla alive no +2 Nicola Tesla profession inventor +2 Thomas Edison alive no +2 Thomas Edison profession inventor diff --git a/tests/queries/0_stateless/03258_multiple_array_joins.sql b/tests/queries/0_stateless/03258_multiple_array_joins.sql new file mode 100644 index 00000000000..5afe7725d3f --- /dev/null +++ b/tests/queries/0_stateless/03258_multiple_array_joins.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS test_multiple_array_join; + +CREATE TABLE test_multiple_array_join ( + id UInt64, + person Nested ( + name String, + surname String + ), + properties Nested ( + key String, + value String + ) +) Engine=MergeTree ORDER BY id; + +INSERT INTO test_multiple_array_join VALUES (1, ['Thomas', 'Michel'], ['Aquinas', 'Foucault'], ['profession', 'alive'], ['philosopher', 'no']); +INSERT INTO test_multiple_array_join VALUES (2, ['Thomas', 'Nicola'], ['Edison', 'Tesla'], ['profession', 'alive'], ['inventor', 'no']); + +SELECT * +FROM test_multiple_array_join +ARRAY JOIN person +ARRAY JOIN properties +ORDER BY ALL; + +DROP TABLE test_multiple_array_join; From 934bd3716092139be4b90d7ef20ca0e9ed3e4c56 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 29 Oct 2024 00:48:31 -0400 Subject: [PATCH 550/816] Add a note that wildcard grants feature will be available only after 24.10 --- docs/en/sql-reference/statements/grant.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index c11299baf38..19305675ec8 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -78,6 +78,10 @@ Specifying privileges you can use asterisk (`*`) instead of a table or a databas Also, you can omit database name. In this case privileges are granted for current database. For example, `GRANT SELECT ON * TO john` grants the privilege on all the tables in the current database, `GRANT SELECT ON mytable TO john` grants the privilege on the `mytable` table in the current database. +:::note +The feature described below is available starting with the 24.10 ClickHouse version. +::: + You can also put asterisks at the end of a table or a database name. This feature allows you to grant privileges on an abstract prefix of the table's path. Example: `GRANT SELECT ON db.my_tables* TO john`. This query allows `john` to execute the `SELECT` query over all the `db` database tables with the prefix `my_tables*`. From b98057da1926189416cff77878f41e6c053395ab Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 04:21:53 +0000 Subject: [PATCH 551/816] fix cursor display --- src/Client/ClientBase.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 23aa7e841cb..499055f39a5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1446,6 +1446,9 @@ void ClientBase::onProfileEvents(Block & block) /// Flush all buffers. void ClientBase::resetOutput() { + if (need_render_progress_table && tty_buf) + progress_table.clearTableOutput(*tty_buf); + /// Order is important: format, compression, file if (output_format) From e431628409cdf62f5d9b677c54bd0c1f2c57f0a0 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 05:49:22 +0000 Subject: [PATCH 552/816] make interactive metrics toggling not sticky --- src/Client/ClientBase.cpp | 9 ++++++--- src/Client/ClientBase.h | 1 + 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 499055f39a5..73885ba522d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -470,8 +470,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) { if (!need_render_progress && select_into_file && !select_into_file_and_stdout) error_stream << "\r"; - bool toggle_enabled = getClientConfiguration().getBool("enable-progress-table-toggle", true); - progress_table.writeTable(*tty_buf, progress_table_toggle_on.load(), toggle_enabled); + progress_table.writeTable(*tty_buf, progress_table_toggle_on.load(), progress_table_toggle_enabled); } } @@ -825,6 +824,9 @@ void ClientBase::initTTYBuffer(ProgressOption progress_option, ProgressOption pr if (!need_render_progress && !need_render_progress_table) return; + progress_table_toggle_enabled = getClientConfiguration().getBool("enable-progress-table-toggle"); + progress_table_toggle_on = !progress_table_toggle_enabled; + /// If need_render_progress and need_render_progress_table are enabled, /// use ProgressOption that was set for the progress bar for progress table as well. ProgressOption progress = progress_option ? progress_option : progress_table_option; @@ -881,7 +883,7 @@ void ClientBase::initTTYBuffer(ProgressOption progress_option, ProgressOption pr void ClientBase::initKeystrokeInterceptor() { - if (is_interactive && need_render_progress_table && getClientConfiguration().getBool("enable-progress-table-toggle", true)) + if (is_interactive && need_render_progress_table && progress_table_toggle_enabled) { keystroke_interceptor = std::make_unique(in_fd, error_stream); keystroke_interceptor->registerCallback(' ', [this]() { progress_table_toggle_on = !progress_table_toggle_on; }); @@ -1151,6 +1153,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b if (keystroke_interceptor) { + progress_table_toggle_on = false; try { keystroke_interceptor->startIntercept(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index b06958f1d14..75f09e1d0a2 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -340,6 +340,7 @@ protected: ProgressTable progress_table; bool need_render_progress = true; bool need_render_progress_table = true; + bool progress_table_toggle_enabled = true; std::atomic_bool progress_table_toggle_on = false; bool need_render_profile_events = true; bool written_first_block = false; From 96f992acca63995ecf7749a335cdc46188d43900 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 06:47:03 +0000 Subject: [PATCH 553/816] fix documentation column width --- src/Client/ProgressTable.cpp | 28 ++++++++++++++++++++++------ src/Client/ProgressTable.h | 3 ++- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/src/Client/ProgressTable.cpp b/src/Client/ProgressTable.cpp index 15da659d3fb..d66df4eded8 100644 --- a/src/Client/ProgressTable.cpp +++ b/src/Client/ProgressTable.cpp @@ -180,9 +180,12 @@ void writeWithWidth(Out & out, std::string_view s, size_t width) template void writeWithWidthStrict(Out & out, std::string_view s, size_t width) { - chassert(width != 0); + constexpr std::string_view ellipsis = "…"; if (s.size() > width) - out << s.substr(0, width - 1) << "…"; + if (width <= ellipsis.size()) + out << s.substr(0, width); + else + out << s.substr(0, width - ellipsis.size()) << ellipsis; else out << s; } @@ -219,7 +222,9 @@ void ProgressTable::writeTable(WriteBufferFromFileDescriptor & message, bool sho writeWithWidth(message, COLUMN_EVENT_NAME, column_event_name_width); writeWithWidth(message, COLUMN_VALUE, COLUMN_VALUE_WIDTH); writeWithWidth(message, COLUMN_PROGRESS, COLUMN_PROGRESS_WIDTH); - writeWithWidth(message, COLUMN_DOCUMENTATION_NAME, COLUMN_DOCUMENTATION_WIDTH); + auto col_doc_width = getColumnDocumentationWith(terminal_width); + if (col_doc_width) + writeWithWidth(message, COLUMN_DOCUMENTATION_NAME, col_doc_width); message << CLEAR_TO_END_OF_LINE; double elapsed_sec = watch.elapsedSeconds(); @@ -257,9 +262,12 @@ void ProgressTable::writeTable(WriteBufferFromFileDescriptor & message, bool sho writeWithWidth(message, formatReadableValue(value_type, progress) + "/s", COLUMN_PROGRESS_WIDTH); - message << setColorForDocumentation(); - const auto * doc = getDocumentation(event_name_to_event.at(name)); - writeWithWidthStrict(message, doc, COLUMN_DOCUMENTATION_WIDTH); + if (col_doc_width) + { + message << setColorForDocumentation(); + const auto * doc = getDocumentation(event_name_to_event.at(name)); + writeWithWidthStrict(message, doc, col_doc_width); + } message << RESET_COLOR; message << CLEAR_TO_END_OF_LINE; @@ -372,6 +380,14 @@ size_t ProgressTable::tableSize() const return metrics.empty() ? 0 : metrics.size() + 1; } +size_t ProgressTable::getColumnDocumentationWith(size_t terminal_width) const +{ + auto fixed_columns_width = column_event_name_width + COLUMN_VALUE_WIDTH + COLUMN_PROGRESS_WIDTH; + if (terminal_width < fixed_columns_width + COLUMN_DOCUMENTATION_MIN_WIDTH) + return 0; + return terminal_width - fixed_columns_width; +} + ProgressTable::MetricInfo::MetricInfo(ProfileEvents::Type t) : type(t) { } diff --git a/src/Client/ProgressTable.h b/src/Client/ProgressTable.h index a55326e8d3a..6599dff4aa2 100644 --- a/src/Client/ProgressTable.h +++ b/src/Client/ProgressTable.h @@ -87,6 +87,7 @@ private: }; size_t tableSize() const; + size_t getColumnDocumentationWith(size_t terminal_width) const; using MetricName = String; @@ -110,7 +111,7 @@ private: static constexpr std::string_view COLUMN_DOCUMENTATION_NAME = "Documentation"; static constexpr size_t COLUMN_VALUE_WIDTH = 20; static constexpr size_t COLUMN_PROGRESS_WIDTH = 20; - static constexpr size_t COLUMN_DOCUMENTATION_WIDTH = 100; + static constexpr size_t COLUMN_DOCUMENTATION_MIN_WIDTH = COLUMN_DOCUMENTATION_NAME.size(); std::ostream & output_stream; int in_fd; From cb4eedec061fe9edba9352b80db0d632f1b5323e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 29 Oct 2024 09:19:49 +0100 Subject: [PATCH 554/816] Disable GWPAsan by default --- programs/server/Server.cpp | 10 ++-------- src/Common/GWPAsan.cpp | 2 +- src/Common/GWPAsan.h | 9 --------- src/Common/PODArray.h | 10 ---------- src/Core/ServerSettings.cpp | 1 - src/IO/BufferWithOwnMemory.h | 15 ++------------- 6 files changed, 5 insertions(+), 42 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 56b43a39351..d061d134e69 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -207,7 +207,6 @@ namespace ServerSetting extern const ServerSettingsBool format_alter_operations_with_parentheses; extern const ServerSettingsUInt64 global_profiler_cpu_time_period_ns; extern const ServerSettingsUInt64 global_profiler_real_time_period_ns; - extern const ServerSettingsDouble gwp_asan_force_sample_probability; extern const ServerSettingsUInt64 http_connections_soft_limit; extern const ServerSettingsUInt64 http_connections_store_limit; extern const ServerSettingsUInt64 http_connections_warn_limit; @@ -622,7 +621,7 @@ void sanityChecks(Server & server) #if defined(OS_LINUX) try { - const std::unordered_set fastClockSources = { + const std::unordered_set fast_clock_sources = { // ARM clock "arch_sys_counter", // KVM guest clock @@ -631,7 +630,7 @@ void sanityChecks(Server & server) "tsc", }; const char * filename = "/sys/devices/system/clocksource/clocksource0/current_clocksource"; - if (!fastClockSources.contains(readLine(filename))) + if (!fast_clock_sources.contains(readLine(filename))) server.context()->addWarningMessage("Linux is not using a fast clock source. Performance can be degraded. Check " + String(filename)); } catch (...) // NOLINT(bugprone-empty-catch) @@ -1930,10 +1929,6 @@ try if (global_context->isServerCompletelyStarted()) CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings[ServerSetting::cannot_allocate_thread_fault_injection_probability]); -#if USE_GWP_ASAN - GWPAsan::setForceSampleProbability(new_server_settings[ServerSetting::gwp_asan_force_sample_probability]); -#endif - ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. @@ -2440,7 +2435,6 @@ try #if USE_GWP_ASAN GWPAsan::initFinished(); - GWPAsan::setForceSampleProbability(server_settings[ServerSetting::gwp_asan_force_sample_probability]); #endif try diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index de6991191ea..a210fb3a73a 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -57,7 +57,7 @@ static bool guarded_alloc_initialized = [] opts.MaxSimultaneousAllocations = 1024; if (!env_options_raw || !std::string_view{env_options_raw}.contains("SampleRate")) - opts.SampleRate = 10000; + opts.SampleRate = 0; const char * collect_stacktraces = std::getenv("GWP_ASAN_COLLECT_STACKTRACES"); // NOLINT(concurrency-mt-unsafe) if (collect_stacktraces && std::string_view{collect_stacktraces} == "1") diff --git a/src/Common/GWPAsan.h b/src/Common/GWPAsan.h index 846c3417db4..c01a1130739 100644 --- a/src/Common/GWPAsan.h +++ b/src/Common/GWPAsan.h @@ -8,7 +8,6 @@ #include #include -#include namespace GWPAsan { @@ -39,14 +38,6 @@ inline bool shouldSample() return init_finished.load(std::memory_order_relaxed) && GuardedAlloc.shouldSample(); } -inline bool shouldForceSample() -{ - if (!init_finished.load(std::memory_order_relaxed)) - return false; - std::bernoulli_distribution dist(force_sample_probability.load(std::memory_order_relaxed)); - return dist(thread_local_rng); -} - } #endif diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 48f2ffee8ce..2d69b8ac26c 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -115,11 +115,6 @@ protected: template void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { -#if USE_GWP_ASAN - if (unlikely(GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - char * allocated = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); c_start = allocated + pad_left; @@ -149,11 +144,6 @@ protected: return; } -#if USE_GWP_ASAN - if (unlikely(GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - unprotect(); ptrdiff_t end_diff = c_end - c_start; diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 011291901eb..8bc6d1f680b 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -183,7 +183,6 @@ namespace DB DECLARE(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ DECLARE(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ DECLARE(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ - DECLARE(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ DECLARE(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ DECLARE(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \ DECLARE(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) \ diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index da38bccdea1..79b1bb67aaa 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -44,16 +44,10 @@ struct Memory : boost::noncopyable, Allocator char * m_data = nullptr; size_t alignment = 0; - [[maybe_unused]] bool allow_gwp_asan_force_sample{false}; - Memory() = default; /// If alignment != 0, then allocate memory aligned to specified value. - explicit Memory(size_t size_, size_t alignment_ = 0, bool allow_gwp_asan_force_sample_ = false) - : alignment(alignment_), allow_gwp_asan_force_sample(allow_gwp_asan_force_sample_) - { - alloc(size_); - } + explicit Memory(size_t size_, size_t alignment_ = 0) : alignment(alignment_) { alloc(size_); } ~Memory() { @@ -133,11 +127,6 @@ private: ProfileEvents::increment(ProfileEvents::IOBufferAllocs); ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, new_capacity); -#if USE_GWP_ASAN - if (unlikely(allow_gwp_asan_force_sample && GWPAsan::shouldForceSample())) - gwp_asan::getThreadLocals()->NextSampleCounter = 1; -#endif - m_data = static_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; m_size = new_size; @@ -165,7 +154,7 @@ protected: public: /// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership. explicit BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) - : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment, /*allow_gwp_asan_force_sample_=*/true) + : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment) { Base::set(existing_memory ? existing_memory : memory.data(), size); Base::padded = !existing_memory; From eac5e9883a24af86c277b674c63700763ee8c9a7 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 29 Oct 2024 08:57:37 +0000 Subject: [PATCH 555/816] Remove StorageExternalDistributed --- src/Storages/StorageExternalDistributed.cpp | 233 -------------------- src/Storages/StorageExternalDistributed.h | 43 ---- src/Storages/registerStorages.cpp | 8 - src/TableFunctions/TableFunctionURL.cpp | 1 - 4 files changed, 285 deletions(-) delete mode 100644 src/Storages/StorageExternalDistributed.cpp delete mode 100644 src/Storages/StorageExternalDistributed.h diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp deleted file mode 100644 index ac560b58962..00000000000 --- a/src/Storages/StorageExternalDistributed.cpp +++ /dev/null @@ -1,233 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace Setting -{ - extern const SettingsUInt64 glob_expansion_max_elements; - extern const SettingsUInt64 postgresql_connection_attempt_timeout; - extern const SettingsBool postgresql_connection_pool_auto_close_connection; - extern const SettingsUInt64 postgresql_connection_pool_retries; - extern const SettingsUInt64 postgresql_connection_pool_size; - extern const SettingsUInt64 postgresql_connection_pool_wait_timeout; -} - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -StorageExternalDistributed::StorageExternalDistributed( - const StorageID & table_id_, - std::unordered_set && shards_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment) - : IStorage(table_id_) - , shards(shards_) -{ - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); -} - -void StorageExternalDistributed::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) -{ - std::vector> plans; - for (const auto & shard : shards) - { - plans.emplace_back(std::make_unique()); - shard->read( - *plans.back(), - column_names, - storage_snapshot, - query_info, - context, - processed_stage, - max_block_size, - num_streams - ); - } - - if (plans.empty()) - { - auto header = storage_snapshot->getSampleBlockForColumns(column_names); - InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info); - } - - if (plans.size() == 1) - { - query_plan = std::move(*plans.front()); - return; - } - - Headers input_headers; - input_headers.reserve(plans.size()); - for (auto & plan : plans) - input_headers.emplace_back(plan->getCurrentHeader()); - - auto union_step = std::make_unique(std::move(input_headers)); - query_plan.unitePlans(std::move(union_step), std::move(plans)); -} - -void registerStorageExternalDistributed(StorageFactory & factory) -{ - factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - if (engine_args.size() < 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Engine ExternalDistributed must have at least 2 arguments: " - "engine_name, named_collection and/or description"); - - auto context = args.getLocalContext(); - const auto & settings = context->getSettingsRef(); - size_t max_addresses = settings[Setting::glob_expansion_max_elements]; - auto get_addresses = [&](const std::string addresses_expr) - { - return parseRemoteDescription(addresses_expr, 0, addresses_expr.size(), ',', max_addresses); - }; - - std::unordered_set shards; - ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end()); - - ASTPtr * address_arg = nullptr; - - /// If there is a named collection argument, named `addresses_expr` - for (auto & node : inner_engine_args) - { - if (ASTFunction * func = node->as(); func && func->name == "equals" && func->arguments) - { - if (ASTExpressionList * func_args = func->arguments->as(); func_args && func_args->children.size() == 2) - { - if (ASTIdentifier * arg_name = func_args->children[0]->as(); arg_name && arg_name->name() == "addresses_expr") - { - address_arg = &func_args->children[1]; - break; - } - } - } - } - - /// Otherwise it is the first argument. - if (!address_arg) - address_arg = &inner_engine_args.at(0); - - String addresses_expr = checkAndGetLiteralArgument(*address_arg, "addresses"); - Strings shards_addresses = get_addresses(addresses_expr); - - auto engine_name = checkAndGetLiteralArgument(engine_args[0], "engine_name"); - if (engine_name == "URL") - { - auto format_settings = StorageURL::getFormatSettingsFromArgs(args); - for (const auto & shard_address : shards_addresses) - { - *address_arg = std::make_shared(shard_address); - auto configuration = StorageURL::getConfiguration(inner_engine_args, context); - auto uri_options = parseRemoteDescription(shard_address, 0, shard_address.size(), '|', max_addresses); - if (uri_options.size() > 1) - { - shards.insert( - std::make_shared( - uri_options, args.table_id, configuration.format, format_settings, - args.columns, args.constraints, context, configuration.compression_method)); - } - else - { - shards.insert(std::make_shared( - shard_address, args.table_id, configuration.format, format_settings, - args.columns, args.constraints, String{}, context, configuration.compression_method)); - } - } - } -#if USE_MYSQL - else if (engine_name == "MySQL") - { - MySQLSettings mysql_settings; - for (const auto & shard_address : shards_addresses) - { - *address_arg = std::make_shared(shard_address); - auto configuration = StorageMySQL::getConfiguration(inner_engine_args, context, mysql_settings); - configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 3306); - auto pool = createMySQLPoolWithFailover(configuration, mysql_settings); - shards.insert(std::make_shared( - args.table_id, std::move(pool), configuration.database, configuration.table, - /* replace_query = */ false, /* on_duplicate_clause = */ "", - args.columns, args.constraints, String{}, context, mysql_settings)); - } - } -#endif -#if USE_LIBPQXX - else if (engine_name == "PostgreSQL") - { - for (const auto & shard_address : shards_addresses) - { - *address_arg = std::make_shared(shard_address); - auto configuration = StoragePostgreSQL::getConfiguration(inner_engine_args, context); - configuration.addresses = parseRemoteDescriptionForExternalDatabase(shard_address, max_addresses, 5432); - auto pool = std::make_shared( - configuration, - settings[Setting::postgresql_connection_pool_size], - settings[Setting::postgresql_connection_pool_wait_timeout], - settings[Setting::postgresql_connection_pool_retries], - settings[Setting::postgresql_connection_pool_auto_close_connection], - settings[Setting::postgresql_connection_attempt_timeout]); - shards.insert(std::make_shared( - args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{}, context)); - } - } -#endif - else - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "External storage engine {} is not supported for StorageExternalDistributed. " - "Supported engines are: MySQL, PostgreSQL, URL", - engine_name); - } - - return std::make_shared( - args.table_id, - std::move(shards), - args.columns, - args.constraints, - args.comment); - }, - { - .source_access_type = AccessType::SOURCES, - }); -} - -} diff --git a/src/Storages/StorageExternalDistributed.h b/src/Storages/StorageExternalDistributed.h deleted file mode 100644 index 56c7fe86f34..00000000000 --- a/src/Storages/StorageExternalDistributed.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -#include "config.h" - -#include - - -namespace DB -{ - -/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas. -/// This class unites multiple storages with replicas into multiple shards with replicas. -/// A query to external database is passed to one replica on each shard, the result is united. -/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue. -/// Similar approach is used for URL storage. -class StorageExternalDistributed final : public DB::IStorage -{ -public: - StorageExternalDistributed( - const StorageID & table_id_, - std::unordered_set && shards_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment); - - std::string getName() const override { return "ExternalDistributed"; } - - void read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; - -private: - using Shards = std::unordered_set; - Shards shards; -}; - -} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index cfd406ccbe2..d2c445c8706 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -93,10 +93,6 @@ void registerStoragePostgreSQL(StorageFactory & factory); void registerStorageMaterializedPostgreSQL(StorageFactory & factory); #endif -#if USE_MYSQL || USE_LIBPQXX -void registerStorageExternalDistributed(StorageFactory & factory); -#endif - #if USE_FILELOG void registerStorageFileLog(StorageFactory & factory); #endif @@ -205,10 +201,6 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]]) registerStorageMaterializedPostgreSQL(factory); #endif - #if USE_MYSQL || USE_LIBPQXX - registerStorageExternalDistributed(factory); - #endif - #if USE_SQLITE registerStorageSQLite(factory); #endif diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 2bdc0b449e0..8f4841a992b 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include From d2762d903e0012d715d31e366a4493d71c386a7d Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Tue, 29 Oct 2024 09:04:38 +0000 Subject: [PATCH 556/816] apply comments --- .../0_stateless/03254_attach_part_order.sql | 35 ------------------- .../03254_attach_part_order.reference | 0 .../queries/bugs/03254_attach_part_order.sql | 34 ++++++++++++++++++ 3 files changed, 34 insertions(+), 35 deletions(-) delete mode 100644 tests/queries/0_stateless/03254_attach_part_order.sql rename tests/queries/{0_stateless => bugs}/03254_attach_part_order.reference (100%) create mode 100644 tests/queries/bugs/03254_attach_part_order.sql diff --git a/tests/queries/0_stateless/03254_attach_part_order.sql b/tests/queries/0_stateless/03254_attach_part_order.sql deleted file mode 100644 index 49500b1c868..00000000000 --- a/tests/queries/0_stateless/03254_attach_part_order.sql +++ /dev/null @@ -1,35 +0,0 @@ -DROP DATABASE IF EXISTS test_attach_order_db; -CREATE DATABASE test_attach_order_db ENGINE=Atomic; - -CREATE TABLE test_attach_order_db.test_table -( - dt DateTime, - id UInt32, - url String, - visits UInt32 -) -ENGINE ReplacingMergeTree -ORDER BY (dt, id) -PARTITION BY toYYYYMM(dt); - -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 100); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 101); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 102); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 103); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 104); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 105); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 106); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 107); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 108); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 109); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 110); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 111); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 112); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 113); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 114); -INSERT INTO test_attach_order_db.test_table VALUES (toDate('2024-10-24'), 1, '/index', 115); - -ALTER TABLE test_attach_order_db.test_table DETACH PARTITION 202410; -ALTER TABLE test_attach_order_db.test_table ATTACH PARTITION 202410; - -SELECT id, visits FROM test_attach_order_db.test_table FINAL ORDER BY id FORMAT Vertical; \ No newline at end of file diff --git a/tests/queries/0_stateless/03254_attach_part_order.reference b/tests/queries/bugs/03254_attach_part_order.reference similarity index 100% rename from tests/queries/0_stateless/03254_attach_part_order.reference rename to tests/queries/bugs/03254_attach_part_order.reference diff --git a/tests/queries/bugs/03254_attach_part_order.sql b/tests/queries/bugs/03254_attach_part_order.sql new file mode 100644 index 00000000000..81439dca030 --- /dev/null +++ b/tests/queries/bugs/03254_attach_part_order.sql @@ -0,0 +1,34 @@ +CREATE TABLE test_table +( + dt DateTime, + id UInt32, + url String, + visits UInt32 +) +ENGINE ReplacingMergeTree +ORDER BY (dt, id) +PARTITION BY toYYYYMM(dt); + +SYSTEM STOP merges test_table; + +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 100); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 101); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 102); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 103); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 104); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 105); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 106); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 107); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 108); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 109); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 110); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 111); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 112); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 113); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 114); +INSERT INTO test_table VALUES (toDate('2024-10-24'), 1, '/index', 115); + +ALTER TABLE test_table DETACH PARTITION 202410; +ALTER TABLE test_table ATTACH PARTITION 202410; + +SELECT id, visits FROM test_table FINAL ORDER BY id FORMAT Vertical; \ No newline at end of file From af7aa7de568063c53d849150be83ee625413dc7d Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 10:03:02 +0000 Subject: [PATCH 557/816] Fix some bugs --- .../ObjectStorage/DataLakes/Common.cpp | 7 +++ .../DataLakes/DataLakeConfiguration.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 46 +++++++++++++++++-- .../ObjectStorage/StorageObjectStorage.h | 2 + .../registerStorageObjectStorage.cpp | 3 +- .../TableFunctionObjectStorage.cpp | 5 +- .../TableFunctionObjectStorageCluster.cpp | 7 +-- .../configs/config.d/filesystem_caches.xml | 1 + .../integration/test_storage_iceberg/test.py | 14 ++++-- 9 files changed, 74 insertions(+), 13 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp index 4830cc52a90..c21c0486eca 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -1,6 +1,9 @@ #include "Common.h" #include #include +#include +#include +#include #include namespace DB @@ -13,6 +16,10 @@ std::vector listFiles( { auto key = std::filesystem::path(configuration.getPath()) / prefix; RelativePathsWithMetadata files_with_metadata; + // time_t now = time(nullptr); + Poco::DateTime now; + std::string formatted = Poco::DateTimeFormatter::format(now, Poco::DateTimeFormat::ISO8601_FORMAT); + LOG_ERROR(&Poco::Logger::get("Inside listFiles"), "Time of files listing: {}", formatted); object_storage.listObjects(key, files_with_metadata, 0); Strings res; for (const auto & file_with_metadata : files_with_metadata) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 18ff6d93c46..8a4147308f3 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -36,7 +36,7 @@ public: void update(ObjectStoragePtr object_storage, ContextPtr local_context) override { - BaseStorageConfiguration::update(object_storage, local_context); + // BaseStorageConfiguration::update(object_storage, local_context); auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); if (current_metadata && *current_metadata == *new_metadata) return; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ddc6276a8a1..6f4c0787e81 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -22,6 +22,7 @@ #include #include #include +#include "Databases/LoadingStrictnessLevel.h" #include "Storages/ColumnsDescription.h" @@ -68,6 +69,27 @@ String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, Con return ""; } +void printConfiguration(const Poco::Util::AbstractConfiguration & config, std::string log_name, const std::string & prefix = "") +{ + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(prefix, keys); + + for (const auto & key : keys) + { + std::string fullKey = prefix.empty() ? key : (prefix + "." + key); + + if (config.hasProperty(fullKey)) + { + std::string value = config.getString(fullKey); + LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", fullKey, value); + } + + // Recursively print sub-configurations + printConfiguration(config, fullKey, log_name); + } +} + + StorageObjectStorage::StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, @@ -77,6 +99,7 @@ StorageObjectStorage::StorageObjectStorage( const ConstraintsDescription & constraints_, const String & comment, std::optional format_settings_, + LoadingStrictnessLevel mode, bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) @@ -87,11 +110,27 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - ColumnsDescription columns{columns_}; - LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size()); - configuration->update(object_storage, context); + // LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size()); + printConfiguration(context->getConfigRef(), "Storage create"); + try + { + // configuration->update(object_storage, context); + } + catch (...) + { + if (mode <= LoadingStrictnessLevel::CREATE) + { + throw; + } + else + { + tryLogCurrentException(__PRETTY_FUNCTION__); + return; + } + } std::string sample_path; + ColumnsDescription columns{columns_}; resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context); configuration->check(context); @@ -271,6 +310,7 @@ void StorageObjectStorage::read( size_t num_streams) { configuration->update(object_storage, local_context); + printConfiguration(local_context->getConfigRef(), "Select query"); if (partition_by && configuration->withPartitionWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index dc461e5861d..6ca1613e65c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -57,6 +57,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, std::optional format_settings_, + LoadingStrictnessLevel mode, bool distributed_processing_ = false, ASTPtr partition_by_ = nullptr); @@ -217,6 +218,7 @@ public: virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context); + protected: virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 9a525b4e21a..a0393ea3e6a 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -51,13 +51,14 @@ static std::shared_ptr createStorageObjectStorage( return std::make_shared( configuration, - configuration->createObjectStorage(context, /* is_readonly */false), + configuration->createObjectStorage(context, /* is_readonly */ false), args.getContext(), args.table_id, args.columns, args.constraints, args.comment, format_settings, + args.mode, /* distributed_processing */ false, partition_by); } diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 66c90b15c0b..6d81269f2d7 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -117,8 +117,9 @@ StoragePtr TableFunctionObjectStorage::executeImpl( columns, ConstraintsDescription{}, String{}, - /* format_settings */std::nullopt, - /* distributed_processing */false, + /* format_settings */ std::nullopt, + /* mode */ LoadingStrictnessLevel::CREATE, + /* distributed_processing */ false, nullptr); storage->startup(); diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 449bd2c8c49..5ca26aabe32 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -41,9 +41,10 @@ StoragePtr TableFunctionObjectStorageCluster::execute StorageID(Base::getDatabaseName(), table_name), columns, ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings - /* distributed_processing */true, + /* comment */ String{}, + /* format_settings */ std::nullopt, /// No format_settings + /* mode */ LoadingStrictnessLevel::CREATE, + /* distributed_processing */ true, /*partition_by_=*/nullptr); } else diff --git a/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml index e91362640fe..3b1b2aeb37e 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml @@ -5,4 +5,5 @@ cache1 + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 36aba550dbd..ca78fbea667 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -6,6 +6,8 @@ import time import uuid from datetime import datetime +from logging import log + import pyspark import pytest from azure.storage.blob import BlobServiceClient @@ -856,14 +858,20 @@ def test_restart_broken_s3(started_cluster): ) minio_client.remove_bucket(bucket) + print("Before restart: ", datetime.now()) + instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error( - f"SELECT count() FROM {TABLE_NAME}" - ) + # assert "NoSuchBucket" in instance.query_and_get_error( + # f"SELECT count() FROM {TABLE_NAME}" + # ) + + time.sleep(10) minio_client.make_bucket(bucket) + print("Before successful select: ", datetime.now()) + files = default_upload_directory( started_cluster, "s3", From 50f6e60eb2e7b0f7b6987b8cc029f2902c85a28a Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 29 Oct 2024 10:27:05 +0000 Subject: [PATCH 558/816] Remove the query from the list as soon as we can There's no need to take again the lock. This comes from a prior version where the logic needed to be different. --- src/Interpreters/QueryMetricLog.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 54a09efba7b..cad0410eac4 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -142,19 +142,18 @@ void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time, /// queries_mutex. So, to prevent a deadblock we need to make sure that we always lock them in /// that order. { - /// Take ownership of the task so that we can destroy it in this scope after unlocking `queries_lock`. + /// Take ownership of the task so that we can destroy it in this scope after unlocking `queries_mutex`. auto task = std::move(it->second.task); /// Build an empty task for the old task to make sure it does not lock any mutex on its destruction. it->second.task = {}; + queries.erase(query_id); + /// Ensure `queries_mutex` is unlocked before calling task's destructor at the end of this /// scope which will lock `exec_mutex`. lock.unlock(); } - - lock.lock(); - queries.erase(query_id); } std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next) @@ -164,7 +163,7 @@ std::optional QueryMetricLog::createLogMetricElement(cons auto query_status_it = queries.find(query_id); /// The query might have finished while the scheduled task is running. - if (query_status_it == queries.end() || !query_status_it->second.task) + if (query_status_it == queries.end()) { lock.unlock(); LOG_TRACE(logger, "Query {} finished already while this collecting task was running", query_id); From d063fc0a1263ed5f0a799156aafc5ec830155a7e Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 29 Oct 2024 10:29:33 +0000 Subject: [PATCH 559/816] Add trace to help debugging the issue on the CI --- src/Interpreters/QueryMetricLog.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index cad0410eac4..e15c29915ad 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -21,10 +21,10 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -}; +// namespace ErrorCodes +// { +// extern const int LOGICAL_ERROR; +// }; static auto logger = getLogger("QueryMetricLog"); @@ -196,9 +196,11 @@ std::optional QueryMetricLog::createLogMetricElement(cons /// Profile event count is monotonically increasing. if (new_value < prev_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", + LOG_TRACE(logger, "PMO: Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); + // throw Exception(ErrorCodes::LOGICAL_ERROR, + // "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", + // ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); elem.profile_events[i] = new_value - prev_value; prev_value = new_value; From 4839c1d9cebd8e0a8f3221b250f4c90ae2910196 Mon Sep 17 00:00:00 2001 From: xmy Date: Tue, 29 Oct 2024 18:42:35 +0800 Subject: [PATCH 560/816] Support write hdfs files with space --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 14 ++++++------- .../HDFS/WriteBufferFromHDFS.cpp | 21 ++++++++++--------- .../ObjectStorage/HDFS/WriteBufferFromHDFS.h | 3 ++- tests/integration/test_storage_hdfs/test.py | 15 +++++++++++++ 4 files changed, 35 insertions(+), 18 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 182534529ea..7698193ee2f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -103,15 +103,15 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); - std::string path = object.remote_path; - if (path.starts_with("/")) - path = path.substr(1); - if (!path.starts_with(url)) - path = fs::path(url) / path; - + auto path = extractObjectKeyFromURL(object); /// Single O_WRONLY in libhdfs adds O_TRUNC return std::make_unique( - path, config, settings->replication, patchSettings(write_settings), buf_size, + url_without_path, + fs::path(data_directory) / path, + config, + settings->replication, + patchSettings(write_settings), + buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); } diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 4f6f8c782f2..4879dc41d53 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -29,6 +29,7 @@ extern const int CANNOT_FSYNC; struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { std::string hdfs_uri; + std::string hdfs_file_path; hdfsFile fout; HDFSBuilderWrapper builder; HDFSFSPtr fs; @@ -36,25 +37,24 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, + const std::string & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_, int flags) : hdfs_uri(hdfs_uri_) + , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri, config_)) , fs(createHDFSFS(builder.get())) , write_settings(write_settings_) { - const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); - const String path = hdfs_uri.substr(begin_of_path); - /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); + fout = hdfsOpenFile(fs.get(), hdfs_file_path.c_str(), flags, 0, replication_, 0); if (fout == nullptr) { throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} ({}) error: {}", - path, hdfs_uri, std::string(hdfsGetLastError())); + hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); } } @@ -71,7 +71,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl rlock.unlock(std::max(0, bytes_written)); if (bytes_written < 0) - throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); + throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {}, hdfs_uri: {}, {}", hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); @@ -83,20 +83,21 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { int result = hdfsSync(fs.get(), fout); if (result < 0) - throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot HDFS sync {} {}", hdfs_uri, std::string(hdfsGetLastError())); + throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot HDFS sync {}, hdfs_url: {}, {}", hdfs_file_path, hdfs_uri, std::string(hdfsGetLastError())); } }; WriteBufferFromHDFS::WriteBufferFromHDFS( - const std::string & hdfs_name_, + const std::string & hdfs_uri_, + const std::string & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_, size_t buf_size_, int flags_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(std::make_unique(hdfs_name_, config_, replication_, write_settings_, flags_)) - , filename(hdfs_name_) + , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, replication_, write_settings_, flags_)) + , filename(hdfs_file_path_) { } diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h index e3f0ae96a8f..8166da92e16 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h @@ -22,7 +22,8 @@ class WriteBufferFromHDFS final : public WriteBufferFromFileBase public: WriteBufferFromHDFS( - const String & hdfs_name_, + const String & hdfs_uri_, + const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, int replication_, const WriteSettings & write_settings_ = {}, diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 362ea7d5bda..366bc28d2c9 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -396,6 +396,21 @@ def test_read_files_with_spaces(started_cluster): node1.query(f"drop table test") +def test_write_files_with_spaces(started_cluster): + fs = HdfsClient(hosts=started_cluster.hdfs_ip) + dir = "/itime=2024-10-24 10%3A02%3A04" + fs.mkdirs(dir) + + node1.query( + f"insert into function hdfs('hdfs://hdfs1:9000{dir}/test.csv', TSVRaw) select 123 settings hdfs_truncate_on_insert=1" + ) + result = node1.query( + f"select * from hdfs('hdfs://hdfs1:9000{dir}/test.csv', TSVRaw)" + ) + assert int(result) == 123 + fs.delete(dir, recursive=True) + + def test_truncate_table(started_cluster): hdfs_api = started_cluster.hdfs_api node1.query( From 646a48e36b1ecc00dd78a1af42c121eacb225575 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Oct 2024 10:51:15 +0000 Subject: [PATCH 561/816] Escape special symbols in files for JSON subcolumns --- src/DataTypes/Serializations/ISerialization.cpp | 2 +- .../0_stateless/03257_json_escape_file_names.reference | 3 +++ .../0_stateless/03257_json_escape_file_names.sql | 10 ++++++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03257_json_escape_file_names.reference create mode 100644 tests/queries/0_stateless/03257_json_escape_file_names.sql diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index fdcdf9e0cda..3c3e9bdc9f9 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -206,7 +206,7 @@ String getNameForSubstreamPath( else if (it->type == SubstreamType::ObjectSharedData) stream_name += ".object_shared_data"; else if (it->type == SubstreamType::ObjectTypedPath || it->type == SubstreamType::ObjectDynamicPath) - stream_name += "." + it->object_path_name; + stream_name += "." + escapeForFileName(it->object_path_name); } return stream_name; diff --git a/tests/queries/0_stateless/03257_json_escape_file_names.reference b/tests/queries/0_stateless/03257_json_escape_file_names.reference new file mode 100644 index 00000000000..f44e7d62cc1 --- /dev/null +++ b/tests/queries/0_stateless/03257_json_escape_file_names.reference @@ -0,0 +1,3 @@ +{"a-b-c":"43","a-b\\/c-d\\/e":"44","a\\/b\\/c":"42"} +42 43 44 +42 43 44 diff --git a/tests/queries/0_stateless/03257_json_escape_file_names.sql b/tests/queries/0_stateless/03257_json_escape_file_names.sql new file mode 100644 index 00000000000..9cc150170fd --- /dev/null +++ b/tests/queries/0_stateless/03257_json_escape_file_names.sql @@ -0,0 +1,10 @@ +set allow_experimental_json_type = 1; +drop table if exists test; +create table test (json JSON) engine=MergeTree order by tuple() settings min_rows_for_wide_part=0, min_bytes_for_wide_part=0; +insert into test format JSONAsObject {"a/b/c" : 42, "a-b-c" : 43, "a-b/c-d/e" : 44}; + +select * from test; +select json.`a/b/c`, json.`a-b-c`, json.`a-b/c-d/e` from test; +select json.`a/b/c`.:Int64, json.`a-b-c`.:Int64, json.`a-b/c-d/e`.:Int64 from test; +drop table test; + From 0d22cbe47fc7b38049157f5f8466e45b43f3e691 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Oct 2024 11:08:27 +0000 Subject: [PATCH 562/816] Fix bad_weak_ptr exception with Dynamic in functions comparison --- src/Functions/FunctionsComparison.h | 4 ++-- src/Functions/transform.cpp | 2 +- .../03258_dynamic_in_functions_weak_ptr_exception.reference | 0 .../03258_dynamic_in_functions_weak_ptr_exception.sql | 6 ++++++ 4 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.reference create mode 100644 tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.sql diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index bd6f0361307..be0875581a5 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1171,7 +1171,7 @@ public: if (left_tuple && right_tuple) { - auto func = FunctionToOverloadResolverAdaptor(std::make_shared>(check_decimal_overflow)); + auto func = std::make_shared(std::make_shared>(check_decimal_overflow)); bool has_nullable = false; bool has_null = false; @@ -1181,7 +1181,7 @@ public: { ColumnsWithTypeAndName args = {{nullptr, left_tuple->getElements()[i], ""}, {nullptr, right_tuple->getElements()[i], ""}}; - auto element_type = func.build(args)->getResultType(); + auto element_type = func->build(args)->getResultType(); has_nullable = has_nullable || element_type->isNullable(); has_null = has_null || element_type->onlyNull(); } diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 45f0a7f5c17..e5445b36809 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -211,7 +211,7 @@ namespace ColumnsWithTypeAndName args = arguments; args[0].column = args[0].column->cloneResized(input_rows_count)->convertToFullColumnIfConst(); - auto impl = FunctionToOverloadResolverAdaptor(std::make_shared()).build(args); + auto impl = std::make_shared(std::make_shared())->build(args); return impl->execute(args, result_type, input_rows_count); } diff --git a/tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.reference b/tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.sql b/tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.sql new file mode 100644 index 00000000000..f825353c135 --- /dev/null +++ b/tests/queries/0_stateless/03258_dynamic_in_functions_weak_ptr_exception.sql @@ -0,0 +1,6 @@ +SET allow_experimental_dynamic_type = 1; +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Tuple(c1 Int,c2 Dynamic)) ENGINE = Memory(); +SELECT 1 FROM t0 tx JOIN t0 ty ON tx.c0 = ty.c0; +DROP TABLE t0; + From 67609e31ad1efa5cf1e077f819f6e6cabaad1697 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Oct 2024 12:15:34 +0100 Subject: [PATCH 563/816] Review fixes --- .../ObjectStorageQueueOrderedFileMetadata.cpp | 1 - .../ObjectStorageQueueUnorderedFileMetadata.cpp | 17 +++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp index b8138cc1377..72e9e073f27 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueOrderedFileMetadata.cpp @@ -389,7 +389,6 @@ void ObjectStorageQueueOrderedFileMetadata::setProcessedImpl() const auto zk_client = getZooKeeper(); std::string failure_reason; - std::map request_id; while (true) { diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp index 32f8e347c0e..2050797a2ea 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueUnorderedFileMetadata.cpp @@ -111,7 +111,7 @@ void ObjectStorageQueueUnorderedFileMetadata::setProcessedImpl() const auto zk_client = getZooKeeper(); Coordination::Requests requests; - std::map request_id; + std::map request_index; if (processing_id_version.has_value()) { @@ -121,14 +121,14 @@ void ObjectStorageQueueUnorderedFileMetadata::setProcessedImpl() /// The order is important: /// we must first check processing nodes and set processed_path the last. - request_id[CHECK_PROCESSING_ID_PATH] = 0; - request_id[REMOVE_PROCESSING_ID_PATH] = 1; - request_id[REMOVE_PROCESSING_PATH] = 2; - request_id[SET_PROCESSED_PATH] = 3; + request_index[CHECK_PROCESSING_ID_PATH] = 0; + request_index[REMOVE_PROCESSING_ID_PATH] = 1; + request_index[REMOVE_PROCESSING_PATH] = 2; + request_index[SET_PROCESSED_PATH] = 3; } else { - request_id[SET_PROCESSED_PATH] = 0; + request_index[SET_PROCESSED_PATH] = 0; } requests.push_back( @@ -138,9 +138,10 @@ void ObjectStorageQueueUnorderedFileMetadata::setProcessedImpl() Coordination::Responses responses; auto is_request_failed = [&](RequestType type) { - if (!request_id.contains(type)) + if (!request_index.contains(type)) return false; - return responses[request_id[type]]->error != Coordination::Error::ZOK; + chassert(request_index[type] < responses.size()); + return responses[request_index[type]]->error != Coordination::Error::ZOK; }; const auto code = zk_client->tryMulti(requests, responses); From 1c81d0fab98cd952f647f0265ef749798644b402 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 29 Oct 2024 12:17:42 +0100 Subject: [PATCH 564/816] fix --- .../0_stateless/03252_check_number_of_arguments_for_dynamic.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql index 86b74b22175..79a8617930e 100644 --- a/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql +++ b/tests/queries/0_stateless/03252_check_number_of_arguments_for_dynamic.sql @@ -1,3 +1,4 @@ +set enable_analyzer=1; set allow_experimental_json_type=1; CREATE TABLE t From 9a9aadc33fff48bbbe3a256876d25e1741f822d0 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 11:21:24 +0000 Subject: [PATCH 565/816] Added info --- .../engines/table-engines/integrations/s3.md | 26 +++++++++++++++++++ docs/en/sql-reference/table-functions/s3.md | 8 ++++++ src/IO/Archives/createArchiveReader.cpp | 5 +++- 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 2675c193519..3e66db112c1 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -320,6 +320,32 @@ The following settings can be specified in configuration file for given endpoint ``` +## Working with archives + +Suppose that we have several archive files with following URIs on S3: + +- 'https://s3-us-west-1.amazonaws.com/umbrella-static/top-1m-2018-01-10.csv.zip' +- 'https://s3-us-west-1.amazonaws.com/umbrella-static/top-1m-2018-01-11.csv.zip' +- 'https://s3-us-west-1.amazonaws.com/umbrella-static/top-1m-2018-01-12.csv.zip' + +Extracting data from these archives is possible using ::. Globs can be used both in the url part as well as in the part after :: (responsible for the name of a file inside the archive). + +``` sql +SELECT * +FROM s3( + 'https://s3-us-west-1.amazonaws.com/umbrella-static/top-1m-2018-01-1{0..2}.csv.zip :: *.csv' +); +``` + +:::note +ClickHouse supports three archive formats: +ZIP +TAR +7Z +While ZIP and TAR archives can be accessed from any supported storage location, 7Z archives can only be read from the local filesystem where ClickHouse is installed. +::: + + ## Accessing public buckets ClickHouse tries to fetch credentials from many different types of sources. diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index df4e10425a5..b14eb84392f 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -284,6 +284,14 @@ FROM s3( ); ``` +:::note +ClickHouse supports three archive formats: +ZIP +TAR +7Z +While ZIP and TAR archives can be accessed from any supported storage location, 7Z archives can only be read from the local filesystem where ClickHouse is installed. +::: + ## Virtual Columns {#virtual-columns} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index dfa098eede0..97597cc4db7 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -43,7 +43,10 @@ std::shared_ptr createArchiveReader( else if (hasSupported7zExtension(path_to_archive)) { #if USE_LIBARCHIVE - return std::make_shared(path_to_archive); + if (archive_read_function) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "7z archive supports only local files reading"); + else + return std::make_shared(path_to_archive); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif From d5029c4af7f4f8f6ca5d851d05a42bf63fc29e1b Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 29 Oct 2024 11:30:02 +0000 Subject: [PATCH 566/816] Add timestamp of QueryInfoStatus to debug trace to ease debugging --- src/Interpreters/QueryMetricLog.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index e15c29915ad..713e0edfb17 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -15,6 +15,7 @@ #include #include +#include #include @@ -158,7 +159,11 @@ void QueryMetricLog::finishQuery(const String & query_id, TimePoint finish_time, std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint query_info_time, bool schedule_next) { - LOG_DEBUG(logger, "Collecting query_metric_log for query {}. Schedule next: {}", query_id, schedule_next); + /// fmtlib supports subsecond formatting in 10.0.0. We're in 9.1.0, so we need to add the milliseconds ourselves. + auto seconds = std::chrono::time_point_cast(query_info_time); + auto milliseconds = std::chrono::duration_cast(query_info_time - seconds).count(); + LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:05}. Schedule next: {}", query_id, seconds, milliseconds, schedule_next); + std::unique_lock lock(queries_mutex); auto query_status_it = queries.find(query_id); From cf0694fd64d7076a43f9043cf9a06c444c634e9a Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 29 Oct 2024 11:32:48 +0000 Subject: [PATCH 567/816] Revert "Add trace to help debugging the issue on the CI" This reverts commit d063fc0a1263ed5f0a799156aafc5ec830155a7e. --- src/Interpreters/QueryMetricLog.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 713e0edfb17..d274d9e139c 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -22,10 +22,10 @@ namespace DB { -// namespace ErrorCodes -// { -// extern const int LOGICAL_ERROR; -// }; +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +}; static auto logger = getLogger("QueryMetricLog"); @@ -201,11 +201,9 @@ std::optional QueryMetricLog::createLogMetricElement(cons /// Profile event count is monotonically increasing. if (new_value < prev_value) - LOG_TRACE(logger, "PMO: Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); - // throw Exception(ErrorCodes::LOGICAL_ERROR, - // "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", - // ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); elem.profile_events[i] = new_value - prev_value; prev_value = new_value; From e2459c663deb7c1f573b0ee5418d0c5042193f16 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 12:38:59 +0100 Subject: [PATCH 568/816] Fix tidy report --- src/Core/BaseSettings.cpp | 6 +++--- src/Core/BaseSettings.h | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 51e99262bdb..9d55179a5db 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -32,14 +32,14 @@ void BaseSettingsHelpers::writeFlags(Flags flags, WriteBuffer & out) } -BaseSettingsHelpers::Flags BaseSettingsHelpers::readFlags(ReadBuffer & in) +UInt64 BaseSettingsHelpers::readFlags(ReadBuffer & in) { UInt64 res; readVarUInt(res, in); - return static_cast(res); + return res; } -SettingsTierType BaseSettingsHelpers::getTier(Flags flags) +SettingsTierType BaseSettingsHelpers::getTier(UInt64 flags) { int8_t tier = (flags & Flags::TIER); if (tier > SettingsTierType::BETA) diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 218460330f4..949b884636f 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -38,9 +38,9 @@ struct BaseSettingsHelpers /// If adding new flags, consider first if Tier might need more bits }; - static SettingsTierType getTier(Flags flags); + static SettingsTierType getTier(UInt64 flags); static void writeFlags(Flags flags, WriteBuffer & out); - static Flags readFlags(ReadBuffer & in); + static UInt64 readFlags(ReadBuffer & in); }; /** Template class to define collections of settings. @@ -481,7 +481,7 @@ void BaseSettings::read(ReadBuffer & in, SettingsWriteFormat format) size_t index = accessor.find(name); using Flags = BaseSettingsHelpers::Flags; - Flags flags{0}; + UInt64 flags{0}; if (format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) flags = BaseSettingsHelpers::readFlags(in); bool is_important = (flags & Flags::IMPORTANT); @@ -860,7 +860,7 @@ using AliasMap = std::unordered_map; String name; \ const char * type; \ const char * description; \ - BaseSettingsHelpers::Flags flags; \ + UInt64 flags; \ Field (*cast_value_util_function)(const Field &); \ String (*value_to_string_util_function)(const Field &); \ Field (*string_to_value_util_function)(const String &); \ @@ -972,7 +972,7 @@ struct DefineAliases #define IMPLEMENT_SETTINGS_TRAITS_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ res.field_infos.emplace_back( \ FieldInfo{#NAME, #TYPE, DESCRIPTION, \ - static_cast(FLAGS), \ + static_cast(FLAGS), \ [](const Field & value) -> Field { return static_cast(SettingField##TYPE{value}); }, \ [](const Field & value) -> String { return SettingField##TYPE{value}.toString(); }, \ [](const String & str) -> Field { SettingField##TYPE temp; temp.parseFromString(str); return static_cast(temp); }, \ From e18f2cf364fa04a437a41dce2278fff4964bda07 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 29 Oct 2024 12:49:00 +0100 Subject: [PATCH 569/816] Don't delete blob when a node is using it --- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 850623157a1..fbfa2916faa 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9891,7 +9891,14 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else if (error_code == Coordination::Error::ZNOTEMPTY) { - LOG_TRACE(logger, "Cannot remove last parent zookeeper lock {} for part {} with id {}, another replica locked part concurrently", zookeeper_part_uniq_node, part_name, part_id); + LOG_TRACE( + logger, + "Cannot remove last parent zookeeper lock {} for part {} with id {}, another replica locked part concurrently", + zookeeper_part_uniq_node, + part_name, + part_id); + part_has_no_more_locks = false; + continue; } else if (error_code == Coordination::Error::ZNONODE) { From 573204c3033a21f5dad745e946e102a596d26e6e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Oct 2024 09:30:44 -0300 Subject: [PATCH 570/816] getbatch --- .../Formats/Impl/Parquet/ParquetDataValuesReader.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 9a79bcffad3..fa38a24fd3c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -320,12 +320,7 @@ void ParquetBitPlainReader::readBatch( }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { - for (UInt32 i = 0; i < count; i++) - { - uint8_t byte; - bit_reader->GetValue(1, &byte); - container[nest_cursor++] = byte; - } + bit_reader->GetBatch(1, &container[nest_cursor], count); } ); } From b5e3df977b3799f2eaaa2590293b0271eeadc073 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 29 Oct 2024 12:48:44 +0000 Subject: [PATCH 571/816] finishing --- src/Interpreters/ConcurrentHashJoin.h | 6 +++++- src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.h | 2 +- src/Processors/QueryPlan/UnionStep.cpp | 3 +-- tests/clickhouse-test | 3 +-- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 355218554ce..b377727a134 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -61,7 +61,11 @@ public: getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; - bool isCloneSupported() const override { return true; } + bool isCloneSupported() const override + { + return !getTotals() && getTotalRowCount() == 0; + } + std::shared_ptr clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override { return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 3f1e0d59287..faa9114c618 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals(); } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index d5abdc2ddb8..8a27961354a 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -127,7 +127,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals() && getTotalRowCount() == 0; } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index b7a87b27be5..d5c2469629b 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -34,8 +34,7 @@ UnionStep::UnionStep(Headers input_headers_, size_t max_threads_) void UnionStep::updateOutputHeader() { - if (input_headers.size() == 1 || !output_header) - output_header = checkHeaders(input_headers); + output_header = checkHeaders(input_headers); } QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 51496c924ac..fa565eb88a7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -921,8 +921,7 @@ class SettingsRandomizer: "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), "query_plan_join_inner_table_selection": lambda: random.choice( - ["left", "auto"] - # ["left", "auto", "right"] + ["left", "auto", "right"] ), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), } From b81e024c70cb27c41daacef6372846cd9478e654 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 13:54:22 +0000 Subject: [PATCH 572/816] Debug prints --- .../DataLakes/DataLakeConfiguration.h | 7 +++++-- .../ObjectStorage/StorageObjectStorage.cpp | 16 ++++++++-------- .../ObjectStorage/StorageObjectStorage.h | 2 +- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 8a4147308f3..9bb02436df1 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -34,9 +34,12 @@ public: std::string getEngineName() const override { return DataLakeMetadata::name; } - void update(ObjectStoragePtr object_storage, ContextPtr local_context) override + void update(ObjectStoragePtr object_storage, ContextPtr local_context, bool update_base) override { - // BaseStorageConfiguration::update(object_storage, local_context); + if (update_base) + { + BaseStorageConfiguratixon::update(object_storage, local_context); + } auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); if (current_metadata && *current_metadata == *new_metadata) return; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 6f4c0787e81..de5a4a08358 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -76,16 +76,16 @@ void printConfiguration(const Poco::Util::AbstractConfiguration & config, std::s for (const auto & key : keys) { - std::string fullKey = prefix.empty() ? key : (prefix + "." + key); + std::string full_key = prefix.empty() ? key : (prefix + "." + key); - if (config.hasProperty(fullKey)) + if (config.hasProperty(full_key)) { - std::string value = config.getString(fullKey); - LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", fullKey, value); + std::string value = config.getString(full_key); + LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", full_key, value); } // Recursively print sub-configurations - printConfiguration(config, fullKey, log_name); + printConfiguration(config, full_key, log_name); } } @@ -114,7 +114,7 @@ StorageObjectStorage::StorageObjectStorage( printConfiguration(context->getConfigRef(), "Storage create"); try { - // configuration->update(object_storage, context); + configuration->update(object_storage, context); } catch (...) { @@ -166,7 +166,7 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); } -void StorageObjectStorage::Configuration::update(ObjectStoragePtr object_storage_ptr, ContextPtr context) +void StorageObjectStorage::Configuration::update(ObjectStoragePtr object_storage_ptr, ContextPtr context, [[maybe_unused]] bool update_base) { IObjectStorage::ApplyNewSettingsOptions options{.allow_client_change = !isStaticConfiguration()}; object_storage_ptr->applyNewSettings(context->getConfigRef(), getTypeName() + ".", context, options); @@ -309,7 +309,7 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - configuration->update(object_storage, local_context); + configuration->update(object_storage, local_context, true); printConfiguration(local_context->getConfigRef(), "Select query"); if (partition_by && configuration->withPartitionWildcard()) { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 6ca1613e65c..3a85a2532f2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -216,7 +216,7 @@ public: String compression_method = "auto"; String structure = "auto"; - virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context); + virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context, [[maybe_unused]] bool update_base = false); protected: From 6fa8153d1aac4d5a0b500cf040ca697a97b0c6f1 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Oct 2024 14:23:34 +0000 Subject: [PATCH 573/816] Fix ignoring format settings in Native format via HTTP and Async Inserts --- src/Processors/Formats/Impl/NativeFormat.cpp | 10 ++++++---- .../Transforms/getSourceFromASTInsertQuery.cpp | 1 + ..._native_http_async_insert_settings.reference | 1 + .../03259_native_http_async_insert_settings.sh | 17 +++++++++++++++++ 4 files changed, 25 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03259_native_http_async_insert_settings.reference create mode 100755 tests/queries/0_stateless/03259_native_http_async_insert_settings.sh diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 5411e2e7811..022cb38596b 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -15,16 +15,17 @@ namespace DB class NativeInputFormat final : public IInputFormat { public: - NativeInputFormat(ReadBuffer & buf, const Block & header_, const FormatSettings & settings) + NativeInputFormat(ReadBuffer & buf, const Block & header_, const FormatSettings & settings_) : IInputFormat(header_, &buf) , reader(std::make_unique( buf, header_, 0, - settings, - settings.defaults_for_omitted_fields ? &block_missing_values : nullptr)) + settings_, + settings_.defaults_for_omitted_fields ? &block_missing_values : nullptr)) , header(header_) , block_missing_values(header.columns()) + , settings(settings_) { } @@ -55,7 +56,7 @@ public: void setReadBuffer(ReadBuffer & in_) override { - reader = std::make_unique(in_, header, 0); + reader = std::make_unique(in_, header, 0, settings, settings.defaults_for_omitted_fields ? &block_missing_values : nullptr); IInputFormat::setReadBuffer(in_); } @@ -67,6 +68,7 @@ private: std::unique_ptr reader; Block header; BlockMissingValues block_missing_values; + const FormatSettings settings; size_t approx_bytes_read_for_chunk = 0; }; diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 648ed9751ff..0c00baeabf7 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -16,6 +16,7 @@ #include "IO/CompressionMethod.h" #include #include +#include namespace DB diff --git a/tests/queries/0_stateless/03259_native_http_async_insert_settings.reference b/tests/queries/0_stateless/03259_native_http_async_insert_settings.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03259_native_http_async_insert_settings.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03259_native_http_async_insert_settings.sh b/tests/queries/0_stateless/03259_native_http_async_insert_settings.sh new file mode 100755 index 00000000000..c0934b06cc7 --- /dev/null +++ b/tests/queries/0_stateless/03259_native_http_async_insert_settings.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt32) engine=Memory"; + +url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" + +$CLICKHOUSE_LOCAL -q "select NULL::Nullable(UInt32) as x format Native" | ${CLICKHOUSE_CURL} -sS "$url&query=INSERT%20INTO%20test%20FORMAT%20Native" --data-binary @- + +$CLICKHOUSE_CLIENT -q "select * from test"; +$CLICKHOUSE_CLIENT -q "drop table test" + From 414d04690e9f431d34b175d3c732eaea350d6cb3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Oct 2024 14:25:27 +0000 Subject: [PATCH 574/816] Remove unneeded include --- src/Processors/Transforms/getSourceFromASTInsertQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 0c00baeabf7..648ed9751ff 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -16,7 +16,6 @@ #include "IO/CompressionMethod.h" #include #include -#include namespace DB From 52a33438080a79c0852acd68e292fcba0476ef1a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 29 Oct 2024 15:37:41 +0100 Subject: [PATCH 575/816] add arm+asan functional tests --- tests/ci/ci_config.py | 13 ++++- tests/ci/ci_definitions.py | 6 ++- tests/ci/test_ci_config.py | 105 +++++++++++++------------------------ 3 files changed, 53 insertions(+), 71 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b9885a89444..9f5d5f1983d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -97,9 +97,9 @@ class CI: ), runner_type=Runners.BUILDER_ARM, ), - BuildNames.PACKAGE_AARCH64_ASAN: CommonJobConfigs.BUILD.with_properties( + BuildNames.PACKAGE_ARM_ASAN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( - name=BuildNames.PACKAGE_AARCH64_ASAN, + name=BuildNames.PACKAGE_ARM_ASAN, compiler="clang-18-aarch64", sanitizer="address", package_type="deb", @@ -283,6 +283,10 @@ class CI: JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN] ), + JobNames.STATEFUL_TEST_ARM_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ARM_ASAN], + runner_type=Runners.FUNC_TESTER_ARM, + ), JobNames.STATEFUL_TEST_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN] ), @@ -331,6 +335,11 @@ class CI: JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 ), + JobNames.STATELESS_TEST_ARM_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ARM_ASAN], + num_batches=2, + runner_type=Runners.FUNC_TESTER_ARM, + ), JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], num_batches=4 ), diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index fc67959013b..58c7825d2c5 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -106,7 +106,7 @@ class BuildNames(metaclass=WithIter): PACKAGE_MSAN = "package_msan" PACKAGE_DEBUG = "package_debug" PACKAGE_AARCH64 = "package_aarch64" - PACKAGE_AARCH64_ASAN = "package_aarch64_asan" + PACKAGE_ARM_ASAN = "package_aarch64_asan" PACKAGE_RELEASE_COVERAGE = "package_release_coverage" BINARY_RELEASE = "binary_release" BINARY_TIDY = "binary_tidy" @@ -141,6 +141,7 @@ class JobNames(metaclass=WithIter): STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" STATELESS_TEST_ASAN = "Stateless tests (asan)" + STATELESS_TEST_ARM_ASAN = "Stateless tests (aarch64, asan)" STATELESS_TEST_TSAN = "Stateless tests (tsan)" STATELESS_TEST_MSAN = "Stateless tests (msan)" STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" @@ -157,6 +158,7 @@ class JobNames(metaclass=WithIter): STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" STATEFUL_TEST_ASAN = "Stateful tests (asan)" + STATEFUL_TEST_ARM_ASAN = "Stateful tests (aarch64, asan)" STATEFUL_TEST_TSAN = "Stateful tests (tsan)" STATEFUL_TEST_MSAN = "Stateful tests (msan)" STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" @@ -632,6 +634,8 @@ REQUIRED_CHECKS = [ JobNames.STATEFUL_TEST_RELEASE, JobNames.STATELESS_TEST_RELEASE, JobNames.STATELESS_TEST_ASAN, + JobNames.STATELESS_TEST_ARM_ASAN, + JobNames.STATEFUL_TEST_ARM_ASAN, JobNames.STATELESS_TEST_FLAKY_ASAN, JobNames.STATEFUL_TEST_ASAN, JobNames.STYLE_CHECK, diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 29b184a4e61..0e396b827ea 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -36,7 +36,7 @@ class TestCIConfig(unittest.TestCase): elif "binary_" in job.lower() or "package_" in job.lower(): if job.lower() in ( CI.BuildNames.PACKAGE_AARCH64, - CI.BuildNames.PACKAGE_AARCH64_ASAN, + CI.BuildNames.PACKAGE_ARM_ASAN, ): self.assertTrue( CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_ARM,), @@ -95,69 +95,39 @@ class TestCIConfig(unittest.TestCase): self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) else: self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) - if "asan" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_ASAN, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + if "asan" in job and "aarch" in job: + expected_builds = [CI.BuildNames.PACKAGE_ARM_ASAN] + elif "asan" in job: + expected_builds = [CI.BuildNames.PACKAGE_ASAN] elif "msan" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_MSAN, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_MSAN] elif "tsan" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_TSAN, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_TSAN] elif "ubsan" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_UBSAN, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_UBSAN] elif "debug" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_DEBUG, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_DEBUG] + elif job in ( + "Unit tests (release)", + "ClickHouse Keeper Jepsen", + "ClickHouse Server Jepsen", + ): + expected_builds = [CI.BuildNames.BINARY_RELEASE] elif "release" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - in ( - CI.BuildNames.PACKAGE_RELEASE, - CI.BuildNames.BINARY_RELEASE, - ), - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_RELEASE] elif "coverage" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_RELEASE_COVERAGE, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_RELEASE_COVERAGE] elif "aarch" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_AARCH64, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_AARCH64] elif "amd64" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_RELEASE, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_RELEASE] elif "uzzer" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] == CI.BuildNames.FUZZERS, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.FUZZERS] elif "Docker" in job: + expected_builds = [ + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.PACKAGE_AARCH64, + ] self.assertTrue( CI.JOB_CONFIGS[job].required_builds[0] in ( @@ -167,20 +137,12 @@ class TestCIConfig(unittest.TestCase): f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", ) elif "SQLTest" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - == CI.BuildNames.PACKAGE_RELEASE, - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [CI.BuildNames.PACKAGE_RELEASE] elif "Jepsen" in job: - self.assertTrue( - CI.JOB_CONFIGS[job].required_builds[0] - in ( - CI.BuildNames.PACKAGE_RELEASE, - CI.BuildNames.BINARY_RELEASE, - ), - f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", - ) + expected_builds = [ + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.BINARY_RELEASE, + ] elif job in ( CI.JobNames.STYLE_CHECK, CI.JobNames.FAST_TEST, @@ -188,9 +150,16 @@ class TestCIConfig(unittest.TestCase): CI.JobNames.DOCS_CHECK, CI.JobNames.BUGFIX_VALIDATE, ): - self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) + expected_builds = [] else: print(f"Job [{job}] required build not checked") + assert False + + self.assertCountEqual( + expected_builds, + CI.JOB_CONFIGS[job].required_builds or [], + f"Required builds are not valid for job [{job}]", + ) def test_job_stage_config(self): """ From a54df544050633074e9680049ffc315a1b143f72 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 15:04:30 +0000 Subject: [PATCH 576/816] Add changes --- src/Storages/ObjectStorage/StorageObjectStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3a85a2532f2..6ca1613e65c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -216,7 +216,7 @@ public: String compression_method = "auto"; String structure = "auto"; - virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context, [[maybe_unused]] bool update_base = false); + virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context); protected: From 858162ce2c229002e808d0d1acc2e100df79b8e0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 29 Oct 2024 15:06:03 +0000 Subject: [PATCH 577/816] add a perf test --- .../replacing_final_non_intersecting.xml | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 tests/performance/replacing_final_non_intersecting.xml diff --git a/tests/performance/replacing_final_non_intersecting.xml b/tests/performance/replacing_final_non_intersecting.xml new file mode 100644 index 00000000000..b3d32f1ca2e --- /dev/null +++ b/tests/performance/replacing_final_non_intersecting.xml @@ -0,0 +1,26 @@ + + + + 0 + 0 + + + + CREATE TABLE replacing_final_non_intersecting (d DateTime, c1 UInt64, c2 String, c3 LowCardinality(String)) + ENGINE = ReplacingMergeTree() + ORDER BY d + + + INSERT INTO replacing_final_non_intersecting SELECT toDateTime('2020-10-10 00:00:00') - number, number, toString(number), toString(number % 1000) FROM numbers(0, 5000000) + OPTIMIZE TABLE replacing_final_non_intersecting FINAL + SYSTEM STOP MERGES replacing_final_non_intersecting + INSERT INTO replacing_final_non_intersecting SELECT toDateTime('2020-10-10 00:00:00') - number, number, toString(number), toString(number % 1000) FROM numbers(5000000, 500000) + + SELECT * FROM replacing_final_non_intersecting FINAL FORMAT Null SETTINGS enable_vertical_final = 0 + SELECT * FROM replacing_final_non_intersecting FINAL FORMAT Null SETTINGS enable_vertical_final = 1 + + DROP TABLE IF EXISTS replacing_final_non_intersecting + From 308763ce678076efdd6d3298f1eb78ba30b4276e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 16:15:41 +0100 Subject: [PATCH 578/816] Improve error and log messages around memory usage --- src/Client/ClientApplicationBase.cpp | 2 +- src/Common/MemoryTracker.cpp | 60 +++++++++++-------- src/Common/ThreadStatus.cpp | 2 +- src/Interpreters/ProcessList.cpp | 4 +- src/Interpreters/ThreadStatusExt.cpp | 6 +- .../test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- ...gate_state_exception_memory_leak.reference | 2 +- ...1_aggregate_state_exception_memory_leak.sh | 2 +- ...01514_distributed_cancel_query_on_error.sh | 2 +- 10 files changed, 47 insertions(+), 37 deletions(-) diff --git a/src/Client/ClientApplicationBase.cpp b/src/Client/ClientApplicationBase.cpp index d26641fe5f9..f7d2d0035d9 100644 --- a/src/Client/ClientApplicationBase.cpp +++ b/src/Client/ClientApplicationBase.cpp @@ -418,7 +418,7 @@ void ClientApplicationBase::init(int argc, char ** argv) UInt64 max_client_memory_usage_int = parseWithSizeSuffix(max_client_memory_usage.c_str(), max_client_memory_usage.length()); total_memory_tracker.setHardLimit(max_client_memory_usage_int); - total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setDescription("Global"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 3ed943f217d..f4af019605e 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -68,15 +68,15 @@ inline std::string_view toDescription(OvercommitResult result) case OvercommitResult::NONE: return ""; case OvercommitResult::DISABLED: - return "Memory overcommit isn't used. Waiting time or overcommit denominator are set to zero."; + return "Memory overcommit isn't used. Waiting time or overcommit denominator are set to zero"; case OvercommitResult::MEMORY_FREED: throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description"); case OvercommitResult::SELECTED: - return "Query was selected to stop by OvercommitTracker."; + return "Query was selected to stop by OvercommitTracker"; case OvercommitResult::TIMEOUTED: - return "Waiting timeout for memory to be freed is reached."; + return "Waiting timeout for memory to be freed is reached"; case OvercommitResult::NOT_ENOUGH_FREED: - return "Memory overcommit has freed not enough memory."; + return "Memory overcommit has not freed enough memory"; } } @@ -150,15 +150,23 @@ void MemoryTracker::logPeakMemoryUsage() auto peak_bytes = peak.load(std::memory_order::relaxed); if (peak_bytes < 128 * 1024) return; - LOG_DEBUG(getLogger("MemoryTracker"), - "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes)); + LOG_DEBUG( + getLogger("MemoryTracker"), + "{}{} memory usage: {}.", + description ? std::string(description) : "", + description ? " peak" : "Peak", + ReadableSize(peak_bytes)); } void MemoryTracker::logMemoryUsage(Int64 current) const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(getLogger("MemoryTracker"), - "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); + LOG_DEBUG( + getLogger("MemoryTracker"), + "{}{} memory usage: {}.", + description ? std::string(description) : "", + description ? " current" : "Current", + ReadableSize(current)); } void MemoryTracker::injectFault() const @@ -178,9 +186,9 @@ void MemoryTracker::injectFault() const const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker{}{}: fault injected (at specific point)", - description ? " " : "", - description ? description : ""); + "{}{}: fault injected (at specific point)", + description ? description : "", + description ? " memory tracker" : "Memory tracker"); } void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused]]) @@ -282,9 +290,9 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory tracker{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - description ? " " : "", + "{}{}: fault injected. Would use {} (attempt to allocate chunk of {} bytes), maximum: {}", description ? description : "", + description ? " memory tracker" : "Memory tracker", formatReadableSizeWithBinarySuffix(will_be), size, formatReadableSizeWithBinarySuffix(current_hard_limit)); @@ -305,6 +313,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed if (overcommit_result != OvercommitResult::MEMORY_FREED) { + bool overcommit_result_ignore + = overcommit_result == OvercommitResult::NONE || overcommit_result == OvercommitResult::DISABLED; /// Revert amount.fetch_sub(size, std::memory_order_relaxed); rss.fetch_sub(size, std::memory_order_relaxed); @@ -314,18 +324,18 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( - DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit{}{} exceeded: " - "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." - "{}{}", - description ? " " : "", - description ? description : "", - formatReadableSizeWithBinarySuffix(will_be), - size, - formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), - formatReadableSizeWithBinarySuffix(current_hard_limit), - overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ", - toDescription(overcommit_result)); + DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, + "{}{} exceeded: " + "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." + "{}{}", + description ? description : "", + description ? " memory limit" : "Memory limit", + formatReadableSizeWithBinarySuffix(will_be), + size, + formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), + formatReadableSizeWithBinarySuffix(current_hard_limit), + overcommit_result_ignore ? "" : " OvercommitTracker decision: ", + overcommit_result_ignore ? "" : toDescription(overcommit_result)); } // If OvercommitTracker::needToStopQuery returned false, it guarantees that enough memory is freed. diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index e38d3480664..268d97e62ef 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -78,7 +78,7 @@ ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) last_rusage = std::make_unique(); - memory_tracker.setDescription("(for thread)"); + memory_tracker.setDescription("Thread"); log = getLogger("ThreadStatus"); current_thread = this; diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 177468f1c8b..21c30a60617 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -276,7 +276,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q thread_group->performance_counters.setTraceProfileEvents(settings[Setting::trace_profile_events]); } - thread_group->memory_tracker.setDescription("(for query)"); + thread_group->memory_tracker.setDescription("Query"); if (settings[Setting::memory_tracker_fault_probability] > 0.0) thread_group->memory_tracker.setFaultProbability(settings[Setting::memory_tracker_fault_probability]); @@ -311,7 +311,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q /// Track memory usage for all simultaneously running queries from single user. user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings[Setting::max_memory_usage_for_user]); user_process_list.user_memory_tracker.setSoftLimit(settings[Setting::memory_overcommit_ratio_denominator_for_user]); - user_process_list.user_memory_tracker.setDescription("(for user)"); + user_process_list.user_memory_tracker.setDescription("User"); if (!total_network_throttler && settings[Setting::max_network_bandwidth_for_all_users]) { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 0544bbcc92e..4d27a840d51 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -119,7 +119,7 @@ void ThreadGroup::unlinkThread() ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); - group->memory_tracker.setDescription("(for query)"); + group->memory_tracker.setDescription("Query"); return group; } @@ -127,7 +127,7 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex { auto group = std::make_shared(storage_context); - group->memory_tracker.setDescription("background process to apply mutate/merge in table"); + group->memory_tracker.setDescription("Background process (mutate/merge)"); /// However settings from storage context have to be applied const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings[Setting::memory_profiler_step]); @@ -384,7 +384,7 @@ void ThreadStatus::initPerformanceCounters() /// TODO: make separate query_thread_performance_counters and thread_performance_counters performance_counters.resetCounters(); memory_tracker.resetCounters(); - memory_tracker.setDescription("(for thread)"); + memory_tracker.setDescription("Thread"); query_start_time.setUp(); diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index 7a843a87ec2..74c35e7f4ea 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -78,7 +78,7 @@ def test_distributed_background_insert_split_batch_on_failure_OFF(started_cluste with pytest.raises( QueryRuntimeException, # no DOTALL in pytest.raises, use '(.|\n)' - match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: (.|\n)*While sending a batch", + match=r"DB::Exception: Received from.*Query memory limit exceeded: (.|\n)*While sending a batch", ): node2.query("system flush distributed dist") assert int(node2.query("select count() from dist_data")) == 0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ad1842f4509..d8326711d84 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1592,7 +1592,7 @@ def test_parallel_reading_with_memory_limit(started_cluster): f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') settings max_memory_usage=1000" ) - assert "Memory limit (for query) exceeded" in result + assert "Query memory limit exceeded" in result time.sleep(5) diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference index 6282bf366d0..76c31901df7 100644 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.reference @@ -1,2 +1,2 @@ -Memory limit exceeded +Query memory limit exceeded Ok diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 5b7cba77432..ceb7b60be0f 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -16,5 +16,5 @@ for _ in {1..1000}; do if [[ $elapsed -gt 30 ]]; then break fi -done 2>&1 | grep -o -P 'Memory limit .+ exceeded' | sed -r -e 's/(Memory limit)(.+)( exceeded)/\1\3/' | uniq +done 2>&1 | grep -o -P 'Query memory limit exceeded' | sed -r -e 's/(.*):([a-Z ]*)([mM]emory limit exceeded)(.*)/\2\3/' | uniq echo 'Ok' diff --git a/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh b/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh index edf3683ccba..245aa3ceb99 100755 --- a/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh +++ b/tests/queries/0_stateless/01514_distributed_cancel_query_on_error.sh @@ -19,6 +19,6 @@ opts=( ) ${CLICKHOUSE_CLIENT} "${opts[@]}" -q "SELECT groupArray(repeat('a', if(_shard_num == 2, 100000, 1))), number%100000 k from remote('127.{2,3}', system.numbers) GROUP BY k LIMIT 10e6" |& { # the query should fail earlier on 127.3 and 127.2 should not even go to the memory limit exceeded error. - grep -F -q 'DB::Exception: Received from 127.3:9000. DB::Exception: Memory limit (for query) exceeded:' + grep -F -q "DB::Exception: Received from 127.3:${CLICKHOUSE_PORT_TCP}. DB::Exception: Query memory limit exceeded:" # while if this will not correctly then it will got the exception from the 127.2:9000 and fail } From 886603d62541818f74d7e206209ef58f87c07e70 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 15:18:05 +0000 Subject: [PATCH 579/816] Fixed some bugs --- .../ObjectStorage/DataLakes/DataLakeConfiguration.h | 9 ++------- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 4 ++-- tests/integration/test_storage_iceberg/test.py | 6 +++--- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 9bb02436df1..1a694a25dff 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -30,16 +30,11 @@ public: bool isDataLakeConfiguration() const override { return true; } - bool isStaticConfiguration() const override { return false; } - std::string getEngineName() const override { return DataLakeMetadata::name; } - void update(ObjectStoragePtr object_storage, ContextPtr local_context, bool update_base) override + void update(ObjectStoragePtr object_storage, ContextPtr local_context) override { - if (update_base) - { - BaseStorageConfiguratixon::update(object_storage, local_context); - } + BaseStorageConfiguration::update(object_storage, local_context); auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context); if (current_metadata && *current_metadata == *new_metadata) return; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index de5a4a08358..89a5bfe9469 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -166,7 +166,7 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); } -void StorageObjectStorage::Configuration::update(ObjectStoragePtr object_storage_ptr, ContextPtr context, [[maybe_unused]] bool update_base) +void StorageObjectStorage::Configuration::update(ObjectStoragePtr object_storage_ptr, ContextPtr context) { IObjectStorage::ApplyNewSettingsOptions options{.allow_client_change = !isStaticConfiguration()}; object_storage_ptr->applyNewSettings(context->getConfigRef(), getTypeName() + ".", context, options); @@ -309,7 +309,7 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - configuration->update(object_storage, local_context, true); + configuration->update(object_storage, local_context); printConfiguration(local_context->getConfigRef(), "Select query"); if (partition_by && configuration->withPartitionWildcard()) { diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index ca78fbea667..3d93c1b163c 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -862,9 +862,9 @@ def test_restart_broken_s3(started_cluster): instance.restart_clickhouse() - # assert "NoSuchBucket" in instance.query_and_get_error( - # f"SELECT count() FROM {TABLE_NAME}" - # ) + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) time.sleep(10) From b81fadc6bfc3e69c8dc8c129de5ad6a2912db106 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 29 Oct 2024 15:18:07 +0000 Subject: [PATCH 580/816] Remove test --- tests/integration/test_storage_mysql/test.py | 94 ------------------- .../test_storage_postgresql/test.py | 83 ---------------- 2 files changed, 177 deletions(-) diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 2fc62d7f511..2d34a52c17b 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -386,100 +386,6 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' conn.close() -def test_mysql_distributed(started_cluster): - table_name = "test_replicas" - - conn1 = get_mysql_conn(started_cluster, started_cluster.mysql8_ip) - conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip) - conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip) - conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip) - - create_mysql_db(conn1, "clickhouse") - create_mysql_db(conn2, "clickhouse") - create_mysql_db(conn3, "clickhouse") - create_mysql_db(conn4, "clickhouse") - - create_mysql_table(conn1, table_name) - create_mysql_table(conn2, table_name) - create_mysql_table(conn3, table_name) - create_mysql_table(conn4, table_name) - - node2.query("DROP TABLE IF EXISTS test_replicas") - - # Storage with with 3 replicas - node2.query( - """ - CREATE TABLE test_replicas - (id UInt32, name String, age UInt32, money UInt32) - ENGINE = MySQL('mysql{2|3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ - ) - - # Fill remote tables with different data to be able to check - nodes = [node1, node2, node2, node2] - for i in range(1, 5): - nodes[i - 1].query("DROP TABLE IF EXISTS test_replica{}".format(i)) - nodes[i - 1].query( - """ - CREATE TABLE test_replica{} - (id UInt32, name String, age UInt32, money UInt32) - ENGINE = MySQL('mysql{}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse');""".format( - i, 80 if i == 1 else i - ) - ) - nodes[i - 1].query( - "INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format( - i, i - ) - ) - - # test multiple ports parsing - result = node2.query( - """SELECT DISTINCT(name) FROM mysql('mysql{80|2|3}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ - ) - assert result == "host1\n" or result == "host2\n" or result == "host3\n" - result = node2.query( - """SELECT DISTINCT(name) FROM mysql('mysql80:3306|mysql2:3306|mysql3:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ - ) - assert result == "host1\n" or result == "host2\n" or result == "host3\n" - - # check all replicas are traversed - query = "SELECT * FROM (" - for i in range(3): - query += "SELECT name FROM test_replicas UNION DISTINCT " - query += "SELECT name FROM test_replicas) ORDER BY name" - - result = node2.query(query) - assert result == "host2\nhost3\nhost4\n" - - # Storage with with two shards, each has 2 replicas - node2.query("DROP TABLE IF EXISTS test_shards") - - node2.query( - """ - CREATE TABLE test_shards - (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('MySQL', 'mysql{80|2}:3306,mysql{3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ - ) - - # Check only one replica in each shard is used - result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - assert result == "host1\nhost3\n" - - # check all replicas are traversed - query = "SELECT name FROM (" - for i in range(3): - query += "SELECT name FROM test_shards UNION DISTINCT " - query += "SELECT name FROM test_shards) ORDER BY name" - result = node2.query(query) - assert result == "host1\nhost2\nhost3\nhost4\n" - - # disconnect mysql - started_cluster.pause_container("mysql80") - result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - started_cluster.unpause_container("mysql80") - assert result == "host2\nhost4\n" or result == "host3\nhost4\n" - - def test_external_settings(started_cluster): table_name = "test_external_settings" node1.query(f"DROP TABLE IF EXISTS {table_name}") diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index aaecc7537cf..0cb551aecc5 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -449,89 +449,6 @@ def test_concurrent_queries(started_cluster): node1.query("DROP TABLE test.stat;") -def test_postgres_distributed(started_cluster): - cursor0 = started_cluster.postgres_conn.cursor() - cursor1 = started_cluster.postgres2_conn.cursor() - cursor2 = started_cluster.postgres3_conn.cursor() - cursor3 = started_cluster.postgres4_conn.cursor() - cursors = [cursor0, cursor1, cursor2, cursor3] - - for i in range(4): - cursors[i].execute("DROP TABLE IF EXISTS test_replicas") - cursors[i].execute("CREATE TABLE test_replicas (id Integer, name Text)") - cursors[i].execute( - f"""INSERT INTO test_replicas select i, 'host{i+1}' from generate_series(0, 99) as t(i);""" - ) - - # test multiple ports parsing - result = node2.query( - """SELECT DISTINCT(name) FROM postgresql('postgres{1|2|3}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """ - ) - assert result == "host1\n" or result == "host2\n" or result == "host3\n" - result = node2.query( - """SELECT DISTINCT(name) FROM postgresql('postgres2:5431|postgres3:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """ - ) - assert result == "host3\n" or result == "host2\n" - - # Create storage with with 3 replicas - node2.query("DROP TABLE IF EXISTS test_replicas") - node2.query( - """ - CREATE TABLE test_replicas - (id UInt32, name String) - ENGINE = PostgreSQL('postgres{2|3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """ - ) - - # Check all replicas are traversed - query = "SELECT name FROM (" - for i in range(3): - query += "SELECT name FROM test_replicas UNION DISTINCT " - query += "SELECT name FROM test_replicas) ORDER BY name" - result = node2.query(query) - assert result == "host2\nhost3\nhost4\n" - - # Create storage with with two two shards, each has 2 replicas - node2.query("DROP TABLE IF EXISTS test_shards") - - node2.query( - """ - CREATE TABLE test_shards - (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('PostgreSQL', 'postgres{1|2}:5432,postgres{3|4}:5432', 'postgres', 'test_replicas', 'postgres', 'mysecretpassword'); """ - ) - - # Check only one replica in each shard is used - result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - assert result == "host1\nhost3\n" - - node2.query( - """ - CREATE TABLE test_shards2 - (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('PostgreSQL', postgres4, addresses_expr='postgres{1|2}:5432,postgres{3|4}:5432'); """ - ) - - result = node2.query("SELECT DISTINCT(name) FROM test_shards2 ORDER BY name") - assert result == "host1\nhost3\n" - - # Check all replicas are traversed - query = "SELECT name FROM (" - for i in range(3): - query += "SELECT name FROM test_shards UNION DISTINCT " - query += "SELECT name FROM test_shards) ORDER BY name" - result = node2.query(query) - assert result == "host1\nhost2\nhost3\nhost4\n" - - # Disconnect postgres1 - started_cluster.pause_container("postgres1") - result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - started_cluster.unpause_container("postgres1") - assert result == "host2\nhost4\n" or result == "host3\nhost4\n" - node2.query("DROP TABLE test_shards2") - node2.query("DROP TABLE test_shards") - node2.query("DROP TABLE test_replicas") - - def test_datetime_with_timezone(started_cluster): cursor = started_cluster.postgres_conn.cursor() cursor.execute("DROP TABLE IF EXISTS test_timezone") From 7a60543670259ee39ac343dd0339a712d1e245aa Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 29 Oct 2024 15:22:15 +0000 Subject: [PATCH 581/816] Take into account that some profile events such as `NetworkReceiveBytes` are inaccurate --- src/Interpreters/MetricLog.cpp | 9 +++++++++ src/Interpreters/QueryMetricLog.cpp | 25 +++++++++++-------------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 16a88b976ba..d0d799ea693 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -70,6 +70,15 @@ void MetricLog::stepFunction(const std::chrono::system_clock::time_point current { const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); auto & old_value = prev_profile_events[i]; + + /// Profile event counters are supposed to be monotonic. However, at least the `NetworkReceiveBytes` can be inaccurate. + /// So, since in the future the counter should always have a bigger value than in the past, we skip this event. + /// It can be reproduced with the following integration tests: + /// - test_hedged_requests/test.py::test_receive_timeout2 + /// - test_secure_socket::test + if (new_value < old_value) + continue; + elem.profile_events[i] = new_value - old_value; old_value = new_value; } diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index d274d9e139c..8a84c95a5a3 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -22,11 +22,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -}; - static auto logger = getLogger("QueryMetricLog"); ColumnsDescription QueryMetricLogElement::getColumnsDescription() @@ -176,7 +171,7 @@ std::optional QueryMetricLog::createLogMetricElement(cons } auto & query_status = query_status_it->second; - if (query_info_time < query_status.last_collect_time) + if (query_info_time <= query_status.last_collect_time) { lock.unlock(); LOG_TRACE(logger, "Query {} has a more recent metrics collected. Skipping this one", query_id); @@ -197,16 +192,18 @@ std::optional QueryMetricLog::createLogMetricElement(cons for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { const auto & new_value = (*(query_info.profile_counters))[i]; - auto & prev_value = query_status.last_profile_events[i]; + auto & old_value = query_status.last_profile_events[i]; - /// Profile event count is monotonically increasing. - if (new_value < prev_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Profile event count is not monotonically increasing for '{}': new value {} is smaller than previous value {}", - ProfileEvents::getName(i), new_value, query_status.last_profile_events[i]); + /// Profile event counters are supposed to be monotonic. However, at least the `NetworkReceiveBytes` can be inaccurate. + /// So, since in the future the counter should always have a bigger value than in the past, we skip this event. + /// It can be reproduced with the following integration tests: + /// - test_hedged_requests/test.py::test_receive_timeout2 + /// - test_secure_socket::test + if (new_value < old_value) + continue; - elem.profile_events[i] = new_value - prev_value; - prev_value = new_value; + elem.profile_events[i] = new_value - old_value; + old_value = new_value; } } else From 9425b19f848ace4c7c183d2c36e1660986ce394d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 29 Oct 2024 15:26:35 +0000 Subject: [PATCH 582/816] Automatic style fix --- tests/integration/test_storage_iceberg/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 3d93c1b163c..690ebeeffbf 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -5,7 +5,6 @@ import os import time import uuid from datetime import datetime - from logging import log import pyspark From a1dd3ea0ac71c733a6729dbc6d53ed7718107bbc Mon Sep 17 00:00:00 2001 From: Zoe Steinkamp Date: Tue, 29 Oct 2024 09:27:00 -0600 Subject: [PATCH 583/816] Update README.md Updating the events into the recent category and adding the new york event --- README.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 3b5209dcbe9..96dec2ca607 100644 --- a/README.md +++ b/README.md @@ -42,17 +42,18 @@ Keep an eye out for upcoming meetups and events around the world. Somewhere else Upcoming meetups -* [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 -* [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3 -* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 * [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 * [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - November 12 * [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 +* [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9 Recently completed meetups +* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 +* [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3 +* [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 * [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 From 69a82191763438fde1412397faa6ac5e5dd79a8a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 29 Oct 2024 15:45:18 +0000 Subject: [PATCH 584/816] Put DateTime64 implementation into another function. --- src/Interpreters/Set.cpp | 66 ++++++++++++++++++++++++---------------- src/Interpreters/Set.h | 2 ++ 2 files changed, 42 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index f6880973743..a910ac5f59b 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -343,6 +343,45 @@ ColumnPtr mergeNullMaps(const ColumnPtr & null_map_column1, const ColumnUInt8::P return merged_null_map_column; } +void Set::processDateTime64Column( + const ColumnWithTypeAndName & column_to_cast, + ColumnPtr & result, + ColumnPtr & null_map_holder, + ConstNullMapPtr & null_map) const +{ + // Check for sub-second precision and create a null map + ColumnUInt8::Ptr filtered_null_map_column = checkDateTimePrecision(column_to_cast); + + // Extract existing null map and nested column from the result + const ColumnNullable * result_nullable_column = typeid_cast(result.get()); + const IColumn * nested_result_column = result_nullable_column + ? &result_nullable_column->getNestedColumn() + : result.get(); + + ColumnPtr existing_null_map_column = result_nullable_column + ? result_nullable_column->getNullMapColumnPtr() + : nullptr; + + if (transform_null_in) + { + if (!null_map_holder) + null_map_holder = filtered_null_map_column; + else + null_map_holder = mergeNullMaps(null_map_holder, filtered_null_map_column); + + const ColumnUInt8 * null_map_column = checkAndGetColumn(null_map_holder.get()); + if (!null_map_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Null map must be ColumnUInt8"); + + null_map = &null_map_column->getData(); + } + else + { + ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); + result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); + } +} + ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const { size_t num_key_columns = columns.size(); @@ -403,32 +442,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co // If the original column is DateTime64, check for sub-second precision if (isDateTime64(column_to_cast.column->getDataType())) { - ColumnUInt8::Ptr filtered_null_map_column = checkDateTimePrecision(column_to_cast); - - // Extract existing null map and nested column from the result - const ColumnNullable * result_nullable_column = typeid_cast(result.get()); - const IColumn * nested_result_column = result_nullable_column - ? &result_nullable_column->getNestedColumn() - : result.get(); - - ColumnPtr existing_null_map_column = result_nullable_column - ? result_nullable_column->getNullMapColumnPtr() - : nullptr; - - if (transform_null_in) - { - if (!null_map_holder) - null_map_holder = filtered_null_map_column; - else - null_map_holder = mergeNullMaps(null_map_holder, filtered_null_map_column); - - null_map = &assert_cast(*null_map_holder).getData(); - } - else - { - ColumnPtr merged_null_map_column = mergeNullMaps(existing_null_map_column, filtered_null_map_column); - result = ColumnNullable::create(nested_result_column->getPtr(), merged_null_map_column); - } + processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } // Append the result to materialized columns diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 8a821d87dfb..240d651352d 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -61,6 +61,8 @@ public: void checkIsCreated() const; + void processDateTime64Column(const ColumnWithTypeAndName & column_to_cast, ColumnPtr & result, ColumnPtr & null_map_holder, ConstNullMapPtr & null_map) const; + /** For columns of 'block', check belonging of corresponding rows to the set. * Return UInt8 column with the result. */ From ea5492d6f854642c410d9e5a38cc6b553ce331b2 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 15:50:13 +0000 Subject: [PATCH 585/816] typo --- src/Client/ProgressTable.cpp | 4 ++-- src/Client/ProgressTable.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ProgressTable.cpp b/src/Client/ProgressTable.cpp index d66df4eded8..f63935440e4 100644 --- a/src/Client/ProgressTable.cpp +++ b/src/Client/ProgressTable.cpp @@ -222,7 +222,7 @@ void ProgressTable::writeTable(WriteBufferFromFileDescriptor & message, bool sho writeWithWidth(message, COLUMN_EVENT_NAME, column_event_name_width); writeWithWidth(message, COLUMN_VALUE, COLUMN_VALUE_WIDTH); writeWithWidth(message, COLUMN_PROGRESS, COLUMN_PROGRESS_WIDTH); - auto col_doc_width = getColumnDocumentationWith(terminal_width); + auto col_doc_width = getColumnDocumentationWidth(terminal_width); if (col_doc_width) writeWithWidth(message, COLUMN_DOCUMENTATION_NAME, col_doc_width); message << CLEAR_TO_END_OF_LINE; @@ -380,7 +380,7 @@ size_t ProgressTable::tableSize() const return metrics.empty() ? 0 : metrics.size() + 1; } -size_t ProgressTable::getColumnDocumentationWith(size_t terminal_width) const +size_t ProgressTable::getColumnDocumentationWidth(size_t terminal_width) const { auto fixed_columns_width = column_event_name_width + COLUMN_VALUE_WIDTH + COLUMN_PROGRESS_WIDTH; if (terminal_width < fixed_columns_width + COLUMN_DOCUMENTATION_MIN_WIDTH) diff --git a/src/Client/ProgressTable.h b/src/Client/ProgressTable.h index 6599dff4aa2..f2563d91217 100644 --- a/src/Client/ProgressTable.h +++ b/src/Client/ProgressTable.h @@ -87,7 +87,7 @@ private: }; size_t tableSize() const; - size_t getColumnDocumentationWith(size_t terminal_width) const; + size_t getColumnDocumentationWidth(size_t terminal_width) const; using MetricName = String; From 2d3f5fabda5aa46e0af4b102e2c809ffadb03dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 17:08:32 +0100 Subject: [PATCH 586/816] Adjust some tests --- tests/integration/test_grpc_protocol/test.py | 2 +- tests/integration/test_peak_memory_usage/test.py | 11 ++++++----- tests/queries/0_stateless/01921_test_progress_bar.py | 2 +- .../03096_text_log_format_string_args_not_empty.sql | 2 +- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 732907eed7a..561f5144aac 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -364,7 +364,7 @@ def test_logs(): ) assert query in logs assert "Read 1000000 rows" in logs - assert "Peak memory usage" in logs + assert "Query peak memory usage" in logs def test_progress(): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 877cf97bb18..f5ebc8bd99c 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -68,7 +68,8 @@ def get_memory_usage_from_client_output_and_close(client_output): for line in client_output: print(f"'{line}'\n") if not peek_memory_usage_str_found: - peek_memory_usage_str_found = "Peak memory usage" in line + # Can be both Peak/peak + peek_memory_usage_str_found = "eak memory usage" in line if peek_memory_usage_str_found: search_obj = re.search(r"[+-]?[0-9]+\.[0-9]+", line) @@ -92,13 +93,13 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): client1.send( "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) - client1.expect("Peak memory usage", timeout=60) + client1.expect("Query peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage assert shard_2.contains_in_log( - f"Peak memory usage (for query): {peak_memory_usage}" + f"Query peak memory usage: {peak_memory_usage}" ) @@ -113,11 +114,11 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): client1.send( "SELECT COUNT(*) FROM (SELECT number FROM numbers(1,300000) INTERSECT SELECT number FROM numbers(10000,1200000))" ) - client1.expect("Peak memory usage", timeout=60) + client1.expect("Query peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage assert shard_1.contains_in_log( - f"Peak memory usage (for query): {peak_memory_usage}" + f"Query peak memory usage: {peak_memory_usage}" ) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index e686698ad9f..4199503ba4a 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,4 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") - client1.expect("Peak memory usage: .*B" + end_of_block) + client1.expect("Query peak memory usage: .*B" + end_of_block) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index a08f35cfc1d..a4eef59f442 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -7,7 +7,7 @@ select conut(); -- { serverError UNKNOWN_FUNCTION } system flush logs; SET max_rows_to_read = 0; -- system.text_log can be really big -select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; +select count() > 0 from system.text_log where message_format_string = '{}{} memory usage: {}.' and not empty(value1) and value3 like '% MiB'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = '`count`' and value4 = 'SELECT count'; From 9858e96e257a926329fb104f451091a4937bf26d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 29 Oct 2024 16:08:40 +0000 Subject: [PATCH 587/816] Fix Style. --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index a910ac5f59b..c6f0455652a 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -347,7 +347,7 @@ void Set::processDateTime64Column( const ColumnWithTypeAndName & column_to_cast, ColumnPtr & result, ColumnPtr & null_map_holder, - ConstNullMapPtr & null_map) const + ConstNullMapPtr & null_map) const { // Check for sub-second precision and create a null map ColumnUInt8::Ptr filtered_null_map_column = checkDateTimePrecision(column_to_cast); From e6969f541930bddfd00ac990f779225cb5acc1ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 16:41:18 +0000 Subject: [PATCH 588/816] Remove unnecessary test diffs --- tests/ci/.mypy.ini | 1 - .../0_stateless/01079_bad_alters_zookeeper_long.sh | 8 +++++++- tests/queries/0_stateless/01162_strange_mutations.sh | 3 ++- tests/queries/0_stateless/01164_alter_memory_database.sql | 3 ++- .../0_stateless/01165_lost_part_empty_partition.sql | 3 ++- .../0_stateless/01166_truncate_multiple_partitions.sql | 3 +++ .../0_stateless/01192_rename_database_zookeeper.sh | 6 +++++- .../01513_optimize_aggregation_in_order_memory_long.sql | 2 +- .../01650_fetch_patition_with_macro_in_zk_path_long.sql | 4 +--- ...an_optimizations_optimize_read_in_window_order_long.sh | 3 ++- .../0_stateless/01700_system_zookeeper_path_in.sql | 3 ++- .../01715_background_checker_blather_zookeeper_long.sql | 3 ++- .../0_stateless/01754_direct_dictionary_complex_key.sql | 2 +- tests/queries/0_stateless/02125_many_mutations.sh | 3 ++- tests/queries/0_stateless/02125_many_mutations_2.sh | 3 ++- tests/queries/0_stateless/02205_ephemeral_1.sql | 2 ++ .../0_stateless/02221_system_zookeeper_unrestricted.sh | 3 ++- tests/queries/0_stateless/02253_empty_part_checksums.sh | 3 ++- tests/queries/0_stateless/02254_projection_broken_part.sh | 2 +- .../0_stateless/02255_broken_parts_chain_on_start.sh | 2 +- .../0_stateless/02369_lost_part_intersecting_merges.sh | 3 ++- .../0_stateless/02370_lost_part_intersecting_merges.sh | 3 ++- .../02421_truncate_isolation_with_mutations.sh | 2 ++ .../0_stateless/02432_s3_parallel_parts_cleanup.sql | 5 ++++- .../queries/0_stateless/02448_clone_replica_lost_part.sql | 3 ++- .../0_stateless/02454_create_table_with_custom_disk.sql | 3 ++- tests/queries/0_stateless/02555_davengers_rename_chain.sh | 4 ++-- tests/queries/0_stateless/02725_memory-for-merges.sql | 2 +- .../0_stateless/02735_system_zookeeper_connection.sql | 3 ++- .../02882_replicated_fetch_checksums_doesnt_match.sql | 2 ++ .../02910_replicated_merge_parameters_must_consistent.sql | 3 ++- .../02916_replication_protocol_wait_for_part.sql | 2 +- .../02919_insert_meet_eternal_hardware_error.sql | 3 ++- .../0_stateless/02922_deduplication_with_zero_copy.sh | 2 +- .../02943_rmt_alter_metadata_merge_checksum_mismatch.sh | 3 ++- tests/queries/0_stateless/03015_optimize_final_rmt.sh | 2 +- .../0_stateless/03155_in_nested_subselects.reference | 2 +- tests/queries/0_stateless/03155_in_nested_subselects.sql | 2 +- .../0_stateless/03223_system_tables_set_not_ready.sql | 3 ++- tests/queries/0_stateless/transactions.lib | 4 ++-- tests/queries/1_stateful/00170_s3_cache.reference | 1 + tests/queries/1_stateful/00170_s3_cache.sql | 1 + 42 files changed, 80 insertions(+), 40 deletions(-) diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini index ecb4aef87dd..bb46a6d24b6 100644 --- a/tests/ci/.mypy.ini +++ b/tests/ci/.mypy.ini @@ -16,4 +16,3 @@ no_implicit_reexport = True strict_equality = True extra_checks = True ignore_missing_imports = True -logging-fstring-interpolation = False \ No newline at end of file diff --git a/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh b/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh index 22f8e5269bd..a619bcdbce2 100755 --- a/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_bad_alters_zookeeper_long.sh @@ -22,7 +22,13 @@ $CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_bad_alters MODIFY COLUMN value sleep 2 -while [[ $($CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='0000000000' and database = '$CLICKHOUSE_DATABASE'" 2>&1) ]]; do +counter=0 retries=60 +while [[ $counter -lt $retries ]]; do + output=$($CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='0000000000' and database = '$CLICKHOUSE_DATABASE'" 2>&1) + if [[ "$output" == *"finished"* ]]; then + break + fi + ((++counter)) sleep 1 done diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index f2428141264..db7ec8e0755 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-replicated-database +# Tags: no-replicated-database, no-shared-merge-tree # Tag no-replicated-database: CREATE AS SELECT is disabled +# Tag no-shared-merge-tree -- implemented separate test, just bad substituion here CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01164_alter_memory_database.sql b/tests/queries/0_stateless/01164_alter_memory_database.sql index f46fc8f9853..0beddbfaa88 100644 --- a/tests/queries/0_stateless/01164_alter_memory_database.sql +++ b/tests/queries/0_stateless/01164_alter_memory_database.sql @@ -1,4 +1,5 @@ --- Tags: zookeeper, no-parallel +-- Tags: zookeeper, no-parallel, no-shared-merge-tree +-- no-shared-merge-tree: doesn't support databases without UUID drop database if exists test_1164_memory; create database test_1164_memory engine=Memory; diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index 787d4567218..2ed46a05823 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -1,4 +1,5 @@ --- Tags: zookeeper +-- Tags: zookeeper, no-shared-merge-tree +-- no-shared-merge-tree: shared merge tree doesn't loose data parts SET max_rows_to_read = 0; -- system.text_log can be really big diff --git a/tests/queries/0_stateless/01166_truncate_multiple_partitions.sql b/tests/queries/0_stateless/01166_truncate_multiple_partitions.sql index 1a7e3ed3bc4..8f5d3ccc1fe 100644 --- a/tests/queries/0_stateless/01166_truncate_multiple_partitions.sql +++ b/tests/queries/0_stateless/01166_truncate_multiple_partitions.sql @@ -1,3 +1,6 @@ +-- Tags: no-shared-catalog +-- no-shared-catalog: standard MergeTree is not supported + drop table if exists trunc; set default_table_engine='ReplicatedMergeTree'; diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index 1ac01fe6abc..e48fc428265 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -1,5 +1,9 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest +# Tags: zookeeper, no-parallel, no-fasttest, no-shared-merge-tree +# no-shared-merge-tree: database ordinary not supported + +# Creation of a database with Ordinary engine emits a warning. +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # Creation of a database with Ordinary engine emits a warning. CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal diff --git a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql index d9430018469..5e7cd2f7da7 100644 --- a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql +++ b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql @@ -1,5 +1,5 @@ -- Tags: long, no-random-merge-tree-settings --- FIXME no-random-merge-tree-settings requires investigation +--- FIXME no-random-merge-tree-settings requires investigation drop table if exists data_01513; create table data_01513 (key String) engine=MergeTree() order by key; diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql index 029a17f87dc..9c6bef4b6b4 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-shared-merge-tree DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; @@ -32,5 +32,3 @@ SELECT _part, * FROM restore_01640; DROP TABLE test_01640; DROP TABLE restore_01640; - - diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh index e5e57ddb78a..fde0fb8a8de 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, no-random-merge-tree-settings +# Tags: long, no-random-merge-tree-settings, no-random-settings +# no sanitizers -- bad idea to check memory usage with sanitizers CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index 3b321d3cea5..0c9f8c3293c 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -1,4 +1,5 @@ --- Tags: zookeeper +-- Tags: zookeeper, no-shared-merge-tree +-- no-shared-merge-tree: depend on replicated merge tree zookeeper structure DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index 32481be1bcd..3de17f8c30b 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -1,4 +1,5 @@ --- Tags: long, zookeeper +-- Tags: long, zookeeper, no-shared-merge-tree +-- no-shared-merge-tree: no replication queue DROP TABLE IF EXISTS i20203_1 SYNC; DROP TABLE IF EXISTS i20203_2 SYNC; diff --git a/tests/queries/0_stateless/01754_direct_dictionary_complex_key.sql b/tests/queries/0_stateless/01754_direct_dictionary_complex_key.sql index a695302161d..8b34eb87fb2 100644 --- a/tests/queries/0_stateless/01754_direct_dictionary_complex_key.sql +++ b/tests/queries/0_stateless/01754_direct_dictionary_complex_key.sql @@ -89,7 +89,7 @@ SELECT dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_compl SELECT 'dictHas'; SELECT dictHas('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; SELECT 'select all values as input stream'; -SELECT * FROM 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes; +SELECT * FROM 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes ORDER BY ALL; DROP DICTIONARY 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes; DROP TABLE 01754_dictionary_db.complex_key_complex_attributes_source_table; diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index 363253371cc..4dd9c6d9648 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-shared-merge-tree +# no-shared-merge-tree -- this test is too slow CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index d0025e6d8cc..e63bd296ca3 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-parallel +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-parallel, no-shared-merge-tree +# no-shared-merge-tree -- this test is too slow CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02205_ephemeral_1.sql b/tests/queries/0_stateless/02205_ephemeral_1.sql index 7a996ee3a8f..fd1d2f5fa18 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.sql +++ b/tests/queries/0_stateless/02205_ephemeral_1.sql @@ -1,3 +1,5 @@ +SET mutations_sync=2; + DROP TABLE IF EXISTS t_ephemeral_02205_1; CREATE TABLE t_ephemeral_02205_1 (x UInt32 DEFAULT y, y UInt32 EPHEMERAL 17, z UInt32 DEFAULT 5) ENGINE = Memory; diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index e23a272a4e8..deb45e20b7c 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, zookeeper +# Tags: no-replicated-database, zookeeper, no-shared-merge-tree +# no-shared-merge-tree: depend on specific paths created by replicated tables CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02253_empty_part_checksums.sh b/tests/queries/0_stateless/02253_empty_part_checksums.sh index 371c0768e3d..66a4434576b 100755 --- a/tests/queries/0_stateless/02253_empty_part_checksums.sh +++ b/tests/queries/0_stateless/02253_empty_part_checksums.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database +# Tags: zookeeper, no-replicated-database, no-shared-merge-tree # no-replicated-database because it adds extra replicas +# no-shared-merge-tree do something with parts on local fs CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02254_projection_broken_part.sh b/tests/queries/0_stateless/02254_projection_broken_part.sh index 3521d1d9d16..04a0c4fb0a1 100755 --- a/tests/queries/0_stateless/02254_projection_broken_part.sh +++ b/tests/queries/0_stateless/02254_projection_broken_part.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: long, zookeeper, no-shared-merge-tree CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02255_broken_parts_chain_on_start.sh b/tests/queries/0_stateless/02255_broken_parts_chain_on_start.sh index de260937b9c..888ac73e4ab 100755 --- a/tests/queries/0_stateless/02255_broken_parts_chain_on_start.sh +++ b/tests/queries/0_stateless/02255_broken_parts_chain_on_start.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: long, zookeeper, no-shared-merge-tree CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index 357c089e040..8853d75a86b 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: zookeeper, no-shared-merge-tree, long +# no-shared-merge-tree: depend on local fs CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh index e34163d0502..de61f5cc23e 100755 --- a/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02370_lost_part_intersecting_merges.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: long, zookeeper, no-shared-merge-tree +# no-shared-merge-tree: depend on local fs (remove parts) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh b/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh index fabc9eab140..da0b132bcbc 100755 --- a/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh +++ b/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh @@ -11,6 +11,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./parts.lib . "$CURDIR"/parts.lib +CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --apply_mutations_on_fly 0" + function reset_table() { table=${1:-"tt"} diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 0e7a14ddf99..c25f4e13023 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -1,10 +1,13 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-shared-merge-tree +-- no-shared-merge-tree: depend on custom storage policy SET send_logs_level = 'fatal'; drop table if exists rmt; drop table if exists rmt2; +set apply_mutations_on_fly = 0; + -- Disable compact parts, because we need hardlinks in mutations. create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index ec669ace620..9aea6aeaa94 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -1,4 +1,5 @@ --- Tags: long +-- Tags: long, no-shared-merge-tree +-- no-shared-merge-tree: depend on replication queue/fetches SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index a2d46cf6d1b..73f4e166ea2 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -1,4 +1,5 @@ --- Tags: no-object-storage, no-replicated-database +-- Tags: no-object-storage, no-replicated-database, no-shared-merge-tree +-- no-shared-merge-tree: custom disk DROP TABLE IF EXISTS test; diff --git a/tests/queries/0_stateless/02555_davengers_rename_chain.sh b/tests/queries/0_stateless/02555_davengers_rename_chain.sh index b770eaba087..eaa455f181d 100755 --- a/tests/queries/0_stateless/02555_davengers_rename_chain.sh +++ b/tests/queries/0_stateless/02555_davengers_rename_chain.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash -# Tags: replica, no-fasttest +# Tags: replica, no-fasttest, no-shared-merge-tree # no-fasttest: Mutation load can be slow - +# no-shared-merge-tree -- have separate test for it CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index 8e4d4f5b3e0..a0adddb1aff 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -1,4 +1,4 @@ --- Tags: no-object-storage, no-random-merge-tree-settings +-- Tags: no-object-storage, no-random-merge-tree-settings, no-fasttest -- We allocate a lot of memory for buffers when reading or writing to S3 DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index 48ada633225..2ea40edddf9 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest, no-replicated-database +-- Tags: no-fasttest, no-replicated-database, no-shared-merge-tree +-- no-shared-merge-tree -- smt doesn't support aux zookeepers DROP TABLE IF EXISTS test_zk_connection_table; diff --git a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql index a745625f17a..45027e0454f 100644 --- a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql +++ b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql @@ -1,3 +1,5 @@ +-- Tags: no-shared-merge-tree + DROP TABLE IF EXISTS checksums_r3; DROP TABLE IF EXISTS checksums_r2; DROP TABLE IF EXISTS checksums_r1; diff --git a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql index 0f452105e6d..ec19e54e9b6 100644 --- a/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql +++ b/tests/queries/0_stateless/02910_replicated_merge_parameters_must_consistent.sql @@ -1,4 +1,5 @@ --- Tags: zookeeper, no-replicated-database +-- Tags: zookeeper, no-replicated-database, no-shared-merge-tree + CREATE TABLE t ( `id` UInt64, diff --git a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql index 010e29a34e8..63c7120a61a 100644 --- a/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql +++ b/tests/queries/0_stateless/02916_replication_protocol_wait_for_part.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database, no-fasttest +-- Tags: no-replicated-database, no-fasttest, no-shared-merge-tree -- Tag no-replicated-database: different number of replicas create table tableIn (n int) diff --git a/tests/queries/0_stateless/02919_insert_meet_eternal_hardware_error.sql b/tests/queries/0_stateless/02919_insert_meet_eternal_hardware_error.sql index 05602b42c6a..b04b22ac9cd 100644 --- a/tests/queries/0_stateless/02919_insert_meet_eternal_hardware_error.sql +++ b/tests/queries/0_stateless/02919_insert_meet_eternal_hardware_error.sql @@ -1,4 +1,5 @@ --- Tags: zookeeper, no-parallel +-- Tags: zookeeper, no-parallel, no-shared-merge-tree +-- no-shared-merge-tree: This failure injection is only RMT specific DROP TABLE IF EXISTS t_hardware_error NO DELAY; diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index 2eccade5c81..67a1e6f5e8d 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database, no-fasttest +# Tags: long, no-replicated-database, no-fasttest, no-shared-merge-tree set -e diff --git a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh index 44af2dbf26f..2a5f957b97b 100755 --- a/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh +++ b/tests/queries/0_stateless/02943_rmt_alter_metadata_merge_checksum_mismatch.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-shared-merge-tree # Tag no-parallel: failpoint is in use +# Tag no-shared-merge-tree: looks like it tests a specific behaviour of ReplicatedMergeTree with failpoints CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03015_optimize_final_rmt.sh b/tests/queries/0_stateless/03015_optimize_final_rmt.sh index e8bd7466503..187c8d54842 100755 --- a/tests/queries/0_stateless/03015_optimize_final_rmt.sh +++ b/tests/queries/0_stateless/03015_optimize_final_rmt.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-random-settings, no-random-merge-tree-settings, no-tsan, no-msan, no-ubsan, no-asan +# Tags: long, no-random-settings, no-random-merge-tree-settings, no-tsan, no-msan, no-ubsan, no-asan, no-debug # no sanitizers: too slow CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/03155_in_nested_subselects.reference b/tests/queries/0_stateless/03155_in_nested_subselects.reference index 5565ed6787f..0463db26710 100644 --- a/tests/queries/0_stateless/03155_in_nested_subselects.reference +++ b/tests/queries/0_stateless/03155_in_nested_subselects.reference @@ -1,4 +1,4 @@ 0 +0 1 -0 1 diff --git a/tests/queries/0_stateless/03155_in_nested_subselects.sql b/tests/queries/0_stateless/03155_in_nested_subselects.sql index faecb73040d..62a25165162 100644 --- a/tests/queries/0_stateless/03155_in_nested_subselects.sql +++ b/tests/queries/0_stateless/03155_in_nested_subselects.sql @@ -16,4 +16,4 @@ using id; INSERT INTO Null SELECT number AS id FROM numbers(2); -select * from Example; -- should return 4 rows +select * from Example order by all; -- should return 4 rows diff --git a/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql b/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql index 907fa47143c..2cbc0286f4c 100644 --- a/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql +++ b/tests/queries/0_stateless/03223_system_tables_set_not_ready.sql @@ -1,5 +1,6 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-shared-merge-tree -- Tag no-fasttest -- due to EmbeddedRocksDB +-- Tag no-shared-merge-tree -- due to system.replication_queue drop table if exists null; drop table if exists dist; diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index 12345ac2799..94125004849 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -11,7 +11,7 @@ function tx() session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" query_id="${session}_${RANDOM}" url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" - url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" + url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE&apply_mutations_on_fly=0" ${CLICKHOUSE_CURL} -m 90 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" } @@ -56,7 +56,7 @@ function tx_async() session="${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}_tx$tx_num" query_id="${session}_${RANDOM}" url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" - url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" + url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE&apply_mutations_on_fly=0" # We cannot be sure that query will actually start execution and appear in system.processes before the next call to tx_wait # Also we cannot use global map in bash to store last query_id for each tx_num, so we use tmp file... diff --git a/tests/queries/1_stateful/00170_s3_cache.reference b/tests/queries/1_stateful/00170_s3_cache.reference index 293fbd7f8cb..79c780b0729 100644 --- a/tests/queries/1_stateful/00170_s3_cache.reference +++ b/tests/queries/1_stateful/00170_s3_cache.reference @@ -3,6 +3,7 @@ SET allow_prefetched_read_pool_for_remote_filesystem=0; SET enable_filesystem_cache_on_write_operations=0; SET max_memory_usage='20G'; +SET read_through_distributed_cache = 1; SYSTEM DROP FILESYSTEM CACHE; SELECT count() FROM test.hits_s3; 8873898 diff --git a/tests/queries/1_stateful/00170_s3_cache.sql b/tests/queries/1_stateful/00170_s3_cache.sql index 23663a1844d..8709d7d14f1 100644 --- a/tests/queries/1_stateful/00170_s3_cache.sql +++ b/tests/queries/1_stateful/00170_s3_cache.sql @@ -5,6 +5,7 @@ SET allow_prefetched_read_pool_for_remote_filesystem=0; SET enable_filesystem_cache_on_write_operations=0; SET max_memory_usage='20G'; +SET read_through_distributed_cache = 1; SYSTEM DROP FILESYSTEM CACHE; SELECT count() FROM test.hits_s3; SELECT count() FROM test.hits_s3 WHERE AdvEngineID != 0; From 7d2fc48b6d37c5120372349892f5382823cafa06 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 17:02:43 +0000 Subject: [PATCH 589/816] Fixed restart broken --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 89a5bfe9469..9fa7b669b79 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -118,14 +118,15 @@ StorageObjectStorage::StorageObjectStorage( } catch (...) { - if (mode <= LoadingStrictnessLevel::CREATE) + if (mode <= LoadingStrictnessLevel::CREATE || columns_.empty() + || (configuration->format + == "auto")) // If we don't have format or schema yet, we can't ignore failed configuration update, because relevant configuration is crucial for format and schema inference { throw; } else { tryLogCurrentException(__PRETTY_FUNCTION__); - return; } } From 9b435388deb183edc2dfee520107391e6b96a2f4 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 17:20:53 +0000 Subject: [PATCH 590/816] Remove useless stuff --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 3 +- .../ObjectStorage/DataLakes/Common.cpp | 7 ----- .../ObjectStorage/StorageObjectStorage.cpp | 28 ++----------------- .../configs/config.d/filesystem_caches.xml | 1 - .../integration/test_storage_iceberg/test.py | 7 ----- 5 files changed, 4 insertions(+), 42 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 44aeabc1c28..47ef97401f2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -501,7 +501,8 @@ void S3ObjectStorage::applyNewSettings( } auto current_settings = s3_settings.get(); - if (options.allow_client_change && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) + if (options.allow_client_change + && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) { auto new_client = getClient(uri, *modified_settings, context, for_disk_s3); client.set(std::move(new_client)); diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp index c21c0486eca..4830cc52a90 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -1,9 +1,6 @@ #include "Common.h" #include #include -#include -#include -#include #include namespace DB @@ -16,10 +13,6 @@ std::vector listFiles( { auto key = std::filesystem::path(configuration.getPath()) / prefix; RelativePathsWithMetadata files_with_metadata; - // time_t now = time(nullptr); - Poco::DateTime now; - std::string formatted = Poco::DateTimeFormatter::format(now, Poco::DateTimeFormat::ISO8601_FORMAT); - LOG_ERROR(&Poco::Logger::get("Inside listFiles"), "Time of files listing: {}", formatted); object_storage.listObjects(key, files_with_metadata, 0); Strings res; for (const auto & file_with_metadata : files_with_metadata) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 9fa7b669b79..1ed6e137a31 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -69,27 +69,6 @@ String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, Con return ""; } -void printConfiguration(const Poco::Util::AbstractConfiguration & config, std::string log_name, const std::string & prefix = "") -{ - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(prefix, keys); - - for (const auto & key : keys) - { - std::string full_key = prefix.empty() ? key : (prefix + "." + key); - - if (config.hasProperty(full_key)) - { - std::string value = config.getString(full_key); - LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", full_key, value); - } - - // Recursively print sub-configurations - printConfiguration(config, full_key, log_name); - } -} - - StorageObjectStorage::StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, @@ -110,17 +89,14 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - // LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size()); - printConfiguration(context->getConfigRef(), "Storage create"); try { configuration->update(object_storage, context); } catch (...) { - if (mode <= LoadingStrictnessLevel::CREATE || columns_.empty() - || (configuration->format - == "auto")) // If we don't have format or schema yet, we can't ignore failed configuration update, because relevant configuration is crucial for format and schema inference + // If we don't have format or schema yet, we can't ignore failed configuration update, because relevant configuration is crucial for format and schema inference + if (mode <= LoadingStrictnessLevel::CREATE || columns_.empty() || (configuration->format == "auto")) { throw; } diff --git a/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml index 3b1b2aeb37e..e91362640fe 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/filesystem_caches.xml @@ -5,5 +5,4 @@ cache1 - diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 690ebeeffbf..36aba550dbd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -5,7 +5,6 @@ import os import time import uuid from datetime import datetime -from logging import log import pyspark import pytest @@ -857,20 +856,14 @@ def test_restart_broken_s3(started_cluster): ) minio_client.remove_bucket(bucket) - print("Before restart: ", datetime.now()) - instance.restart_clickhouse() assert "NoSuchBucket" in instance.query_and_get_error( f"SELECT count() FROM {TABLE_NAME}" ) - time.sleep(10) - minio_client.make_bucket(bucket) - print("Before successful select: ", datetime.now()) - files = default_upload_directory( started_cluster, "s3", From 98c9afda2e48053877ec38a5dbe3eb48f0b5d8a4 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 29 Oct 2024 17:24:30 +0000 Subject: [PATCH 591/816] Remove build ifdef issue --- src/Storages/registerStorages.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 4eb90955a6c..6f6d9c3148f 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -145,6 +145,10 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]]) registerStorageAzureQueue(factory); #endif +#if USE_AVRO + registerStorageIceberg(factory); +#endif + #if USE_AWS_S3 registerStorageHudi(factory); registerStorageS3Queue(factory); @@ -153,14 +157,10 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]]) registerStorageDeltaLake(factory); #endif - #if USE_AVRO - registerStorageIceberg(factory); - #endif +#endif - #endif - - #if USE_HDFS - #if USE_HIVE +#if USE_HDFS +# if USE_HIVE registerStorageHive(factory); #endif #endif From c3a7c25c519f2966c700d80a4fa9152ab2878927 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 19:53:35 +0100 Subject: [PATCH 592/816] Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 7 +- ...calar_in_format_table_expression.reference | 5 ++ ...3257_scalar_in_format_table_expression.sql | 82 +++++++++++++++++++ 3 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03257_scalar_in_format_table_expression.reference create mode 100644 tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 381edee607d..cb3087af707 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -227,8 +227,13 @@ void QueryAnalyzer::resolveConstantExpression(QueryTreeNodePtr & node, const Que scope.context = context; auto node_type = node->getNodeType(); + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + { + evaluateScalarSubqueryIfNeeded(node, scope); + return; + } - if (table_expression && node_type != QueryTreeNodeType::QUERY && node_type != QueryTreeNodeType::UNION) + if (table_expression) { scope.expression_join_tree_node = table_expression; validateTableExpressionModifiers(scope.expression_join_tree_node, scope); diff --git a/tests/queries/0_stateless/03257_scalar_in_format_table_expression.reference b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.reference new file mode 100644 index 00000000000..5d60960bee9 --- /dev/null +++ b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.reference @@ -0,0 +1,5 @@ +Hello 111 +World 123 +Hello 111 +World 123 +6 6 diff --git a/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql new file mode 100644 index 00000000000..1d74b0c3775 --- /dev/null +++ b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql @@ -0,0 +1,82 @@ +SELECT * FROM format( + JSONEachRow, +$$ +{"a": "Hello", "b": 111} +{"a": "World", "b": 123} +$$ + ); + +-- Should be equivalent to the previous one +SELECT * FROM format( + JSONEachRow, + ( + SELECT $$ +{"a": "Hello", "b": 111} +{"a": "World", "b": 123} +$$ + ) + ); + +-- The scalar subquery is incorrect so it should throw the proper error +SELECT * FROM format( + JSONEachRow, + ( + SELECT $$ +{"a": "Hello", "b": 111} +{"a": "World", "b": 123} +$$ + WHERE column_does_not_exists = 4 + ) + ); -- { serverError UNKNOWN_IDENTIFIER } + +-- https://github.com/ClickHouse/ClickHouse/issues/70177 + +-- Resolution of the scalar subquery should work ok (already did, adding a test just for safety) +WITH (SELECT sum(number)::String as s FROM numbers(4)) as s +SELECT *, s +FROM format(TSVRaw, s); + +SELECT count() +FROM format(TSVRaw, ( + SELECT where_qualified__fuzz_19 + FROM numbers(10000) +)); -- { serverError UNKNOWN_IDENTIFIER } + +SELECT count() +FROM format(TSVRaw, ( + SELECT where_qualified__fuzz_19 + FROM numbers(10000) + UNION ALL + SELECT where_qualified__fuzz_35 + FROM numbers(10000) +)); -- { serverError UNKNOWN_IDENTIFIER } + +WITH ( + SELECT where_qualified__fuzz_19 + FROM numbers(10000) +) as s SELECT count() +FROM format(TSVRaw, s); -- { serverError UNKNOWN_IDENTIFIER } + +-- https://github.com/ClickHouse/ClickHouse/issues/70675 +SELECT count() +FROM format(TSVRaw, ( + SELECT CAST(arrayStringConcat(groupArray(format(TSVRaw, ( + SELECT CAST(arrayStringConcat(1 GLOBAL IN ( + SELECT 1 + WHERE 1 GLOBAL IN ( + SELECT toUInt128(1) + GROUP BY + GROUPING SETS ((1)) + WITH ROLLUP + ) + GROUP BY 1 + WITH CUBE + ), groupArray('some long string')), 'LowCardinality(String)') + FROM numbers(10000) + )), toLowCardinality('some long string')) RESPECT NULLS, '\n'), 'LowCardinality(String)') + FROM numbers(10000) +)) +FORMAT TSVRaw; -- { serverError UNKNOWN_IDENTIFIER } + +-- Same but for table function numbers +SELECT 1 FROM numbers((SELECT DEFAULT)); -- { serverError UNKNOWN_IDENTIFIER } From f73ff65edb2824a744462dbb4dc760fe22ee3648 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Oct 2024 19:00:38 +0000 Subject: [PATCH 593/816] Fix tests --- src/DataTypes/Serializations/ISerialization.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 3c3e9bdc9f9..7522248e088 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -161,7 +161,7 @@ String getNameForSubstreamPath( String stream_name, SubstreamIterator begin, SubstreamIterator end, - bool escape_tuple_delimiter) + bool escape_for_file_name) { using Substream = ISerialization::Substream; @@ -186,7 +186,7 @@ String getNameForSubstreamPath( /// Because nested data may be represented not by Array of Tuple, /// but by separate Array columns with names in a form of a.b, /// and name is encoded as a whole. - if (it->type == Substream::TupleElement && escape_tuple_delimiter) + if (it->type == Substream::TupleElement && escape_for_file_name) stream_name += escapeForFileName(substream_name); else stream_name += substream_name; @@ -206,7 +206,7 @@ String getNameForSubstreamPath( else if (it->type == SubstreamType::ObjectSharedData) stream_name += ".object_shared_data"; else if (it->type == SubstreamType::ObjectTypedPath || it->type == SubstreamType::ObjectDynamicPath) - stream_name += "." + escapeForFileName(it->object_path_name); + stream_name += "." + (escape_for_file_name ? escapeForFileName(it->object_path_name) : it->object_path_name); } return stream_name; From 50de2f4073aa13ac3b8130d065fe7fc5ea681bd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 20:15:11 +0100 Subject: [PATCH 594/816] Fix style --- tests/integration/test_peak_memory_usage/test.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index f5ebc8bd99c..b4f33b54bbf 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -98,9 +98,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage - assert shard_2.contains_in_log( - f"Query peak memory usage: {peak_memory_usage}" - ) + assert shard_2.contains_in_log(f"Query peak memory usage: {peak_memory_usage}") def test_clickhouse_client_max_peak_memory_single_node(started_cluster): @@ -119,6 +117,4 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) assert peak_memory_usage - assert shard_1.contains_in_log( - f"Query peak memory usage: {peak_memory_usage}" - ) + assert shard_1.contains_in_log(f"Query peak memory usage: {peak_memory_usage}") From 078bf2ea5b564ed536330b1b44351c7edb82006e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 20:28:51 +0100 Subject: [PATCH 595/816] Disable enable_named_columns_in_function_tuple for 24.10 --- src/Core/Settings.cpp | 3 +-- src/Core/SettingsChangesHistory.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0aecb7cf941..bebafd145a2 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4451,9 +4451,8 @@ Optimize GROUP BY when all keys in block are constant DECLARE(Bool, legacy_column_name_of_tuple_literal, false, R"( List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher. )", 0) \ - DECLARE(Bool, enable_named_columns_in_function_tuple, true, R"( + DECLARE(Bool, enable_named_columns_in_function_tuple, false, R"( Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers. -Beware that this setting might currently result in broken queries. It's not recommended to use in production )", 0) \ \ DECLARE(Bool, query_plan_enable_optimizations, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e2488901802..266085cf6db 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -87,7 +87,6 @@ static std::initializer_list Date: Tue, 29 Oct 2024 20:48:55 +0100 Subject: [PATCH 596/816] Better --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 266085cf6db..eb64d59ce84 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -87,6 +87,7 @@ static std::initializer_list Date: Tue, 29 Oct 2024 21:22:26 +0100 Subject: [PATCH 597/816] Revert "[RFC] Fix optimize_functions_to_subcolumns optimization" --- src/Functions/array/FunctionsMapMiscellaneous.cpp | 9 ++------- .../03252_optimize_functions_to_subcolumns_map.reference | 1 - .../03252_optimize_functions_to_subcolumns_map.sql | 9 --------- 3 files changed, 2 insertions(+), 17 deletions(-) delete mode 100644 tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.reference delete mode 100644 tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.sql diff --git a/src/Functions/array/FunctionsMapMiscellaneous.cpp b/src/Functions/array/FunctionsMapMiscellaneous.cpp index 368c0ad620f..c3586a57161 100644 --- a/src/Functions/array/FunctionsMapMiscellaneous.cpp +++ b/src/Functions/array/FunctionsMapMiscellaneous.cpp @@ -349,19 +349,14 @@ struct MapKeyLikeAdapter } }; -struct FunctionIdentityMap : public FunctionIdentity -{ - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } -}; - struct NameMapConcat { static constexpr auto name = "mapConcat"; }; using FunctionMapConcat = FunctionMapToArrayAdapter, NameMapConcat>; struct NameMapKeys { static constexpr auto name = "mapKeys"; }; -using FunctionMapKeys = FunctionMapToArrayAdapter, NameMapKeys>; +using FunctionMapKeys = FunctionMapToArrayAdapter, NameMapKeys>; struct NameMapValues { static constexpr auto name = "mapValues"; }; -using FunctionMapValues = FunctionMapToArrayAdapter, NameMapValues>; +using FunctionMapValues = FunctionMapToArrayAdapter, NameMapValues>; struct NameMapContains { static constexpr auto name = "mapContains"; }; using FunctionMapContains = FunctionMapToArrayAdapter, MapToSubcolumnAdapter, NameMapContains>; diff --git a/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.reference b/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.reference deleted file mode 100644 index 3bc835eaeac..00000000000 --- a/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.reference +++ /dev/null @@ -1 +0,0 @@ -['foo'] ['bar'] diff --git a/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.sql b/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.sql deleted file mode 100644 index e0cc932783d..00000000000 --- a/tests/queries/0_stateless/03252_optimize_functions_to_subcolumns_map.sql +++ /dev/null @@ -1,9 +0,0 @@ -drop table if exists x; -create table x -( - kv Map(LowCardinality(String), LowCardinality(String)), - k Array(LowCardinality(String)) alias mapKeys(kv), - v Array(LowCardinality(String)) alias mapValues(kv) -) engine=Memory(); -insert into x values (map('foo', 'bar')); -select k, v from x settings optimize_functions_to_subcolumns=1; From 6e57a41401f2a75602f02282ff00c2c8470daef7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 29 Oct 2024 21:45:13 +0100 Subject: [PATCH 598/816] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c26e47a78fa..adb3fbe22ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,6 @@ #### Backward Incompatible Change * Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#68614](https://github.com/ClickHouse/ClickHouse/pull/68614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Reordering of filter conditions from `[PRE]WHERE` clause is now allowed by default. It could be disabled by setting `allow_reorder_prewhere_conditions` to `false`. [#70657](https://github.com/ClickHouse/ClickHouse/pull/70657) ([Nikita Taranov](https://github.com/nickitat)). -* Fix `optimize_functions_to_subcolumns` optimization (previously could lead to `Invalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)` error), by preserving `LowCardinality` type in `mapKeys`/`mapValues`. [#70716](https://github.com/ClickHouse/ClickHouse/pull/70716) ([Azat Khuzhin](https://github.com/azat)). * Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature From 20ee02e97a6935052eb72b6451dee09cb7de72e9 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Tue, 29 Oct 2024 14:41:08 -0700 Subject: [PATCH 599/816] Update README.md - Update meetups Remove Oslo, add Amsterdam, remove old ones cc @tylerhannan --- README.md | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/README.md b/README.md index 96dec2ca607..9d55d1fe9da 100644 --- a/README.md +++ b/README.md @@ -42,11 +42,11 @@ Keep an eye out for upcoming meetups and events around the world. Somewhere else Upcoming meetups -* [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31 * [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - November 12 * [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19 * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 +* [Amsterdam Meetup](https://www.meetup.com/clickhouse-netherlands-user-group/events/303638814) - December 3 * [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9 Recently completed meetups @@ -54,20 +54,6 @@ Recently completed meetups * [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22 * [Singapore Meetup](https://www.meetup.com/clickhouse-singapore-meetup-group/events/303212064/) - October 3 * [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 -* [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 -* [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 -* [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 -* [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 -* [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 -* [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 -* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 -* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 -* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 -* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 -* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 -* [Bangalore Meetup](https://www.meetup.com/clickhouse-bangalore-user-group/events/303208274/) - September 18 -* [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From bb9355b3d3fd2748ed1877d839ff555580f1be70 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 29 Oct 2024 22:52:36 +0100 Subject: [PATCH 600/816] stash --- src/Planner/findParallelReplicasQuery.cpp | 101 +++++++++++++++++++--- 1 file changed, 88 insertions(+), 13 deletions(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index b97a9a36381..91cbc492fdc 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -17,10 +17,12 @@ #include #include #include +#include #include #include #include #include +#include "Processors/QueryPlan/SortingStep.h" namespace DB { @@ -52,22 +54,30 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre { case QueryTreeNodeType::TABLE: { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto & table_node = query_tree_node->as(); const auto & storage = table_node.getStorage(); /// Here we check StorageDummy as well, to support a query tree with replaced storages. if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); /// parallel replicas is not supported with FINAL if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; + } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; } case QueryTreeNodeType::TABLE_FUNCTION: { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; } case QueryTreeNodeType::QUERY: @@ -75,6 +85,7 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & query_node_to_process = query_tree_node->as(); query_tree_node = query_node_to_process.getJoinTree().get(); res.push(&query_node_to_process); + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::UNION: @@ -83,15 +94,20 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & union_queries = union_node.getQueries().getNodes(); if (union_queries.empty()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; + } query_tree_node = union_queries.front().get(); + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::ARRAY_JOIN: { const auto & array_join_node = query_tree_node->as(); query_tree_node = array_join_node.getTableExpression().get(); + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::JOIN: @@ -105,9 +121,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre || (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All); if (!can_parallelize_join) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; + } query_tree_node = join_node.getLeftTableExpression().get(); + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } default: @@ -173,75 +193,114 @@ const QueryNode * findQueryForParallelReplicas( const QueryPlan::Node * prev_checked_node = nullptr; const QueryNode * res = nullptr; + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); while (!stack.empty()) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const QueryNode * subquery_node = stack.top(); stack.pop(); auto it = mapping.find(subquery_node); /// This should not happen ideally. if (it == mapping.end()) - break; - - const QueryPlan::Node * curr_node = it->second; - const QueryPlan::Node * next_node_to_check = curr_node; - bool can_distribute_full_node = true; - - while (next_node_to_check && next_node_to_check != prev_checked_node) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); + break; + } + + const QueryPlan::Node * const curr_node = it->second; + std::deque> nodes_to_check; + nodes_to_check.push_front(std::make_pair(curr_node, false)); + bool can_distribute_full_node = true; + bool in = false; + + while (!nodes_to_check.empty() /* && nodes_to_check.front() != prev_checked_node*/) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); + const auto & [next_node_to_check, digging_into_rabbit_hole] = nodes_to_check.front(); + LOG_DEBUG( + &Poco::Logger::get("debug"), + "next_node_to_check->step->getName()={}, next_node_to_check->step->getStepDescription());={}", + next_node_to_check->step->getName(), + next_node_to_check->step->getStepDescription()); + nodes_to_check.pop_front(); const auto & children = next_node_to_check->children; auto * step = next_node_to_check->step.get(); if (children.empty()) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); /// Found a source step. This should be possible only in the first iteration. if (prev_checked_node) - return nullptr; + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); + // return nullptr; + } - next_node_to_check = nullptr; + nodes_to_check = {}; } else if (children.size() == 1) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto * expression = typeid_cast(step); const auto * filter = typeid_cast(step); + const auto * sorting = typeid_cast(step); const auto * creating_sets = typeid_cast(step); bool allowed_creating_sets = settings[Setting::parallel_replicas_allow_in_with_subquery] && creating_sets; - if (!expression && !filter && !allowed_creating_sets) + if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->getStepDescription().contains("before JOIN"))) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); can_distribute_full_node = false; + in = digging_into_rabbit_hole; + } - next_node_to_check = children.front(); + nodes_to_check.push_front(std::pair(children.front(), digging_into_rabbit_hole)); } else { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto * join = typeid_cast(step); /// We've checked that JOIN is INNER/LEFT in query tree. /// Don't distribute UNION node. if (!join) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; + } - next_node_to_check = children.front(); + for (const auto & child : children) + nodes_to_check.push_front(std::make_pair(child, true)); } } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); + /// Current node contains steps like GROUP BY / DISTINCT /// Will try to execute query up to WithMergableStage if (!can_distribute_full_node) { /// Current query node does not contain subqueries. /// We can execute parallel replicas over storage::read. + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); if (!res) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; + } - return subquery_node; + return in ? res : subquery_node; } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); + /// Query is simple enough to be fully distributed. res = subquery_node; prev_checked_node = curr_node; } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } @@ -261,16 +320,26 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr auto context = query_node ? query_node->getContext() : union_node->getContext(); if (!context->canUseParallelReplicasOnInitiator()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; + } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); auto stack = getSupportingParallelReplicasQuery(query_tree_node.get()); /// Empty stack means that storage does not support parallel replicas. if (stack.empty()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; + } /// We don't have any subquery and storage can process parallel replicas by itself. if (stack.top() == query_tree_node.get()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; + } /// This is needed to avoid infinite recursion. auto mutable_context = Context::createCopy(context); @@ -295,16 +364,22 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr /// Now, return a query from initial stack. if (res) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); while (!new_stack.empty()) { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); if (res == new_stack.top()) + { + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return stack.top(); + } stack.pop(); new_stack.pop(); } } + LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } From 41bd99510a3de0936ff6aab8c28f93a7f78107fb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 29 Oct 2024 23:08:51 +0100 Subject: [PATCH 601/816] stash --- src/Planner/findParallelReplicasQuery.cpp | 74 +---------------------- 1 file changed, 2 insertions(+), 72 deletions(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 91cbc492fdc..a5d3e863521 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -17,12 +17,11 @@ #include #include #include -#include +#include #include #include #include #include -#include "Processors/QueryPlan/SortingStep.h" namespace DB { @@ -54,30 +53,22 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre { case QueryTreeNodeType::TABLE: { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto & table_node = query_tree_node->as(); const auto & storage = table_node.getStorage(); /// Here we check StorageDummy as well, to support a query tree with replaced storages. if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); /// parallel replicas is not supported with FINAL if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; - } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; } case QueryTreeNodeType::TABLE_FUNCTION: { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; } case QueryTreeNodeType::QUERY: @@ -85,7 +76,6 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & query_node_to_process = query_tree_node->as(); query_tree_node = query_node_to_process.getJoinTree().get(); res.push(&query_node_to_process); - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::UNION: @@ -94,20 +84,15 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & union_queries = union_node.getQueries().getNodes(); if (union_queries.empty()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; - } query_tree_node = union_queries.front().get(); - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::ARRAY_JOIN: { const auto & array_join_node = query_tree_node->as(); query_tree_node = array_join_node.getTableExpression().get(); - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } case QueryTreeNodeType::JOIN: @@ -121,13 +106,9 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre || (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All); if (!can_parallelize_join) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return {}; - } query_tree_node = join_node.getLeftTableExpression().get(); - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; } default: @@ -190,23 +171,17 @@ const QueryNode * findQueryForParallelReplicas( const std::unordered_map & mapping, const Settings & settings) { - const QueryPlan::Node * prev_checked_node = nullptr; const QueryNode * res = nullptr; - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); while (!stack.empty()) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const QueryNode * subquery_node = stack.top(); stack.pop(); auto it = mapping.find(subquery_node); /// This should not happen ideally. if (it == mapping.end()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); break; - } const QueryPlan::Node * const curr_node = it->second; std::deque> nodes_to_check; @@ -214,34 +189,20 @@ const QueryNode * findQueryForParallelReplicas( bool can_distribute_full_node = true; bool in = false; - while (!nodes_to_check.empty() /* && nodes_to_check.front() != prev_checked_node*/) + while (!nodes_to_check.empty()) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto & [next_node_to_check, digging_into_rabbit_hole] = nodes_to_check.front(); - LOG_DEBUG( - &Poco::Logger::get("debug"), - "next_node_to_check->step->getName()={}, next_node_to_check->step->getStepDescription());={}", - next_node_to_check->step->getName(), - next_node_to_check->step->getStepDescription()); nodes_to_check.pop_front(); const auto & children = next_node_to_check->children; auto * step = next_node_to_check->step.get(); if (children.empty()) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); /// Found a source step. This should be possible only in the first iteration. - if (prev_checked_node) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); - // return nullptr; - } - nodes_to_check = {}; } else if (children.size() == 1) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto * expression = typeid_cast(step); const auto * filter = typeid_cast(step); const auto * sorting = typeid_cast(step); @@ -251,7 +212,6 @@ const QueryNode * findQueryForParallelReplicas( if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->getStepDescription().contains("before JOIN"))) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); can_distribute_full_node = false; in = digging_into_rabbit_hole; } @@ -260,47 +220,33 @@ const QueryNode * findQueryForParallelReplicas( } else { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); const auto * join = typeid_cast(step); /// We've checked that JOIN is INNER/LEFT in query tree. /// Don't distribute UNION node. if (!join) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; - } for (const auto & child : children) nodes_to_check.push_front(std::make_pair(child, true)); } } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); - /// Current node contains steps like GROUP BY / DISTINCT /// Will try to execute query up to WithMergableStage if (!can_distribute_full_node) { /// Current query node does not contain subqueries. /// We can execute parallel replicas over storage::read. - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); if (!res) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; - } return in ? res : subquery_node; } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); - /// Query is simple enough to be fully distributed. res = subquery_node; - prev_checked_node = curr_node; } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } @@ -320,26 +266,16 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr auto context = query_node ? query_node->getContext() : union_node->getContext(); if (!context->canUseParallelReplicasOnInitiator()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; - } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); auto stack = getSupportingParallelReplicasQuery(query_tree_node.get()); /// Empty stack means that storage does not support parallel replicas. if (stack.empty()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; - } /// We don't have any subquery and storage can process parallel replicas by itself. if (stack.top() == query_tree_node.get()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return nullptr; - } /// This is needed to avoid infinite recursion. auto mutable_context = Context::createCopy(context); @@ -364,22 +300,16 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr /// Now, return a query from initial stack. if (res) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); while (!new_stack.empty()) { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); if (res == new_stack.top()) - { - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return stack.top(); - } stack.pop(); new_stack.pop(); } } - LOG_DEBUG(&Poco::Logger::get("debug"), "__PRETTY_FUNCTION__={}, __LINE__={}", __PRETTY_FUNCTION__, __LINE__); return res; } From 1ad1d372b2461101c1cf4d7180c1423b6424bdf0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 29 Oct 2024 23:08:56 +0100 Subject: [PATCH 602/816] stash --- src/Planner/findParallelReplicasQuery.cpp | 30 ++++++++++++++--------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index a5d3e863521..fbcf5386620 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -171,11 +171,17 @@ const QueryNode * findQueryForParallelReplicas( const std::unordered_map & mapping, const Settings & settings) { + struct Frame + { + const QueryPlan::Node * node = nullptr; + bool inside_join = false; + }; + const QueryNode * res = nullptr; while (!stack.empty()) { - const QueryNode * subquery_node = stack.top(); + const QueryNode * const subquery_node = stack.top(); stack.pop(); auto it = mapping.find(subquery_node); @@ -183,23 +189,22 @@ const QueryNode * findQueryForParallelReplicas( if (it == mapping.end()) break; - const QueryPlan::Node * const curr_node = it->second; - std::deque> nodes_to_check; - nodes_to_check.push_front(std::make_pair(curr_node, false)); + std::stack nodes_to_check; + nodes_to_check.push({.node = it->second, .inside_join = false}); bool can_distribute_full_node = true; - bool in = false; + bool currently_inside_join = false; while (!nodes_to_check.empty()) { - const auto & [next_node_to_check, digging_into_rabbit_hole] = nodes_to_check.front(); - nodes_to_check.pop_front(); + const auto & [next_node_to_check, inside_join] = nodes_to_check.top(); + nodes_to_check.pop(); const auto & children = next_node_to_check->children; auto * step = next_node_to_check->step.get(); if (children.empty()) { /// Found a source step. This should be possible only in the first iteration. - nodes_to_check = {}; + break; } else if (children.size() == 1) { @@ -213,10 +218,10 @@ const QueryNode * findQueryForParallelReplicas( if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->getStepDescription().contains("before JOIN"))) { can_distribute_full_node = false; - in = digging_into_rabbit_hole; + currently_inside_join = inside_join; } - nodes_to_check.push_front(std::pair(children.front(), digging_into_rabbit_hole)); + nodes_to_check.push({.node = children.front(), .inside_join = inside_join}); } else { @@ -227,7 +232,7 @@ const QueryNode * findQueryForParallelReplicas( return res; for (const auto & child : children) - nodes_to_check.push_front(std::make_pair(child, true)); + nodes_to_check.push({.node = child, .inside_join = true}); } } @@ -240,7 +245,8 @@ const QueryNode * findQueryForParallelReplicas( if (!res) return nullptr; - return in ? res : subquery_node; + /// todo + return currently_inside_join ? res : subquery_node; } /// Query is simple enough to be fully distributed. From d9f427deba385b6ab708c8e57cb6caad14cfdfc4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 29 Oct 2024 23:33:45 +0100 Subject: [PATCH 603/816] stash --- src/Planner/PlannerJoinTree.cpp | 5 +- src/Planner/findParallelReplicasQuery.cpp | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 6 +-- src/Processors/QueryPlan/SortingStep.h | 6 ++- ...rallel_replicas_join_with_totals.reference | 10 ++++ ...3254_parallel_replicas_join_with_totals.sh | 46 +++++++++++++++++++ 6 files changed, 65 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.reference create mode 100755 tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 39c1352c9cf..5c153f6db39 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1555,10 +1555,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ SortingStep::Settings sort_settings(*query_context); auto sorting_step = std::make_unique( - plan.getCurrentHeader(), - std::move(sort_description), - 0 /*limit*/, - sort_settings); + plan.getCurrentHeader(), std::move(sort_description), 0 /*limit*/, sort_settings, true /*is_sorting_for_merge_join*/); sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_table_side)); plan.addStep(std::move(sorting_step)); }; diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index fbcf5386620..66c7c6440c4 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -215,7 +215,7 @@ const QueryNode * findQueryForParallelReplicas( const auto * creating_sets = typeid_cast(step); bool allowed_creating_sets = settings[Setting::parallel_replicas_allow_in_with_subquery] && creating_sets; - if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->getStepDescription().contains("before JOIN"))) + if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->isSortingForMergeJoin())) { can_distribute_full_node = false; currently_inside_join = inside_join; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 5ad2f1f62d5..c15c45ee269 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -77,13 +77,11 @@ static ITransformingStep::Traits getTraits(size_t limit) } SortingStep::SortingStep( - const Header & input_header, - SortDescription description_, - UInt64 limit_, - const Settings & settings_) + const Header & input_header, SortDescription description_, UInt64 limit_, const Settings & settings_, bool is_sorting_for_merge_join_) : ITransformingStep(input_header, input_header, getTraits(limit_)) , type(Type::Full) , result_description(std::move(description_)) + , is_sorting_for_merge_join(is_sorting_for_merge_join_) , limit(limit_) , sort_settings(settings_) { diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 6cdf626d4c8..9af591d603a 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -39,7 +39,8 @@ public: const Header & input_header, SortDescription description_, UInt64 limit_, - const Settings & settings_); + const Settings & settings_, + bool is_sorting_for_merge_join_ = false); /// Full with partitioning SortingStep( @@ -81,6 +82,8 @@ public: bool hasPartitions() const { return !partition_by_description.empty(); } + bool isSortingForMergeJoin() const { return is_sorting_for_merge_join; } + void convertToFinishSorting(SortDescription prefix_description, bool use_buffering_); Type getType() const { return type; } @@ -124,6 +127,7 @@ private: const SortDescription result_description; SortDescription partition_by_description; + bool is_sorting_for_merge_join = false; UInt64 limit; bool always_read_till_end = false; diff --git a/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.reference b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.reference new file mode 100644 index 00000000000..f87bb786c46 --- /dev/null +++ b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.reference @@ -0,0 +1,10 @@ +1 1 +1 1 + +0 0 +----- +1 1 +1 1 + +0 0 +----- diff --git a/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh new file mode 100755 index 00000000000..d3780d12ae0 --- /dev/null +++ b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query=" +CREATE TABLE t +( + item_id UInt64, + price_sold Float32, + date Date +) +ENGINE = MergeTree +ORDER BY item_id; + +INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); +" + +for enable_parallel_replicas in {0..1}; do + ${CLICKHOUSE_CLIENT} --query=" + set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; + + SELECT * + FROM + ( + SELECT item_id + FROM t + ) AS l + LEFT JOIN + ( + SELECT item_id + FROM t + GROUP BY item_id + WITH TOTALS + ORDER BY item_id ASC + ) AS r ON l.item_id = r.item_id; + + SELECT '-----'; + " +done + +${CLICKHOUSE_CLIENT} --query=" +DROP TABLE t; +" From 339c8fd94922b3fdc0c64b4c3dad0cf4973e5e44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 29 Oct 2024 23:09:37 +0000 Subject: [PATCH 604/816] Fix and simplify test --- .../configs/display_name.xml | 3 ++ .../test_config_reload/configs/kafka.xml | 11 ----- tests/integration/test_config_reload/test.py | 44 +++---------------- 3 files changed, 10 insertions(+), 48 deletions(-) create mode 100644 tests/integration/test_config_reload/configs/display_name.xml delete mode 100644 tests/integration/test_config_reload/configs/kafka.xml diff --git a/tests/integration/test_config_reload/configs/display_name.xml b/tests/integration/test_config_reload/configs/display_name.xml new file mode 100644 index 00000000000..ddb7f0be8be --- /dev/null +++ b/tests/integration/test_config_reload/configs/display_name.xml @@ -0,0 +1,3 @@ + + 424242 + diff --git a/tests/integration/test_config_reload/configs/kafka.xml b/tests/integration/test_config_reload/configs/kafka.xml deleted file mode 100644 index 8ac6ff89156..00000000000 --- a/tests/integration/test_config_reload/configs/kafka.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - consumer - - - config_test - 424242 - - - - diff --git a/tests/integration/test_config_reload/test.py b/tests/integration/test_config_reload/test.py index ccd4338b455..c2882b7f776 100644 --- a/tests/integration/test_config_reload/test.py +++ b/tests/integration/test_config_reload/test.py @@ -1,15 +1,11 @@ import pytest -from helpers.cluster import ClickHouseCluster, is_arm - -if is_arm(): - pytestmark = pytest.mark.skip +from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", - main_configs=["configs/kafka.xml"], - with_kafka=True, + main_configs=["configs/display_name.xml"], stay_alive=True, ) @@ -24,48 +20,22 @@ def start_cluster(): DEFAULT_VALUE = "424242" -CHANGED_VALUE = "414141" - - -def check_value(value): - instance.query( - f""" - CREATE TABLE test (x Int64) ENGINE = Kafka - SETTINGS - kafka_broker_list = '{cluster.kafka_host}:{cluster.kafka_port}', - kafka_topic_list = 'config_test', - kafka_group_name = 'config_test_group', - kafka_format = 'JSON'; - """ - ) - - instance.query( - "SELECT * FROM test SETTINGS stream_like_engine_allow_direct_select=1", - ignore_error=True, - ) - - assert instance.wait_for_log_line("Consumer set property session.timeout.ms") - instance.query("DROP TABLE test SYNC") - - instance.contains_in_log(f"Consumer set property session.timeout.ms:{value}") +CHANGED_VALUE = "434343" def test_system_reload_config_with_global_context(start_cluster): # When running the this test multiple times, make sure failure of one test won't cause the failure of every subsequent tests - instance.query("DROP TABLE IF EXISTS test SYNC") instance.replace_in_config( - "/etc/clickhouse-server/config.d/kafka.xml", CHANGED_VALUE, DEFAULT_VALUE + "/etc/clickhouse-server/config.d/display_name.xml", CHANGED_VALUE, DEFAULT_VALUE ) instance.restart_clickhouse() - check_value(DEFAULT_VALUE) - - instance.rotate_logs() + assert DEFAULT_VALUE == instance.query("SELECT displayName()").strip() instance.replace_in_config( - "/etc/clickhouse-server/config.d/kafka.xml", DEFAULT_VALUE, CHANGED_VALUE + "/etc/clickhouse-server/config.d/display_name.xml", DEFAULT_VALUE, CHANGED_VALUE ) instance.query("SYSTEM RELOAD CONFIG") - check_value(CHANGED_VALUE) + assert CHANGED_VALUE == instance.query("SELECT displayName()").strip() From 4a0a0446e40f8e77342e10a1742bc214ed02b6ee Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 17 Oct 2024 04:56:59 +0000 Subject: [PATCH 605/816] Fix extensive listObject object storage API calls --- .../ObjectStorages/InMemoryDirectoryPathMap.h | 2 + .../MetadataStorageFromPlainObjectStorage.cpp | 26 +++-- .../MetadataStorageFromPlainObjectStorage.h | 15 ++- ...torageFromPlainObjectStorageOperations.cpp | 82 +++++++++++++++- ...aStorageFromPlainObjectStorageOperations.h | 34 +++++++ ...torageFromPlainRewritableObjectStorage.cpp | 96 ++++++++++--------- ...aStorageFromPlainRewritableObjectStorage.h | 2 - 7 files changed, 195 insertions(+), 62 deletions(-) diff --git a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h index ac07f3558a2..4077c72ac23 100644 --- a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ struct InMemoryDirectoryPathMap { std::string path; time_t last_modified = 0; + std::unordered_set filenames; }; using Map = std::map; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 5462a27c0a7..e1235bf19ea 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -220,6 +220,20 @@ void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std } } +void MetadataStorageFromPlainObjectStorageTransaction::createEmptyMetadataFile(const std::string & path) +{ + if (metadata_storage.object_storage->isWriteOnce()) + return; + + addOperation(std::make_unique(path, *metadata_storage.getPathMap())); +} + +void MetadataStorageFromPlainObjectStorageTransaction::createMetadataFile( + const std::string & path, ObjectStorageKey /*object_key*/, uint64_t /* size_in_bytes */) +{ + return createEmptyMetadataFile(path); +} + void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std::string & path) { if (metadata_storage.object_storage->isWriteOnce()) @@ -252,12 +266,6 @@ void MetadataStorageFromPlainObjectStorageTransaction::moveDirectory(const std:: metadata_storage.getMetadataKeyPrefix())); } -void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( - const std::string &, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) -{ - /// Noop, local metadata files is only one file, it is the metadata file itself. -} - UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string & path) { /// The record has become stale, remove it from cache. @@ -270,7 +278,11 @@ UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTrans } /// No hardlinks, so will always remove file. - return std::make_shared(UnlinkMetadataFileOperationOutcome{0}); + auto result = std::make_shared(UnlinkMetadataFileOperationOutcome{0}); + if (!metadata_storage.object_storage->isWriteOnce()) + addOperation(std::make_unique( + path, *metadata_storage.getPathMap(), object_storage)); + return result; } void MetadataStorageFromPlainObjectStorageTransaction::commit() diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index c8854bc6d19..db7390af5fd 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -114,22 +114,19 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const override; - void addBlobToMetadata(const std::string & path, ObjectStorageKey object_key, uint64_t size_in_bytes) override; + void addBlobToMetadata(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override + { + // Noop + } void setLastModified(const String &, const Poco::Timestamp &) override { /// Noop } - void createEmptyMetadataFile(const std::string & /* path */) override - { - /// No metadata, no need to create anything. - } + void createEmptyMetadataFile(const std::string & /* path */) override; - void createMetadataFile(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override - { - /// Noop - } + void createMetadataFile(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override; void createDirectory(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index d2e0243a4cf..f1c4d31b3d8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -1,6 +1,8 @@ #include "MetadataStorageFromPlainObjectStorageOperations.h" #include +#include +#include #include #include #include @@ -76,7 +78,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: std::lock_guard lock(path_map.mutex); auto & map = path_map.map; [[maybe_unused]] auto result - = map.emplace(base_path, InMemoryDirectoryPathMap::RemotePathInfo{object_key_prefix, Poco::Timestamp{}.epochTime()}); + = map.emplace(base_path, InMemoryDirectoryPathMap::RemotePathInfo{object_key_prefix, Poco::Timestamp{}.epochTime(), {}}); chassert(result.second); } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -287,4 +289,82 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un CurrentMetrics::add(metric, 1); } +MetadataStorageFromPlainObjectStorageWriteFileOperation::MetadataStorageFromPlainObjectStorageWriteFileOperation( + const std::string & path_, InMemoryDirectoryPathMap & path_map_) + : path(path_), path_map(path_map_) +{ +} + +void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::unique_lock &) +{ + LOG_TEST(getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), "Creating metadata for a file '{}'", path); + + std::lock_guard lock(path_map.mutex); + + auto it = path_map.map.find(path.parent_path()); + /// Some paths (e.g., clickhouse_access_check) may not have parent directories. + if (it == path_map.map.end()) + LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), "{}", path); + else + written = it->second.filenames.emplace(path.filename()).second; +} + +void MetadataStorageFromPlainObjectStorageWriteFileOperation::undo(std::unique_lock &) +{ + if (written) + { + std::lock_guard lock(path_map.mutex); + auto it = path_map.map.find(path.parent_path()); + chassert(it != path_map.map.end()); + if (it != path_map.map.end()) + { + [[maybe_unused]] auto res = it->second.filenames.erase(path.filename()); + chassert(res > 0); + } + } +} + +MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation( + std::filesystem::path && path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_) + : path(path_) + , remote_path(std::filesystem::path(object_storage_->generateObjectKeyForPath(path_, std::nullopt).serialize())) + , path_map(path_map_) +{ + auto common_key_prefix = object_storage_->getCommonKeyPrefix(); + chassert(remote_path.string().starts_with(common_key_prefix)); + auto rel_path = remote_path.lexically_relative(common_key_prefix); + remote_parent_path = rel_path.parent_path() / ""; + filename = rel_path.filename(); +} + +void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(std::unique_lock &) +{ + LOG_TEST( + getLogger("MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation"), + "Unlinking metadata for a write '{}' with remote path '{}'", + path, + remote_path); + + std::lock_guard lock(path_map.mutex); + auto it = path_map.map.find(path.parent_path()); + if (it != path_map.map.end()) + { + auto res = it->second.filenames.erase(filename); + unlinked = res > 0; + } +} + +void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::undo(std::unique_lock &) +{ + if (unlinked) + { + std::lock_guard lock(path_map.mutex); + auto it = path_map.map.find(path.parent_path()); + chassert(it != path_map.map.end()); + if (it != path_map.map.end()) + { + it->second.filenames.emplace(filename); + } + } +} } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 00f1d191b47..c0a7d306a6a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -87,4 +87,38 @@ public: void undo(std::unique_lock & metadata_lock) override; }; +class MetadataStorageFromPlainObjectStorageWriteFileOperation final : public IMetadataOperation +{ +private: + std::filesystem::path path; + InMemoryDirectoryPathMap & path_map; + + bool written = false; + +public: + MetadataStorageFromPlainObjectStorageWriteFileOperation(const std::string & path, InMemoryDirectoryPathMap & path_map_); + + void execute(std::unique_lock & metadata_lock) override; + void undo(std::unique_lock & metadata_lock) override; +}; + +class MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation final : public IMetadataOperation +{ +private: + std::filesystem::path path; + std::filesystem::path remote_path; + InMemoryDirectoryPathMap & path_map; + + std::filesystem::path remote_parent_path; + std::string filename; + + bool unlinked = false; + +public: + MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation( + std::filesystem::path && path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_); + + void execute(std::unique_lock & metadata_lock) override; + void undo(std::unique_lock & metadata_lock) override; +}; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 115b3bc0616..4dd0a566378 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -5,17 +5,22 @@ #include #include +#include +#include #include +#include #include #include #include #include #include #include "Common/Exception.h" +#include "Common/Logger.h" #include #include #include #include "CommonPathPrefixKeyGenerator.h" +#include "Disks/ObjectStorages/IObjectStorage_fwd.h" namespace DB @@ -45,6 +50,28 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } +void loadDirectory(InMemoryDirectoryPathMap::Map & map, ObjectStoragePtr object_storage) +{ + const auto common_key_prefix = object_storage->getCommonKeyPrefix(); + LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory structure"); + for (auto & [local_path, info] : map) + { + LOG_TRACE(getLogger("loadDirectory"), "Loading directories for local path: {}", local_path); + const auto remote_path = std::filesystem::path(common_key_prefix) / info.path / ""; + for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next()) + { + auto file = iterator->current(); + String path = file->getPath(); + LOG_TRACE(getLogger("loadDirectory"), "Remote path: {}", path); + chassert(path.starts_with(remote_path.string())); + auto filename = std::filesystem::path(path).filename(); + /// Check that the file is a direct child. + if (path.substr(remote_path.string().size()) == filename) + info.filenames.emplace(filename); + } + } +} + std::shared_ptr loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) { auto result = std::make_shared(); @@ -62,6 +89,9 @@ std::shared_ptr loadPathPrefixMap(const std::string & LOG_DEBUG(log, "Loading metadata"); size_t num_files = 0; + + std::mutex mutex; + InMemoryDirectoryPathMap::Map map; for (auto iterator = object_storage->iterate(metadata_key_prefix, 0); iterator->isValid(); iterator->next()) { ++num_files; @@ -72,7 +102,7 @@ std::shared_ptr loadPathPrefixMap(const std::string & continue; runner( - [remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix] + [remote_metadata_path, path, &object_storage, &mutex, &map, &log, &settings, &metadata_key_prefix] { setThreadName("PlainRWMetaLoad"); @@ -109,13 +139,13 @@ std::shared_ptr loadPathPrefixMap(const std::string & chassert(remote_metadata_path.has_parent_path()); chassert(remote_metadata_path.string().starts_with(metadata_key_prefix)); auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size()); - auto remote_path = std::filesystem::path(std::move(suffix)); + auto rel_path = std::filesystem::path(std::move(suffix)); std::pair res; { - std::lock_guard lock(result->mutex); - res = result->map.emplace( + std::lock_guard lock(mutex); + res = map.emplace( std::filesystem::path(local_path).parent_path(), - InMemoryDirectoryPathMap::RemotePathInfo{remote_path.parent_path(), last_modified.epochTime()}); + InMemoryDirectoryPathMap::RemotePathInfo{rel_path.parent_path(), last_modified.epochTime(), {}}); } /// This can happen if table replication is enabled, then the same local path is written @@ -126,13 +156,16 @@ std::shared_ptr loadPathPrefixMap(const std::string & "The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'", local_path, res.first->second.path, - remote_path.parent_path().string()); + rel_path.parent_path().string()); }); } runner.waitForAllToFinishAndRethrowFirstError(); + + loadDirectory(map, object_storage); { - SharedLockGuard lock(result->mutex); + std::lock_guard lock(result->mutex); + result->map = std::move(map); LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size()); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -142,50 +175,36 @@ std::shared_ptr loadPathPrefixMap(const std::string & } void getDirectChildrenOnDiskImpl( - const std::string & storage_key, - const RelativePathsWithMetadata & remote_paths, - const std::string & local_path, + const std::filesystem::path & local_path, const InMemoryDirectoryPathMap & path_map, std::unordered_set & result) { - /// Directories are retrieved from the in-memory path map. { + /// Directories are retrieved from the in-memory path map. SharedLockGuard lock(path_map.mutex); const auto & local_path_prefixes = path_map.map; const auto end_it = local_path_prefixes.end(); for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) { const auto & [k, _] = std::make_tuple(it->first.string(), it->second); - if (!k.starts_with(local_path)) + if (!k.starts_with(local_path.string())) break; - auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); + auto slash_num = count(k.begin() + local_path.string().size(), k.end(), '/'); /// The local_path_prefixes comparator ensures that the paths with the smallest number of /// hops from the local_path are iterated first. The paths do not end with '/', hence /// break the loop if the number of slashes is greater than 0. if (slash_num != 0) break; - result.emplace(std::string(k.begin() + local_path.size(), k.end()) + "/"); + result.emplace(std::string(k.begin() + local_path.string().size(), k.end()) + "/"); } - } - /// Files. - auto skip_list = std::set{PREFIX_PATH_FILE_NAME}; - for (const auto & elem : remote_paths) - { - const auto & path = elem->relative_path; - chassert(path.find(storage_key) == 0); - const auto child_pos = storage_key.size(); - - auto slash_pos = path.find('/', child_pos); - - if (slash_pos == std::string::npos) + /// Files. + auto it = path_map.map.find(local_path.parent_path()); + if (it != path_map.map.end()) { - /// File names. - auto filename = path.substr(child_pos); - if (!skip_list.contains(filename)) - result.emplace(std::move(filename)); + result.insert(it->second.filenames.begin(), it->second.filenames.end()); } } } @@ -246,17 +265,10 @@ bool MetadataStorageFromPlainRewritableObjectStorage::existsDirectory(const std: std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - auto key_prefix = object_storage->generateObjectKeyForPath(path, "" /* key_prefix */).serialize(); + std::unordered_set children; + getDirectChildrenOnDisk(std::filesystem::path(path) / "", children); - RelativePathsWithMetadata files; - auto absolute_key = std::filesystem::path(object_storage->getCommonKeyPrefix()) / key_prefix / ""; - - object_storage->listObjects(absolute_key, files, 0); - - std::unordered_set directories; - getDirectChildrenOnDisk(absolute_key, files, std::filesystem::path(path) / "", directories); - - return std::vector(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end())); + return std::vector(std::make_move_iterator(children.begin()), std::make_move_iterator(children.end())); } std::optional MetadataStorageFromPlainRewritableObjectStorage::getLastModifiedIfExists(const String & path) const @@ -272,12 +284,10 @@ std::optional MetadataStorageFromPlainRewritableObjectStorage:: } void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( - const std::string & storage_key, - const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const { - getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result); + getDirectChildrenOnDiskImpl(local_path, *getPathMap(), result); } bool MetadataStorageFromPlainRewritableObjectStorage::useSeparateLayoutForMetadata() const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index 31a7dbe8307..b3d8dcde27b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -36,8 +36,6 @@ protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } void getDirectChildrenOnDisk( - const std::string & storage_key, - const RelativePathsWithMetadata & remote_paths, const std::string & local_path, std::unordered_set & result) const; From ee2564244f3dd1f0f3cf72fb0b765b9812926f33 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 25 Oct 2024 00:12:46 +0000 Subject: [PATCH 606/816] cleanup --- .../MetadataStorageFromPlainObjectStorage.cpp | 1 - ...torageFromPlainObjectStorageOperations.cpp | 12 ++- ...torageFromPlainRewritableObjectStorage.cpp | 80 ++++++++----------- ...aStorageFromPlainRewritableObjectStorage.h | 4 +- 4 files changed, 46 insertions(+), 51 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index e1235bf19ea..98168687a9e 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -277,7 +277,6 @@ UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTrans metadata_storage.object_metadata_cache->remove(hash.get128()); } - /// No hardlinks, so will always remove file. auto result = std::make_shared(UnlinkMetadataFileOperationOutcome{0}); if (!metadata_storage.object_storage->isWriteOnce()) addOperation(std::make_unique( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index f1c4d31b3d8..8cf6e666048 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -304,7 +304,10 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::uniqu auto it = path_map.map.find(path.parent_path()); /// Some paths (e.g., clickhouse_access_check) may not have parent directories. if (it == path_map.map.end()) - LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), "{}", path); + LOG_TRACE( + getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), + "Parrent dirrectory does not exist, skipping path {}", + path); else written = it->second.filenames.emplace(path.filename()).second; } @@ -347,7 +350,12 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(s std::lock_guard lock(path_map.mutex); auto it = path_map.map.find(path.parent_path()); - if (it != path_map.map.end()) + if (it == path_map.map.end()) + LOG_TRACE( + getLogger("MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation"), + "Parent directory does not exist, skipping path {}", + path); + else { auto res = it->second.filenames.erase(filename); unlinked = res > 0; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 4dd0a566378..6176ef169d0 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -50,7 +50,7 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -void loadDirectory(InMemoryDirectoryPathMap::Map & map, ObjectStoragePtr object_storage) +void loadDirectoryTree(InMemoryDirectoryPathMap::Map & map, ObjectStoragePtr object_storage) { const auto common_key_prefix = object_storage->getCommonKeyPrefix(); LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory structure"); @@ -162,7 +162,7 @@ std::shared_ptr loadPathPrefixMap(const std::string & runner.waitForAllToFinishAndRethrowFirstError(); - loadDirectory(map, object_storage); + loadDirectoryTree(map, object_storage); { std::lock_guard lock(result->mutex); result->map = std::move(map); @@ -174,41 +174,6 @@ std::shared_ptr loadPathPrefixMap(const std::string & return result; } -void getDirectChildrenOnDiskImpl( - const std::filesystem::path & local_path, - const InMemoryDirectoryPathMap & path_map, - std::unordered_set & result) -{ - { - /// Directories are retrieved from the in-memory path map. - SharedLockGuard lock(path_map.mutex); - const auto & local_path_prefixes = path_map.map; - const auto end_it = local_path_prefixes.end(); - for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it) - { - const auto & [k, _] = std::make_tuple(it->first.string(), it->second); - if (!k.starts_with(local_path.string())) - break; - - auto slash_num = count(k.begin() + local_path.string().size(), k.end(), '/'); - /// The local_path_prefixes comparator ensures that the paths with the smallest number of - /// hops from the local_path are iterated first. The paths do not end with '/', hence - /// break the loop if the number of slashes is greater than 0. - if (slash_num != 0) - break; - - result.emplace(std::string(k.begin() + local_path.string().size(), k.end()) + "/"); - } - - /// Files. - auto it = path_map.map.find(local_path.parent_path()); - if (it != path_map.map.end()) - { - result.insert(it->second.filenames.begin(), it->second.filenames.end()); - } - } -} - } MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewritableObjectStorage( @@ -265,10 +230,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::existsDirectory(const std: std::vector MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const { - std::unordered_set children; - getDirectChildrenOnDisk(std::filesystem::path(path) / "", children); - - return std::vector(std::make_move_iterator(children.begin()), std::make_move_iterator(children.end())); + std::unordered_set result = getDirectChildrenOnDisk(std::filesystem::path(path) / ""); + return std::vector(std::make_move_iterator(result.begin()), std::make_move_iterator(result.end())); } std::optional MetadataStorageFromPlainRewritableObjectStorage::getLastModifiedIfExists(const String & path) const @@ -283,11 +246,38 @@ std::optional MetadataStorageFromPlainRewritableObjectStorage:: return std::nullopt; } -void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk( - const std::string & local_path, - std::unordered_set & result) const +std::unordered_set +MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(const std::filesystem::path & local_path) const { - getDirectChildrenOnDiskImpl(local_path, *getPathMap(), result); + std::unordered_set result; + SharedLockGuard lock(path_map->mutex); + // const auto & map = path_map->map; + const auto end_it = path_map->map.end(); + /// Directories. + for (auto it = path_map->map.lower_bound(local_path); it != end_it; ++it) + { + const auto & [k, _] = std::make_tuple(it->first.string(), it->second); + if (!k.starts_with(local_path.string())) + break; + + auto slash_num = count(k.begin() + local_path.string().size(), k.end(), '/'); + /// The directory map comparator ensures that the paths with the smallest number of + /// hops from the local_path are iterated first. The paths do not end with '/', hence + /// break the loop if the number of slashes to the right from the offset is greater than 0. + if (slash_num != 0) + break; + + result.emplace(std::string(k.begin() + local_path.string().size(), k.end()) + "/"); + } + + /// Files. + auto it = path_map->map.find(local_path.parent_path()); + if (it != path_map->map.end()) + { + result.insert(it->second.filenames.begin(), it->second.filenames.end()); + } + + return result; } bool MetadataStorageFromPlainRewritableObjectStorage::useSeparateLayoutForMetadata() const diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h index b3d8dcde27b..983e379d292 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h @@ -35,9 +35,7 @@ public: protected: std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; } std::shared_ptr getPathMap() const override { return path_map; } - void getDirectChildrenOnDisk( - const std::string & local_path, - std::unordered_set & result) const; + std::unordered_set getDirectChildrenOnDisk(const std::filesystem::path & local_path) const; private: bool useSeparateLayoutForMetadata() const; From be985bcd9f9ae8b258b6350787e105a3abfe4af0 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 25 Oct 2024 04:40:09 +0000 Subject: [PATCH 607/816] tidy --- .../MetadataStorageFromPlainObjectStorage.cpp | 2 +- ...etadataStorageFromPlainObjectStorageOperations.cpp | 11 +++-------- .../MetadataStorageFromPlainObjectStorageOperations.h | 3 --- 3 files changed, 4 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 98168687a9e..a48de83bb3a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -231,7 +231,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::createEmptyMetadataFile(c void MetadataStorageFromPlainObjectStorageTransaction::createMetadataFile( const std::string & path, ObjectStorageKey /*object_key*/, uint64_t /* size_in_bytes */) { - return createEmptyMetadataFile(path); + createEmptyMetadataFile(path); } void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std::string & path) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 8cf6e666048..c9c48004b67 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -306,7 +306,7 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::uniqu if (it == path_map.map.end()) LOG_TRACE( getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), - "Parrent dirrectory does not exist, skipping path {}", + "Parent dirrectory does not exist, skipping path {}", path); else written = it->second.filenames.emplace(path.filename()).second; @@ -333,11 +333,6 @@ MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::MetadataStorag , remote_path(std::filesystem::path(object_storage_->generateObjectKeyForPath(path_, std::nullopt).serialize())) , path_map(path_map_) { - auto common_key_prefix = object_storage_->getCommonKeyPrefix(); - chassert(remote_path.string().starts_with(common_key_prefix)); - auto rel_path = remote_path.lexically_relative(common_key_prefix); - remote_parent_path = rel_path.parent_path() / ""; - filename = rel_path.filename(); } void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(std::unique_lock &) @@ -357,7 +352,7 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(s path); else { - auto res = it->second.filenames.erase(filename); + auto res = it->second.filenames.erase(path.filename()); unlinked = res > 0; } } @@ -371,7 +366,7 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::undo(std: chassert(it != path_map.map.end()); if (it != path_map.map.end()) { - it->second.filenames.emplace(filename); + it->second.filenames.emplace(path.filename()); } } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index c0a7d306a6a..72e57e80705 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -109,9 +109,6 @@ private: std::filesystem::path remote_path; InMemoryDirectoryPathMap & path_map; - std::filesystem::path remote_parent_path; - std::string filename; - bool unlinked = false; public: From bdb284cf05ac22e71d11d30a58264d894b3ec0b5 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 26 Oct 2024 03:04:27 +0000 Subject: [PATCH 608/816] store filenames compactly --- .../ObjectStorages/InMemoryDirectoryPathMap.h | 15 ++++++++++-- ...torageFromPlainObjectStorageOperations.cpp | 23 +++++++++++++----- ...torageFromPlainRewritableObjectStorage.cpp | 24 +++++++++++++++---- 3 files changed, 50 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h index 4077c72ac23..117cbad6203 100644 --- a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h @@ -2,9 +2,10 @@ #include #include +#include #include +#include #include -#include #include #include #include @@ -26,11 +27,19 @@ struct InMemoryDirectoryPathMap return path1 < path2; } }; + + using FileNames = std::set; + using FileNamesIterator = FileNames::iterator; + struct FileNameIteratorComparator + { + bool operator()(const FileNames::iterator & lhs, const FileNames::iterator & rhs) const { return *lhs < *rhs; } + }; + struct RemotePathInfo { std::string path; time_t last_modified = 0; - std::unordered_set filenames; + std::set filename_iterators; }; using Map = std::map; @@ -51,9 +60,11 @@ struct InMemoryDirectoryPathMap mutable SharedMutex mutex; #ifdef OS_LINUX + FileNames TSA_GUARDED_BY(mutex) unique_filenames; Map TSA_GUARDED_BY(mutex) map; /// std::shared_mutex may not be annotated with the 'capability' attribute in libcxx. #else + FileNames unique_filenames; Map map; #endif }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index c9c48004b67..fe88f592cec 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -309,7 +309,10 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::uniqu "Parent dirrectory does not exist, skipping path {}", path); else - written = it->second.filenames.emplace(path.filename()).second; + { + auto filename_it = path_map.unique_filenames.emplace(path.filename()).first; + written = it->second.filename_iterators.emplace(filename_it).second; + } } void MetadataStorageFromPlainObjectStorageWriteFileOperation::undo(std::unique_lock &) @@ -321,8 +324,12 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::undo(std::unique_l chassert(it != path_map.map.end()); if (it != path_map.map.end()) { - [[maybe_unused]] auto res = it->second.filenames.erase(path.filename()); - chassert(res > 0); + auto filename_it = path_map.unique_filenames.find(path.filename()); + if (filename_it != path_map.unique_filenames.end()) + { + [[maybe_unused]] auto res = it->second.filename_iterators.erase(filename_it); + chassert(res > 0); + } } } } @@ -352,8 +359,10 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(s path); else { - auto res = it->second.filenames.erase(path.filename()); - unlinked = res > 0; + auto & filename_iterators = it->second.filename_iterators; + auto filename_it = path_map.unique_filenames.find(path.filename()); + if (filename_it != path_map.unique_filenames.end()) + unlinked = (filename_iterators.erase(filename_it) > 0); } } @@ -366,7 +375,9 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::undo(std: chassert(it != path_map.map.end()); if (it != path_map.map.end()) { - it->second.filenames.emplace(path.filename()); + auto filename_it = path_map.unique_filenames.find(path.filename()); + if (filename_it != path_map.unique_filenames.end()) + it->second.filename_iterators.emplace(filename_it); } } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 6176ef169d0..41e769cbf42 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -3,6 +3,8 @@ #include #include +#include +#include #include #include #include @@ -50,14 +52,18 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage) : metadata_key_prefix; } -void loadDirectoryTree(InMemoryDirectoryPathMap::Map & map, ObjectStoragePtr object_storage) +void loadDirectoryTree( + InMemoryDirectoryPathMap::Map & map, InMemoryDirectoryPathMap::FileNames & unique_filenames, ObjectStoragePtr object_storage) { + using FileNamesIterator = InMemoryDirectoryPathMap::FileNamesIterator; + using FileNameIteratorComparator = InMemoryDirectoryPathMap::FileNameIteratorComparator; const auto common_key_prefix = object_storage->getCommonKeyPrefix(); LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory structure"); for (auto & [local_path, info] : map) { LOG_TRACE(getLogger("loadDirectory"), "Loading directories for local path: {}", local_path); const auto remote_path = std::filesystem::path(common_key_prefix) / info.path / ""; + std::set filename_iterators; for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next()) { auto file = iterator->current(); @@ -67,8 +73,12 @@ void loadDirectoryTree(InMemoryDirectoryPathMap::Map & map, ObjectStoragePtr obj auto filename = std::filesystem::path(path).filename(); /// Check that the file is a direct child. if (path.substr(remote_path.string().size()) == filename) - info.filenames.emplace(filename); + { + auto filename_it = unique_filenames.emplace(filename).first; + filename_iterators.emplace(filename_it); + } } + info.filename_iterators = std::move(filename_iterators); } } @@ -162,10 +172,12 @@ std::shared_ptr loadPathPrefixMap(const std::string & runner.waitForAllToFinishAndRethrowFirstError(); - loadDirectoryTree(map, object_storage); + InMemoryDirectoryPathMap::FileNames unique_filenames; + loadDirectoryTree(map, unique_filenames, object_storage); { std::lock_guard lock(result->mutex); result->map = std::move(map); + result->unique_filenames = std::move(unique_filenames); LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size()); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; @@ -274,7 +286,11 @@ MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(const s auto it = path_map->map.find(local_path.parent_path()); if (it != path_map->map.end()) { - result.insert(it->second.filenames.begin(), it->second.filenames.end()); + for (const auto & filename_it : it->second.filename_iterators) + { + chassert(filename_it != path_map->unique_filenames.end()); + result.insert(*filename_it); + } } return result; From a0d58ad9e6402f116a7313c484ecda0316dd481a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Sat, 26 Oct 2024 04:06:09 +0000 Subject: [PATCH 609/816] add counters --- src/Common/CurrentMetrics.cpp | 6 +++ .../MetadataStorageFromPlainObjectStorage.cpp | 3 +- ...torageFromPlainObjectStorageOperations.cpp | 38 ++++++++++++++++--- ...aStorageFromPlainObjectStorageOperations.h | 5 ++- ...torageFromPlainRewritableObjectStorage.cpp | 9 ++++- .../ObjectStorages/MetadataStorageMetrics.h | 2 + .../createMetadataStorageMetrics.h | 18 +++++++-- 7 files changed, 68 insertions(+), 13 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index c4318fb0fda..0c850fd4d36 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -183,8 +183,14 @@ M(BuildVectorSimilarityIndexThreadsScheduled, "Number of queued or active jobs in the build vector similarity index thread pool.") \ \ M(DiskPlainRewritableAzureDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \ + M(DiskPlainRewritableAzureFileCount, "Number of file entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \ + M(DiskPlainRewritableAzureUniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \ M(DiskPlainRewritableLocalDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \ + M(DiskPlainRewritableLocalFileCount, "Number of file entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \ + M(DiskPlainRewritableLocalUniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \ M(DiskPlainRewritableS3DirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \ + M(DiskPlainRewritableS3FileCount, "Number of file entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \ + M(DiskPlainRewritableS3UniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \ \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a48de83bb3a..d56c5d9143c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -225,7 +225,8 @@ void MetadataStorageFromPlainObjectStorageTransaction::createEmptyMetadataFile(c if (metadata_storage.object_storage->isWriteOnce()) return; - addOperation(std::make_unique(path, *metadata_storage.getPathMap())); + addOperation( + std::make_unique(path, *metadata_storage.getPathMap(), object_storage)); } void MetadataStorageFromPlainObjectStorageTransaction::createMetadataFile( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index fe88f592cec..ea57d691908 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -290,8 +290,8 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un } MetadataStorageFromPlainObjectStorageWriteFileOperation::MetadataStorageFromPlainObjectStorageWriteFileOperation( - const std::string & path_, InMemoryDirectoryPathMap & path_map_) - : path(path_), path_map(path_map_) + const std::string & path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_) + : path(path_), path_map(path_map_), object_storage(object_storage_) { } @@ -310,8 +310,18 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::uniqu path); else { - auto filename_it = path_map.unique_filenames.emplace(path.filename()).first; + auto [filename_it, inserted] = path_map.unique_filenames.emplace(path.filename()); + if (inserted) + { + auto metric = object_storage->getMetadataStorageMetrics().unique_filenames_count; + CurrentMetrics::add(metric, 1); + } written = it->second.filename_iterators.emplace(filename_it).second; + if (written) + { + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::add(metric, 1); + } } } @@ -327,8 +337,11 @@ void MetadataStorageFromPlainObjectStorageWriteFileOperation::undo(std::unique_l auto filename_it = path_map.unique_filenames.find(path.filename()); if (filename_it != path_map.unique_filenames.end()) { - [[maybe_unused]] auto res = it->second.filename_iterators.erase(filename_it); - chassert(res > 0); + if (it->second.filename_iterators.erase(filename_it) > 0) + { + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::sub(metric, 1); + } } } } @@ -339,6 +352,7 @@ MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::MetadataStorag : path(path_) , remote_path(std::filesystem::path(object_storage_->generateObjectKeyForPath(path_, std::nullopt).serialize())) , path_map(path_map_) + , object_storage(object_storage_) { } @@ -363,6 +377,12 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(s auto filename_it = path_map.unique_filenames.find(path.filename()); if (filename_it != path_map.unique_filenames.end()) unlinked = (filename_iterators.erase(filename_it) > 0); + + if (unlinked) + { + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::sub(metric, 1); + } } } @@ -377,7 +397,13 @@ void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::undo(std: { auto filename_it = path_map.unique_filenames.find(path.filename()); if (filename_it != path_map.unique_filenames.end()) - it->second.filename_iterators.emplace(filename_it); + { + if (it->second.filename_iterators.emplace(filename_it).second) + { + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::add(metric, 1); + } + } } } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 72e57e80705..565d4429548 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -92,11 +92,13 @@ class MetadataStorageFromPlainObjectStorageWriteFileOperation final : public IMe private: std::filesystem::path path; InMemoryDirectoryPathMap & path_map; + ObjectStoragePtr object_storage; bool written = false; public: - MetadataStorageFromPlainObjectStorageWriteFileOperation(const std::string & path, InMemoryDirectoryPathMap & path_map_); + MetadataStorageFromPlainObjectStorageWriteFileOperation( + const std::string & path, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_); void execute(std::unique_lock & metadata_lock) override; void undo(std::unique_lock & metadata_lock) override; @@ -108,6 +110,7 @@ private: std::filesystem::path path; std::filesystem::path remote_path; InMemoryDirectoryPathMap & path_map; + ObjectStoragePtr object_storage; bool unlinked = false; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 41e769cbf42..dc02a21e986 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -61,14 +61,12 @@ void loadDirectoryTree( LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory structure"); for (auto & [local_path, info] : map) { - LOG_TRACE(getLogger("loadDirectory"), "Loading directories for local path: {}", local_path); const auto remote_path = std::filesystem::path(common_key_prefix) / info.path / ""; std::set filename_iterators; for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next()) { auto file = iterator->current(); String path = file->getPath(); - LOG_TRACE(getLogger("loadDirectory"), "Remote path: {}", path); chassert(path.starts_with(remote_path.string())); auto filename = std::filesystem::path(path).filename(); /// Check that the file is a direct child. @@ -78,6 +76,10 @@ void loadDirectoryTree( filename_iterators.emplace(filename_it); } } + + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::add(metric, filename_iterators.size()); + info.filename_iterators = std::move(filename_iterators); } } @@ -211,6 +213,9 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita auto keys_gen = std::make_shared(object_storage->getCommonKeyPrefix(), path_map); object_storage->setKeysGenerator(keys_gen); } + + auto metric = object_storage->getMetadataStorageMetrics().unique_filenames_count; + CurrentMetrics::add(metric, path_map->unique_filenames.size()); } MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage() diff --git a/src/Disks/ObjectStorages/MetadataStorageMetrics.h b/src/Disks/ObjectStorages/MetadataStorageMetrics.h index 365fd3c8145..ab21f68f90d 100644 --- a/src/Disks/ObjectStorages/MetadataStorageMetrics.h +++ b/src/Disks/ObjectStorages/MetadataStorageMetrics.h @@ -13,6 +13,8 @@ struct MetadataStorageMetrics const ProfileEvents::Event directory_removed = ProfileEvents::end(); CurrentMetrics::Metric directory_map_size = CurrentMetrics::end(); + CurrentMetrics::Metric unique_filenames_count = CurrentMetrics::end(); + CurrentMetrics::Metric file_count = CurrentMetrics::end(); template static MetadataStorageMetrics create() diff --git a/src/Disks/ObjectStorages/createMetadataStorageMetrics.h b/src/Disks/ObjectStorages/createMetadataStorageMetrics.h index 5cf1fbef2ab..bc2ccec9d85 100644 --- a/src/Disks/ObjectStorages/createMetadataStorageMetrics.h +++ b/src/Disks/ObjectStorages/createMetadataStorageMetrics.h @@ -24,8 +24,14 @@ extern const Event DiskPlainRewritableS3DirectoryRemoved; namespace CurrentMetrics { extern const Metric DiskPlainRewritableAzureDirectoryMapSize; +extern const Metric DiskPlainRewritableAzureUniqueFileNamesCount; +extern const Metric DiskPlainRewritableAzureFileCount; extern const Metric DiskPlainRewritableLocalDirectoryMapSize; +extern const Metric DiskPlainRewritableLocalUniqueFileNamesCount; +extern const Metric DiskPlainRewritableLocalFileCount; extern const Metric DiskPlainRewritableS3DirectoryMapSize; +extern const Metric DiskPlainRewritableS3UniqueFileNamesCount; +extern const Metric DiskPlainRewritableS3FileCount; } namespace DB @@ -38,7 +44,9 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create Date: Mon, 28 Oct 2024 13:41:51 -0700 Subject: [PATCH 610/816] Update src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index dc02a21e986..0efe0d8992e 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -73,7 +73,8 @@ void loadDirectoryTree( if (path.substr(remote_path.string().size()) == filename) { auto filename_it = unique_filenames.emplace(filename).first; - filename_iterators.emplace(filename_it); + [[maybe_unused]] auto inserted = filename_iterators.emplace(filename_it).second; + chassert(inserted); } } From 3c0eb1a80f6429b32e2bf20b1bd8d207eab1bcff Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 28 Oct 2024 20:37:20 +0000 Subject: [PATCH 611/816] review fixes --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 0efe0d8992e..37c37734996 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -269,23 +269,22 @@ MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(const s { std::unordered_set result; SharedLockGuard lock(path_map->mutex); - // const auto & map = path_map->map; const auto end_it = path_map->map.end(); /// Directories. for (auto it = path_map->map.lower_bound(local_path); it != end_it; ++it) { - const auto & [k, _] = std::make_tuple(it->first.string(), it->second); - if (!k.starts_with(local_path.string())) + const auto & subdirectory = it->first.string(); + if (!subdirectory.starts_with(local_path.string())) break; - auto slash_num = count(k.begin() + local_path.string().size(), k.end(), '/'); + auto slash_num = count(subdirectory.begin() + local_path.string().size(), subdirectory.end(), '/'); /// The directory map comparator ensures that the paths with the smallest number of /// hops from the local_path are iterated first. The paths do not end with '/', hence /// break the loop if the number of slashes to the right from the offset is greater than 0. if (slash_num != 0) break; - result.emplace(std::string(k.begin() + local_path.string().size(), k.end()) + "/"); + result.emplace(std::string(subdirectory.begin() + local_path.string().size(), subdirectory.end()) + "/"); } /// Files. From 13e28eeb61c22ec15f1236f3369ed5349b898d3c Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 28 Oct 2024 21:30:41 +0000 Subject: [PATCH 612/816] parallelize directory tree loading --- ...torageFromPlainRewritableObjectStorage.cpp | 69 ++++++++++++------- 1 file changed, 45 insertions(+), 24 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 37c37734996..0da68eaa803 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -16,13 +16,12 @@ #include #include #include -#include "Common/Exception.h" -#include "Common/Logger.h" +#include #include #include #include +#include #include "CommonPathPrefixKeyGenerator.h" -#include "Disks/ObjectStorages/IObjectStorage_fwd.h" namespace DB @@ -58,31 +57,53 @@ void loadDirectoryTree( using FileNamesIterator = InMemoryDirectoryPathMap::FileNamesIterator; using FileNameIteratorComparator = InMemoryDirectoryPathMap::FileNameIteratorComparator; const auto common_key_prefix = object_storage->getCommonKeyPrefix(); - LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory structure"); - for (auto & [local_path, info] : map) + ThreadPool & pool = getIOThreadPool().get(); + ThreadPoolCallbackRunnerLocal runner(pool, "PlainRWTreeLoad"); + + std::atomic num_files = 0; + LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory tree"); + std::mutex mutex; + for (auto & item : map) { - const auto remote_path = std::filesystem::path(common_key_prefix) / info.path / ""; - std::set filename_iterators; - for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next()) - { - auto file = iterator->current(); - String path = file->getPath(); - chassert(path.starts_with(remote_path.string())); - auto filename = std::filesystem::path(path).filename(); - /// Check that the file is a direct child. - if (path.substr(remote_path.string().size()) == filename) + auto & remote_path_info = item.second; + const auto remote_path = std::filesystem::path(common_key_prefix) / remote_path_info.path / ""; + runner( + [remote_path, &remote_path_info, &mutex, &unique_filenames, &object_storage, &num_files] { - auto filename_it = unique_filenames.emplace(filename).first; - [[maybe_unused]] auto inserted = filename_iterators.emplace(filename_it).second; - chassert(inserted); - } - } + setThreadName("PlainRWTreeLoad"); + std::set filename_iterators; + for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next()) + { + auto file = iterator->current(); + String path = file->getPath(); + chassert(path.starts_with(remote_path.string())); + auto filename = std::filesystem::path(path).filename(); + /// Check that the file is a direct child. + if (path.substr(remote_path.string().size()) == filename) + { + auto filename_it = unique_filenames.emplace(filename).first; + auto inserted = filename_iterators.emplace(filename_it).second; + chassert(inserted); + if (inserted) + ++num_files; + } + } - auto metric = object_storage->getMetadataStorageMetrics().file_count; - CurrentMetrics::add(metric, filename_iterators.size()); + auto metric = object_storage->getMetadataStorageMetrics().file_count; + CurrentMetrics::add(metric, filename_iterators.size()); - info.filename_iterators = std::move(filename_iterators); + { + std::lock_guard lock(mutex); + remote_path_info.filename_iterators = std::move(filename_iterators); + } + }); } + runner.waitForAllToFinishAndRethrowFirstError(); + LOG_DEBUG( + getLogger("MetadataStorageFromPlainObjectStorage"), + "Loaded directory tree for {} directories, found {} files", + map.size(), + num_files); } std::shared_ptr loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage) @@ -176,12 +197,12 @@ std::shared_ptr loadPathPrefixMap(const std::string & runner.waitForAllToFinishAndRethrowFirstError(); InMemoryDirectoryPathMap::FileNames unique_filenames; + LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, map.size()); loadDirectoryTree(map, unique_filenames, object_storage); { std::lock_guard lock(result->mutex); result->map = std::move(map); result->unique_filenames = std::move(unique_filenames); - LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size()); auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, result->map.size()); From a62b0a34662ad8557919e5b66fd689131335b298 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 16:04:29 +0000 Subject: [PATCH 613/816] fix race --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index 0da68eaa803..c106979fd80 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -68,7 +68,7 @@ void loadDirectoryTree( auto & remote_path_info = item.second; const auto remote_path = std::filesystem::path(common_key_prefix) / remote_path_info.path / ""; runner( - [remote_path, &remote_path_info, &mutex, &unique_filenames, &object_storage, &num_files] + [remote_path, &mutex, &remote_path_info, &unique_filenames, &object_storage, &num_files] { setThreadName("PlainRWTreeLoad"); std::set filename_iterators; @@ -81,7 +81,11 @@ void loadDirectoryTree( /// Check that the file is a direct child. if (path.substr(remote_path.string().size()) == filename) { - auto filename_it = unique_filenames.emplace(filename).first; + auto filename_it = unique_filenames.end(); + { + std::lock_guard lock(mutex); + filename_it = unique_filenames.emplace(filename).first; + } auto inserted = filename_iterators.emplace(filename_it).second; chassert(inserted); if (inserted) From 0387331fe33bb31fb03afa0af8552802353da62a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 29 Oct 2024 19:33:14 +0000 Subject: [PATCH 614/816] redundant --- .../MetadataStorageFromPlainRewritableObjectStorage.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index c106979fd80..6966c0053b3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -95,11 +95,7 @@ void loadDirectoryTree( auto metric = object_storage->getMetadataStorageMetrics().file_count; CurrentMetrics::add(metric, filename_iterators.size()); - - { - std::lock_guard lock(mutex); - remote_path_info.filename_iterators = std::move(filename_iterators); - } + remote_path_info.filename_iterators = std::move(filename_iterators); }); } runner.waitForAllToFinishAndRethrowFirstError(); From b0977923fb1504ae1ea3fcade862c8e77e306c39 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 24 Oct 2024 09:32:29 +0000 Subject: [PATCH 615/816] Add new type of headers for S3 endpoints --- .../engines/table-engines/integrations/s3.md | 1 + .../engines/table-engines/integrations/s3.md | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 +- src/IO/S3AuthSettings.cpp | 21 ++- src/IO/S3AuthSettings.h | 3 + src/IO/S3Common.cpp | 4 +- .../test_s3_access_headers/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 ++ .../configs/config.d/s3_headers.xml | 8 ++ .../configs/users.d/users.xml | 9 ++ .../s3_mocks/mocker_s3.py | 98 ++++++++++++++ .../test_s3_access_headers/test.py | 125 ++++++++++++++++++ 12 files changed, 277 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_s3_access_headers/__init__.py create mode 100644 tests/integration/test_s3_access_headers/configs/config.d/named_collections.xml create mode 100644 tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml create mode 100644 tests/integration/test_s3_access_headers/configs/users.d/users.xml create mode 100644 tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py create mode 100644 tests/integration/test_s3_access_headers/test.py diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 2675c193519..20ee4823c1c 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -290,6 +290,7 @@ The following settings can be specified in configuration file for given endpoint - `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`. - `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. +- `access_header` - Adds specified HTTP header to a request to given endpoint, in cases where there are no other credentials from another source. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `server_side_encryption_kms_key_id` - If specified, required headers for accessing S3 objects with [SSE-KMS encryption](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) will be set. If an empty string is specified, the AWS managed S3 key will be used. Optional. - `server_side_encryption_kms_encryption_context` - If specified alongside `server_side_encryption_kms_key_id`, the given encryption context header for SSE-KMS will be set. Optional. diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index a1c69df4d0a..2bab78c0612 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -138,6 +138,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) - `use_insecure_imds_request` — признак использования менее безопасного соединения при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. - `region` — название региона S3. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. +- `access_header` - добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса, в случая если не указаны другие способы авторизации. - `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. - `single_read_retries` — Максимальное количество попыток запроса при единичном чтении. Значение по умолчанию — `4`. diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 1ae3730e4c7..92be835560b 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -177,7 +177,7 @@ std::unique_ptr getClient( auth_settings[S3AuthSetting::secret_access_key], auth_settings[S3AuthSetting::server_side_encryption_customer_key_base64], auth_settings.server_side_encryption_kms_config, - auth_settings.headers, + auth_settings.getHeaders(), credentials_configuration, auth_settings[S3AuthSetting::session_token]); } diff --git a/src/IO/S3AuthSettings.cpp b/src/IO/S3AuthSettings.cpp index 799dc6692fa..2362d60674d 100644 --- a/src/IO/S3AuthSettings.cpp +++ b/src/IO/S3AuthSettings.cpp @@ -105,7 +105,9 @@ S3AuthSettings::S3AuthSettings( } } - headers = getHTTPHeaders(config_prefix, config); + headers = getHTTPHeaders(config_prefix, config, "header"); + access_headers = getHTTPHeaders(config_prefix, config, "access_header"); + server_side_encryption_kms_config = getSSEKMSConfig(config_prefix, config); Poco::Util::AbstractConfiguration::Keys keys; @@ -157,6 +159,9 @@ bool S3AuthSettings::operator==(const S3AuthSettings & right) if (headers != right.headers) return false; + if (access_headers != right.access_headers) + return false; + if (users != right.users) return false; @@ -196,6 +201,9 @@ void S3AuthSettings::updateIfChanged(const S3AuthSettings & settings) if (!settings.headers.empty()) headers = settings.headers; + if (!settings.access_headers.empty()) + access_headers = settings.access_headers; + if (!settings.users.empty()) users.insert(settings.users.begin(), settings.users.end()); @@ -205,6 +213,17 @@ void S3AuthSettings::updateIfChanged(const S3AuthSettings & settings) server_side_encryption_kms_config = settings.server_side_encryption_kms_config; } +HTTPHeaderEntries S3AuthSettings::getHeaders() const +{ + bool auth_settings_is_default = !impl->isChanged("access_key_id"); + if (access_headers.empty() || !auth_settings_is_default) + return headers; + + HTTPHeaderEntries result(headers); + result.insert(result.end(), access_headers.begin(), access_headers.end()); + + return result; +} } } diff --git a/src/IO/S3AuthSettings.h b/src/IO/S3AuthSettings.h index 4026adb1e68..38f46cfeccd 100644 --- a/src/IO/S3AuthSettings.h +++ b/src/IO/S3AuthSettings.h @@ -55,8 +55,11 @@ struct S3AuthSettings bool hasUpdates(const S3AuthSettings & other) const; void updateIfChanged(const S3AuthSettings & settings); bool canBeUsedByUser(const String & user) const { return users.empty() || users.contains(user); } + HTTPHeaderEntries getHeaders() const; HTTPHeaderEntries headers; + HTTPHeaderEntries access_headers; + std::unordered_set users; ServerSideEncryptionKMSConfig server_side_encryption_kms_config; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 5c1ee6ccc78..f12de6a7b54 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -74,14 +74,14 @@ namespace ErrorCodes namespace S3 { -HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config) +HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config, const std::string header_key) { HTTPHeaderEntries headers; Poco::Util::AbstractConfiguration::Keys subconfig_keys; config.keys(config_elem, subconfig_keys); for (const std::string & subkey : subconfig_keys) { - if (subkey.starts_with("header")) + if (subkey.starts_with(header_key)) { auto header_str = config.getString(config_elem + "." + subkey); auto delimiter = header_str.find(':'); diff --git a/tests/integration/test_s3_access_headers/__init__.py b/tests/integration/test_s3_access_headers/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_access_headers/configs/config.d/named_collections.xml b/tests/integration/test_s3_access_headers/configs/config.d/named_collections.xml new file mode 100644 index 00000000000..d08d3401778 --- /dev/null +++ b/tests/integration/test_s3_access_headers/configs/config.d/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://resolver:8081/root/test_named_colections.csv + minio + minio123 + + + diff --git a/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml b/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml new file mode 100644 index 00000000000..c364d22ec96 --- /dev/null +++ b/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml @@ -0,0 +1,8 @@ + + + + http://resolver:8081/ + custom-auth-token: ValidToken1234 + + + \ No newline at end of file diff --git a/tests/integration/test_s3_access_headers/configs/users.d/users.xml b/tests/integration/test_s3_access_headers/configs/users.d/users.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_s3_access_headers/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py b/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py new file mode 100644 index 00000000000..64c4857731c --- /dev/null +++ b/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py @@ -0,0 +1,98 @@ +import http.client +import http.server +import random +import socketserver +import sys +import urllib.parse + + +UPSTREAM_HOST = "minio1:9001" +random.seed("No list objects/1.0") + + +def request(command, url, headers={}, data=None): + """Mini-requests.""" + + class Dummy: + pass + + parts = urllib.parse.urlparse(url) + c = http.client.HTTPConnection(parts.hostname, parts.port) + c.request( + command, + urllib.parse.urlunparse(parts._replace(scheme="", netloc="")), + headers=headers, + body=data, + ) + r = c.getresponse() + result = Dummy() + result.status_code = r.status + result.headers = r.headers + result.content = r.read() + return result + + +CUSTOM_AUTH_TOKEN_HEADER = "custom-auth-token" +CUSTOM_AUTH_TOKEN_VALID_VALUE = "ValidToken1234" + + +class RequestHandler(http.server.BaseHTTPRequestHandler): + def do_GET(self): + if self.path == "/": + self.send_response(200) + self.send_header("Content-Type", "text/plain") + self.end_headers() + self.wfile.write(b"OK") + return + self.do_HEAD() + + def do_PUT(self): + self.do_HEAD() + + def do_DELETE(self): + self.do_HEAD() + + def do_POST(self): + self.do_HEAD() + + def do_HEAD(self): + + custom_auth_token = self.headers.get(CUSTOM_AUTH_TOKEN_HEADER) + if custom_auth_token and custom_auth_token != CUSTOM_AUTH_TOKEN_VALID_VALUE: + self.send_response(403) + self.send_header("Content-Type", "application/xml") + self.end_headers() + + body = f""" + + AccessDenied + Access Denied. Custom token was {custom_auth_token}, the correct one: {CUSTOM_AUTH_TOKEN_VALID_VALUE}. + RESOURCE + REQUEST_ID + +""" + self.wfile.write(body.encode()) + return + + content_length = self.headers.get("Content-Length") + data = self.rfile.read(int(content_length)) if content_length else None + r = request( + self.command, + f"http://{UPSTREAM_HOST}{self.path}", + headers=self.headers, + data=data, + ) + self.send_response(r.status_code) + for k, v in r.headers.items(): + self.send_header(k, v) + self.end_headers() + self.wfile.write(r.content) + self.wfile.close() + + +class ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): + """Handle requests in a separate thread.""" + + +httpd = ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) +httpd.serve_forever() diff --git a/tests/integration/test_s3_access_headers/test.py b/tests/integration/test_s3_access_headers/test.py new file mode 100644 index 00000000000..d5eaa5b23e1 --- /dev/null +++ b/tests/integration/test_s3_access_headers/test.py @@ -0,0 +1,125 @@ +import logging +import pytest +import os + +from helpers.cluster import ClickHouseCluster +from helpers.mock_servers import start_mock_servers + +from helpers.s3_tools import prepare_s3_bucket + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def run_s3_mocks(started_cluster): + script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") + start_mock_servers( + started_cluster, + script_dir, + [ + ("mocker_s3.py", "resolver", "8081"), + ], + ) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster = ClickHouseCluster(__file__, with_spark=True) + try: + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/named_collections.xml", + "configs/config.d/s3_headers.xml", + ], + user_configs=["configs/users.d/users.xml"], + with_minio=True, + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + run_s3_mocks(cluster) + yield cluster + + finally: + cluster.shutdown() + + +CUSTOM_AUTH_TOKEN = "custom-auth-token" +CORRECT_TOKEN = "ValidToken1234" +INCORRECT_TOKEN = "InvalidToken1234" + + +@pytest.mark.parametrize( + "table_name, engine, query_with_invalid_token_must_fail", + [ + pytest.param( + "test_access_header", + "S3('http://resolver:8081/root/test_access_header.csv', 'CSV')", + True, + id="test_access_over_custom_header", + ), + pytest.param( + "test_static_override", + "S3('http://resolver:8081/root/test_static_override.csv', 'minio', 'minio123', 'CSV')", + False, + id="test_access_key_id_overrides_access_header", + ), + pytest.param( + "test_named_colections", + "S3(s3_mock, format='CSV')", + False, + id="test_named_coll_overrides_access_header", + ), + ], +) +def test_custom_access_header( + started_cluster, table_name, engine, query_with_invalid_token_must_fail +): + instance = started_cluster.instances["node1"] + + instance.query( + f""" + SET s3_truncate_on_insert=1; + INSERT INTO FUNCTION s3('http://minio1:9001/root/{table_name}.csv', 'minio', 'minio123','CSV') + SELECT number as a, toString(number) as b FROM numbers(3); + """ + ) + instance.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} (name String, value UInt32) + ENGINE={engine}; + """ + ) + instance.query("SYSTEM DROP QUERY CACHE") + + assert instance.query(f"SELECT count(*) FROM {table_name}") == "3\n" + + config_path = "/etc/clickhouse-server/config.d/s3_headers.xml" + + instance.replace_in_config( + config_path, + f"{CUSTOM_AUTH_TOKEN}: {CORRECT_TOKEN}", + f"{CUSTOM_AUTH_TOKEN}: {INCORRECT_TOKEN}", + ) + instance.query("SYSTEM RELOAD CONFIG") + + if query_with_invalid_token_must_fail: + instance.query_and_get_error(f"SELECT count(*) FROM {table_name}") + + else: + assert instance.query(f"SELECT count(*) FROM {table_name}") == "3\n" + + instance.replace_in_config( + config_path, + f"{CUSTOM_AUTH_TOKEN}: {INCORRECT_TOKEN}", + f"{CUSTOM_AUTH_TOKEN}: {CORRECT_TOKEN}", + ) + + instance.query("SYSTEM RELOAD CONFIG") + assert instance.query(f"SELECT count(*) FROM {table_name}") == "3\n" From 961fb88901d4dadec09f043741d7f7dc1d5e7482 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 24 Oct 2024 10:38:19 +0000 Subject: [PATCH 616/816] fix isort --- .../integration/test_s3_access_headers/s3_mocks/mocker_s3.py | 1 - tests/integration/test_s3_access_headers/test.py | 5 ++--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py b/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py index 64c4857731c..0bbcb2e60e8 100644 --- a/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py +++ b/tests/integration/test_s3_access_headers/s3_mocks/mocker_s3.py @@ -5,7 +5,6 @@ import socketserver import sys import urllib.parse - UPSTREAM_HOST = "minio1:9001" random.seed("No list objects/1.0") diff --git a/tests/integration/test_s3_access_headers/test.py b/tests/integration/test_s3_access_headers/test.py index d5eaa5b23e1..4d4a5b81230 100644 --- a/tests/integration/test_s3_access_headers/test.py +++ b/tests/integration/test_s3_access_headers/test.py @@ -1,13 +1,12 @@ import logging -import pytest import os +import pytest + from helpers.cluster import ClickHouseCluster from helpers.mock_servers import start_mock_servers - from helpers.s3_tools import prepare_s3_bucket - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) From 5a92eb69f14e26ff8b2fb02c036777a531d41ad4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Oct 2024 10:16:39 +0100 Subject: [PATCH 617/816] Ignore No such key exceptions in some cases --- tests/docker_scripts/stress_tests.lib | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index 3ab52c19dbd..b9efe57d501 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -263,8 +263,12 @@ function check_logs_for_critical_errors() # Remove file logical_errors.txt if it's empty [ -s /test_output/logical_errors.txt ] || rm /test_output/logical_errors.txt - # No such key errors (ignore a.myext which is used in 02724_database_s3.sh and does not exist) - rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v "a.myext" > /test_output/no_such_key_errors.txt \ + # ignore: + # - a.myext which is used in 02724_database_s3.sh and does not exist + # - "DistributedCacheTCPHandler" and "caller id: None:DistribCache" because they happen inside distributed cache server + # - "Will read from object storage directly" printed internally by ReadBufferFromDistributedCache, exception will be rethrown and handled correctly + # - "Caught exception while reading S3 object" printed internally by ReadBufferFromS3, exception will be rethrown and handled correctly + rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v -e "a.myext" -e "DistributedCacheTCPHandler" -e "Will read from object storage directly" -e "Caught exception while reading S3 object" -e "caller id: None:DistribCache" > /test_output/no_such_key_errors.txt \ && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \ || echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv From c9810bbea0db857ad8afe8d67346047aa3549c83 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 30 Oct 2024 09:27:50 +0000 Subject: [PATCH 618/816] Rebase --- src/IO/S3AuthSettings.cpp | 5 ++++- src/IO/S3Common.h | 2 +- .../test_s3_access_headers/configs/config.d/s3_headers.xml | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/IO/S3AuthSettings.cpp b/src/IO/S3AuthSettings.cpp index 2362d60674d..5d7d4678977 100644 --- a/src/IO/S3AuthSettings.cpp +++ b/src/IO/S3AuthSettings.cpp @@ -121,6 +121,7 @@ S3AuthSettings::S3AuthSettings( S3AuthSettings::S3AuthSettings(const S3AuthSettings & settings) : headers(settings.headers) + , access_headers(settings.access_headers) , users(settings.users) , server_side_encryption_kms_config(settings.server_side_encryption_kms_config) , impl(std::make_unique(*settings.impl)) @@ -129,6 +130,7 @@ S3AuthSettings::S3AuthSettings(const S3AuthSettings & settings) S3AuthSettings::S3AuthSettings(S3AuthSettings && settings) noexcept : headers(std::move(settings.headers)) + , access_headers(std::move(settings.access_headers)) , users(std::move(settings.users)) , server_side_encryption_kms_config(std::move(settings.server_side_encryption_kms_config)) , impl(std::make_unique(std::move(*settings.impl))) @@ -147,6 +149,7 @@ S3AUTH_SETTINGS_SUPPORTED_TYPES(S3AuthSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPER S3AuthSettings & S3AuthSettings::operator=(S3AuthSettings && settings) noexcept { headers = std::move(settings.headers); + access_headers = std::move(settings.access_headers); users = std::move(settings.users); server_side_encryption_kms_config = std::move(settings.server_side_encryption_kms_config); *impl = std::move(*settings.impl); @@ -202,7 +205,7 @@ void S3AuthSettings::updateIfChanged(const S3AuthSettings & settings) headers = settings.headers; if (!settings.access_headers.empty()) - access_headers = settings.access_headers; + access_headers = settings.access_headers; if (!settings.users.empty()) users.insert(settings.users.begin(), settings.users.end()); diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 1e40108b09f..22b590dcb18 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -69,7 +69,7 @@ struct ProxyConfigurationResolver; namespace S3 { -HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); +HTTPHeaderEntries getHTTPHeaders(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config, std::string header_key = "header"); ServerSideEncryptionKMSConfig getSSEKMSConfig(const std::string & config_elem, const Poco::Util::AbstractConfiguration & config); } diff --git a/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml b/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml index c364d22ec96..2d2eeb3c7b1 100644 --- a/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml +++ b/tests/integration/test_s3_access_headers/configs/config.d/s3_headers.xml @@ -5,4 +5,4 @@ custom-auth-token: ValidToken1234 - \ No newline at end of file + From b4eb69ad4e438dea6e07a873451faf07926102d3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Oct 2024 11:13:14 +0100 Subject: [PATCH 619/816] Better --- tests/docker_scripts/stress_tests.lib | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index b9efe57d501..b0d6cf6d532 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -266,9 +266,9 @@ function check_logs_for_critical_errors() # ignore: # - a.myext which is used in 02724_database_s3.sh and does not exist # - "DistributedCacheTCPHandler" and "caller id: None:DistribCache" because they happen inside distributed cache server - # - "Will read from object storage directly" printed internally by ReadBufferFromDistributedCache, exception will be rethrown and handled correctly - # - "Caught exception while reading S3 object" printed internally by ReadBufferFromS3, exception will be rethrown and handled correctly - rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v -e "a.myext" -e "DistributedCacheTCPHandler" -e "Will read from object storage directly" -e "Caught exception while reading S3 object" -e "caller id: None:DistribCache" > /test_output/no_such_key_errors.txt \ + # - "ReadBufferFromDistributedCache" exception printed internally by ReadBufferFromDistributedCache, exception will be rethrown and handled correctly + # - "ReadBufferFromS3" exception printed internally by ReadBufferFromS3, exception will be rethrown and handled correctly + rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v -e "a.myext" -e "DistributedCacheTCPHandler" -e "ReadBufferFromDistributedCache" -e "ReadBufferFromS3" -e "caller id: None:DistribCache" > /test_output/no_such_key_errors.txt \ && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \ || echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv From 091db0a9845099a2b88b22eb2a73996c9ab8d1bf Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 30 Oct 2024 10:19:08 +0000 Subject: [PATCH 620/816] Fix kafka test --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 0bade55415f..999324b563a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4193,7 +4193,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator ], "expected": { "raw_message": "050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", - "error": "Cannot convert: String to UInt16", + "error": "Cannot parse string \'BAD\' as UInt16", }, "printable": False, }, From ba9587c728d7af72f01618e44e58dfe9cc156e06 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 30 Oct 2024 10:34:12 +0000 Subject: [PATCH 621/816] Removed trash --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 1ed6e137a31..a72fd16abc2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -287,7 +287,6 @@ void StorageObjectStorage::read( size_t num_streams) { configuration->update(object_storage, local_context); - printConfiguration(local_context->getConfigRef(), "Select query"); if (partition_by && configuration->withPartitionWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, From d6acaeae5ac604816159e33dbd29abbde819086c Mon Sep 17 00:00:00 2001 From: Hiroaki Nakamura Date: Wed, 30 Oct 2024 19:43:04 +0900 Subject: [PATCH 622/816] Fix doc for CREATE MATERIALIZED VIEW ON CLUSTER --- docs/en/sql-reference/statements/create/view.md | 2 +- docs/ru/sql-reference/statements/create/view.md | 2 +- docs/zh/sql-reference/statements/create/view.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 0e5d5250e0f..c770348bce0 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -55,7 +55,7 @@ SELECT * FROM view(column1=value1, column2=value2 ...) ## Materialized View ``` sql -CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] +CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] [TO[db.]name] [ENGINE = engine] [POPULATE] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... [COMMENT 'comment'] diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 8fa30446bb3..5dbffd90205 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -39,7 +39,7 @@ SELECT a, b, c FROM (SELECT ...) ## Материализованные представления {#materialized} ``` sql -CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] +CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] [TO[db.]name] [ENGINE = engine] [POPULATE] [DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... ``` diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index 49a1d66bdf1..6c93240644d 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -39,7 +39,7 @@ SELECT a, b, c FROM (SELECT ...) ## Materialized {#materialized} ``` sql -CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... ``` 物化视图存储由相应的[SELECT](../../../sql-reference/statements/select/index.md)管理. From 623b2f11d30af6ff0d00caa56bffbd4590bc4fff Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 30 Oct 2024 02:40:51 +0000 Subject: [PATCH 623/816] Fix test --- tests/integration/test_storage_postgresql/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 0cb551aecc5..78bb1167d79 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -767,6 +767,7 @@ def test_filter_pushdown(started_cluster): "INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)" ) + node1.query("DROP TABLE IF EXISTS test_filter_pushdown_pg_table") node1.query( """ CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32) @@ -774,12 +775,14 @@ def test_filter_pushdown(started_cluster): """ ) + node1.query("DROP TABLE IF EXISTS test_filter_pushdown_local_table") node1.query( """ CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table """ ) + node1.query("DROP TABLE IF EXISTS ch_table") node1.query( "CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id" ) From 486f4512d17492d2e950843f18d358f5a060a3c4 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Wed, 30 Oct 2024 10:53:54 +0000 Subject: [PATCH 624/816] Add missing sources grants for Kafka, NATS and RabbitMQ. --- src/Access/Common/AccessType.h | 3 +++ src/Access/ContextAccess.cpp | 5 ++++- src/Storages/Kafka/StorageKafkaUtils.cpp | 1 + src/Storages/NATS/StorageNATS.cpp | 8 +++++++- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +++++++- 5 files changed, 22 insertions(+), 3 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..fe34618c490 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -237,6 +237,9 @@ enum class AccessType : uint8_t M(S3, "", GLOBAL, SOURCES) \ M(HIVE, "", GLOBAL, SOURCES) \ M(AZURE, "", GLOBAL, SOURCES) \ + M(KAFKA, "", GLOBAL, SOURCES) \ + M(NATS, "", GLOBAL, SOURCES) \ + M(RABBITMQ, "", GLOBAL, SOURCES) \ M(SOURCES, "", GROUP, ALL) \ \ M(CLUSTER, "", GLOBAL, ALL) /* ON CLUSTER queries */ \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 949fd37e403..f47cd53b137 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -52,7 +52,10 @@ namespace {AccessType::HDFS, "HDFS"}, {AccessType::S3, "S3"}, {AccessType::HIVE, "Hive"}, - {AccessType::AZURE, "AzureBlobStorage"} + {AccessType::AZURE, "AzureBlobStorage"}, + {AccessType::KAFKA, "Kafka"}, + {AccessType::NATS, "NATS"}, + {AccessType::RABBITMQ, "RabbitMQ"} }; diff --git a/src/Storages/Kafka/StorageKafkaUtils.cpp b/src/Storages/Kafka/StorageKafkaUtils.cpp index dd954d6a7c2..119aadd11d8 100644 --- a/src/Storages/Kafka/StorageKafkaUtils.cpp +++ b/src/Storages/Kafka/StorageKafkaUtils.cpp @@ -308,6 +308,7 @@ void registerStorageKafka(StorageFactory & factory) creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, + .source_access_type = AccessType::KAFKA, }); } diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 123f5adc22d..5a51f078e7b 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -786,7 +786,13 @@ void registerStorageNATS(StorageFactory & factory) return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(nats_settings), args.mode); }; - factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); + factory.registerStorage( + "NATS", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + .source_access_type = AccessType::NATS, + }); } } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0f3ac2d5289..3e922b541f7 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1322,7 +1322,13 @@ void registerStorageRabbitMQ(StorageFactory & factory) return std::make_shared(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode); }; - factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); + factory.registerStorage( + "RabbitMQ", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + .source_access_type = AccessType::RABBITMQ, + }); } } From 3b0273a5d30b447ca00c1ded40ce937fd358604f Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Wed, 30 Oct 2024 11:02:22 +0000 Subject: [PATCH 625/816] Docs --- docs/en/sql-reference/statements/grant.md | 6 ++++ docs/ru/sql-reference/statements/grant.md | 38 +++++++++++++++++------ docs/zh/sql-reference/statements/grant.md | 38 +++++++++++++++++------ 3 files changed, 62 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index c11299baf38..d4a3e128b13 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -238,10 +238,13 @@ Hierarchy of privileges: - `HDFS` - `HIVE` - `JDBC` + - `KAFKA` - `MONGO` - `MYSQL` + - `NATS` - `ODBC` - `POSTGRES` + - `RABBITMQ` - `REDIS` - `REMOTE` - `S3` @@ -520,10 +523,13 @@ Allows using external data sources. Applies to [table engines](../../engines/tab - `HDFS`. Level: `GLOBAL` - `HIVE`. Level: `GLOBAL` - `JDBC`. Level: `GLOBAL` + - `KAFKA`. Level: `GLOBAL` - `MONGO`. Level: `GLOBAL` - `MYSQL`. Level: `GLOBAL` + - `NATS`. Level: `GLOBAL` - `ODBC`. Level: `GLOBAL` - `POSTGRES`. Level: `GLOBAL` + - `RABBITMQ`. Level: `GLOBAL` - `REDIS`. Level: `GLOBAL` - `REMOTE`. Level: `GLOBAL` - `S3`. Level: `GLOBAL` diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index 2ccc2d05452..79682dc42cd 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -192,14 +192,23 @@ GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION - `addressToSymbol` - `demangle` - [SOURCES](#grant-sources) + - `AZURE` - `FILE` - - `URL` - - `REMOTE` - - `MYSQL` - - `ODBC` - - `JDBC` - `HDFS` + - `HIVE` + - `JDBC` + - `KAFKA` + - `MONGO` + - `MYSQL` + - `NATS` + - `ODBC` + - `POSTGRES` + - `RABBITMQ` + - `REDIS` + - `REMOTE` - `S3` + - `SQLITE` + - `URL` - [dictGet](#grant-dictget) Примеры того, как трактуется данная иерархия: @@ -461,14 +470,23 @@ GRANT INSERT(x,y) ON db.table TO john Разрешает использовать внешние источники данных. Применяется к [движкам таблиц](../../engines/table-engines/index.md) и [табличным функциям](../table-functions/index.md#table-functions). - `SOURCES`. Уровень: `GROUP` + - `AZURE`. Уровень: `GLOBAL` - `FILE`. Уровень: `GLOBAL` - - `URL`. Уровень: `GLOBAL` - - `REMOTE`. Уровень: `GLOBAL` - - `MYSQL`. Уровень: `GLOBAL` - - `ODBC`. Уровень: `GLOBAL` - - `JDBC`. Уровень: `GLOBAL` - `HDFS`. Уровень: `GLOBAL` + - `HIVE`. Уровень: `GLOBAL` + - `JDBC`. Уровень: `GLOBAL` + - `KAFKA`. Уровень: `GLOBAL` + - `MONGO`. Уровень: `GLOBAL` + - `MYSQL`. Уровень: `GLOBAL` + - `NATS`. Уровень: `GLOBAL` + - `ODBC`. Уровень: `GLOBAL` + - `POSTGRES`. Уровень: `GLOBAL` + - `RABBITMQ`. Уровень: `GLOBAL` + - `REDIS`. Уровень: `GLOBAL` + - `REMOTE`. Уровень: `GLOBAL` - `S3`. Уровень: `GLOBAL` + - `SQLITE`. Уровень: `GLOBAL` + - `URL`. Уровень: `GLOBAL` Привилегия `SOURCES` разрешает использование всех источников. Также вы можете присвоить привилегию для каждого источника отдельно. Для использования источников необходимы дополнительные привилегии. diff --git a/docs/zh/sql-reference/statements/grant.md b/docs/zh/sql-reference/statements/grant.md index fea51d590d5..3fd314c791f 100644 --- a/docs/zh/sql-reference/statements/grant.md +++ b/docs/zh/sql-reference/statements/grant.md @@ -170,14 +170,23 @@ GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION - `addressToSymbol` - `demangle` - [SOURCES](#grant-sources) + - `AZURE` - `FILE` - - `URL` - - `REMOTE` - - `YSQL` - - `ODBC` - - `JDBC` - `HDFS` + - `HIVE` + - `JDBC` + - `KAFKA` + - `MONGO` + - `MYSQL` + - `NATS` + - `ODBC` + - `POSTGRES` + - `RABBITMQ` + - `REDIS` + - `REMOTE` - `S3` + - `SQLITE` + - `URL` - [dictGet](#grant-dictget) 如何对待该层级的示例: @@ -428,14 +437,23 @@ GRANT INSERT(x,y) ON db.table TO john 允许在 [table engines](../../engines/table-engines/index.md) 和 [table functions](../../sql-reference/table-functions/index.md#table-functions)中使用外部数据源。 - `SOURCES`. 级别: `GROUP` + - `AZURE`. 级别: `GLOBAL` - `FILE`. 级别: `GLOBAL` - - `URL`. 级别: `GLOBAL` - - `REMOTE`. 级别: `GLOBAL` - - `YSQL`. 级别: `GLOBAL` - - `ODBC`. 级别: `GLOBAL` - - `JDBC`. 级别: `GLOBAL` - `HDFS`. 级别: `GLOBAL` + - `HIVE`. 级别: `GLOBAL` + - `JDBC`. 级别: `GLOBAL` + - `KAFKA`. 级别: `GLOBAL` + - `MONGO`. 级别: `GLOBAL` + - `MYSQL`. 级别: `GLOBAL` + - `NATS`. 级别: `GLOBAL` + - `ODBC`. 级别: `GLOBAL` + - `POSTGRES`. 级别: `GLOBAL` + - `RABBITMQ`. 级别: `GLOBAL` + - `REDIS`. 级别: `GLOBAL` + - `REMOTE`. 级别: `GLOBAL` - `S3`. 级别: `GLOBAL` + - `SQLITE`. 级别: `GLOBAL` + - `URL`. 级别: `GLOBAL` `SOURCES` 权限允许使用所有数据源。当然也可以单独对每个数据源进行授权。要使用数据源时,还需要额外的权限。 From e7fe8fed22db3c8772f9b6fe1bd9eb233e50c36c Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 30 Oct 2024 11:13:03 +0000 Subject: [PATCH 626/816] Added flag for parquet files --- .../registerStorageObjectStorage.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index a0393ea3e6a..e94f1860176 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -16,10 +16,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static std::shared_ptr createStorageObjectStorage( - const StorageFactory::Arguments & args, - StorageObjectStorage::ConfigurationPtr configuration, - ContextPtr context) +namespace +{ + +// LocalObjectStorage is only supported for Iceberg Datalake operations where Avro format is required. For regular file access, use FileStorage instead. +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS || USE_AVRO + +std::shared_ptr +createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObjectStorage::ConfigurationPtr configuration, ContextPtr context) { auto & engine_args = args.engine_args; if (engine_args.empty()) @@ -63,6 +67,9 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } +#endif +} + #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { From 62362d31660a96ac9857d5c3b6db34df855cf9b2 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 30 Oct 2024 12:36:58 +0100 Subject: [PATCH 627/816] Update ASTLiteral.cpp --- src/Parsers/ASTLiteral.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index fc9749e6f1e..515f4f0cb9f 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -1,4 +1,3 @@ - #include #include #include From 4a821e81a1e5397889793c59a2332c0dce4045e3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Oct 2024 12:48:13 +0100 Subject: [PATCH 628/816] Revert "SQL syntax for workload and resource management" --- .gitignore | 1 - .../settings.md | 28 - docs/en/operations/system-tables/resources.md | 37 - docs/en/operations/system-tables/workloads.md | 40 - docs/en/operations/workload-scheduling.md | 53 -- programs/server/Server.cpp | 5 +- programs/server/config.xml | 4 - src/Access/Common/AccessType.h | 4 - src/Access/ContextAccess.cpp | 6 +- src/CMakeLists.txt | 1 - src/Common/Priority.h | 5 +- src/Common/Scheduler/IResourceManager.h | 8 +- src/Common/Scheduler/ISchedulerConstraint.h | 29 +- src/Common/Scheduler/ISchedulerNode.h | 63 +- src/Common/Scheduler/ISchedulerQueue.h | 9 - .../Scheduler/Nodes/ClassifiersConfig.cpp | 9 +- .../Scheduler/Nodes/ClassifiersConfig.h | 1 - ...Manager.cpp => DynamicResourceManager.cpp} | 45 +- ...urceManager.h => DynamicResourceManager.h} | 12 +- src/Common/Scheduler/Nodes/FairPolicy.h | 19 +- src/Common/Scheduler/Nodes/FifoQueue.h | 35 +- .../Scheduler/Nodes/IOResourceManager.cpp | 532 ------------ .../Scheduler/Nodes/IOResourceManager.h | 281 ------- src/Common/Scheduler/Nodes/PriorityPolicy.h | 19 +- .../Scheduler/Nodes/SemaphoreConstraint.h | 73 +- .../Scheduler/Nodes/ThrottlerConstraint.h | 56 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 606 -------------- .../Nodes/registerResourceManagers.cpp | 15 + .../Nodes/registerResourceManagers.h | 8 + .../Scheduler/Nodes/tests/ResourceTest.h | 209 +---- ...cpp => gtest_dynamic_resource_manager.cpp} | 28 +- .../Nodes/tests/gtest_event_queue.cpp | 6 - .../Nodes/tests/gtest_io_resource_manager.cpp | 335 -------- .../Nodes/tests/gtest_resource_class_fair.cpp | 15 +- .../tests/gtest_resource_class_priority.cpp | 13 +- .../Nodes/tests/gtest_resource_scheduler.cpp | 26 +- .../tests/gtest_throttler_constraint.cpp | 34 +- .../tests/gtest_unified_scheduler_node.cpp | 748 ----------------- src/Common/Scheduler/ResourceGuard.h | 20 - src/Common/Scheduler/ResourceManagerFactory.h | 55 ++ src/Common/Scheduler/ResourceRequest.cpp | 25 +- src/Common/Scheduler/ResourceRequest.h | 27 +- src/Common/Scheduler/SchedulerRoot.h | 37 +- src/Common/Scheduler/SchedulingSettings.cpp | 130 --- src/Common/Scheduler/SchedulingSettings.h | 39 - .../Workload/IWorkloadEntityStorage.h | 91 --- .../Workload/WorkloadEntityDiskStorage.cpp | 287 ------- .../Workload/WorkloadEntityDiskStorage.h | 44 - .../Workload/WorkloadEntityKeeperStorage.cpp | 273 ------- .../Workload/WorkloadEntityKeeperStorage.h | 71 -- .../Workload/WorkloadEntityStorageBase.cpp | 773 ------------------ .../Workload/WorkloadEntityStorageBase.h | 126 --- .../Workload/createWorkloadEntityStorage.cpp | 45 - .../Workload/createWorkloadEntityStorage.h | 11 - .../Scheduler/createResourceManager.cpp | 104 --- src/Common/Scheduler/createResourceManager.h | 11 - .../ObjectStorages/DiskObjectStorage.cpp | 127 +-- src/Disks/ObjectStorages/DiskObjectStorage.h | 13 +- src/Interpreters/Context.cpp | 22 +- src/Interpreters/Context.h | 3 - .../InterpreterCreateResourceQuery.cpp | 68 -- .../InterpreterCreateResourceQuery.h | 25 - .../InterpreterCreateWorkloadQuery.cpp | 68 -- .../InterpreterCreateWorkloadQuery.h | 25 - .../InterpreterDropResourceQuery.cpp | 60 -- .../InterpreterDropResourceQuery.h | 21 - .../InterpreterDropWorkloadQuery.cpp | 60 -- .../InterpreterDropWorkloadQuery.h | 21 - src/Interpreters/InterpreterFactory.cpp | 20 - src/Interpreters/registerInterpreters.cpp | 8 - src/Parsers/ASTCreateResourceQuery.cpp | 83 -- src/Parsers/ASTCreateResourceQuery.h | 48 -- src/Parsers/ASTCreateWorkloadQuery.cpp | 95 --- src/Parsers/ASTCreateWorkloadQuery.h | 53 -- src/Parsers/ASTDropResourceQuery.cpp | 25 - src/Parsers/ASTDropResourceQuery.h | 28 - src/Parsers/ASTDropWorkloadQuery.cpp | 25 - src/Parsers/ASTDropWorkloadQuery.h | 28 - src/Parsers/CommonParsers.h | 4 - src/Parsers/ParserCreateResourceQuery.cpp | 144 ---- src/Parsers/ParserCreateResourceQuery.h | 16 - src/Parsers/ParserCreateWorkloadEntity.cpp | 16 - src/Parsers/ParserCreateWorkloadEntity.h | 17 - src/Parsers/ParserCreateWorkloadQuery.cpp | 155 ---- src/Parsers/ParserCreateWorkloadQuery.h | 16 - src/Parsers/ParserDropResourceQuery.cpp | 52 -- src/Parsers/ParserDropResourceQuery.h | 14 - src/Parsers/ParserDropWorkloadQuery.cpp | 52 -- src/Parsers/ParserDropWorkloadQuery.h | 14 - src/Parsers/ParserQuery.cpp | 12 - .../System/StorageSystemResources.cpp | 71 -- src/Storages/System/StorageSystemResources.h | 29 - .../System/StorageSystemScheduler.cpp | 18 +- .../System/StorageSystemWorkloads.cpp | 48 -- src/Storages/System/StorageSystemWorkloads.h | 29 - src/Storages/System/attachSystemTables.cpp | 4 - .../configs/storage_configuration.xml | 17 - tests/integration/test_scheduler/test.py | 394 --------- .../01271_show_privileges.reference | 4 - .../03232_resource_create_and_drop.reference | 5 - .../03232_resource_create_and_drop.sql | 11 - .../03232_workload_create_and_drop.reference | 5 - .../03232_workload_create_and_drop.sql | 11 - .../03232_workloads_and_resources.reference | 0 .../03232_workloads_and_resources.sql | 68 -- 105 files changed, 336 insertions(+), 7403 deletions(-) delete mode 100644 docs/en/operations/system-tables/resources.md delete mode 100644 docs/en/operations/system-tables/workloads.md rename src/Common/Scheduler/Nodes/{CustomResourceManager.cpp => DynamicResourceManager.cpp} (84%) rename src/Common/Scheduler/Nodes/{CustomResourceManager.h => DynamicResourceManager.h} (86%) delete mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.cpp delete mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.h delete mode 100644 src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h create mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.cpp create mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.h rename src/Common/Scheduler/Nodes/tests/{gtest_custom_resource_manager.cpp => gtest_dynamic_resource_manager.cpp} (82%) delete mode 100644 src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp delete mode 100644 src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp create mode 100644 src/Common/Scheduler/ResourceManagerFactory.h delete mode 100644 src/Common/Scheduler/SchedulingSettings.cpp delete mode 100644 src/Common/Scheduler/SchedulingSettings.h delete mode 100644 src/Common/Scheduler/Workload/IWorkloadEntityStorage.h delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp delete mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h delete mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp delete mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.h delete mode 100644 src/Common/Scheduler/createResourceManager.cpp delete mode 100644 src/Common/Scheduler/createResourceManager.h delete mode 100644 src/Interpreters/InterpreterCreateResourceQuery.cpp delete mode 100644 src/Interpreters/InterpreterCreateResourceQuery.h delete mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.cpp delete mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.h delete mode 100644 src/Interpreters/InterpreterDropResourceQuery.cpp delete mode 100644 src/Interpreters/InterpreterDropResourceQuery.h delete mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.cpp delete mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.h delete mode 100644 src/Parsers/ASTCreateResourceQuery.cpp delete mode 100644 src/Parsers/ASTCreateResourceQuery.h delete mode 100644 src/Parsers/ASTCreateWorkloadQuery.cpp delete mode 100644 src/Parsers/ASTCreateWorkloadQuery.h delete mode 100644 src/Parsers/ASTDropResourceQuery.cpp delete mode 100644 src/Parsers/ASTDropResourceQuery.h delete mode 100644 src/Parsers/ASTDropWorkloadQuery.cpp delete mode 100644 src/Parsers/ASTDropWorkloadQuery.h delete mode 100644 src/Parsers/ParserCreateResourceQuery.cpp delete mode 100644 src/Parsers/ParserCreateResourceQuery.h delete mode 100644 src/Parsers/ParserCreateWorkloadEntity.cpp delete mode 100644 src/Parsers/ParserCreateWorkloadEntity.h delete mode 100644 src/Parsers/ParserCreateWorkloadQuery.cpp delete mode 100644 src/Parsers/ParserCreateWorkloadQuery.h delete mode 100644 src/Parsers/ParserDropResourceQuery.cpp delete mode 100644 src/Parsers/ParserDropResourceQuery.h delete mode 100644 src/Parsers/ParserDropWorkloadQuery.cpp delete mode 100644 src/Parsers/ParserDropWorkloadQuery.h delete mode 100644 src/Storages/System/StorageSystemResources.cpp delete mode 100644 src/Storages/System/StorageSystemResources.h delete mode 100644 src/Storages/System/StorageSystemWorkloads.cpp delete mode 100644 src/Storages/System/StorageSystemWorkloads.h delete mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.reference delete mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.sql delete mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.reference delete mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.sql delete mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.reference delete mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.sql diff --git a/.gitignore b/.gitignore index 8a745655cbf..4bc162c1b0f 100644 --- a/.gitignore +++ b/.gitignore @@ -159,7 +159,6 @@ website/package-lock.json /programs/server/store /programs/server/uuid /programs/server/coordination -/programs/server/workload # temporary test files tests/queries/0_stateless/test_* diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 02fa5a8ca58..76d6f5388e3 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3224,34 +3224,6 @@ Default value: "default" **See Also** - [Workload Scheduling](/docs/en/operations/workload-scheduling.md) -## workload_path {#workload_path} - -The directory used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. By default `/workload/` folder under server working directory is used. - -**Example** - -``` xml -/var/lib/clickhouse/workload/ -``` - -**See Also** -- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) -- [workload_zookeeper_path](#workload_zookeeper_path) - -## workload_zookeeper_path {#workload_zookeeper_path} - -The path to a ZooKeeper node, which is used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. For consistency all SQL definitions are stored as a value of this single znode. By default ZooKeeper is not used and definitions are stored on [disk](#workload_path). - -**Example** - -``` xml -/clickhouse/workload/definitions.sql -``` - -**See Also** -- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) -- [workload_path](#workload_path) - ## max_authentication_methods_per_user {#max_authentication_methods_per_user} The maximum number of authentication methods a user can be created with or altered to. diff --git a/docs/en/operations/system-tables/resources.md b/docs/en/operations/system-tables/resources.md deleted file mode 100644 index 6329f05f610..00000000000 --- a/docs/en/operations/system-tables/resources.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -slug: /en/operations/system-tables/resources ---- -# resources - -Contains information for [resources](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every resource. - -Example: - -``` sql -SELECT * -FROM system.resources -FORMAT Vertical -``` - -``` text -Row 1: -────── -name: io_read -read_disks: ['s3'] -write_disks: [] -create_query: CREATE RESOURCE io_read (READ DISK s3) - -Row 2: -────── -name: io_write -read_disks: [] -write_disks: ['s3'] -create_query: CREATE RESOURCE io_write (WRITE DISK s3) -``` - -Columns: - -- `name` (`String`) - Resource name. -- `read_disks` (`Array(String)`) - The array of disk names that uses this resource for read operations. -- `write_disks` (`Array(String)`) - The array of disk names that uses this resource for write operations. -- `create_query` (`String`) - The definition of the resource. diff --git a/docs/en/operations/system-tables/workloads.md b/docs/en/operations/system-tables/workloads.md deleted file mode 100644 index d9c62372044..00000000000 --- a/docs/en/operations/system-tables/workloads.md +++ /dev/null @@ -1,40 +0,0 @@ ---- -slug: /en/operations/system-tables/workloads ---- -# workloads - -Contains information for [workloads](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every workload. - -Example: - -``` sql -SELECT * -FROM system.workloads -FORMAT Vertical -``` - -``` text -Row 1: -────── -name: production -parent: all -create_query: CREATE WORKLOAD production IN `all` SETTINGS weight = 9 - -Row 2: -────── -name: development -parent: all -create_query: CREATE WORKLOAD development IN `all` - -Row 3: -────── -name: all -parent: -create_query: CREATE WORKLOAD `all` -``` - -Columns: - -- `name` (`String`) - Workload name. -- `parent` (`String`) - Parent workload name. -- `create_query` (`String`) - The definition of the workload. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index a43bea7a5b1..08629492ec6 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -43,20 +43,6 @@ Example: ``` -An alternative way to express which disks are used by a resource is SQL syntax: - -```sql -CREATE RESOURCE resource_name (WRITE DISK disk1, READ DISK disk2) -``` - -Resource could be used for any number of disk for READ or WRITE or both for READ and WRITE. There a syntax allowing to use a resource for all the disks: - -```sql -CREATE RESOURCE all_io (READ ANY DISK, WRITE ANY DISK); -``` - -Note that server configuration options have priority over SQL way to define resources. - ## Workload markup {#workload_markup} Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. @@ -167,48 +153,9 @@ Example: ``` -## Workload hierarchy (SQL only) {#workloads} - -Defining resources and classifiers in XML could be challenging. ClickHouse provides SQL syntax that is much more convenient. All resources that were created with `CREATE RESOURCE` share the same structure of the hierarchy, but could differ in some aspects. Every workload created with `CREATE WORKLOAD` maintains a few automatically created scheduling nodes for every resource. A child workload can be created inside another parent workload. Here is the example that defines exactly the same hierarchy as XML configuration above: - -```sql -CREATE RESOURCE network_write (WRITE DISK s3) -CREATE RESOURCE network_read (READ DISK s3) -CREATE WORKLOAD all SETTINGS max_requests = 100 -CREATE WORKLOAD development IN all -CREATE WORKLOAD production IN all SETTINGS weight = 3 -``` - -The name of a leaf workload without children could be used in query settings `SETTINGS workload = 'name'`. Note that workload classifiers are also created automatically when using SQL syntax. - -To customize workload the following settings could be used: -* `priority` - sibling workloads are served according to static priority values (lower value means higher priority). -* `weight` - sibling workloads having the same static priority share resources according to weights. -* `max_requests` - the limit on the number of concurrent resource requests in this workload. -* `max_cost` - the limit on the total inflight bytes count of concurrent resource requests in this workload. -* `max_speed` - the limit on byte processing rate of this workload (the limit is independent for every resource). -* `max_burst` - maximum number of bytes that could be processed by the workload without being throttled (for every resource independently). - -Note that workload settings are translated into a proper set of scheduling nodes. For more details, see the description of the scheduling node [types and options](#hierarchy). - -There is no way to specify different hierarchies of workloads for different resources. But there is a way to specify different workload setting value for a specific resource: - -```sql -CREATE OR REPLACE WORKLOAD all SETTINGS max_requests = 100, max_speed = 1000000 FOR network_read, max_speed = 2000000 FOR network_write -``` - -Also note that workload or resource could not be dropped if it is referenced from another workload. To update a definition of a workload use `CREATE OR REPLACE WORKLOAD` query. - -## Workloads and resources storage {#workload_entity_storage} -Definitions of all workloads and resources in the form of `CREATE WORKLOAD` and `CREATE RESOURCE` queries are stored persistently either on disk at `workload_path` or in ZooKeeper at `workload_zookeeper_path`. ZooKeeper storage is recommended to achieve consistency between nodes. Alternatively `ON CLUSTER` clause could be used along with disk storage. - ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) - - [system.workloads](/docs/en/operations/system-tables/workloads.md) - - [system.resources](/docs/en/operations/system-tables/resources.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting - - [workload_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_path) global server setting - - [workload_zookeeper_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_zookeeper_path) global server setting diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 826100f68e2..d061d134e69 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -86,7 +86,7 @@ #include #include #include -#include +#include #include #include #include "MetricsTransmitter.h" @@ -920,6 +920,7 @@ try registerFormats(); registerRemoteFileMetadatas(); registerSchedulerNodes(); + registerResourceManagers(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); @@ -2252,8 +2253,6 @@ try database_catalog.assertDatabaseExists(default_database); /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); - /// Load WORKLOADs and RESOURCEs. - global_context->getWorkloadEntityStorage().loadEntities(); global_context->getRefreshSet().setRefreshesStopped(false); } diff --git a/programs/server/config.xml b/programs/server/config.xml index 9807f8c0d5a..15649b5c95d 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1399,10 +1399,6 @@ If not specified they will be stored locally. --> - - - diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 242dfcd8c35..010d11e533a 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -99,8 +99,6 @@ enum class AccessType : uint8_t M(CREATE_ARBITRARY_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables with arbitrary table engine */\ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ - M(CREATE_WORKLOAD, "", GLOBAL, CREATE) /* allows to execute CREATE WORKLOAD */ \ - M(CREATE_RESOURCE, "", GLOBAL, CREATE) /* allows to execute CREATE RESOURCE */ \ M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ @@ -110,8 +108,6 @@ enum class AccessType : uint8_t implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ - M(DROP_WORKLOAD, "", GLOBAL, DROP) /* allows to execute DROP WORKLOAD */\ - M(DROP_RESOURCE, "", GLOBAL, DROP) /* allows to execute DROP RESOURCE */\ M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index a5d0451714b..949fd37e403 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -701,17 +701,15 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; - const AccessFlags workload_ddl = AccessType::CREATE_WORKLOAD | AccessType::DROP_WORKLOAD; - const AccessFlags resource_ddl = AccessType::CREATE_RESOURCE | AccessType::DROP_RESOURCE; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; const AccessFlags table_and_dictionary_and_function_ddl = table_ddl | dictionary_ddl | function_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; - const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | workload_ddl | resource_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; + const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; - const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl | workload_ddl | resource_ddl; + const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; const AccessFlags introspection_flags = AccessType::INTROSPECTION; }; static const PrecalculatedFlags precalc; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3627d760d4c..39499cc577d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -136,7 +136,6 @@ add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) -add_headers_and_sources(dbms Common/Scheduler/Workload) if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) diff --git a/src/Common/Priority.h b/src/Common/Priority.h index f0e5787ae91..8952fe4dd5a 100644 --- a/src/Common/Priority.h +++ b/src/Common/Priority.h @@ -6,7 +6,6 @@ /// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize struct Priority { - using Value = Int64; - Value value = 0; /// Note that lower value means higher priority. - constexpr operator Value() const { return value; } /// NOLINT + Int64 value = 0; /// Note that lower value means higher priority. + constexpr operator Int64() const { return value; } /// NOLINT }; diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index c6f41346e11..8a7077ac3d5 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -26,9 +26,6 @@ class IClassifier : private boost::noncopyable public: virtual ~IClassifier() = default; - /// Returns true iff resource access is allowed by this classifier - virtual bool has(const String & resource_name) = 0; - /// Returns ResourceLink that should be used to access resource. /// Returned link is valid until classifier destruction. virtual ResourceLink get(const String & resource_name) = 0; @@ -49,15 +46,12 @@ public: /// Initialize or reconfigure manager. virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0; - /// Returns true iff given resource is controlled through this manager. - virtual bool hasResource(const String & resource_name) const = 0; - /// Obtain a classifier instance required to get access to resources. /// Note that it holds resource configuration, so should be destructed when query is done. virtual ClassifierPtr acquire(const String & classifier_name) = 0; /// For introspection, see `system.scheduler` table - using VisitorFunc = std::function; + using VisitorFunc = std::function; virtual void forEachNode(VisitorFunc visitor) = 0; }; diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index 3bee9c1b424..a976206de74 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -15,7 +15,8 @@ namespace DB * When constraint is again satisfied, scheduleActivation() is called from finishRequest(). * * Derived class behaviour requirements: - * - dequeueRequest() must call `request->addConstraint()`. + * - dequeueRequest() must fill `request->constraint` iff it is nullptr; + * - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`. */ class ISchedulerConstraint : public ISchedulerNode { @@ -24,16 +25,34 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} - ISchedulerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_) - : ISchedulerNode(event_queue_, info_) - {} - /// Resource consumption by `request` is finished. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; + void setParent(ISchedulerNode * parent_) override + { + ISchedulerNode::setParent(parent_); + + // Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint + for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent) + { + if (auto * constraint = dynamic_cast(node)) + { + parent_constraint = constraint; + break; + } + } + } + /// For introspection of current state (true = satisfied, false = violated) virtual bool isSatisfied() = 0; + +protected: + // Reference to nearest parent that is also derived from ISchedulerConstraint. + // Request can traverse through multiple constraints while being dequeue from hierarchy, + // while finishing request should traverse the same chain in reverse order. + // NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest() + ISchedulerConstraint * parent_constraint = nullptr; }; } diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 5e1239de274..0705c4f0a35 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -57,13 +57,7 @@ struct SchedulerNodeInfo SchedulerNodeInfo() = default; - explicit SchedulerNodeInfo(double weight_, Priority priority_ = {}) - { - setWeight(weight_); - setPriority(priority_); - } - - explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config, const String & config_prefix = {}) + explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) { setWeight(config.getDouble(config_prefix + ".weight", weight)); setPriority(config.getInt64(config_prefix + ".priority", priority)); @@ -74,7 +68,7 @@ struct SchedulerNodeInfo if (value <= 0 || !isfinite(value)) throw Exception( ErrorCodes::INVALID_SCHEDULER_NODE, - "Zero, negative and non-finite node weights are not allowed: {}", + "Negative and non-finite node weights are not allowed: {}", value); weight = value; } @@ -84,11 +78,6 @@ struct SchedulerNodeInfo priority.value = value; } - void setPriority(Priority value) - { - priority = value; - } - // To check if configuration update required bool equals(const SchedulerNodeInfo & o) const { @@ -134,14 +123,7 @@ public: , info(config, config_prefix) {} - ISchedulerNode(EventQueue * event_queue_, const SchedulerNodeInfo & info_) - : event_queue(event_queue_) - , info(info_) - {} - - virtual ~ISchedulerNode(); - - virtual const String & getTypeName() const = 0; + virtual ~ISchedulerNode() = default; /// Checks if two nodes configuration is equal virtual bool equals(ISchedulerNode * other) @@ -152,11 +134,10 @@ public: /// Attach new child virtual void attachChild(const std::shared_ptr & child) = 0; - /// Detach child - /// NOTE: child might be destroyed if the only reference was stored in parent + /// Detach and destroy child virtual void removeChild(ISchedulerNode * child) = 0; - /// Get attached child by name (for tests only) + /// Get attached child by name virtual ISchedulerNode * getChild(const String & child_name) = 0; /// Activation of child due to the first pending request @@ -166,7 +147,7 @@ public: /// Returns true iff node is active virtual bool isActive() = 0; - /// Returns number of active children (for introspection only). + /// Returns number of active children virtual size_t activeChildren() = 0; /// Returns the first request to be executed as the first component of resulting pair. @@ -174,10 +155,10 @@ public: virtual std::pair dequeueRequest() = 0; /// Returns full path string using names of every parent - String getPath() const + String getPath() { String result; - const ISchedulerNode * ptr = this; + ISchedulerNode * ptr = this; while (ptr->parent) { result = "/" + ptr->basename + result; @@ -187,7 +168,10 @@ public: } /// Attach to a parent (used by attachChild) - void setParent(ISchedulerNode * parent_); + virtual void setParent(ISchedulerNode * parent_) + { + parent = parent_; + } protected: /// Notify parents about the first pending request or constraint becoming satisfied. @@ -323,15 +307,6 @@ public: pending.notify_one(); } - /// Removes an activation from queue - void cancelActivation(ISchedulerNode * node) - { - std::unique_lock lock{mutex}; - if (node->is_linked()) - activations.erase(activations.iterator_to(*node)); - node->activation_event_id = 0; - } - /// Process single event if it exists /// Note that postponing constraint are ignored, use it to empty the queue including postponed events on shutdown /// Returns `true` iff event has been processed @@ -496,20 +471,6 @@ private: std::atomic manual_time{TimePoint()}; // for tests only }; -inline ISchedulerNode::~ISchedulerNode() -{ - // Make sure there is no dangling reference in activations queue - event_queue->cancelActivation(this); -} - -inline void ISchedulerNode::setParent(ISchedulerNode * parent_) -{ - parent = parent_; - // Avoid activation of a detached node - if (parent == nullptr) - event_queue->cancelActivation(this); -} - inline void ISchedulerNode::scheduleActivation() { if (likely(parent)) diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index 6c77cee6b9d..b7a51870a24 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -21,10 +21,6 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} - ISchedulerQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) - : ISchedulerNode(event_queue_, info_) - {} - // Wrapper for `enqueueRequest()` that should be used to account for available resource budget // Returns `estimated_cost` that should be passed later to `adjustBudget()` [[ nodiscard ]] ResourceCost enqueueRequestUsingBudget(ResourceRequest * request) @@ -51,11 +47,6 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual bool cancelRequest(ResourceRequest * request) = 0; - /// Fails all the resource requests in queue and marks this queue as not usable. - /// Afterwards any new request will be failed on `enqueueRequest()`. - /// NOTE: This is done for queues that are about to be destructed. - virtual void purgeQueue() = 0; - /// For introspection ResourceCost getBudget() const { diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 455d0880aa6..3be61801149 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_NOT_FOUND; +} + ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { Poco::Util::AbstractConfiguration::Keys keys; @@ -26,11 +31,9 @@ ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & c const ClassifierDescription & ClassifiersConfig::get(const String & classifier_name) { - static ClassifierDescription empty; if (auto it = classifiers.find(classifier_name); it != classifiers.end()) return it->second; - else - return empty; + throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name); } } diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.h b/src/Common/Scheduler/Nodes/ClassifiersConfig.h index 62db719568b..186c49943ad 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.h +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.h @@ -10,7 +10,6 @@ namespace DB /// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class") struct ClassifierDescription : std::unordered_map { - ClassifierDescription() = default; ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); }; diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp similarity index 84% rename from src/Common/Scheduler/Nodes/CustomResourceManager.cpp rename to src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index b9ab89ee2b8..5bf884fc3df 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -1,6 +1,7 @@ -#include +#include #include +#include #include #include @@ -20,7 +21,7 @@ namespace ErrorCodes extern const int INVALID_SCHEDULER_NODE; } -CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) +DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) : classifiers(config) { Poco::Util::AbstractConfiguration::Keys keys; @@ -34,7 +35,7 @@ CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util:: } } -CustomResourceManager::State::Resource::Resource( +DynamicResourceManager::State::Resource::Resource( const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, @@ -91,7 +92,7 @@ CustomResourceManager::State::Resource::Resource( throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name); } -CustomResourceManager::State::Resource::~Resource() +DynamicResourceManager::State::Resource::~Resource() { // NOTE: we should rely on `attached_to` and cannot use `parent`, // NOTE: because `parent` can be `nullptr` in case attachment is still in event queue @@ -105,14 +106,14 @@ CustomResourceManager::State::Resource::~Resource() } } -CustomResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +DynamicResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) : type(config.getString(config_prefix + ".type", "fifo")) , ptr(SchedulerNodeFactory::instance().get(type, event_queue, config, config_prefix)) { ptr->basename = name; } -bool CustomResourceManager::State::Resource::equals(const CustomResourceManager::State::Resource & o) const +bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManager::State::Resource & o) const { if (nodes.size() != o.nodes.size()) return false; @@ -129,14 +130,14 @@ bool CustomResourceManager::State::Resource::equals(const CustomResourceManager: return true; } -bool CustomResourceManager::State::Node::equals(const CustomResourceManager::State::Node & o) const +bool DynamicResourceManager::State::Node::equals(const DynamicResourceManager::State::Node & o) const { if (type != o.type) return false; return ptr->equals(o.ptr.get()); } -CustomResourceManager::Classifier::Classifier(const CustomResourceManager::StatePtr & state_, const String & classifier_name) +DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::StatePtr & state_, const String & classifier_name) : state(state_) { // State is immutable, but nodes are mutable and thread-safe @@ -161,25 +162,20 @@ CustomResourceManager::Classifier::Classifier(const CustomResourceManager::State } } -bool CustomResourceManager::Classifier::has(const String & resource_name) -{ - return resources.contains(resource_name); -} - -ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) +ResourceLink DynamicResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) return iter->second; throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); } -CustomResourceManager::CustomResourceManager() +DynamicResourceManager::DynamicResourceManager() : state(new State()) { scheduler.start(); } -void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { StatePtr new_state = std::make_shared(scheduler.event_queue, config); @@ -221,13 +217,7 @@ void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfig // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } -bool CustomResourceManager::hasResource(const String & resource_name) const -{ - std::lock_guard lock{mutex}; - return state->resources.contains(resource_name); -} - -ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) +ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state StatePtr state_ref; @@ -239,7 +229,7 @@ ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) return std::make_shared(state_ref, classifier_name); } -void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { // Acquire a reference to the current state StatePtr state_ref; @@ -254,7 +244,7 @@ void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { for (auto & [name, resource] : state_ref->resources) for (auto & [path, node] : resource->nodes) - visitor(name, path, node.ptr.get()); + visitor(name, path, node.type, node.ptr); promise.set_value(); }); @@ -262,4 +252,9 @@ void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) future.get(); } +void registerDynamicResourceManager(ResourceManagerFactory & factory) +{ + factory.registerMethod("dynamic"); +} + } diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.h b/src/Common/Scheduler/Nodes/DynamicResourceManager.h similarity index 86% rename from src/Common/Scheduler/Nodes/CustomResourceManager.h rename to src/Common/Scheduler/Nodes/DynamicResourceManager.h index 900a9c4e50b..4b0a3a48b61 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.h +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.h @@ -10,9 +10,7 @@ namespace DB { /* - * Implementation of `IResourceManager` supporting arbitrary hierarchy of scheduler nodes. - * Scheduling hierarchies for every resource is described through server xml or yaml configuration. - * Configuration could be changed dynamically without server restart. + * Implementation of `IResourceManager` supporting arbitrary dynamic hierarchy of scheduler nodes. * All resources are controlled by single root `SchedulerRoot`. * * State of manager is set of resources attached to the scheduler. States are referenced by classifiers. @@ -26,12 +24,11 @@ namespace DB * violation will apply to fairness. Old version exists as long as there is at least one classifier * instance referencing it. Classifiers are typically attached to queries and will be destructed with them. */ -class CustomResourceManager : public IResourceManager +class DynamicResourceManager : public IResourceManager { public: - CustomResourceManager(); + DynamicResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; - bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; @@ -82,7 +79,6 @@ private: { public: Classifier(const StatePtr & state_, const String & classifier_name); - bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; private: std::unordered_map resources; // accessible resources by names @@ -90,7 +86,7 @@ private: }; SchedulerRoot scheduler; - mutable std::mutex mutex; + std::mutex mutex; StatePtr state; }; diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index a865711c460..246642ff2fd 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -28,7 +28,7 @@ namespace ErrorCodes * of a child is set to vruntime of "start" of the last request. This guarantees immediate processing * of at least single request of newly activated children and thus best isolation and scheduling latency. */ -class FairPolicy final : public ISchedulerNode +class FairPolicy : public ISchedulerNode { /// Scheduling state of a child struct Item @@ -48,23 +48,6 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} - FairPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & info_) - : ISchedulerNode(event_queue_, info_) - {} - - ~FairPolicy() override - { - // We need to clear `parent` in all children to avoid dangling references - while (!children.empty()) - removeChild(children.begin()->second.get()); - } - - const String & getTypeName() const override - { - static String type_name("fair"); - return type_name; - } - bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 9502fae1a45..90f8fffe665 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -23,28 +23,13 @@ namespace ErrorCodes /* * FIFO queue to hold pending resource requests */ -class FifoQueue final : public ISchedulerQueue +class FifoQueue : public ISchedulerQueue { public: FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) : ISchedulerQueue(event_queue_, config, config_prefix) {} - FifoQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) - : ISchedulerQueue(event_queue_, info_) - {} - - ~FifoQueue() override - { - purgeQueue(); - } - - const String & getTypeName() const override - { - static String type_name("fifo"); - return type_name; - } - bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -57,8 +42,6 @@ public: void enqueueRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); - if (is_not_usable) - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue is about to be destructed"); queue_cost += request->cost; bool was_empty = requests.empty(); requests.push_back(*request); @@ -83,8 +66,6 @@ public: bool cancelRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); - if (is_not_usable) - return false; // Any request should already be failed or executed if (request->is_linked()) { // It's impossible to check that `request` is indeed inserted to this queue and not another queue. @@ -107,19 +88,6 @@ public: return false; } - void purgeQueue() override - { - std::lock_guard lock(mutex); - is_not_usable = true; - while (!requests.empty()) - { - ResourceRequest * request = &requests.front(); - requests.pop_front(); - request->failed(std::make_exception_ptr( - Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request is about to be destructed"))); - } - } - bool isActive() override { std::lock_guard lock(mutex); @@ -163,7 +131,6 @@ private: std::mutex mutex; Int64 queue_cost = 0; boost::intrusive::list requests; - bool is_not_usable = false; }; } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp deleted file mode 100644 index e2042a29a80..00000000000 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ /dev/null @@ -1,532 +0,0 @@ -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int RESOURCE_NOT_FOUND; - extern const int INVALID_SCHEDULER_NODE; - extern const int LOGICAL_ERROR; -} - -namespace -{ - String getEntityName(const ASTPtr & ast) - { - if (auto * create = typeid_cast(ast.get())) - return create->getWorkloadName(); - if (auto * create = typeid_cast(ast.get())) - return create->getResourceName(); - return "unknown-workload-entity"; - } -} - -IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resource_name) -{ - auto * create = assert_cast(ast.get()); - name = create->getWorkloadName(); - parent = create->getWorkloadParent(); - settings.updateFromChanges(create->changes, resource_name); -} - -IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) - : resource_entity(resource_entity_) - , resource_name(getEntityName(resource_entity)) -{ - scheduler.start(); -} - -IOResourceManager::Resource::~Resource() -{ - scheduler.stop(); -} - -void IOResourceManager::Resource::createNode(const NodeInfo & info) -{ - if (info.name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", - resource_name); - - if (info.name == info.parent) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", - info.name, resource_name); - - if (node_for_workload.contains(info.name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", - info.name, resource_name); - - if (!info.parent.empty() && !node_for_workload.contains(info.parent)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", - info.parent, info.name, resource_name); - - if (info.parent.empty() && root_node) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", - info.name, root_node->basename, resource_name); - - executeInSchedulerThread([&, this] - { - auto node = std::make_shared(scheduler.event_queue, info.settings); - node->basename = info.name; - if (!info.parent.empty()) - node_for_workload[info.parent]->attachUnifiedChild(node); - else - { - root_node = node; - scheduler.attachChild(root_node); - } - node_for_workload[info.name] = node; - - updateCurrentVersion(); - }); -} - -void IOResourceManager::Resource::deleteNode(const NodeInfo & info) -{ - if (!node_for_workload.contains(info.name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for removing workload '{}' does not exist in resource '{}'", - info.name, resource_name); - - if (!info.parent.empty() && !node_for_workload.contains(info.parent)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for removing workload '{}' does not exist in resource '{}'", - info.parent, info.name, resource_name); - - auto node = node_for_workload[info.name]; - - if (node->hasUnifiedChildren()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", - info.name, resource_name); - - executeInSchedulerThread([&] - { - if (!info.parent.empty()) - node_for_workload[info.parent]->detachUnifiedChild(node); - else - { - chassert(node == root_node); - scheduler.removeChild(root_node.get()); - root_node.reset(); - } - - node_for_workload.erase(info.name); - - updateCurrentVersion(); - }); -} - -void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const NodeInfo & new_info) -{ - if (old_info.name != new_info.name) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", - old_info.name, new_info.name, resource_name); - - if (old_info.parent != new_info.parent && (old_info.parent.empty() || new_info.parent.empty())) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", - old_info.name, old_info.parent, new_info.parent, resource_name); - - if (!node_for_workload.contains(old_info.name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for updating workload '{}' does not exist in resource '{}'", - old_info.name, resource_name); - - if (!old_info.parent.empty() && !node_for_workload.contains(old_info.parent)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Old parent node '{}' for updating workload '{}' does not exist in resource '{}'", - old_info.parent, old_info.name, resource_name); - - if (!new_info.parent.empty() && !node_for_workload.contains(new_info.parent)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "New parent node '{}' for updating workload '{}' does not exist in resource '{}'", - new_info.parent, new_info.name, resource_name); - - executeInSchedulerThread([&, this] - { - auto node = node_for_workload[old_info.name]; - bool detached = false; - if (UnifiedSchedulerNode::updateRequiresDetach(old_info.parent, new_info.parent, old_info.settings, new_info.settings)) - { - if (!old_info.parent.empty()) - node_for_workload[old_info.parent]->detachUnifiedChild(node); - detached = true; - } - - node->updateSchedulingSettings(new_info.settings); - - if (detached) - { - if (!new_info.parent.empty()) - node_for_workload[new_info.parent]->attachUnifiedChild(node); - } - updateCurrentVersion(); - }); -} - -void IOResourceManager::Resource::updateCurrentVersion() -{ - auto previous_version = current_version; - - // Create a full list of constraints and queues in the current hierarchy - current_version = std::make_shared(); - if (root_node) - root_node->addRawPointerNodes(current_version->nodes); - - // See details in version control section of description in IOResourceManager.h - if (previous_version) - { - previous_version->newer_version = current_version; - previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it - } -} - -IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_) - : resource_manager(resource_manager_) - , workload_entity(workload_entity_) -{ - try - { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->createNode(NodeInfo(workload_entity, resource_name)); - } - catch (...) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", - getCurrentExceptionMessage(/* with_stacktrace = */ true)); - } -} - -IOResourceManager::Workload::~Workload() -{ - try - { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->deleteNode(NodeInfo(workload_entity, resource_name)); - } - catch (...) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", - getCurrentExceptionMessage(/* with_stacktrace = */ true)); - } -} - -void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) -{ - try - { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); - workload_entity = new_entity; - } - catch (...) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", - getCurrentExceptionMessage(/* with_stacktrace = */ true)); - } -} - -String IOResourceManager::Workload::getParent() const -{ - return assert_cast(workload_entity.get())->getWorkloadParent(); -} - -IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) - : storage(storage_) - , log{getLogger("IOResourceManager")} -{ - subscription = storage.getAllEntitiesAndSubscribe( - [this] (const std::vector & events) - { - for (const auto & [entity_type, entity_name, entity] : events) - { - switch (entity_type) - { - case WorkloadEntityType::Workload: - { - if (entity) - createOrUpdateWorkload(entity_name, entity); - else - deleteWorkload(entity_name); - break; - } - case WorkloadEntityType::Resource: - { - if (entity) - createOrUpdateResource(entity_name, entity); - else - deleteResource(entity_name); - break; - } - case WorkloadEntityType::MAX: break; - } - } - }); -} - -IOResourceManager::~IOResourceManager() -{ - subscription.reset(); - resources.clear(); - workloads.clear(); -} - -void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) -{ - // No-op -} - -void IOResourceManager::createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast) -{ - std::unique_lock lock{mutex}; - if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) - workload_iter->second->updateWorkload(ast); - else - workloads.emplace(workload_name, std::make_shared(this, ast)); -} - -void IOResourceManager::deleteWorkload(const String & workload_name) -{ - std::unique_lock lock{mutex}; - if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) - { - // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent - workloads.erase(workload_iter); - } - else // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - LOG_ERROR(log, "Delete workload that doesn't exist: {}", workload_name); -} - -void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) -{ - std::unique_lock lock{mutex}; - if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) - resource_iter->second->updateResource(ast); - else - { - // Add all workloads into the new resource - auto resource = std::make_shared(ast); - for (Workload * workload : topologicallySortedWorkloads()) - resource->createNode(NodeInfo(workload->workload_entity, resource_name)); - - // Attach the resource - resources.emplace(resource_name, resource); - } -} - -void IOResourceManager::deleteResource(const String & resource_name) -{ - std::unique_lock lock{mutex}; - if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) - { - resources.erase(resource_iter); - } - else // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - LOG_ERROR(log, "Delete resource that doesn't exist: {}", resource_name); -} - -IOResourceManager::Classifier::~Classifier() -{ - // Detach classifier from all resources in parallel (executed in every scheduler thread) - std::vector> futures; - { - std::unique_lock lock{mutex}; - futures.reserve(attachments.size()); - for (auto & [resource_name, attachment] : attachments) - { - futures.emplace_back(attachment.resource->detachClassifier(std::move(attachment.version))); - attachment.link.reset(); // Just in case because it is not valid any longer - } - } - - // Wait for all tasks to finish (to avoid races in case of exceptions) - for (auto & future : futures) - future.wait(); - - // There should not be any exceptions because it just destruct few objects, but let's rethrow just in case - for (auto & future : futures) - future.get(); - - // This unreferences and probably destroys `Resource` objects. - // NOTE: We cannot do it in the scheduler threads (because thread cannot join itself). - attachments.clear(); -} - -std::future IOResourceManager::Resource::detachClassifier(VersionPtr && version) -{ - auto detach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semanticss - auto future = detach_promise->get_future(); - scheduler.event_queue->enqueue([detached_version = std::move(version), promise = std::move(detach_promise)] mutable - { - try - { - // Unreferences and probably destroys the version and scheduler nodes it owns. - // The main reason from moving destruction into the scheduler thread is to - // free memory in the same thread it was allocated to avoid memtrackers drift. - detached_version.reset(); - promise->set_value(); - } - catch (...) - { - promise->set_exception(std::current_exception()); - } - }); - return future; -} - -bool IOResourceManager::Classifier::has(const String & resource_name) -{ - std::unique_lock lock{mutex}; - return attachments.contains(resource_name); -} - -ResourceLink IOResourceManager::Classifier::get(const String & resource_name) -{ - std::unique_lock lock{mutex}; - if (auto iter = attachments.find(resource_name); iter != attachments.end()) - return iter->second.link; - else - throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Access denied to resource '{}'", resource_name); -} - -void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link) -{ - std::unique_lock lock{mutex}; - chassert(!attachments.contains(resource->getName())); - attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; -} - -void IOResourceManager::Resource::updateResource(const ASTPtr & new_resource_entity) -{ - chassert(getEntityName(new_resource_entity) == resource_name); - resource_entity = new_resource_entity; -} - -std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) -{ - auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics - auto future = attach_promise->get_future(); - scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] - { - try - { - if (auto iter = node_for_workload.find(workload_name); iter != node_for_workload.end()) - { - auto queue = iter->second->getQueue(); - if (!queue) - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to use workload '{}' that have children for resource '{}'", - workload_name, resource_name); - classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); - } - else - { - // This resource does not have specified workload. It is either unknown or managed by another resource manager. - // We leave this resource not attached to the classifier. Access denied will be thrown later on `classifier->get(resource_name)` - } - promise->set_value(); - } - catch (...) - { - promise->set_exception(std::current_exception()); - } - }); - return future; -} - -bool IOResourceManager::hasResource(const String & resource_name) const -{ - std::unique_lock lock{mutex}; - return resources.contains(resource_name); -} - -ClassifierPtr IOResourceManager::acquire(const String & workload_name) -{ - auto classifier = std::make_shared(); - - // Attach classifier to all resources in parallel (executed in every scheduler thread) - std::vector> futures; - { - std::unique_lock lock{mutex}; - futures.reserve(resources.size()); - for (auto & [resource_name, resource] : resources) - futures.emplace_back(resource->attachClassifier(*classifier, workload_name)); - } - - // Wait for all tasks to finish (to avoid races in case of exceptions) - for (auto & future : futures) - future.wait(); - - // Rethrow exceptions if any - for (auto & future : futures) - future.get(); - - return classifier; -} - -void IOResourceManager::Resource::forEachResourceNode(IResourceManager::VisitorFunc & visitor) -{ - executeInSchedulerThread([&, this] - { - for (auto & [path, node] : node_for_workload) - { - node->forEachSchedulerNode([&] (ISchedulerNode * scheduler_node) - { - visitor(resource_name, scheduler_node->getPath(), scheduler_node); - }); - } - }); -} - -void IOResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) -{ - // Copy resource to avoid holding mutex for a long time - std::unordered_map resources_copy; - { - std::unique_lock lock{mutex}; - resources_copy = resources; - } - - /// Run tasks one by one to avoid concurrent calls to visitor - for (auto & [resource_name, resource] : resources_copy) - resource->forEachResourceNode(visitor); -} - -void IOResourceManager::topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads) -{ - if (visited.contains(workload)) - return; - visited.insert(workload); - - // Recurse into parent (if any) - String parent = workload->getParent(); - if (!parent.empty()) - { - auto parent_iter = workloads.find(parent); - chassert(parent_iter != workloads.end()); // validations check that all parents exist - topologicallySortedWorkloadsImpl(parent_iter->second.get(), visited, sorted_workloads); - } - - sorted_workloads.push_back(workload); -} - -std::vector IOResourceManager::topologicallySortedWorkloads() -{ - std::vector sorted_workloads; - std::unordered_set visited; - for (auto & [workload_name, workload] : workloads) - topologicallySortedWorkloadsImpl(workload.get(), visited, sorted_workloads); - return sorted_workloads; -} - -} diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h deleted file mode 100644 index cfd8a234b37..00000000000 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ /dev/null @@ -1,281 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include -#include -#include -#include -#include - -namespace DB -{ - -/* - * Implementation of `IResourceManager` that creates hierarchy of scheduler nodes according to - * workload entities (WORKLOADs and RESOURCEs). It subscribes for updates in IWorkloadEntityStorage and - * creates hierarchy of UnifiedSchedulerNode identical to the hierarchy of WORKLOADs. - * For every RESOURCE an independent hierarchy of scheduler nodes is created. - * - * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. - * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). - * After DROP RESOURCE parts of hierarchy might be kept alive while at least one query uses it. - * - * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having - * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. - * - * Classifiers are used (1) to access IO resources and (2) to keep shared ownership of scheduling nodes. - * This allows `ResourceRequest` and `ResourceLink` to hold raw pointers as long as - * `ClassifierPtr` is acquired and held. - * - * === RESOURCE ARCHITECTURE === - * Let's consider how a single resource is implemented. Every workload is represented by corresponding UnifiedSchedulerNode. - * Every UnifiedSchedulerNode manages its own subtree of ISchedulerNode objects (see details in UnifiedSchedulerNode.h) - * UnifiedSchedulerNode for workload w/o children has a queue, which provide a ResourceLink for consumption. - * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. - * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). - * - * Here is an example of SQL and corresponding hierarchy of scheduler nodes: - * CREATE RESOURCE my_io_resource (...) - * CREATE WORKLOAD all - * CREATE WORKLOAD production PARENT all - * CREATE WORKLOAD development PARENT all - * - * root - SchedulerRoot (with scheduler thread and EventQueue) - * | - * all - UnifiedSchedulerNode - * | - * p0_fair - FairPolicy (part of parent UnifiedSchedulerNode internal structure) - * / \ - * production development - UnifiedSchedulerNode - * | | - * queue queue - FifoQueue (part of parent UnifiedSchedulerNode internal structure) - * - * === UPDATING WORKLOADS === - * Workload may be created, updated or deleted. - * Updating a child of a workload might lead to updating other workloads: - * 1. Workload itself: it's structure depend on settings of children workloads - * (e.g. fifo node of a leaf workload is remove when the first child is added; - * and a fair node is inserted after the first two children are added). - * 2. Other children: for them path to root might be changed (e.g. intermediate priority node is inserted) - * - * === VERSION CONTROL === - * Versions are created on hierarchy updates and hold ownership of nodes that are used through raw pointers. - * Classifier reference version of every resource it use. Older version reference newer version. - * Here is a diagram explaining version control based on Version objects (for 1 resource): - * - * [nodes] [nodes] [nodes] - * ^ ^ ^ - * | | | - * version1 --> version2 -...-> versionN - * ^ ^ ^ - * | | | - * old_classifier new_classifier current_version - * - * Previous version should hold reference to a newer version. It is required for proper handling of updates. - * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). - * It may move a queue to a new position in the hierarchy or create/destroy constraints, thus resource requests - * created by old classifier may reference constraints of newer versions through `request->constraints` which - * is filled during dequeueRequest(). - * - * === THREADS === - * scheduler thread: - * - one thread per resource - * - uses event_queue (per resource) for processing w/o holding mutex for every scheduler node - * - handle resource requests - * - node activations - * - scheduler hierarchy updates - * query thread: - * - multiple independent threads - * - send resource requests - * - acquire and release classifiers (via scheduler event queues) - * control thread: - * - modify workload and resources through subscription - * - * === SYNCHRONIZATION === - * List of related sync primitives and their roles: - * IOResourceManager::mutex - * - protects resource manager data structures - resource and workloads - * - serialize control thread actions - * IOResourceManager::Resource::scheduler->event_queue - * - serializes scheduler hierarchy events - * - events are created in control and query threads - * - all events are processed by specific scheduler thread - * - hierarchy-wide actions: requests dequeueing, activations propagation and nodes updates. - * - resource version control management - * FifoQueue::mutex and SemaphoreContraint::mutex - * - serializes query and scheduler threads on specific node accesses - * - resource request processing: enqueueRequest(), dequeueRequest() and finishRequest() - */ -class IOResourceManager : public IResourceManager -{ -public: - explicit IOResourceManager(IWorkloadEntityStorage & storage_); - ~IOResourceManager() override; - void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; - bool hasResource(const String & resource_name) const override; - ClassifierPtr acquire(const String & workload_name) override; - void forEachNode(VisitorFunc visitor) override; - -private: - // Forward declarations - struct NodeInfo; - struct Version; - class Resource; - struct Workload; - class Classifier; - - friend struct Workload; - - using VersionPtr = std::shared_ptr; - using ResourcePtr = std::shared_ptr; - using WorkloadPtr = std::shared_ptr; - - /// Helper for parsing workload AST for a specific resource - struct NodeInfo - { - String name; // Workload name - String parent; // Name of parent workload - SchedulingSettings settings; // Settings specific for a given resource - - NodeInfo(const ASTPtr & ast, const String & resource_name); - }; - - /// Ownership control for scheduler nodes, which could be referenced by raw pointers - struct Version - { - std::vector nodes; - VersionPtr newer_version; - }; - - /// Holds a thread and hierarchy of unified scheduler nodes for specific RESOURCE - class Resource : public std::enable_shared_from_this, boost::noncopyable - { - public: - explicit Resource(const ASTPtr & resource_entity_); - ~Resource(); - - const String & getName() const { return resource_name; } - - /// Hierarchy management - void createNode(const NodeInfo & info); - void deleteNode(const NodeInfo & info); - void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); - - /// Updates resource entity - void updateResource(const ASTPtr & new_resource_entity); - - /// Updates a classifier to contain a reference for specified workload - std::future attachClassifier(Classifier & classifier, const String & workload_name); - - /// Remove classifier reference. This destroys scheduler nodes in proper scheduler thread - std::future detachClassifier(VersionPtr && version); - - /// Introspection - void forEachResourceNode(IOResourceManager::VisitorFunc & visitor); - - private: - void updateCurrentVersion(); - - template - void executeInSchedulerThread(Task && task) - { - std::promise promise; - auto future = promise.get_future(); - scheduler.event_queue->enqueue([&] - { - try - { - task(); - promise.set_value(); - } - catch (...) - { - promise.set_exception(std::current_exception()); - } - }); - future.get(); // Blocks until execution is done in the scheduler thread - } - - ASTPtr resource_entity; - const String resource_name; - SchedulerRoot scheduler; - - // TODO(serxa): consider using resource_manager->mutex + scheduler thread for updates and mutex only for reading to avoid slow acquire/release of classifier - /// These field should be accessed only by the scheduler thread - std::unordered_map node_for_workload; - UnifiedSchedulerNodePtr root_node; - VersionPtr current_version; - }; - - struct Workload : boost::noncopyable - { - IOResourceManager * resource_manager; - ASTPtr workload_entity; - - Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_); - ~Workload(); - - void updateWorkload(const ASTPtr & new_entity); - String getParent() const; - }; - - class Classifier : public IClassifier - { - public: - ~Classifier() override; - - /// Implements IClassifier interface - /// NOTE: It is called from query threads (possibly multiple) - bool has(const String & resource_name) override; - ResourceLink get(const String & resource_name) override; - - /// Attaches/detaches a specific resource - /// NOTE: It is called from scheduler threads (possibly multiple) - void attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link); - void detach(const ResourcePtr & resource); - - private: - IOResourceManager * resource_manager; - std::mutex mutex; - struct Attachment - { - ResourcePtr resource; - VersionPtr version; - ResourceLink link; - }; - std::unordered_map attachments; // TSA_GUARDED_BY(mutex); - }; - - void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); - void deleteWorkload(const String & workload_name); - void createOrUpdateResource(const String & resource_name, const ASTPtr & ast); - void deleteResource(const String & resource_name); - - // Topological sorting of workloads - void topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads); - std::vector topologicallySortedWorkloads(); - - IWorkloadEntityStorage & storage; - scope_guard subscription; - - mutable std::mutex mutex; - std::unordered_map workloads; // TSA_GUARDED_BY(mutex); - std::unordered_map resources; // TSA_GUARDED_BY(mutex); - - LoggerPtr log; -}; - -} diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index cfbe242c13e..b170ab0dbee 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -19,7 +19,7 @@ namespace ErrorCodes * Scheduler node that implements priority scheduling policy. * Requests are scheduled in order of priorities. */ -class PriorityPolicy final : public ISchedulerNode +class PriorityPolicy : public ISchedulerNode { /// Scheduling state of a child struct Item @@ -39,23 +39,6 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} - explicit PriorityPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & node_info) - : ISchedulerNode(event_queue_, node_info) - {} - - ~PriorityPolicy() override - { - // We need to clear `parent` in all children to avoid dangling references - while (!children.empty()) - removeChild(children.begin()->second.get()); - } - - const String & getTypeName() const override - { - static String type_name("priority"); - return type_name; - } - bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index e223100a646..fe1b03b74bd 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -1,6 +1,5 @@ #pragma once -#include "Common/Scheduler/ISchedulerNode.h" #include #include @@ -14,7 +13,7 @@ namespace DB * Limited concurrency constraint. * Blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost` */ -class SemaphoreConstraint final : public ISchedulerConstraint +class SemaphoreConstraint : public ISchedulerConstraint { static constexpr Int64 default_max_requests = std::numeric_limits::max(); static constexpr Int64 default_max_cost = std::numeric_limits::max(); @@ -25,25 +24,6 @@ public: , max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost))) {} - SemaphoreConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, Int64 max_requests_, Int64 max_cost_) - : ISchedulerConstraint(event_queue_, info_) - , max_requests(max_requests_) - , max_cost(max_cost_) - {} - - ~SemaphoreConstraint() override - { - // We need to clear `parent` in child to avoid dangling references - if (child) - removeChild(child.get()); - } - - const String & getTypeName() const override - { - static String type_name("inflight_limit"); - return type_name; - } - bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -88,14 +68,15 @@ public: if (!request) return {nullptr, false}; - std::unique_lock lock(mutex); - if (request->addConstraint(this)) - { - // Update state on request arrival - requests++; - cost += request->cost; - } + // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. + // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. + if (!request->constraint) + request->constraint = this; + // Update state on request arrival + std::unique_lock lock(mutex); + requests++; + cost += request->cost; child_active = child_now_active; if (!active()) busy_periods++; @@ -105,6 +86,10 @@ public: void finishRequest(ResourceRequest * request) override { + // Recursive traverse of parent flow controls in reverse order + if (parent_constraint) + parent_constraint->finishRequest(request); + // Update state on request departure std::unique_lock lock(mutex); bool was_active = active(); @@ -124,32 +109,6 @@ public: parent->activateChild(this); } - /// Update limits. - /// Should be called from the scheduler thread because it could lead to activation or deactivation - void updateConstraints(const SchedulerNodePtr & self, Int64 new_max_requests, UInt64 new_max_cost) - { - std::unique_lock lock(mutex); - bool was_active = active(); - max_requests = new_max_requests; - max_cost = new_max_cost; - - if (parent) - { - // Activate on transition from inactive state - if (!was_active && active()) - parent->activateChild(this); - // Deactivate on transition into inactive state - else if (was_active && !active()) - { - // Node deactivation is usually done in dequeueRequest(), but we do not want to - // do extra call to active() on every request just to make sure there was no update(). - // There is no interface method to do deactivation, so we do the following trick. - parent->removeChild(this); - parent->attachChild(self); // This call is the only reason we have `recursive_mutex` - } - } - } - bool isActive() override { std::unique_lock lock(mutex); @@ -191,10 +150,10 @@ private: return satisfied() && child_active; } - Int64 max_requests = default_max_requests; - Int64 max_cost = default_max_cost; + const Int64 max_requests = default_max_requests; + const Int64 max_cost = default_max_cost; - std::recursive_mutex mutex; + std::mutex mutex; Int64 requests = 0; Int64 cost = 0; bool child_active = false; diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index a2594b7ff2e..b279cbe972b 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -3,6 +3,8 @@ #include #include +#include +#include #include @@ -13,7 +15,7 @@ namespace DB * Limited throughput constraint. Blocks if token-bucket constraint is violated: * i.e. more than `max_burst + duration * max_speed` cost units (aka tokens) dequeued from this node in last `duration` seconds. */ -class ThrottlerConstraint final : public ISchedulerConstraint +class ThrottlerConstraint : public ISchedulerConstraint { public: static constexpr double default_burst_seconds = 1.0; @@ -26,28 +28,10 @@ public: , tokens(max_burst) {} - ThrottlerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, double max_speed_, double max_burst_) - : ISchedulerConstraint(event_queue_, info_) - , max_speed(max_speed_) - , max_burst(max_burst_) - , last_update(event_queue_->now()) - , tokens(max_burst) - {} - ~ThrottlerConstraint() override { // We should cancel event on destruction to avoid dangling references from event queue event_queue->cancelPostponed(postponed); - - // We need to clear `parent` in child to avoid dangling reference - if (child) - removeChild(child.get()); - } - - const String & getTypeName() const override - { - static String type_name("bandwidth_limit"); - return type_name; } bool equals(ISchedulerNode * other) override @@ -94,7 +78,10 @@ public: if (!request) return {nullptr, false}; - // We don't do `request->addConstraint(this)` because `finishRequest()` is no-op + // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. + // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. + if (!request->constraint) + request->constraint = this; updateBucket(request->cost); @@ -105,8 +92,12 @@ public: return {request, active()}; } - void finishRequest(ResourceRequest *) override + void finishRequest(ResourceRequest * request) override { + // Recursive traverse of parent flow controls in reverse order + if (parent_constraint) + parent_constraint->finishRequest(request); + // NOTE: Token-bucket constraint does not require any action when consumption ends } @@ -117,21 +108,6 @@ public: parent->activateChild(this); } - /// Update limits. - /// Should be called from the scheduler thread because it could lead to activation - void updateConstraints(double new_max_speed, double new_max_burst) - { - event_queue->cancelPostponed(postponed); - postponed = EventQueue::not_postponed; - bool was_active = active(); - updateBucket(0, true); // To apply previous params for duration since `last_update` - max_speed = new_max_speed; - max_burst = new_max_burst; - updateBucket(0, false); // To postpone (if needed) using new params - if (!was_active && active() && parent) - parent->activateChild(this); - } - bool isActive() override { return active(); @@ -174,7 +150,7 @@ private: parent->activateChild(this); } - void updateBucket(ResourceCost use = 0, bool do_not_postpone = false) + void updateBucket(ResourceCost use = 0) { auto now = event_queue->now(); if (max_speed > 0.0) @@ -184,7 +160,7 @@ private: tokens -= use; // This is done outside min() to avoid passing large requests w/o token consumption after long idle period // Postpone activation until there is positive amount of tokens - if (!do_not_postpone && tokens < 0.0) + if (tokens < 0.0) { auto delay_ns = std::chrono::nanoseconds(static_cast(-tokens / max_speed * 1e9)); if (postponed == EventQueue::not_postponed) @@ -208,8 +184,8 @@ private: return satisfied() && child_active; } - double max_speed{0}; /// in tokens per second - double max_burst{0}; /// in tokens + const double max_speed{0}; /// in tokens per second + const double max_burst{0}; /// in tokens EventQueue::TimePoint last_update; UInt64 postponed = EventQueue::not_postponed; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h deleted file mode 100644 index 84923c49c62..00000000000 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ /dev/null @@ -1,606 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SCHEDULER_NODE; - extern const int LOGICAL_ERROR; -} - -class UnifiedSchedulerNode; -using UnifiedSchedulerNodePtr = std::shared_ptr; - -/* - * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. - * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes - * are also present. This approach is easiers for manipulations in runtime than using multiple types of nodes. - * - * Unified node is capable of updating its internal structure based on: - * 1. Number of children (fifo if =0 or fairness/priority if >0). - * 2. Priorities of its children (for subtree structure). - * 3. `SchedulingSettings` associated with unified node (for throttler and semaphore constraints). - * - * In general, unified node has "internal" subtree with the following structure: - * - * THIS <-- UnifiedSchedulerNode object - * | - * THROTTLER <-- [Optional] Throttling scheduling constraint - * | - * [If no children]------ SEMAPHORE <-- [Optional] Semaphore constraint - * | | - * FIFO PRIORITY <-- [Optional] Scheduling policy distinguishing priorities - * .-------' '-------. - * FAIRNESS[p1] ... FAIRNESS[pN] <-- [Optional] Policies for fairness if priorities are equal - * / \ / \ - * CHILD[p1,w1] ... CHILD[p1,wM] CHILD[pN,w1] ... CHILD[pN,wM] <-- Unified children (UnifiedSchedulerNode objects) - * - * NOTE: to distinguish different kinds of children we use the following terms: - * - immediate child: child of unified object (THROTTLER); - * - unified child: leaf of this "internal" subtree (CHILD[p,w]); - * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) - */ -class UnifiedSchedulerNode final : public ISchedulerNode -{ -private: - /// Helper function for managing a parent of a node - static void reparent(const SchedulerNodePtr & node, const SchedulerNodePtr & new_parent) - { - reparent(node, new_parent.get()); - } - - /// Helper function for managing a parent of a node - static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) - { - chassert(node); - chassert(new_parent); - if (new_parent == node->parent) - return; - if (node->parent) - node->parent->removeChild(node.get()); - new_parent->attachChild(node); - } - - /// Helper function for managing a parent of a node - static void detach(const SchedulerNodePtr & node) - { - if (node->parent) - node->parent->removeChild(node.get()); - } - - /// A branch of the tree for a specific priority value - struct FairnessBranch - { - SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached - std::unordered_map children; // basename -> child - - bool empty() const { return children.empty(); } - - SchedulerNodePtr getRoot() - { - chassert(!children.empty()); - if (root) - return root; - chassert(children.size() == 1); - return children.begin()->second; - } - - /// Attaches a new child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) - throw Exception( - ErrorCodes::INVALID_SCHEDULER_NODE, - "Can't add another child with the same path: {}", - it->second->getPath()); - - if (children.size() == 2) - { - // Insert fair node if we have just added the second child - chassert(!root); - root = std::make_shared(event_queue_, SchedulerNodeInfo{}); - root->info.setPriority(child->info.priority); - root->basename = fmt::format("p{}_fair", child->info.priority.value); - for (auto & [_, node] : children) - reparent(node, root); - return root; // New root has been created - } - else if (children.size() == 1) - return child; // We have added single child so far and it is the new root - else - reparent(child, root); - return {}; // Root is the same - } - - /// Detaches a child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - /// NOTE: It could also return null if `empty()` after detaching - [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue *, const UnifiedSchedulerNodePtr & child) - { - auto it = children.find(child->basename); - if (it == children.end()) - return {}; // unknown child - - detach(child); - children.erase(it); - if (children.size() == 1) - { - // Remove fair if the only child has left - chassert(root); - detach(root); - root.reset(); - return children.begin()->second; // The last child is a new root now - } - else if (children.empty()) - return {}; // We have detached the last child - else - return {}; // Root is the same (two or more children have left) - } - }; - - /// Handles all the children nodes with intermediate fair and/or priority nodes - struct ChildrenBranch - { - SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached - std::unordered_map branches; /// Branches for different priority values - - // Returns true iff there are no unified children attached - bool empty() const { return branches.empty(); } - - SchedulerNodePtr getRoot() - { - chassert(!branches.empty()); - if (root) - return root; - return branches.begin()->second.getRoot(); // There should be exactly one child-branch - } - - /// Attaches a new child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - auto [it, new_branch] = branches.try_emplace(child->info.priority); - auto & child_branch = it->second; - auto branch_root = child_branch.attachUnifiedChild(event_queue_, child); - if (!new_branch) - { - if (branch_root) - { - if (root) - reparent(branch_root, root); - else - return branch_root; - } - return {}; - } - else - { - chassert(branch_root); - if (branches.size() == 2) - { - // Insert priority node if we have just added the second branch - chassert(!root); - root = std::make_shared(event_queue_, SchedulerNodeInfo{}); - root->basename = "prio"; - for (auto & [_, branch] : branches) - reparent(branch.getRoot(), root); - return root; // New root has been created - } - else if (branches.size() == 1) - return child; // We have added single child so far and it is the new root - else - reparent(child, root); - return {}; // Root is the same - } - } - - /// Detaches a child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - /// NOTE: It could also return null if `empty()` after detaching - [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - auto it = branches.find(child->info.priority); - if (it == branches.end()) - return {}; // unknown child - - auto & child_branch = it->second; - auto branch_root = child_branch.detachUnifiedChild(event_queue_, child); - if (child_branch.empty()) - { - branches.erase(it); - if (branches.size() == 1) - { - // Remove priority node if the only child-branch has left - chassert(root); - detach(root); - root.reset(); - return branches.begin()->second.getRoot(); // The last child-branch is a new root now - } - else if (branches.empty()) - return {}; // We have detached the last child - else - return {}; // Root is the same (two or more children-branches have left) - } - if (branch_root) - { - if (root) - reparent(branch_root, root); - else - return branch_root; - } - return {}; // Root is the same - } - }; - - /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. - struct QueueOrChildrenBranch - { - SchedulerNodePtr queue; /// FifoQueue node is used if there are no children - ChildrenBranch branch; /// Used if there is at least one child - - SchedulerNodePtr getRoot() - { - if (queue) - return queue; - else - return branch.getRoot(); - } - - // Should be called after constructor, before any other methods - [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) - { - createQueue(event_queue_); - return queue; - } - - /// Attaches a new child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - if (queue) - removeQueue(); - return branch.attachUnifiedChild(event_queue_, child); - } - - /// Detaches a child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - if (queue) - return {}; // No-op, it already has no children - auto branch_root = branch.detachUnifiedChild(event_queue_, child); - if (branch.empty()) - { - createQueue(event_queue_); - return queue; - } - return branch_root; - } - - private: - void createQueue(EventQueue * event_queue_) - { - queue = std::make_shared(event_queue_, SchedulerNodeInfo{}); - queue->basename = "fifo"; - } - - void removeQueue() - { - // This unified node will not be able to process resource requests any longer - // All remaining resource requests are be aborted on queue destruction - detach(queue); - std::static_pointer_cast(queue)->purgeQueue(); - queue.reset(); - } - }; - - /// Handles all the nodes under this unified node - /// Specifically handles constraints with `QueueOrChildrenBranch` under it - struct ConstraintsBranch - { - SchedulerNodePtr throttler; - SchedulerNodePtr semaphore; - QueueOrChildrenBranch branch; - SchedulingSettings settings; - - // Should be called after constructor, before any other methods - [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_, const SchedulingSettings & settings_) - { - settings = settings_; - SchedulerNodePtr node = branch.initialize(event_queue_); - if (settings.hasSemaphore()) - { - semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_requests, settings.max_cost); - semaphore->basename = "semaphore"; - reparent(node, semaphore); - node = semaphore; - } - if (settings.hasThrottler()) - { - throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_speed, settings.max_burst); - throttler->basename = "throttler"; - reparent(node, throttler); - node = throttler; - } - return node; - } - - /// Attaches a new child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) - { - // If both semaphore and throttler exist we should reparent to the farthest from the root - if (semaphore) - reparent(branch_root, semaphore); - else if (throttler) - reparent(branch_root, throttler); - else - return branch_root; - } - return {}; - } - - /// Detaches a child. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) - { - if (auto branch_root = branch.detachUnifiedChild(event_queue_, child)) - { - if (semaphore) - reparent(branch_root, semaphore); - else if (throttler) - reparent(branch_root, throttler); - else - return branch_root; - } - return {}; - } - - /// Updates constraint-related nodes. - /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr updateSchedulingSettings(EventQueue * event_queue_, const SchedulingSettings & new_settings) - { - SchedulerNodePtr node = branch.getRoot(); - - if (!settings.hasSemaphore() && new_settings.hasSemaphore()) // Add semaphore - { - semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_requests, new_settings.max_cost); - semaphore->basename = "semaphore"; - reparent(node, semaphore); - node = semaphore; - } - else if (settings.hasSemaphore() && !new_settings.hasSemaphore()) // Remove semaphore - { - detach(semaphore); - semaphore.reset(); - } - else if (settings.hasSemaphore() && new_settings.hasSemaphore()) // Update semaphore - { - static_cast(*semaphore).updateConstraints(semaphore, new_settings.max_requests, new_settings.max_cost); - node = semaphore; - } - - if (!settings.hasThrottler() && new_settings.hasThrottler()) // Add throttler - { - throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_speed, new_settings.max_burst); - throttler->basename = "throttler"; - reparent(node, throttler); - node = throttler; - } - else if (settings.hasThrottler() && !new_settings.hasThrottler()) // Remove throttler - { - detach(throttler); - throttler.reset(); - } - else if (settings.hasThrottler() && new_settings.hasThrottler()) // Update throttler - { - static_cast(*throttler).updateConstraints(new_settings.max_speed, new_settings.max_burst); - node = throttler; - } - - settings = new_settings; - return node; - } - }; - -public: - explicit UnifiedSchedulerNode(EventQueue * event_queue_, const SchedulingSettings & settings) - : ISchedulerNode(event_queue_, SchedulerNodeInfo(settings.weight, settings.priority)) - { - immediate_child = impl.initialize(event_queue, settings); - reparent(immediate_child, this); - } - - ~UnifiedSchedulerNode() override - { - // We need to clear `parent` in child to avoid dangling references - if (immediate_child) - removeChild(immediate_child.get()); - } - - /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes - /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children - void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) - { - if (auto new_child = impl.attachUnifiedChild(event_queue, child)) - reparent(new_child, this); - } - - /// Detaches unified child and update all the intermediate nodes. - /// Detached child could be safely attached to another parent. - /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children - void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) - { - if (auto new_child = impl.detachUnifiedChild(event_queue, child)) - reparent(new_child, this); - } - - static bool updateRequiresDetach(const String & old_parent, const String & new_parent, const SchedulingSettings & old_settings, const SchedulingSettings & new_settings) - { - return old_parent != new_parent || old_settings.priority != new_settings.priority; - } - - /// Updates scheduling settings. Set of constraints might change. - /// NOTE: Caller is responsible for detaching and attaching if `updateRequiresDetach` returns true - void updateSchedulingSettings(const SchedulingSettings & new_settings) - { - info.setPriority(new_settings.priority); - info.setWeight(new_settings.weight); - if (auto new_child = impl.updateSchedulingSettings(event_queue, new_settings)) - reparent(new_child, this); - } - - const SchedulingSettings & getSettings() const - { - return impl.settings; - } - - /// Returns the queue to be used for resource requests or `nullptr` if it has unified children - std::shared_ptr getQueue() const - { - return static_pointer_cast(impl.branch.queue); - } - - /// Collects nodes that could be accessed with raw pointers by resource requests (queue and constraints) - /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints - /// for that queue might change in future, and `request->constraints` might reference nodes not in - /// the initial set of nodes returned by `addRawPointerNodes()`. To avoid destruction of such additional nodes - /// classifier must (indirectly) hold nodes return by `addRawPointerNodes()` for all future versions of - /// all unified nodes. Such a version control is done by `IOResourceManager`. - void addRawPointerNodes(std::vector & nodes) - { - // NOTE: `impl.throttler` could be skipped, because ThrottlerConstraint does not call `request->addConstraint()` - if (impl.semaphore) - nodes.push_back(impl.semaphore); - if (impl.branch.queue) - nodes.push_back(impl.branch.queue); - for (auto & [_, branch] : impl.branch.branch.branches) - { - for (auto & [_, child] : branch.children) - child->addRawPointerNodes(nodes); - } - } - - bool hasUnifiedChildren() const - { - return impl.branch.queue == nullptr; - } - - /// Introspection. Calls a visitor for self and every internal node. Do not recurse into unified children. - void forEachSchedulerNode(std::function visitor) - { - visitor(this); - if (impl.throttler) - visitor(impl.throttler.get()); - if (impl.semaphore) - visitor(impl.semaphore.get()); - if (impl.branch.queue) - visitor(impl.branch.queue.get()); - if (impl.branch.branch.root) // priority - visitor(impl.branch.branch.root.get()); - for (auto & [_, branch] : impl.branch.branch.branches) - { - if (branch.root) // fairness - visitor(branch.root.get()); - } - } - -protected: // Hide all the ISchedulerNode interface methods as an implementation details - const String & getTypeName() const override - { - static String type_name("unified"); - return type_name; - } - - bool equals(ISchedulerNode *) override - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "UnifiedSchedulerNode should not be used with CustomResourceManager"); - } - - /// Attaches an immediate child (used through `reparent()`) - void attachChild(const SchedulerNodePtr & child_) override - { - immediate_child = child_; - immediate_child->setParent(this); - - // Activate if required - if (immediate_child->isActive()) - activateChild(immediate_child.get()); - } - - /// Removes an immediate child (used through `reparent()`) - void removeChild(ISchedulerNode * child) override - { - if (immediate_child.get() == child) - { - child_active = false; // deactivate - immediate_child->setParent(nullptr); // detach - immediate_child.reset(); - } - } - - ISchedulerNode * getChild(const String & child_name) override - { - if (immediate_child->basename == child_name) - return immediate_child.get(); - else - return nullptr; - } - - std::pair dequeueRequest() override - { - auto [request, child_now_active] = immediate_child->dequeueRequest(); - if (!request) - return {nullptr, false}; - - child_active = child_now_active; - if (!child_active) - busy_periods++; - incrementDequeued(request->cost); - return {request, child_active}; - } - - bool isActive() override - { - return child_active; - } - - /// Shows number of immediate active children (for introspection) - size_t activeChildren() override - { - return child_active; - } - - /// Activate an immediate child - void activateChild(ISchedulerNode * child) override - { - if (child == immediate_child.get()) - if (!std::exchange(child_active, true) && parent) - parent->activateChild(this); - } - -private: - ConstraintsBranch impl; - SchedulerNodePtr immediate_child; // An immediate child (actually the root of the whole subtree) - bool child_active = false; -}; - -} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp b/src/Common/Scheduler/Nodes/registerResourceManagers.cpp new file mode 100644 index 00000000000..c5d5ba5b981 --- /dev/null +++ b/src/Common/Scheduler/Nodes/registerResourceManagers.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +void registerDynamicResourceManager(ResourceManagerFactory &); + +void registerResourceManagers() +{ + auto & factory = ResourceManagerFactory::instance(); + registerDynamicResourceManager(factory); +} + +} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.h b/src/Common/Scheduler/Nodes/registerResourceManagers.h new file mode 100644 index 00000000000..243b25a9587 --- /dev/null +++ b/src/Common/Scheduler/Nodes/registerResourceManagers.h @@ -0,0 +1,8 @@ +#pragma once + +namespace DB +{ + +void registerResourceManagers(); + +} diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 927f87d5aa6..c787a686a09 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,8 +1,5 @@ #pragma once -#include - -#include #include #include #include @@ -10,35 +7,26 @@ #include #include #include -#include #include +#include #include #include #include -#include -#include -#include #include #include #include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int RESOURCE_ACCESS_DENIED; -} - struct ResourceTestBase { ResourceTestBase() { - [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); return true; }(); + [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }(); } template @@ -49,16 +37,10 @@ struct ResourceTestBase Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)}; String config_prefix = "node"; - return add(event_queue, root_node, path, std::ref(*config), config_prefix); - } - - template - static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) - { if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); - root_node.reset(new TClass(event_queue, std::forward(args)...)); + root_node.reset(new TClass(event_queue, *config, config_prefix)); return static_cast(root_node.get()); } @@ -83,114 +65,73 @@ struct ResourceTestBase } EXPECT_TRUE(!child_name.empty()); // wrong path - SchedulerNodePtr node = std::make_shared(event_queue, std::forward(args)...); + SchedulerNodePtr node = std::make_shared(event_queue, *config, config_prefix); node->basename = child_name; parent->attachChild(node); return static_cast(node.get()); } }; + +struct ConstraintTest : public SemaphoreConstraint +{ + explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + : SemaphoreConstraint(event_queue_, config, config_prefix) + {} + + std::pair dequeueRequest() override + { + auto [request, active] = SemaphoreConstraint::dequeueRequest(); + if (request) + { + std::unique_lock lock(mutex); + requests.insert(request); + } + return {request, active}; + } + + void finishRequest(ResourceRequest * request) override + { + { + std::unique_lock lock(mutex); + requests.erase(request); + } + SemaphoreConstraint::finishRequest(request); + } + + std::mutex mutex; + std::set requests; +}; + class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest { - ResourceTestClass * test; String name; - Request(ResourceTestClass * test_, ResourceCost cost_, const String & name_) + Request(ResourceCost cost_, const String & name_) : ResourceRequest(cost_) - , test(test_) , name(name_) {} void execute() override { } - - void failed(const std::exception_ptr &) override - { - test->failed_cost += cost; - delete this; - } }; public: - ~ResourceTestClass() - { - if (root_node) - dequeue(); // Just to avoid any leaks of `Request` object - } - template void add(const String & path, const String & xml = {}) { ResourceTestBase::add(&event_queue, root_node, path, xml); } - template - void addCustom(const String & path, Args... args) - { - ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); - } - - UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const SchedulingSettings & settings = {}) - { - return createUnifiedNode(basename, {}, settings); - } - - UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const UnifiedSchedulerNodePtr & parent, const SchedulingSettings & settings = {}) - { - auto node = std::make_shared(&event_queue, settings); - node->basename = basename; - if (parent) - { - parent->attachUnifiedChild(node); - } - else - { - EXPECT_TRUE(root_node.get() == nullptr); - root_node = node; - } - return node; - } - - // Updates the parent and/or scheduling settings for a specidfied `node`. - // Unit test implementation must make sure that all needed queues and constraints are not going to be destroyed. - // Normally it is the responsibility of IOResourceManager, but we do not use it here, so manual version control is required. - // (see IOResourceManager::Resource::updateCurrentVersion() fo details) - void updateUnifiedNode(const UnifiedSchedulerNodePtr & node, const UnifiedSchedulerNodePtr & old_parent, const UnifiedSchedulerNodePtr & new_parent, const SchedulingSettings & new_settings) - { - EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported - bool detached = false; - if (UnifiedSchedulerNode::updateRequiresDetach( - old_parent ? old_parent->basename : "", - new_parent ? new_parent->basename : "", - node->getSettings(), - new_settings)) - { - if (old_parent) - old_parent->detachUnifiedChild(node); - detached = true; - } - - node->updateSchedulingSettings(new_settings); - - if (detached && new_parent) - new_parent->attachUnifiedChild(node); - } - - - void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) - { - enqueueImpl(node->getQueue().get(), costs, node->basename); - } - void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first ISchedulerNode * node = root_node.get(); size_t pos = 1; - while (node && pos < path.length()) + while (pos < path.length()) { size_t slash = path.find('/', pos); if (slash != String::npos) @@ -205,17 +146,13 @@ public: pos = String::npos; } } - if (node) - enqueueImpl(dynamic_cast(node), costs); - } - - void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs, const String & name = {}) - { + ISchedulerQueue * queue = dynamic_cast(node); ASSERT_TRUE(queue != nullptr); // not a queue - if (!queue) - return; // to make clang-analyzer-core.NonNullParamChecker happy + for (ResourceCost cost : costs) - queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name)); + { + queue->enqueueRequest(new Request(cost, queue->basename)); + } processEvents(); // to activate queues } @@ -271,12 +208,6 @@ public: consumed_cost[name] -= value; } - void failed(ResourceCost value) - { - EXPECT_EQ(failed_cost, value); - failed_cost -= value; - } - void processEvents() { while (event_queue.tryProcess()) {} @@ -286,11 +217,8 @@ private: EventQueue event_queue; SchedulerNodePtr root_node; std::unordered_map consumed_cost; - ResourceCost failed_cost = 0; }; -enum EnqueueOnlyEnum { EnqueueOnly }; - template struct ResourceTestManager : public ResourceTestBase { @@ -302,49 +230,16 @@ struct ResourceTestManager : public ResourceTestBase struct Guard : public ResourceGuard { ResourceTestManager & t; - ResourceCost cost; - /// Works like regular ResourceGuard, ready for consumption after constructor - Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) , t(t_) - , cost(cost_) { t.onEnqueue(link); - waitExecute(); - } - - /// Just enqueue resource request, do not block (needed for tests to sync). Call `waitExecuted()` afterwards - Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) - , t(t_) - , cost(cost_) - { - t.onEnqueue(link); - } - - /// Waits for ResourceRequest::execute() to be called for enqueued request - void waitExecute() - { lock(); t.onExecute(link); consume(cost); } - - /// Waits for ResourceRequest::failure() to be called for enqueued request - void waitFailed(const String & pattern) - { - try - { - lock(); - FAIL(); - } - catch (Exception & e) - { - ASSERT_EQ(e.code(), ErrorCodes::RESOURCE_ACCESS_DENIED); - ASSERT_TRUE(e.message().contains(pattern)); - } - } }; struct TItem @@ -369,24 +264,10 @@ struct ResourceTestManager : public ResourceTestBase , busy_period(thread_count) {} - enum DoNotInitManagerEnum { DoNotInitManager }; - - explicit ResourceTestManager(size_t thread_count, DoNotInitManagerEnum) - : busy_period(thread_count) - {} - ~ResourceTestManager() - { - wait(); - } - - void wait() { for (auto & thread : threads) - { - if (thread.joinable()) - thread.join(); - } + thread.join(); } void update(const String & xml) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp similarity index 82% rename from src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp rename to src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 37432128606..3328196cced 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -2,15 +2,15 @@ #include -#include +#include #include using namespace DB; -using ResourceTest = ResourceTestManager; +using ResourceTest = ResourceTestManager; using TestGuard = ResourceTest::Guard; -TEST(SchedulerCustomResourceManager, Smoke) +TEST(SchedulerDynamicResourceManager, Smoke) { ResourceTest t; @@ -31,25 +31,25 @@ TEST(SchedulerCustomResourceManager, Smoke) )CONFIG"); - ClassifierPtr c_a = t.manager->acquire("A"); - ClassifierPtr c_b = t.manager->acquire("B"); + ClassifierPtr cA = t.manager->acquire("A"); + ClassifierPtr cB = t.manager->acquire("B"); for (int i = 0; i < 10; i++) { - ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); - g_a.lock(); - g_a.consume(1); - g_a.unlock(); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); + gA.lock(); + gA.consume(1); + gA.unlock(); - ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); - g_b.unlock(); + ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); + gB.unlock(); - ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); - g_b.consume(2); + ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); + gB.consume(2); } } -TEST(SchedulerCustomResourceManager, Fairness) +TEST(SchedulerDynamicResourceManager, Fairness) { // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). // Requests from A use `value = 1` and from B `value = -1` is used. diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp index 9989215ba7b..07798f78080 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -13,12 +13,6 @@ public: , log(log_) {} - const String & getTypeName() const override - { - static String type_name("fake"); - return type_name; - } - void attachChild(const SchedulerNodePtr & child) override { log += " +" + child->basename; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp deleted file mode 100644 index 2bac69185d3..00000000000 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ /dev/null @@ -1,335 +0,0 @@ -#include - -#include -#include - -#include -#include -#include - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -using namespace DB; - -class WorkloadEntityTestStorage : public WorkloadEntityStorageBase -{ -public: - WorkloadEntityTestStorage() - : WorkloadEntityStorageBase(Context::getGlobalContextInstance()) - {} - - void loadEntities() override {} - - void executeQuery(const String & query) - { - ParserCreateWorkloadQuery create_workload_p; - ParserDropWorkloadQuery drop_workload_p; - ParserCreateResourceQuery create_resource_p; - ParserDropResourceQuery drop_resource_p; - - auto parse = [&] (IParser & parser) - { - String error; - const char * end = query.data(); - return tryParseQuery( - parser, - end, - query.data() + query.size(), - error, - false, - "", - false, - 0, - DBMS_DEFAULT_MAX_PARSER_DEPTH, - DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, - true); - }; - - if (ASTPtr create_workload = parse(create_workload_p)) - { - auto & parsed = create_workload->as(); - auto workload_name = parsed.getWorkloadName(); - bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; - bool replace_if_exists = parsed.or_replace; - - storeEntity( - nullptr, - WorkloadEntityType::Workload, - workload_name, - create_workload, - throw_if_exists, - replace_if_exists, - {}); - } - else if (ASTPtr create_resource = parse(create_resource_p)) - { - auto & parsed = create_resource->as(); - auto resource_name = parsed.getResourceName(); - bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; - bool replace_if_exists = parsed.or_replace; - - storeEntity( - nullptr, - WorkloadEntityType::Resource, - resource_name, - create_resource, - throw_if_exists, - replace_if_exists, - {}); - } - else if (ASTPtr drop_workload = parse(drop_workload_p)) - { - auto & parsed = drop_workload->as(); - bool throw_if_not_exists = !parsed.if_exists; - removeEntity( - nullptr, - WorkloadEntityType::Workload, - parsed.workload_name, - throw_if_not_exists); - } - else if (ASTPtr drop_resource = parse(drop_resource_p)) - { - auto & parsed = drop_resource->as(); - bool throw_if_not_exists = !parsed.if_exists; - removeEntity( - nullptr, - WorkloadEntityType::Resource, - parsed.resource_name, - throw_if_not_exists); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query in WorkloadEntityTestStorage: {}", query); - } - -private: - WorkloadEntityStorageBase::OperationResult storeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) override - { - UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); - return OperationResult::Ok; - } - - WorkloadEntityStorageBase::OperationResult removeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) override - { - UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); - return OperationResult::Ok; - } -}; - -struct ResourceTest : ResourceTestManager -{ - WorkloadEntityTestStorage storage; - - explicit ResourceTest(size_t thread_count = 1) - : ResourceTestManager(thread_count, DoNotInitManager) - { - manager = std::make_shared(storage); - } - - void query(const String & query_str) - { - storage.executeQuery(query_str); - } - - template - void async(const String & workload, Func func) - { - threads.emplace_back([=, this, func2 = std::move(func)] - { - ClassifierPtr classifier = manager->acquire(workload); - func2(classifier); - }); - } - - template - void async(const String & workload, const String & resource, Func func) - { - threads.emplace_back([=, this, func2 = std::move(func)] - { - ClassifierPtr classifier = manager->acquire(workload); - ResourceLink link = classifier->get(resource); - func2(link); - }); - } -}; - -using TestGuard = ResourceTest::Guard; - -TEST(SchedulerIOResourceManager, Smoke) -{ - ResourceTest t; - - t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); - t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); - t.query("CREATE WORKLOAD A in all"); - t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); - - ClassifierPtr c_a = t.manager->acquire("A"); - ClassifierPtr c_b = t.manager->acquire("B"); - - for (int i = 0; i < 10; i++) - { - ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); - g_a.lock(); - g_a.consume(1); - g_a.unlock(); - - ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); - g_b.unlock(); - - ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); - g_b.consume(2); - } -} - -TEST(SchedulerIOResourceManager, Fairness) -{ - // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). - // Requests from A use `value = 1` and from B `value = -1` is used. - std::atomic unfairness = 0; - auto fairness_diff = [&] (Int64 value) - { - Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; - EXPECT_NEAR(cur_unfairness, 0, 1); - }; - - constexpr size_t threads_per_queue = 2; - int requests_per_thread = 100; - ResourceTest t(2 * threads_per_queue + 1); - - t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); - t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); - t.query("CREATE WORKLOAD A IN all"); - t.query("CREATE WORKLOAD B IN all"); - t.query("CREATE WORKLOAD leader IN all"); - - for (int thread = 0; thread < threads_per_queue; thread++) - { - t.threads.emplace_back([&] - { - ClassifierPtr c = t.manager->acquire("A"); - ResourceLink link = c->get("res1"); - t.startBusyPeriod(link, 1, requests_per_thread); - for (int request = 0; request < requests_per_thread; request++) - { - TestGuard g(t, link, 1); - fairness_diff(1); - } - }); - } - - for (int thread = 0; thread < threads_per_queue; thread++) - { - t.threads.emplace_back([&] - { - ClassifierPtr c = t.manager->acquire("B"); - ResourceLink link = c->get("res1"); - t.startBusyPeriod(link, 1, requests_per_thread); - for (int request = 0; request < requests_per_thread; request++) - { - TestGuard g(t, link, 1); - fairness_diff(-1); - } - }); - } - - ClassifierPtr c = t.manager->acquire("leader"); - ResourceLink link = c->get("res1"); - t.blockResource(link); - - t.wait(); // Wait for threads to finish before destructing locals -} - -TEST(SchedulerIOResourceManager, DropNotEmptyQueue) -{ - ResourceTest t; - - t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); - t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); - t.query("CREATE WORKLOAD intermediate IN all"); - - std::barrier sync_before_enqueue(2); - std::barrier sync_before_drop(3); - std::barrier sync_after_drop(2); - t.async("intermediate", "res1", [&] (ResourceLink link) - { - TestGuard g(t, link, 1); - sync_before_enqueue.arrive_and_wait(); - sync_before_drop.arrive_and_wait(); // 1st resource request is consuming - sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming - }); - - sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests - - t.async("intermediate", "res1", [&] (ResourceLink link) - { - TestGuard g(t, link, 1, EnqueueOnly); - sync_before_drop.arrive_and_wait(); // 2nd resource request is enqueued - g.waitFailed("is about to be destructed"); - }); - - sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child - t.query("CREATE WORKLOAD leaf IN intermediate"); - sync_after_drop.arrive_and_wait(); - - t.wait(); // Wait for threads to finish before destructing locals -} - -TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) -{ - ResourceTest t; - - t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); - t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); - t.query("CREATE WORKLOAD intermediate IN all"); - - static constexpr int queue_size = 100; - std::barrier sync_before_enqueue(2); - std::barrier sync_before_drop(2 + queue_size); - std::barrier sync_after_drop(2); - t.async("intermediate", "res1", [&] (ResourceLink link) - { - TestGuard g(t, link, 1); - sync_before_enqueue.arrive_and_wait(); - sync_before_drop.arrive_and_wait(); // 1st resource request is consuming - sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming - }); - - sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests - - for (int i = 0; i < queue_size; i++) - { - t.async("intermediate", "res1", [&] (ResourceLink link) - { - TestGuard g(t, link, 1, EnqueueOnly); - sync_before_drop.arrive_and_wait(); // many resource requests are enqueued - g.waitFailed("is about to be destructed"); - }); - } - - sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child - t.query("CREATE WORKLOAD leaf IN intermediate"); - sync_after_drop.arrive_and_wait(); - - t.wait(); // Wait for threads to finish before destructing locals -} diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp index d859693eba5..16cce309c2a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp @@ -8,17 +8,18 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerFairPolicy, Factory) +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerFairPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - EventQueue event_queue; - SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", &event_queue, *cfg, ""); + SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", /* event_queue = */ nullptr, *cfg, ""); EXPECT_TRUE(dynamic_cast(fair.get()) != nullptr); } -TEST(SchedulerFairPolicy, FairnessWeights) +TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) { ResourceTest t; @@ -42,7 +43,7 @@ TEST(SchedulerFairPolicy, FairnessWeights) t.consumed("B", 20); } -TEST(SchedulerFairPolicy, Activation) +TEST(DISABLED_SchedulerFairPolicy, Activation) { ResourceTest t; @@ -78,7 +79,7 @@ TEST(SchedulerFairPolicy, Activation) t.consumed("B", 10); } -TEST(SchedulerFairPolicy, FairnessMaxMin) +TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) { ResourceTest t; @@ -102,7 +103,7 @@ TEST(SchedulerFairPolicy, FairnessMaxMin) t.consumed("A", 20); } -TEST(SchedulerFairPolicy, HierarchicalFairness) +TEST(DISABLED_SchedulerFairPolicy, HierarchicalFairness) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp index ab248209635..d3d38aae048 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp @@ -8,17 +8,18 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerPriorityPolicy, Factory) +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerPriorityPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - EventQueue event_queue; - SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", &event_queue, *cfg, ""); + SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", /* event_queue = */ nullptr, *cfg, ""); EXPECT_TRUE(dynamic_cast(prio.get()) != nullptr); } -TEST(SchedulerPriorityPolicy, Priorities) +TEST(DISABLED_SchedulerPriorityPolicy, Priorities) { ResourceTest t; @@ -52,7 +53,7 @@ TEST(SchedulerPriorityPolicy, Priorities) t.consumed("C", 0); } -TEST(SchedulerPriorityPolicy, Activation) +TEST(DISABLED_SchedulerPriorityPolicy, Activation) { ResourceTest t; @@ -93,7 +94,7 @@ TEST(SchedulerPriorityPolicy, Activation) t.consumed("C", 0); } -TEST(SchedulerPriorityPolicy, SinglePriority) +TEST(DISABLED_SchedulerPriorityPolicy, SinglePriority) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index 85d35fab0a6..ddfe0cfbc6f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,6 +1,5 @@ #include -#include #include #include @@ -102,11 +101,6 @@ struct MyRequest : public ResourceRequest if (on_execute) on_execute(); } - - void failed(const std::exception_ptr &) override - { - FAIL(); - } }; TEST(SchedulerRoot, Smoke) @@ -114,14 +108,14 @@ TEST(SchedulerRoot, Smoke) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); r1.registerResource(); ResourceHolder r2(t); - auto * fc2 = r2.add("/", "1"); + auto * fc2 = r2.add("/", "1"); r2.add("/prio"); auto c = r2.addQueue("/prio/C", "-1"); auto d = r2.addQueue("/prio/D", "-2"); @@ -129,25 +123,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); - EXPECT_TRUE(fc1->getInflights().first == 1); + EXPECT_TRUE(fc1->requests.contains(&rg.request)); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); - EXPECT_TRUE(fc1->getInflights().first == 1); + EXPECT_TRUE(fc1->requests.contains(&rg.request)); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); - EXPECT_TRUE(fc2->getInflights().first == 1); + EXPECT_TRUE(fc2->requests.contains(&rg.request)); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); - EXPECT_TRUE(fc2->getInflights().first == 1); + EXPECT_TRUE(fc2->requests.contains(&rg.request)); rg.consume(1); } } @@ -157,7 +151,7 @@ TEST(SchedulerRoot, Budget) ResourceTest t; ResourceHolder r1(t); - r1.add("/", "1"); + r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", ""); r1.registerResource(); @@ -182,7 +176,7 @@ TEST(SchedulerRoot, Cancel) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); @@ -195,7 +189,7 @@ TEST(SchedulerRoot, Cancel) MyRequest request(1,[&] { sync.arrive_and_wait(); // (A) - EXPECT_TRUE(fc1->getInflights().first == 1); + EXPECT_TRUE(fc1->requests.contains(&request)); sync.arrive_and_wait(); // (B) request.finish(); destruct_sync.arrive_and_wait(); // (C) @@ -220,5 +214,5 @@ TEST(SchedulerRoot, Cancel) consumer1.join(); consumer2.join(); - EXPECT_TRUE(fc1->getInflights().first == 0); + EXPECT_TRUE(fc1->requests.empty()); } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 585bb738b27..2bc24cdb292 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -10,7 +10,9 @@ using namespace DB; using ResourceTest = ResourceTestClass; -TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) +/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 + +TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -40,7 +42,7 @@ TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) t.consumed("A", 10); } -TEST(SchedulerThrottlerConstraint, Unlimited) +TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -57,7 +59,7 @@ TEST(SchedulerThrottlerConstraint, Unlimited) } } -TEST(SchedulerThrottlerConstraint, Pacing) +TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -77,7 +79,7 @@ TEST(SchedulerThrottlerConstraint, Pacing) } } -TEST(SchedulerThrottlerConstraint, BucketFilling) +TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -111,7 +113,7 @@ TEST(SchedulerThrottlerConstraint, BucketFilling) t.consumed("A", 3); } -TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) +TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -139,7 +141,7 @@ TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) } } -TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness) +TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -158,22 +160,22 @@ TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness) t.enqueue("/fair/B", {req_cost}); } - double share_a = 0.1; - double share_b = 0.9; + double shareA = 0.1; + double shareB = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); - auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); + auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); + auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); - double consumed_a = 0; - double consumed_b = 0; + double consumedA = 0; + double consumedB = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); - t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); - consumed_a = arrival_curve * share_a; - consumed_b = arrival_curve * share_b; + t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); + t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); + consumedA = arrival_curve * shareA; + consumedB = arrival_curve * shareB; } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp deleted file mode 100644 index b5bcc07f71a..00000000000 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ /dev/null @@ -1,748 +0,0 @@ -#include -#include - -#include -#include -#include - -#include -#include -#include - -using namespace DB; - -using ResourceTest = ResourceTestClass; - -TEST(SchedulerUnifiedNode, Smoke) -{ - ResourceTest t; - - t.addCustom("/", SchedulingSettings{}); - - t.enqueue("/fifo", {10, 10}); - t.dequeue(2); - t.consumed("fifo", 20); -} - -TEST(SchedulerUnifiedNode, FairnessWeight) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); - auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(4); - t.consumed("A", 10); - t.consumed("B", 30); - - t.dequeue(4); - t.consumed("A", 10); - t.consumed("B", 30); - - t.dequeue(); - t.consumed("A", 60); - t.consumed("B", 20); -} - -TEST(SchedulerUnifiedNode, FairnessActivation) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all); - auto b = t.createUnifiedNode("B", all); - auto c = t.createUnifiedNode("C", all); - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10}); - t.enqueue(c, {10, 10}); - - t.dequeue(3); - t.consumed("A", 10); - t.consumed("B", 10); - t.consumed("C", 10); - - t.dequeue(4); - t.consumed("A", 30); - t.consumed("B", 0); - t.consumed("C", 10); - - t.enqueue(b, {10, 10}); - t.dequeue(1); - t.consumed("B", 10); - - t.enqueue(c, {10, 10}); - t.dequeue(1); - t.consumed("C", 10); - - t.dequeue(2); // A B or B A - t.consumed("A", 10); - t.consumed("B", 10); -} - -TEST(SchedulerUnifiedNode, FairnessMaxMin) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all); - auto b = t.createUnifiedNode("B", all); - - t.enqueue(a, {10, 10}); // make sure A is never empty - - for (int i = 0; i < 10; i++) - { - t.enqueue(a, {10, 10, 10, 10}); - t.enqueue(b, {10, 10}); - - t.dequeue(6); - t.consumed("A", 40); - t.consumed("B", 20); - } - - t.dequeue(2); - t.consumed("A", 20); -} - -TEST(SchedulerUnifiedNode, FairnessHierarchical) -{ - ResourceTest t; - - - auto all = t.createUnifiedNode("all"); - auto x = t.createUnifiedNode("X", all); - auto y = t.createUnifiedNode("Y", all); - auto a = t.createUnifiedNode("A", x); - auto b = t.createUnifiedNode("B", x); - auto c = t.createUnifiedNode("C", y); - auto d = t.createUnifiedNode("D", y); - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("A", 20); - t.consumed("B", 20); - t.consumed("C", 20); - t.consumed("D", 20); - } - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("A", 40); - t.consumed("C", 20); - t.consumed("D", 20); - } - - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("B", 40); - t.consumed("C", 20); - t.consumed("D", 20); - } - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("A", 20); - t.consumed("B", 20); - t.consumed("C", 40); - } - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("A", 20); - t.consumed("B", 20); - t.consumed("D", 40); - } - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); - for (int i = 0; i < 4; i++) - { - t.dequeue(8); - t.consumed("A", 40); - t.consumed("D", 40); - } -} - -TEST(SchedulerUnifiedNode, Priority) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); - auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); - auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); - - t.enqueue(a, {10, 10, 10}); - t.enqueue(b, {10, 10, 10}); - t.enqueue(c, {10, 10, 10}); - - t.dequeue(2); - t.consumed("A", 0); - t.consumed("B", 0); - t.consumed("C", 20); - - t.dequeue(2); - t.consumed("A", 0); - t.consumed("B", 10); - t.consumed("C", 10); - - t.dequeue(2); - t.consumed("A", 0); - t.consumed("B", 20); - t.consumed("C", 0); - - t.dequeue(); - t.consumed("A", 30); - t.consumed("B", 0); - t.consumed("C", 0); -} - -TEST(SchedulerUnifiedNode, PriorityActivation) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); - auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); - auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); - - t.enqueue(a, {10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10}); - t.enqueue(c, {10, 10}); - - t.dequeue(3); - t.consumed("A", 0); - t.consumed("B", 10); - t.consumed("C", 20); - - t.dequeue(2); - t.consumed("A", 20); - t.consumed("B", 0); - t.consumed("C", 0); - - t.enqueue(b, {10, 10, 10}); - t.dequeue(2); - t.consumed("A", 0); - t.consumed("B", 20); - t.consumed("C", 0); - - t.enqueue(c, {10, 10}); - t.dequeue(3); - t.consumed("A", 0); - t.consumed("B", 10); - t.consumed("C", 20); - - t.dequeue(2); - t.consumed("A", 20); - t.consumed("B", 0); - t.consumed("C", 0); -} - -TEST(SchedulerUnifiedNode, List) -{ - ResourceTest t; - - std::list list; - list.push_back(t.createUnifiedNode("all")); - - for (int length = 1; length < 5; length++) - { - String name = fmt::format("L{}", length); - list.push_back(t.createUnifiedNode(name, list.back())); - - for (int i = 0; i < 3; i++) - { - t.enqueue(list.back(), {10, 10}); - t.dequeue(1); - t.consumed(name, 10); - - for (int j = 0; j < 3; j++) - { - t.enqueue(list.back(), {10, 10, 10}); - t.dequeue(1); - t.consumed(name, 10); - t.dequeue(1); - t.consumed(name, 10); - t.dequeue(1); - t.consumed(name, 10); - } - - t.dequeue(1); - t.consumed(name, 10); - } - } -} - -TEST(SchedulerUnifiedNode, ThrottlerLeakyBucket) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); - - t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.process(start + std::chrono::seconds(0)); - t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request - - t.process(start + std::chrono::seconds(1)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(2)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(3)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(4)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(100500)); - t.consumed("all", 10); -} - -TEST(SchedulerUnifiedNode, ThrottlerPacing) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - // Zero burst allows you to send one request of any `size` and than throttle for `size/max_speed` seconds. - // Useful if outgoing traffic should be "paced", i.e. have the least possible burstiness. - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 1.0, .max_burst = 0.0}); - - t.enqueue(all, {1, 2, 3, 1, 2, 1}); - int output[] = {1, 2, 0, 3, 0, 0, 1, 2, 0, 1, 0}; - for (int i = 0; i < std::size(output); i++) - { - t.process(start + std::chrono::seconds(i)); - t.consumed("all", output[i]); - } -} - -TEST(SchedulerUnifiedNode, ThrottlerBucketFilling) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); - - t.enqueue(all, {100}); - - t.process(start + std::chrono::seconds(0)); - t.consumed("all", 100); // consume all tokens, but it is still active (not negative) - - t.process(start + std::chrono::seconds(5)); - t.consumed("all", 0); // There was nothing to consume - - t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); - t.process(start + std::chrono::seconds(5)); - t.consumed("all", 60); // 5 sec * 10 tokens/sec = 50 tokens + 1 extra request to go below zero - - t.process(start + std::chrono::seconds(100)); - t.consumed("all", 40); // Consume rest - - t.process(start + std::chrono::seconds(200)); - - t.enqueue(all, {95, 1, 1, 1, 1, 1, 1, 1, 1, 1}); - t.process(start + std::chrono::seconds(200)); - t.consumed("all", 101); // check we cannot consume more than max_burst + 1 request - - t.process(start + std::chrono::seconds(100500)); - t.consumed("all", 3); -} - -TEST(SchedulerUnifiedNode, ThrottlerAndFairness) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); - auto a = t.createUnifiedNode("A", all, {.weight = 10.0, .priority = Priority{}}); - auto b = t.createUnifiedNode("B", all, {.weight = 90.0, .priority = Priority{}}); - - ResourceCost req_cost = 1; - ResourceCost total_cost = 2000; - for (int i = 0; i < total_cost / req_cost; i++) - { - t.enqueue(a, {req_cost}); - t.enqueue(b, {req_cost}); - } - - double share_a = 0.1; - double share_b = 0.9; - - // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); - auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - - double consumed_a = 0; - double consumed_b = 0; - for (int seconds = 0; seconds < 100; seconds++) - { - t.process(start + std::chrono::seconds(seconds)); - double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); - t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); - consumed_a = arrival_curve * share_a; - consumed_b = arrival_curve * share_b; - } -} - -TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - - t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled - - // This will destroy the queue and fail both requests - auto a = t.createUnifiedNode("A", all); - t.failed(20); - - // Check that everything works fine after destruction - auto b = t.createUnifiedNode("B", all); - t.enqueue(a, {10, 10}); // make sure A is never empty - for (int i = 0; i < 10; i++) - { - t.enqueue(a, {10, 10, 10, 10}); - t.enqueue(b, {10, 10}); - - t.dequeue(6); - t.consumed("A", 40); - t.consumed("B", 20); - } - t.dequeue(2); - t.consumed("A", 20); -} - -TEST(SchedulerUnifiedNode, ResourceGuardException) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - - t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled - - std::thread consumer([queue = all->getQueue()] - { - ResourceLink link{.queue = queue.get()}; - bool caught = false; - try - { - ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link); - } - catch (...) - { - caught = true; - } - ASSERT_TRUE(caught); - }); - - // This will destroy the queue and fail both requests - auto a = t.createUnifiedNode("A", all); - t.failed(20); - consumer.join(); - - // Check that everything works fine after destruction - auto b = t.createUnifiedNode("B", all); - t.enqueue(a, {10, 10}); // make sure A is never empty - for (int i = 0; i < 10; i++) - { - t.enqueue(a, {10, 10, 10, 10}); - t.enqueue(b, {10, 10}); - - t.dequeue(6); - t.consumed("A", 40); - t.consumed("B", 20); - } - t.dequeue(2); - t.consumed("A", 20); -} - -TEST(SchedulerUnifiedNode, UpdateWeight) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); - auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(4); - t.consumed("A", 10); - t.consumed("B", 30); - - t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{}}); - - t.dequeue(4); - t.consumed("A", 20); - t.consumed("B", 20); - - t.dequeue(4); - t.consumed("A", 20); - t.consumed("B", 20); -} - -TEST(SchedulerUnifiedNode, UpdatePriority) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); - auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{}}); - - t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(2); - t.consumed("A", 10); - t.consumed("B", 10); - - t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-1}}); - - t.dequeue(2); - t.consumed("A", 20); - t.consumed("B", 0); - - t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{-2}}); - - t.dequeue(2); - t.consumed("A", 0); - t.consumed("B", 20); - - t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-2}}); - - t.dequeue(2); - t.consumed("A", 10); - t.consumed("B", 10); -} - -TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); - auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); - auto x = t.createUnifiedNode("X", a, {}); - auto y = t.createUnifiedNode("Y", b, {}); - - t.enqueue(x, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(y, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(2); - t.consumed("X", 20); - t.consumed("Y", 0); - - t.updateUnifiedNode(x, a, b, {}); - - t.dequeue(2); - t.consumed("X", 10); - t.consumed("Y", 10); - - t.updateUnifiedNode(y, b, a, {}); - - t.dequeue(2); - t.consumed("X", 0); - t.consumed("Y", 20); - - t.updateUnifiedNode(y, a, all, {}); - t.updateUnifiedNode(x, b, all, {}); - - t.dequeue(4); - t.consumed("X", 20); - t.consumed("Y", 20); -} - -TEST(SchedulerUnifiedNode, UpdatePriorityOfIntermediateNode) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); - auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); - auto x1 = t.createUnifiedNode("X1", a, {}); - auto y1 = t.createUnifiedNode("Y1", b, {}); - auto x2 = t.createUnifiedNode("X2", a, {}); - auto y2 = t.createUnifiedNode("Y2", b, {}); - - t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(4); - t.consumed("X1", 20); - t.consumed("Y1", 0); - t.consumed("X2", 20); - t.consumed("Y2", 0); - - t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{2}}); - - t.dequeue(4); - t.consumed("X1", 10); - t.consumed("Y1", 10); - t.consumed("X2", 10); - t.consumed("Y2", 10); - - t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{1}}); - - t.dequeue(4); - t.consumed("X1", 0); - t.consumed("Y1", 20); - t.consumed("X2", 0); - t.consumed("Y2", 20); -} - -TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) -{ - ResourceTest t; - - auto all = t.createUnifiedNode("all"); - auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); - auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); - auto c = t.createUnifiedNode("C", a, {}); - auto d = t.createUnifiedNode("D", b, {}); - auto x1 = t.createUnifiedNode("X1", c, {}); - auto y1 = t.createUnifiedNode("Y1", d, {}); - auto x2 = t.createUnifiedNode("X2", c, {}); - auto y2 = t.createUnifiedNode("Y2", d, {}); - - t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); - t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.dequeue(4); - t.consumed("X1", 20); - t.consumed("Y1", 0); - t.consumed("X2", 20); - t.consumed("Y2", 0); - - t.updateUnifiedNode(c, a, b, {}); - - t.dequeue(4); - t.consumed("X1", 10); - t.consumed("Y1", 10); - t.consumed("X2", 10); - t.consumed("Y2", 10); - - t.updateUnifiedNode(d, b, a, {}); - - t.dequeue(4); - t.consumed("X1", 0); - t.consumed("Y1", 20); - t.consumed("X2", 0); - t.consumed("Y2", 20); -} - -TEST(SchedulerUnifiedNode, UpdateThrottlerMaxSpeed) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); - - t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); - - t.process(start + std::chrono::seconds(0)); - t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request - - t.process(start + std::chrono::seconds(1)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(2)); - t.consumed("all", 10); - - t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 1.0, .max_burst = 20.0}); - - t.process(start + std::chrono::seconds(12)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(22)); - t.consumed("all", 10); - - t.process(start + std::chrono::seconds(100500)); - t.consumed("all", 10); -} - -TEST(SchedulerUnifiedNode, UpdateThrottlerMaxBurst) -{ - ResourceTest t; - EventQueue::TimePoint start = std::chrono::system_clock::now(); - t.process(start, 0); - - auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); - - t.enqueue(all, {100}); - - t.process(start + std::chrono::seconds(0)); - t.consumed("all", 100); // consume all tokens, but it is still active (not negative) - - t.process(start + std::chrono::seconds(2)); - t.consumed("all", 0); // There was nothing to consume - t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 30.0}); - - t.process(start + std::chrono::seconds(5)); - t.consumed("all", 0); // There was nothing to consume - - t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); - t.process(start + std::chrono::seconds(5)); - t.consumed("all", 40); // min(30 tokens, 5 sec * 10 tokens/sec) = 30 tokens + 1 extra request to go below zero - - t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); - - t.process(start + std::chrono::seconds(100)); - t.consumed("all", 60); // Consume rest - - t.process(start + std::chrono::seconds(150)); - t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 100.0, .max_burst = 200.0}); - - t.process(start + std::chrono::seconds(200)); - - t.enqueue(all, {195, 1, 1, 1, 1, 1, 1, 1, 1, 1}); - t.process(start + std::chrono::seconds(200)); - t.consumed("all", 201); // check we cannot consume more than max_burst + 1 request - - t.process(start + std::chrono::seconds(100500)); - t.consumed("all", 3); -} diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index ba3532598af..cf97f7acf93 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -12,7 +12,6 @@ #include #include -#include #include @@ -35,11 +34,6 @@ namespace CurrentMetrics namespace DB { -namespace ErrorCodes -{ - extern const int RESOURCE_ACCESS_DENIED; -} - /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor @@ -115,25 +109,12 @@ public: dequeued_cv.notify_one(); } - // This function is executed inside scheduler thread and wakes thread that issued this `request`. - // That thread will throw an exception. - void failed(const std::exception_ptr & ptr) override - { - std::unique_lock lock(mutex); - chassert(state == Enqueued); - state = Dequeued; - exception = ptr; - dequeued_cv.notify_one(); - } - void wait() { CurrentMetrics::Increment scheduled(metrics->scheduled_count); auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); - if (exception) - throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Resource request failed: {}", getExceptionMessage(exception, /* with_stacktrace = */ false)); } void finish(ResourceCost real_cost_, ResourceLink link_) @@ -170,7 +151,6 @@ public: std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; - std::exception_ptr exception; }; /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) diff --git a/src/Common/Scheduler/ResourceManagerFactory.h b/src/Common/Scheduler/ResourceManagerFactory.h new file mode 100644 index 00000000000..52f271e51b1 --- /dev/null +++ b/src/Common/Scheduler/ResourceManagerFactory.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + +#include + +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_SCHEDULER_NODE; +} + +class ResourceManagerFactory : private boost::noncopyable +{ +public: + static ResourceManagerFactory & instance() + { + static ResourceManagerFactory ret; + return ret; + } + + ResourceManagerPtr get(const String & name) + { + std::lock_guard lock{mutex}; + if (auto iter = methods.find(name); iter != methods.end()) + return iter->second(); + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name); + } + + template + void registerMethod(const String & name) + { + std::lock_guard lock{mutex}; + methods[name] = [] () + { + return std::make_shared(); + }; + } + +private: + std::mutex mutex; + using Method = std::function; + std::unordered_map methods; +}; + +} diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 674c7650adf..26e8084cdfa 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -1,34 +1,13 @@ #include #include -#include - -#include - namespace DB { void ResourceRequest::finish() { - // Iterate over constraints in reverse order - for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints)) - { - if (constraint) - constraint->finishRequest(this); - } -} - -bool ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) -{ - for (auto & constraint : constraints) - { - if (!constraint) - { - constraint = new_constraint; - return true; - } - } - return false; + if (constraint) + constraint->finishRequest(this); } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index bb9bfbfc8fd..7b6a5af0fe6 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -2,9 +2,7 @@ #include #include -#include #include -#include namespace DB { @@ -17,9 +15,6 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -/// Max number of constraints for a request to pass though (depth of constraints chain) -constexpr size_t ResourceMaxConstraints = 8; - /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -44,7 +39,8 @@ constexpr size_t ResourceMaxConstraints = 8; * * Request can also be canceled before (3) using ISchedulerQueue::cancelRequest(). * Returning false means it is too late for request to be canceled. It should be processed in a regular way. - * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen. + * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen + * and step (6) MUST be omitted. */ class ResourceRequest : public boost::intrusive::list_base_hook<> { @@ -53,10 +49,9 @@ public: /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; - /// Scheduler nodes to be notified on consumption finish - /// Auto-filled during request dequeue - /// Vector is not used to avoid allocations in the scheduler thread - std::array constraints; + /// Scheduler node to be notified on consumption finish + /// Auto-filled during request enqueue/dequeue + ISchedulerConstraint * constraint; explicit ResourceRequest(ResourceCost cost_ = 1) { @@ -67,8 +62,7 @@ public: void reset(ResourceCost cost_) { cost = cost_; - for (auto & constraint : constraints) - constraint = nullptr; + constraint = nullptr; // Note that list_base_hook should be reset independently (by intrusive list) } @@ -80,18 +74,11 @@ public: /// (e.g. setting an std::promise or creating a job in a thread pool) virtual void execute() = 0; - /// Callback to trigger an error in case if resource is unavailable. - virtual void failed(const std::exception_ptr & ptr) = 0; - /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// It is okay to call finish() even for failed and canceled requests (it will be no-op) + /// WARNING: this function MUST not be called if request was canceled. void finish(); - - /// Is called from the scheduler thread to fill `constraints` chain - /// Returns `true` iff constraint was added successfully - bool addConstraint(ISchedulerConstraint * new_constraint); }; } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 451f29f33f2..6a3c3962eb1 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -28,27 +28,27 @@ namespace ErrorCodes * Resource scheduler root node with a dedicated thread. * Immediate children correspond to different resources. */ -class SchedulerRoot final : public ISchedulerNode +class SchedulerRoot : public ISchedulerNode { private: - struct Resource + struct TResource { SchedulerNodePtr root; // Intrusive cyclic list of active resources - Resource * next = nullptr; - Resource * prev = nullptr; + TResource * next = nullptr; + TResource * prev = nullptr; - explicit Resource(const SchedulerNodePtr & root_) + explicit TResource(const SchedulerNodePtr & root_) : root(root_) { root->info.parent.ptr = this; } // Get pointer stored by ctor in info - static Resource * get(SchedulerNodeInfo & info) + static TResource * get(SchedulerNodeInfo & info) { - return reinterpret_cast(info.parent.ptr); + return reinterpret_cast(info.parent.ptr); } }; @@ -60,8 +60,6 @@ public: ~SchedulerRoot() override { stop(); - while (!children.empty()) - removeChild(children.begin()->first); } /// Runs separate scheduler thread @@ -97,12 +95,6 @@ public: } } - const String & getTypeName() const override - { - static String type_name("scheduler"); - return type_name; - } - bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -187,11 +179,16 @@ public: void activateChild(ISchedulerNode * child) override { - activate(Resource::get(child->info)); + activate(TResource::get(child->info)); + } + + void setParent(ISchedulerNode *) override + { + abort(); // scheduler must be the root and this function should not be called } private: - void activate(Resource * value) + void activate(TResource * value) { assert(value->next == nullptr && value->prev == nullptr); if (current == nullptr) // No active children @@ -209,7 +206,7 @@ private: } } - void deactivate(Resource * value) + void deactivate(TResource * value) { if (value->next == nullptr) return; // Already deactivated @@ -254,8 +251,8 @@ private: request->execute(); } - Resource * current = nullptr; // round-robin pointer - std::unordered_map children; // resources by pointer + TResource * current = nullptr; // round-robin pointer + std::unordered_map children; // resources by pointer std::atomic stop_flag = false; EventQueue events; ThreadFromGlobalPool scheduler; diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp deleted file mode 100644 index 60319cdd54c..00000000000 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ /dev/null @@ -1,130 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) -{ - struct { - std::optional new_weight; - std::optional new_priority; - std::optional new_max_speed; - std::optional new_max_burst; - std::optional new_max_requests; - std::optional new_max_cost; - - static Float64 getNotNegativeFloat64(const String & name, const Field & field) - { - { - UInt64 val; - if (field.tryGet(val)) - return static_cast(val); // We dont mind slight loss of precision - } - - { - Int64 val; - if (field.tryGet(val)) - { - if (val < 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); - return static_cast(val); // We dont mind slight loss of precision - } - } - - return field.safeGet(); - } - - static Int64 getNotNegativeInt64(const String & name, const Field & field) - { - { - UInt64 val; - if (field.tryGet(val)) - { - // Saturate on overflow - if (val > static_cast(std::numeric_limits::max())) - val = std::numeric_limits::max(); - return static_cast(val); - } - } - - { - Int64 val; - if (field.tryGet(val)) - { - if (val < 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); - return val; - } - } - - return field.safeGet(); - } - - void read(const String & name, const Field & value) - { - if (name == "weight") - new_weight = getNotNegativeFloat64(name, value); - else if (name == "priority") - new_priority = Priority{value.safeGet()}; - else if (name == "max_speed") - new_max_speed = getNotNegativeFloat64(name, value); - else if (name == "max_burst") - new_max_burst = getNotNegativeFloat64(name, value); - else if (name == "max_requests") - new_max_requests = getNotNegativeInt64(name, value); - else if (name == "max_cost") - new_max_cost = getNotNegativeInt64(name, value); - } - } regular, specific; - - // Read changed setting values - for (const auto & [name, value, resource] : changes) - { - if (resource.empty()) - regular.read(name, value); - else if (resource == resource_name) - specific.read(name, value); - } - - auto get_value = [] (const std::optional & specific_new, const std::optional & regular_new, T & old) - { - if (specific_new) - return *specific_new; - if (regular_new) - return *regular_new; - return old; - }; - - // Validate that we could use values read in a scheduler node - { - SchedulerNodeInfo validating_node( - get_value(specific.new_weight, regular.new_weight, weight), - get_value(specific.new_priority, regular.new_priority, priority)); - } - - // Commit new values. - // Previous values are left intentionally for ALTER query to be able to skip not mentioned setting values - weight = get_value(specific.new_weight, regular.new_weight, weight); - priority = get_value(specific.new_priority, regular.new_priority, priority); - if (specific.new_max_speed || regular.new_max_speed) - { - max_speed = get_value(specific.new_max_speed, regular.new_max_speed, max_speed); - // We always set max_burst if max_speed is changed. - // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed - max_burst = default_burst_seconds * max_speed; - } - max_burst = get_value(specific.new_max_burst, regular.new_max_burst, max_burst); - max_requests = get_value(specific.new_max_requests, regular.new_max_requests, max_requests); - max_cost = get_value(specific.new_max_cost, regular.new_max_cost, max_cost); -} - -} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h deleted file mode 100644 index 6db3ef0dce9..00000000000 --- a/src/Common/Scheduler/SchedulingSettings.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include - -#include -#include - -#include - -namespace DB -{ - -struct SchedulingSettings -{ - /// Priority and weight among siblings - Float64 weight = 1.0; - Priority priority; - - /// Throttling constraints. - /// Up to 2 independent throttlers: one for average speed and one for peek speed. - static constexpr Float64 default_burst_seconds = 1.0; - Float64 max_speed = 0; // Zero means unlimited - Float64 max_burst = 0; // default is `default_burst_seconds * max_speed` - - /// Limits total number of concurrent resource requests that are allowed to consume - static constexpr Int64 default_max_requests = std::numeric_limits::max(); - Int64 max_requests = default_max_requests; - - /// Limits total cost of concurrent resource requests that are allowed to consume - static constexpr Int64 default_max_cost = std::numeric_limits::max(); - Int64 max_cost = default_max_cost; - - bool hasThrottler() const { return max_speed != 0; } - bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - - void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name = {}); -}; - -} diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h deleted file mode 100644 index adb3a808eea..00000000000 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ /dev/null @@ -1,91 +0,0 @@ -#pragma once - -#include -#include - -#include - -#include - - -namespace DB -{ - -class IAST; -struct Settings; - -enum class WorkloadEntityType : uint8_t -{ - Workload, - Resource, - - MAX -}; - -/// Interface for a storage of workload entities (WORKLOAD and RESOURCE). -class IWorkloadEntityStorage -{ -public: - virtual ~IWorkloadEntityStorage() = default; - - /// Whether this storage can replicate entities to another node. - virtual bool isReplicated() const { return false; } - virtual String getReplicationID() const { return ""; } - - /// Loads all entities. Can be called once - if entities are already loaded the function does nothing. - virtual void loadEntities() = 0; - - /// Get entity by name. If no entity stored with entity_name throws exception. - virtual ASTPtr get(const String & entity_name) const = 0; - - /// Get entity by name. If no entity stored with entity_name return nullptr. - virtual ASTPtr tryGet(const String & entity_name) const = 0; - - /// Check if entity with entity_name is stored. - virtual bool has(const String & entity_name) const = 0; - - /// Get all entity names. - virtual std::vector getAllEntityNames() const = 0; - - /// Get all entity names of specified type. - virtual std::vector getAllEntityNames(WorkloadEntityType entity_type) const = 0; - - /// Get all entities. - virtual std::vector> getAllEntities() const = 0; - - /// Check whether any entity have been stored. - virtual bool empty() const = 0; - - /// Stops watching. - virtual void stopWatching() {} - - /// Stores an entity. - virtual bool storeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) = 0; - - /// Removes an entity. - virtual bool removeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) = 0; - - struct Event - { - WorkloadEntityType type; - String name; - ASTPtr entity; /// new or changed entity, null if removed - }; - using OnChangedHandler = std::function &)>; - - /// Gets all current entries, pass them through `handler` and subscribes for all later changes. - virtual scope_guard getAllEntitiesAndSubscribe(const OnChangedHandler & handler) = 0; -}; - -} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp deleted file mode 100644 index 1bff672c150..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ /dev/null @@ -1,287 +0,0 @@ -#include - -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include - -#include - -#include -#include -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - - -namespace DB -{ - -namespace Setting -{ - extern const SettingsUInt64 max_parser_backtracks; - extern const SettingsUInt64 max_parser_depth; - extern const SettingsBool fsync_metadata; -} - -namespace ErrorCodes -{ - extern const int DIRECTORY_DOESNT_EXIST; - extern const int BAD_ARGUMENTS; -} - - -namespace -{ - constexpr std::string_view workload_prefix = "workload_"; - constexpr std::string_view resource_prefix = "resource_"; - constexpr std::string_view sql_suffix = ".sql"; - - /// Converts a path to an absolute path and append it with a separator. - String makeDirectoryPathCanonical(const String & directory_path) - { - auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path); - if (canonical_directory_path.has_filename()) - canonical_directory_path += std::filesystem::path::preferred_separator; - return canonical_directory_path; - } -} - -WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) - : WorkloadEntityStorageBase(global_context_) - , dir_path{makeDirectoryPathCanonical(dir_path_)} -{ - log = getLogger("WorkloadEntityDiskStorage"); -} - - -ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name) -{ - return tryLoadEntity(entity_type, entity_name, getFilePath(entity_type, entity_name), /* check_file_exists= */ true); -} - - -ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & path, bool check_file_exists) -{ - LOG_DEBUG(log, "Loading workload entity {} from file {}", backQuote(entity_name), path); - - try - { - if (check_file_exists && !fs::exists(path)) - return nullptr; - - /// There is .sql file with workload entity creation statement. - ReadBufferFromFile in(path); - - String entity_create_query; - readStringUntilEOF(entity_create_query, in); - - auto parse = [&] (auto parser) - { - return parseQuery( - parser, - entity_create_query.data(), - entity_create_query.data() + entity_create_query.size(), - "", - 0, - global_context->getSettingsRef()[Setting::max_parser_depth], - global_context->getSettingsRef()[Setting::max_parser_backtracks]); - }; - - switch (entity_type) - { - case WorkloadEntityType::Workload: return parse(ParserCreateWorkloadQuery()); - case WorkloadEntityType::Resource: return parse(ParserCreateResourceQuery()); - case WorkloadEntityType::MAX: return nullptr; - } - } - catch (...) - { - tryLogCurrentException(log, fmt::format("while loading workload entity {} from path {}", backQuote(entity_name), path)); - return nullptr; /// Failed to load this entity, will ignore it - } -} - - -void WorkloadEntityDiskStorage::loadEntities() -{ - if (!entities_loaded) - loadEntitiesImpl(); -} - - -void WorkloadEntityDiskStorage::loadEntitiesImpl() -{ - LOG_INFO(log, "Loading workload entities from {}", dir_path); - - if (!std::filesystem::exists(dir_path)) - { - LOG_DEBUG(log, "The directory for workload entities ({}) does not exist: nothing to load", dir_path); - return; - } - - std::vector> entities_name_and_queries; - - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) - { - if (it->isDirectory()) - continue; - - const String & file_name = it.name(); - - if (file_name.starts_with(workload_prefix) && file_name.ends_with(sql_suffix)) - { - String name = unescapeForFileName(file_name.substr( - workload_prefix.size(), - file_name.size() - workload_prefix.size() - sql_suffix.size())); - - if (name.empty()) - continue; - - ASTPtr ast = tryLoadEntity(WorkloadEntityType::Workload, name, dir_path + it.name(), /* check_file_exists= */ false); - if (ast) - entities_name_and_queries.emplace_back(name, ast); - } - - if (file_name.starts_with(resource_prefix) && file_name.ends_with(sql_suffix)) - { - String name = unescapeForFileName(file_name.substr( - resource_prefix.size(), - file_name.size() - resource_prefix.size() - sql_suffix.size())); - - if (name.empty()) - continue; - - ASTPtr ast = tryLoadEntity(WorkloadEntityType::Resource, name, dir_path + it.name(), /* check_file_exists= */ false); - if (ast) - entities_name_and_queries.emplace_back(name, ast); - } - } - - setAllEntities(entities_name_and_queries); - entities_loaded = true; - - LOG_DEBUG(log, "Workload entities loaded"); -} - - -void WorkloadEntityDiskStorage::createDirectory() -{ - std::error_code create_dir_error_code; - fs::create_directories(dir_path, create_dir_error_code); - if (!fs::exists(dir_path) || !fs::is_directory(dir_path) || create_dir_error_code) - throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "Couldn't create directory {} reason: '{}'", - dir_path, create_dir_error_code.message()); -} - - -WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::storeEntityImpl( - const ContextPtr & /*current_context*/, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) -{ - createDirectory(); - String file_path = getFilePath(entity_type, entity_name); - LOG_DEBUG(log, "Storing workload entity {} to file {}", backQuote(entity_name), file_path); - - if (fs::exists(file_path)) - { - if (throw_if_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); - else if (!replace_if_exists) - return OperationResult::Failed; - } - - - String temp_file_path = file_path + ".tmp"; - - try - { - WriteBufferFromFile out(temp_file_path); - formatAST(*create_entity_query, out, false); - writeChar('\n', out); - out.next(); - if (settings[Setting::fsync_metadata]) - out.sync(); - out.close(); - - if (replace_if_exists) - fs::rename(temp_file_path, file_path); - else - renameNoReplace(temp_file_path, file_path); - } - catch (...) - { - fs::remove(temp_file_path); - throw; - } - - LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); - return OperationResult::Ok; -} - - -WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::removeEntityImpl( - const ContextPtr & /*current_context*/, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) -{ - String file_path = getFilePath(entity_type, entity_name); - LOG_DEBUG(log, "Removing workload entity {} stored in file {}", backQuote(entity_name), file_path); - - bool existed = fs::remove(file_path); - - if (!existed) - { - if (throw_if_not_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); - else - return OperationResult::Failed; - } - - LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); - return OperationResult::Ok; -} - - -String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, const String & entity_name) const -{ - String file_path; - switch (entity_type) - { - case WorkloadEntityType::Workload: - { - file_path = dir_path + "workload_" + escapeForFileName(entity_name) + ".sql"; - break; - } - case WorkloadEntityType::Resource: - { - file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; - break; - } - case WorkloadEntityType::MAX: break; - } - return file_path; -} - -} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h deleted file mode 100644 index cb3fb600182..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -/// Loads workload entities from a specified folder. -class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase -{ -public: - WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); - void loadEntities() override; - -private: - OperationResult storeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) override; - - OperationResult removeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) override; - - void createDirectory(); - void loadEntitiesImpl(); - ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name); - ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & file_path, bool check_file_exists); - String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; - - String dir_path; - std::atomic entities_loaded = false; -}; - -} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp deleted file mode 100644 index 4b60a7ec57e..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ /dev/null @@ -1,273 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace Setting -{ -extern const SettingsUInt64 max_parser_backtracks; -extern const SettingsUInt64 max_parser_depth; -} - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; -} - -WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( - const ContextPtr & global_context_, const String & zookeeper_path_) - : WorkloadEntityStorageBase(global_context_) - , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} - , zookeeper_path{zookeeper_path_} - , watch{std::make_shared()} -{ - log = getLogger("WorkloadEntityKeeperStorage"); - if (zookeeper_path.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); - - if (zookeeper_path.back() == '/') - zookeeper_path.pop_back(); - - /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. - if (zookeeper_path.front() != '/') - zookeeper_path = "/" + zookeeper_path; -} - -WorkloadEntityKeeperStorage::~WorkloadEntityKeeperStorage() -{ - SCOPE_EXIT_SAFE(stopWatchingThread()); -} - -void WorkloadEntityKeeperStorage::startWatchingThread() -{ - if (!watching_flag.exchange(true)) - watching_thread = ThreadFromGlobalPool(&WorkloadEntityKeeperStorage::processWatchQueue, this); -} - -void WorkloadEntityKeeperStorage::stopWatchingThread() -{ - if (watching_flag.exchange(false)) - { - watch->cv.notify_one(); - if (watching_thread.joinable()) - watching_thread.join(); - } -} - -zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() -{ - auto [zookeeper, session_status] = zookeeper_getter.getZooKeeper(); - - if (session_status == zkutil::ZooKeeperCachingGetter::SessionStatus::New) - { - /// It's possible that we connected to different [Zoo]Keeper instance - /// so we may read a bit stale state. - zookeeper->sync(zookeeper_path); - - createRootNodes(zookeeper); - - auto lock = getLock(); - refreshEntities(zookeeper); - } - - return zookeeper; -} - -void WorkloadEntityKeeperStorage::loadEntities() -{ - /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. - /// However the watching thread must be started anyway in case the connection will be established later. - try - { - auto lock = getLock(); - refreshEntities(getZooKeeper()); - } - catch (...) - { - tryLogCurrentException(log, "Failed to load workload entities"); - } - startWatchingThread(); -} - - -void WorkloadEntityKeeperStorage::processWatchQueue() -{ - LOG_DEBUG(log, "Started watching thread"); - setThreadName("WrkldEntWatch"); - - UInt64 handled = 0; - while (watching_flag) - { - try - { - /// Re-initialize ZooKeeper session if expired - getZooKeeper(); - - { - std::unique_lock lock{watch->mutex}; - if (!watch->cv.wait_for(lock, std::chrono::seconds(10), [&] { return !watching_flag || handled != watch->triggered; })) - continue; - handled = watch->triggered; - } - - auto lock = getLock(); - refreshEntities(getZooKeeper()); - } - catch (...) - { - tryLogCurrentException(log, "Will try to restart watching thread after error"); - zookeeper_getter.resetCache(); - sleepForSeconds(5); - } - } - - LOG_DEBUG(log, "Stopped watching thread"); -} - - -void WorkloadEntityKeeperStorage::stopWatching() -{ - stopWatchingThread(); -} - -void WorkloadEntityKeeperStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper) -{ - zookeeper->createAncestors(zookeeper_path); - // If node does not exist we consider it to be equal to empty node: no workload entities - zookeeper->createIfNotExists(zookeeper_path, ""); -} - -WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEntityImpl( - const ContextPtr & /*current_context*/, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool /*throw_if_exists*/, - bool /*replace_if_exists*/, - const Settings &) -{ - LOG_DEBUG(log, "Storing workload entity {}", backQuote(entity_name)); - - String new_data = serializeAllEntities(Event{entity_type, entity_name, create_entity_query}); - auto zookeeper = getZooKeeper(); - - Coordination::Stat stat; - auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); - if (code != Coordination::Error::ZOK) - { - refreshEntities(zookeeper); - return OperationResult::Retry; - } - - current_version = stat.version; - - LOG_DEBUG(log, "Workload entity {} stored", backQuote(entity_name)); - - return OperationResult::Ok; -} - - -WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEntityImpl( - const ContextPtr & /*current_context*/, - WorkloadEntityType entity_type, - const String & entity_name, - bool /*throw_if_not_exists*/) -{ - LOG_DEBUG(log, "Removing workload entity {}", backQuote(entity_name)); - - String new_data = serializeAllEntities(Event{entity_type, entity_name, {}}); - auto zookeeper = getZooKeeper(); - - Coordination::Stat stat; - auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); - if (code != Coordination::Error::ZOK) - { - refreshEntities(zookeeper); - return OperationResult::Retry; - } - - current_version = stat.version; - - LOG_DEBUG(log, "Workload entity {} removed", backQuote(entity_name)); - - return OperationResult::Ok; -} - -std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) -{ - const auto data_watcher = [my_watch = watch](const Coordination::WatchResponse & response) - { - if (response.type == Coordination::Event::CHANGED) - { - std::unique_lock lock{my_watch->mutex}; - my_watch->triggered++; - my_watch->cv.notify_one(); - } - }; - - Coordination::Stat stat; - String data; - bool exists = zookeeper->tryGetWatch(zookeeper_path, data, &stat, data_watcher); - if (!exists) - { - createRootNodes(zookeeper); - data = zookeeper->getWatch(zookeeper_path, &stat, data_watcher); - } - return {data, stat.version}; -} - -void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) -{ - auto [data, version] = getDataAndSetWatch(zookeeper); - if (version == current_version) - return; - - LOG_DEBUG(log, "Refreshing workload entities from keeper"); - ASTs queries; - ParserCreateWorkloadEntity parser; - const char * begin = data.data(); /// begin of current query - const char * pos = begin; /// parser moves pos from begin to the end of current query - const char * end = begin + data.size(); - while (pos < end) - { - queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS)); - while (isWhitespaceASCII(*pos) || *pos == ';') - ++pos; - } - - /// Read and parse all SQL entities from data we just read from ZooKeeper - std::vector> new_entities; - for (const auto & query : queries) - { - LOG_TRACE(log, "Read keeper entity definition: {}", serializeAST(*query)); - if (auto * create_workload_query = query->as()) - new_entities.emplace_back(create_workload_query->getWorkloadName(), query); - else if (auto * create_resource_query = query->as()) - new_entities.emplace_back(create_resource_query->getResourceName(), query); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity query in keeper storage: {}", query->getID()); - } - - setAllEntities(new_entities); - current_version = version; - - LOG_DEBUG(log, "Workload entities refreshing is done"); -} - -} - diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h deleted file mode 100644 index 25dcd6d8c9a..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ /dev/null @@ -1,71 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -/// Loads RESOURCE and WORKLOAD sql objects from Keeper. -class WorkloadEntityKeeperStorage : public WorkloadEntityStorageBase -{ -public: - WorkloadEntityKeeperStorage(const ContextPtr & global_context_, const String & zookeeper_path_); - ~WorkloadEntityKeeperStorage() override; - - bool isReplicated() const override { return true; } - String getReplicationID() const override { return zookeeper_path; } - - void loadEntities() override; - void stopWatching() override; - -private: - OperationResult storeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) override; - - OperationResult removeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) override; - - void processWatchQueue(); - - zkutil::ZooKeeperPtr getZooKeeper(); - - void startWatchingThread(); - void stopWatchingThread(); - - void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); - std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); - void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); - - zkutil::ZooKeeperCachingGetter zookeeper_getter; - String zookeeper_path; - Int32 current_version = 0; - - ThreadFromGlobalPool watching_thread; - std::atomic watching_flag = false; - - struct WatchEvent - { - std::mutex mutex; - std::condition_variable cv; - UInt64 triggered = 0; - }; - std::shared_ptr watch; -}; - -} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp deleted file mode 100644 index 1b7a559698c..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ /dev/null @@ -1,773 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; -} - -namespace -{ - -/// Removes details from a CREATE query to be used as workload entity definition -ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) -{ - auto ptr = create_query.clone(); - if (auto * res = typeid_cast(ptr.get())) - { - res->if_not_exists = false; - res->or_replace = false; - } - if (auto * res = typeid_cast(ptr.get())) - { - res->if_not_exists = false; - res->or_replace = false; - } - return ptr; -} - -/// Returns a type of a workload entity `ptr` -WorkloadEntityType getEntityType(const ASTPtr & ptr) -{ - if (auto * res = typeid_cast(ptr.get())) - return WorkloadEntityType::Workload; - if (auto * res = typeid_cast(ptr.get())) - return WorkloadEntityType::Resource; - chassert(false); - return WorkloadEntityType::MAX; -} - -bool entityEquals(const ASTPtr & lhs, const ASTPtr & rhs) -{ - if (auto * a = typeid_cast(lhs.get())) - { - if (auto * b = typeid_cast(rhs.get())) - { - return std::forward_as_tuple(a->getWorkloadName(), a->getWorkloadParent(), a->changes) - == std::forward_as_tuple(b->getWorkloadName(), b->getWorkloadParent(), b->changes); - } - } - if (auto * a = typeid_cast(lhs.get())) - { - if (auto * b = typeid_cast(rhs.get())) - return std::forward_as_tuple(a->getResourceName(), a->operations) - == std::forward_as_tuple(b->getResourceName(), b->operations); - } - return false; -} - -/// Workload entities could reference each other. -/// This enum defines all possible reference types -enum class ReferenceType -{ - Parent, // Source workload references target workload as a parent - ForResource // Source workload references target resource in its `SETTINGS x = y FOR resource` clause -}; - -/// Runs a `func` callback for every reference from `source` to `target`. -/// This function is the source of truth defining what `target` references are stored in a workload `source_entity` -void forEachReference( - const ASTPtr & source_entity, - std::function func) -{ - if (auto * res = typeid_cast(source_entity.get())) - { - // Parent reference - String parent = res->getWorkloadParent(); - if (!parent.empty()) - func(parent, res->getWorkloadName(), ReferenceType::Parent); - - // References to RESOURCEs mentioned in SETTINGS clause after FOR keyword - std::unordered_set resources; - for (const auto & [name, value, resource] : res->changes) - { - if (!resource.empty()) - resources.insert(resource); - } - for (const String & resource : resources) - func(resource, res->getWorkloadName(), ReferenceType::ForResource); - } - if (auto * res = typeid_cast(source_entity.get())) - { - // RESOURCE has no references to be validated, we allow mentioned disks to be created later - } -} - -/// Helper for recursive DFS -void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) -{ - if (visited.contains(name)) - return; - visited.insert(name); - - // Recurse into parent (if any) - String parent = typeid_cast(ast.get())->getWorkloadParent(); - if (!parent.empty()) - { - auto parent_iter = workloads.find(parent); - if (parent_iter == workloads.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload metadata inconsistency: Workload '{}' parent '{}' does not exist. This must be fixed manually.", name, parent); - topologicallySortedWorkloadsImpl(parent, parent_iter->second, workloads, visited, sorted_workloads); - } - - sorted_workloads.emplace_back(name, ast); -} - -/// Returns pairs {worload_name, create_workload_ast} in order that respect child-parent relation (parent first, then children) -std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) -{ - std::vector> sorted_workloads; - std::unordered_set visited; - for (const auto & [name, ast] : workloads) - topologicallySortedWorkloadsImpl(name, ast, workloads, visited, sorted_workloads); - return sorted_workloads; -} - -/// Helper for recursive DFS -void topologicallySortedDependenciesImpl( - const String & name, - const std::unordered_map> & dependencies, - std::unordered_set & visited, - std::vector & result) -{ - if (visited.contains(name)) - return; - visited.insert(name); - - if (auto it = dependencies.find(name); it != dependencies.end()) - { - for (const String & dep : it->second) - topologicallySortedDependenciesImpl(dep, dependencies, visited, result); - } - - result.emplace_back(name); -} - -/// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) -std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) -{ - std::unordered_set visited; // Set to track visited nodes - std::vector result; // Result to store nodes in topologically sorted order - - // Perform DFS for each node in the graph - for (const auto & [name, _] : dependencies) - topologicallySortedDependenciesImpl(name, dependencies, visited, result); - - return result; -} - -/// Represents a change of a workload entity (WORKLOAD or RESOURCE) -struct EntityChange -{ - String name; /// Name of entity - ASTPtr before; /// Entity before change (CREATE if not set) - ASTPtr after; /// Entity after change (DROP if not set) - - std::vector toEvents() const - { - if (!after) - return {{getEntityType(before), name, {}}}; - else if (!before) - return {{getEntityType(after), name, after}}; - else - { - auto type_before = getEntityType(before); - auto type_after = getEntityType(after); - // If type changed, we have to remove an old entity and add a new one - if (type_before != type_after) - return {{type_before, name, {}}, {type_after, name, after}}; - else - return {{type_after, name, after}}; - } - } -}; - -/// Returns `changes` ordered for execution. -/// Every intemediate state during execution will be consistent (i.e. all references will be valid) -/// NOTE: It does not validate changes, any problem will be detected during execution. -/// NOTE: There will be no error if valid order does not exist. -std::vector topologicallySortedChanges(const std::vector & changes) -{ - // Construct map from entity name into entity change - std::unordered_map change_by_name; - for (const auto & change : changes) - change_by_name[change.name] = &change; - - // Construct references maps (before changes and after changes) - std::unordered_map> old_sources; // Key is target. Value is set of names of source entities. - std::unordered_map> new_targets; // Key is source. Value is set of names of target entities. - for (const auto & change : changes) - { - if (change.before) - { - forEachReference(change.before, - [&] (const String & target, const String & source, ReferenceType) - { - old_sources[target].insert(source); - }); - } - if (change.after) - { - forEachReference(change.after, - [&] (const String & target, const String & source, ReferenceType) - { - new_targets[source].insert(target); - }); - } - } - - // There are consistency rules that regulate order in which changes must be applied (see below). - // Construct DAG of dependencies between changes. - std::unordered_map> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. - for (const auto & change : changes) - { - dependencies.emplace(change.name, std::unordered_set{}); // Make sure we create nodes that have no dependencies - for (const auto & event : change.toEvents()) - { - if (!event.entity) // DROP - { - // Rule 1: Entity can only be removed after all existing references to it are removed as well. - for (const String & source : old_sources[event.name]) - { - if (change_by_name.contains(source)) - dependencies[event.name].insert(source); - } - } - else // CREATE || CREATE OR REPLACE - { - // Rule 2: Entity can only be created after all entities it references are created as well. - for (const String & target : new_targets[event.name]) - { - if (auto it = change_by_name.find(target); it != change_by_name.end()) - { - const EntityChange & target_change = *it->second; - // If target is creating, it should be created first. - // (But if target is updating, there is no dependency). - if (!target_change.before) - dependencies[event.name].insert(target); - } - } - } - } - } - - // Topological sort of changes to respect consistency rules - std::vector result; - for (const String & name : topologicallySortedDependencies(dependencies)) - result.push_back(*change_by_name[name]); - - return result; -} - -} - -WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) - : handlers(std::make_shared()) - , global_context(std::move(global_context_)) - , log{getLogger("WorkloadEntityStorage")} // could be overridden in derived class -{} - -ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const -{ - if (auto result = tryGet(entity_name)) - return result; - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The workload entity name '{}' is not saved", - entity_name); -} - -ASTPtr WorkloadEntityStorageBase::tryGet(const String & entity_name) const -{ - std::lock_guard lock(mutex); - - auto it = entities.find(entity_name); - if (it == entities.end()) - return nullptr; - - return it->second; -} - -bool WorkloadEntityStorageBase::has(const String & entity_name) const -{ - return tryGet(entity_name) != nullptr; -} - -std::vector WorkloadEntityStorageBase::getAllEntityNames() const -{ - std::vector entity_names; - - std::lock_guard lock(mutex); - entity_names.reserve(entities.size()); - - for (const auto & [name, _] : entities) - entity_names.emplace_back(name); - - return entity_names; -} - -std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const -{ - std::vector entity_names; - - std::lock_guard lock(mutex); - for (const auto & [name, entity] : entities) - { - if (getEntityType(entity) == entity_type) - entity_names.emplace_back(name); - } - - return entity_names; -} - -bool WorkloadEntityStorageBase::empty() const -{ - std::lock_guard lock(mutex); - return entities.empty(); -} - -bool WorkloadEntityStorageBase::storeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) -{ - if (entity_name.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); - - create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); - auto * workload = typeid_cast(create_entity_query.get()); - auto * resource = typeid_cast(create_entity_query.get()); - - while (true) - { - std::unique_lock lock{mutex}; - - ASTPtr old_entity; // entity to be REPLACED - if (auto it = entities.find(entity_name); it != entities.end()) - { - if (throw_if_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); - else if (!replace_if_exists) - return false; - else - old_entity = it->second; - } - - // Validate CREATE OR REPLACE - if (old_entity) - { - auto * old_workload = typeid_cast(old_entity.get()); - auto * old_resource = typeid_cast(old_entity.get()); - if (workload && !old_workload) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); - if (resource && !old_resource) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); - if (workload && !old_workload->hasParent() && workload->hasParent()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); - } - - // Validate workload - if (workload) - { - if (!workload->hasParent()) - { - if (!root_name.empty() && root_name != workload->getWorkloadName()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); - } - - SchedulingSettings validator; - validator.updateFromChanges(workload->changes); - } - - forEachReference(create_entity_query, - [this, workload] (const String & target, const String & source, ReferenceType type) - { - if (auto it = entities.find(target); it == entities.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); - - switch (type) - { - case ReferenceType::Parent: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); - break; - } - case ReferenceType::ForResource: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - - // Validate that we could parse the settings for specific resource - SchedulingSettings validator; - validator.updateFromChanges(workload->changes, target); - break; - } - } - - // Detect reference cycles. - // The only way to create a cycle is to add an edge that will be a part of a new cycle. - // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. - if (isIndirectlyReferenced(source, target)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); - }); - - auto result = storeEntityImpl( - current_context, - entity_type, - entity_name, - create_entity_query, - throw_if_exists, - replace_if_exists, - settings); - - if (result == OperationResult::Retry) - continue; // Entities were updated, we need to rerun all the validations - - if (result == OperationResult::Ok) - { - Event event{entity_type, entity_name, create_entity_query}; - applyEvent(lock, event); - unlockAndNotify(lock, {std::move(event)}); - } - - return result == OperationResult::Ok; - } -} - -bool WorkloadEntityStorageBase::removeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) -{ - while (true) - { - std::unique_lock lock(mutex); - auto it = entities.find(entity_name); - if (it == entities.end()) - { - if (throw_if_not_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); - else - return false; - } - - if (auto reference_it = references.find(entity_name); reference_it != references.end()) - { - String names; - for (const String & name : reference_it->second) - names += " " + name; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); - } - - auto result = removeEntityImpl( - current_context, - entity_type, - entity_name, - throw_if_not_exists); - - if (result == OperationResult::Retry) - continue; // Entities were updated, we need to rerun all the validations - - if (result == OperationResult::Ok) - { - Event event{entity_type, entity_name, {}}; - applyEvent(lock, event); - unlockAndNotify(lock, {std::move(event)}); - } - - return result == OperationResult::Ok; - } -} - -scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) -{ - scope_guard result; - - std::vector current_state; - { - std::lock_guard lock{mutex}; - current_state = orderEntities(entities); - - std::lock_guard lock2{handlers->mutex}; - handlers->list.push_back(handler); - auto handler_it = std::prev(handlers->list.end()); - result = [my_handlers = handlers, handler_it] - { - std::lock_guard lock3{my_handlers->mutex}; - my_handlers->list.erase(handler_it); - }; - } - - // When you subscribe you get all the entities back to your handler immediately if already loaded, or later when loaded - handler(current_state); - - return result; -} - -void WorkloadEntityStorageBase::unlockAndNotify( - std::unique_lock & lock, - std::vector tx) -{ - if (tx.empty()) - return; - - std::vector current_handlers; - { - std::lock_guard handlers_lock{handlers->mutex}; - boost::range::copy(handlers->list, std::back_inserter(current_handlers)); - } - - lock.unlock(); - - for (const auto & handler : current_handlers) - { - try - { - handler(tx); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } -} - -std::unique_lock WorkloadEntityStorageBase::getLock() const -{ - return std::unique_lock{mutex}; -} - -void WorkloadEntityStorageBase::setAllEntities(const std::vector> & raw_new_entities) -{ - std::unordered_map new_entities; - for (const auto & [entity_name, create_query] : raw_new_entities) - new_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); - - std::unique_lock lock(mutex); - - // Fill vector of `changes` based on difference between current `entities` and `new_entities` - std::vector changes; - for (const auto & [entity_name, entity] : entities) - { - if (auto it = new_entities.find(entity_name); it != new_entities.end()) - { - if (!entityEquals(entity, it->second)) - { - changes.emplace_back(entity_name, entity, it->second); // Update entities that are present in both `new_entities` and `entities` - LOG_TRACE(log, "Entity {} was updated", entity_name); - } - else - LOG_TRACE(log, "Entity {} is the same", entity_name); - } - else - { - changes.emplace_back(entity_name, entity, ASTPtr{}); // Remove entities that are not present in `new_entities` - LOG_TRACE(log, "Entity {} was dropped", entity_name); - } - } - for (const auto & [entity_name, entity] : new_entities) - { - if (!entities.contains(entity_name)) - { - changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` - LOG_TRACE(log, "Entity {} was created", entity_name); - } - } - - // Sort `changes` to respect consistency of references and apply them one by one. - std::vector tx; - for (const auto & change : topologicallySortedChanges(changes)) - { - for (const auto & event : change.toEvents()) - { - // TODO(serxa): do validation and throw LOGICAL_ERROR if failed - applyEvent(lock, event); - tx.push_back(event); - } - } - - // Notify subscribers - unlockAndNotify(lock, tx); -} - -void WorkloadEntityStorageBase::applyEvent( - std::unique_lock &, - const Event & event) -{ - if (event.entity) // CREATE || CREATE OR REPLACE - { - LOG_DEBUG(log, "Create or replace workload entity: {}", serializeAST(*event.entity)); - - auto * workload = typeid_cast(event.entity.get()); - - // Validate workload - if (workload && !workload->hasParent()) - root_name = workload->getWorkloadName(); - - // Remove references of a replaced entity (only for CREATE OR REPLACE) - if (auto it = entities.find(event.name); it != entities.end()) - removeReferences(it->second); - - // Insert references of created entity - insertReferences(event.entity); - - // Store in memory - entities[event.name] = event.entity; - } - else // DROP - { - auto it = entities.find(event.name); - chassert(it != entities.end()); - - LOG_DEBUG(log, "Drop workload entity: {}", event.name); - - if (event.name == root_name) - root_name.clear(); - - // Clean up references - removeReferences(it->second); - - // Remove from memory - entities.erase(it); - } -} - -std::vector> WorkloadEntityStorageBase::getAllEntities() const -{ - std::lock_guard lock{mutex}; - std::vector> all_entities; - all_entities.reserve(entities.size()); - std::copy(entities.begin(), entities.end(), std::back_inserter(all_entities)); - return all_entities; -} - -bool WorkloadEntityStorageBase::isIndirectlyReferenced(const String & target, const String & source) -{ - std::queue bfs; - std::unordered_set visited; - visited.insert(target); - bfs.push(target); - while (!bfs.empty()) - { - String current = bfs.front(); - bfs.pop(); - if (current == source) - return true; - if (auto it = references.find(current); it != references.end()) - { - for (const String & node : it->second) - { - if (visited.contains(node)) - continue; - visited.insert(node); - bfs.push(node); - } - } - } - return false; -} - -void WorkloadEntityStorageBase::insertReferences(const ASTPtr & entity) -{ - if (!entity) - return; - forEachReference(entity, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].insert(source); - }); -} - -void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) -{ - if (!entity) - return; - forEachReference(entity, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].erase(source); - if (references[target].empty()) - references.erase(target); - }); -} - -std::vector WorkloadEntityStorageBase::orderEntities( - const std::unordered_map & all_entities, - std::optional change) -{ - std::vector result; - - std::unordered_map workloads; - for (const auto & [entity_name, ast] : all_entities) - { - if (typeid_cast(ast.get())) - { - if (change && change->name == entity_name) - continue; // Skip this workload if it is removed or updated - workloads.emplace(entity_name, ast); - } - else if (typeid_cast(ast.get())) - { - if (change && change->name == entity_name) - continue; // Skip this resource if it is removed or updated - // Resources should go first because workloads could reference them - result.emplace_back(WorkloadEntityType::Resource, entity_name, ast); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); - } - - // Introduce new entity described by `change` - if (change && change->entity) - { - if (change->type == WorkloadEntityType::Workload) - workloads.emplace(change->name, change->entity); - else if (change->type == WorkloadEntityType::Resource) - result.emplace_back(WorkloadEntityType::Resource, change->name, change->entity); - } - - // Workloads should go in an order such that children are enlisted only after its parent - for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) - result.emplace_back(WorkloadEntityType::Workload, entity_name, ast); - - return result; -} - -String WorkloadEntityStorageBase::serializeAllEntities(std::optional change) -{ - std::unique_lock lock; - auto ordered_entities = orderEntities(entities, change); - WriteBufferFromOwnString buf; - for (const auto & event : ordered_entities) - { - formatAST(*event.entity, buf, false, true); - buf.write(";\n", 2); - } - return buf.str(); -} - -} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h deleted file mode 100644 index d57bf8201b3..00000000000 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ /dev/null @@ -1,126 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -#include -#include - -#include - -namespace DB -{ - -class WorkloadEntityStorageBase : public IWorkloadEntityStorage -{ -public: - explicit WorkloadEntityStorageBase(ContextPtr global_context_); - ASTPtr get(const String & entity_name) const override; - - ASTPtr tryGet(const String & entity_name) const override; - - bool has(const String & entity_name) const override; - - std::vector getAllEntityNames() const override; - std::vector getAllEntityNames(WorkloadEntityType entity_type) const override; - - std::vector> getAllEntities() const override; - - bool empty() const override; - - bool storeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) override; - - bool removeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) override; - - scope_guard getAllEntitiesAndSubscribe( - const OnChangedHandler & handler) override; - -protected: - enum class OperationResult - { - Ok, - Failed, - Retry - }; - - virtual OperationResult storeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - ASTPtr create_entity_query, - bool throw_if_exists, - bool replace_if_exists, - const Settings & settings) = 0; - - virtual OperationResult removeEntityImpl( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) = 0; - - std::unique_lock getLock() const; - - /// Replace current `entities` with `new_entities` and notifies subscribers. - /// Note that subscribers will be notified with a sequence of events. - /// It is guaranteed that all itermediate states (between every pair of consecutive events) - /// will be consistent (all references between entities will be valid) - void setAllEntities(const std::vector> & new_entities); - - /// Serialize `entities` stored in memory plus one optional `change` into multiline string - String serializeAllEntities(std::optional change = {}); - -private: - /// Change state in memory - void applyEvent(std::unique_lock & lock, const Event & event); - - /// Notify subscribers about changes describe by vector of events `tx` - void unlockAndNotify(std::unique_lock & lock, std::vector tx); - - /// Return true iff `references` has a path from `source` to `target` - bool isIndirectlyReferenced(const String & target, const String & source); - - /// Adds references that are described by `entity` to `references` - void insertReferences(const ASTPtr & entity); - - /// Removes references that are described by `entity` from `references` - void removeReferences(const ASTPtr & entity); - - /// Returns an ordered vector of `entities` - std::vector orderEntities( - const std::unordered_map & all_entities, - std::optional change = {}); - - struct Handlers - { - std::mutex mutex; - std::list list; - }; - /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. - std::shared_ptr handlers; - - mutable std::recursive_mutex mutex; - std::unordered_map entities; /// Maps entity name into CREATE entity query - - // Validation - std::unordered_map> references; /// Keep track of references between entities. Key is target. Value is set of sources - String root_name; /// current root workload name - -protected: - ContextPtr global_context; - LoggerPtr log; -}; - -} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp deleted file mode 100644 index 5dc1265e31d..00000000000 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; - - -namespace DB -{ - - -namespace ErrorCodes -{ - extern const int INVALID_CONFIG_PARAMETER; -} - -std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context) -{ - const String zookeeper_path_key = "workload_zookeeper_path"; - const String disk_path_key = "workload_path"; - - const auto & config = global_context->getConfigRef(); - if (config.has(zookeeper_path_key)) - { - if (config.has(disk_path_key)) - { - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "'{}' and '{}' must not be both specified in the config", - zookeeper_path_key, - disk_path_key); - } - return std::make_unique(global_context, config.getString(zookeeper_path_key)); - } - - String default_path = fs::path{global_context->getPath()} / "workload" / ""; - String path = config.getString(disk_path_key, default_path); - return std::make_unique(global_context, path); -} - -} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h deleted file mode 100644 index 936e1275010..00000000000 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context); - -} diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp deleted file mode 100644 index fd9743dbf72..00000000000 --- a/src/Common/Scheduler/createResourceManager.cpp +++ /dev/null @@ -1,104 +0,0 @@ -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int RESOURCE_ACCESS_DENIED; -} - -class ResourceManagerDispatcher : public IResourceManager -{ -private: - class Classifier : public IClassifier - { - public: - void addClassifier(const ClassifierPtr & classifier) - { - classifiers.push_back(classifier); - } - - bool has(const String & resource_name) override - { - for (const auto & classifier : classifiers) - { - if (classifier->has(resource_name)) - return true; - } - return false; - } - - ResourceLink get(const String & resource_name) override - { - for (auto & classifier : classifiers) - { - if (classifier->has(resource_name)) - return classifier->get(resource_name); - } - throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); - } - private: - std::vector classifiers; // should be constant after initialization to avoid races - }; - -public: - void addManager(const ResourceManagerPtr & manager) - { - managers.push_back(manager); - } - - void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override - { - for (auto & manager : managers) - manager->updateConfiguration(config); - } - - bool hasResource(const String & resource_name) const override - { - for (const auto & manager : managers) - { - if (manager->hasResource(resource_name)) - return true; - } - return false; - } - - ClassifierPtr acquire(const String & workload_name) override - { - auto classifier = std::make_shared(); - for (const auto & manager : managers) - classifier->addClassifier(manager->acquire(workload_name)); - return classifier; - } - - void forEachNode(VisitorFunc visitor) override - { - for (const auto & manager : managers) - manager->forEachNode(visitor); - } - -private: - std::vector managers; // Should be constant after initialization to avoid races -}; - -ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) -{ - auto dispatcher = std::make_shared(); - - // NOTE: if the same resource is described by both managers, then manager added earlier will be used. - dispatcher->addManager(std::make_shared()); - dispatcher->addManager(std::make_shared(global_context->getWorkloadEntityStorage())); - - return dispatcher; -} - -} diff --git a/src/Common/Scheduler/createResourceManager.h b/src/Common/Scheduler/createResourceManager.h deleted file mode 100644 index d80a17f3bff..00000000000 --- a/src/Common/Scheduler/createResourceManager.h +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context); - -} diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index cc8a873c544..fbab25490c1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -18,8 +18,7 @@ #include #include #include -#include -#include + namespace DB { @@ -72,8 +71,8 @@ DiskObjectStorage::DiskObjectStorage( , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) - , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) - , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) + , read_resource_name(config.getString(config_prefix + ".read_resource", "")) + , write_resource_name(config.getString(config_prefix + ".write_resource", "")) , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) { data_source_description = DataSourceDescription{ @@ -84,98 +83,6 @@ DiskObjectStorage::DiskObjectStorage( .is_encrypted = false, .is_cached = object_storage->supportsCache(), }; - resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe( - [this] (const std::vector & events) - { - std::unique_lock lock{resource_mutex}; - - // Sets of matching resource names. Required to resolve possible conflicts in deterministic way - std::set new_read_resource_name_from_sql; - std::set new_write_resource_name_from_sql; - std::set new_read_resource_name_from_sql_any; - std::set new_write_resource_name_from_sql_any; - - // Current state - if (!read_resource_name_from_sql.empty()) - new_read_resource_name_from_sql.insert(read_resource_name_from_sql); - if (!write_resource_name_from_sql.empty()) - new_write_resource_name_from_sql.insert(write_resource_name_from_sql); - if (!read_resource_name_from_sql_any.empty()) - new_read_resource_name_from_sql_any.insert(read_resource_name_from_sql_any); - if (!write_resource_name_from_sql_any.empty()) - new_write_resource_name_from_sql_any.insert(write_resource_name_from_sql_any); - - // Process all updates in specified order - for (const auto & [entity_type, resource_name, resource] : events) - { - if (entity_type == WorkloadEntityType::Resource) - { - if (resource) // CREATE RESOURCE - { - auto * create = typeid_cast(resource.get()); - chassert(create); - for (const auto & [mode, disk] : create->operations) - { - if (!disk) - { - switch (mode) - { - case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql_any.insert(resource_name); break; - case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql_any.insert(resource_name); break; - } - } - else if (*disk == name) - { - switch (mode) - { - case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql.insert(resource_name); break; - case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql.insert(resource_name); break; - } - } - } - } - else // DROP RESOURCE - { - new_read_resource_name_from_sql.erase(resource_name); - new_write_resource_name_from_sql.erase(resource_name); - new_read_resource_name_from_sql_any.erase(resource_name); - new_write_resource_name_from_sql_any.erase(resource_name); - } - } - } - - String old_read_resource = getReadResourceNameNoLock(); - String old_write_resource = getWriteResourceNameNoLock(); - - // Apply changes - if (!new_read_resource_name_from_sql_any.empty()) - read_resource_name_from_sql_any = *new_read_resource_name_from_sql_any.begin(); - else - read_resource_name_from_sql_any.clear(); - - if (!new_write_resource_name_from_sql_any.empty()) - write_resource_name_from_sql_any = *new_write_resource_name_from_sql_any.begin(); - else - write_resource_name_from_sql_any.clear(); - - if (!new_read_resource_name_from_sql.empty()) - read_resource_name_from_sql = *new_read_resource_name_from_sql.begin(); - else - read_resource_name_from_sql.clear(); - - if (!new_write_resource_name_from_sql.empty()) - write_resource_name_from_sql = *new_write_resource_name_from_sql.begin(); - else - write_resource_name_from_sql.clear(); - - String new_read_resource = getReadResourceNameNoLock(); - String new_write_resource = getWriteResourceNameNoLock(); - - if (old_read_resource != new_read_resource) - LOG_INFO(log, "Using resource '{}' instead of '{}' for READ", new_read_resource, old_read_resource); - if (old_write_resource != new_write_resource) - LOG_INFO(log, "Using resource '{}' instead of '{}' for WRITE", new_write_resource, old_write_resource); - }); } StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const @@ -573,29 +480,13 @@ static inline Settings updateIOSchedulingSettings(const Settings & settings, con String DiskObjectStorage::getReadResourceName() const { std::unique_lock lock(resource_mutex); - return getReadResourceNameNoLock(); + return read_resource_name; } String DiskObjectStorage::getWriteResourceName() const { std::unique_lock lock(resource_mutex); - return getWriteResourceNameNoLock(); -} - -String DiskObjectStorage::getReadResourceNameNoLock() const -{ - if (read_resource_name_from_config.empty()) - return read_resource_name_from_sql.empty() ? read_resource_name_from_sql_any : read_resource_name_from_sql; - else - return read_resource_name_from_config; -} - -String DiskObjectStorage::getWriteResourceNameNoLock() const -{ - if (write_resource_name_from_config.empty()) - return write_resource_name_from_sql.empty() ? write_resource_name_from_sql_any : write_resource_name_from_sql; - else - return write_resource_name_from_config; + return write_resource_name; } std::unique_ptr DiskObjectStorage::readFile( @@ -716,10 +607,10 @@ void DiskObjectStorage::applyNewSettings( { std::unique_lock lock(resource_mutex); - if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name_from_config) - read_resource_name_from_config = new_read_resource_name; - if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name_from_config) - write_resource_name_from_config = new_write_resource_name; + if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name) + read_resource_name = new_read_resource_name; + if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name) + write_resource_name = new_write_resource_name; } IDisk::applyNewSettings(config, context_, config_prefix, disk_map); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 6657ee352c9..b4cdf620555 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -6,8 +6,6 @@ #include #include -#include - #include "config.h" @@ -230,8 +228,6 @@ private: String getReadResourceName() const; String getWriteResourceName() const; - String getReadResourceNameNoLock() const; - String getWriteResourceNameNoLock() const; const String object_key_prefix; LoggerPtr log; @@ -250,13 +246,8 @@ private: const bool send_metadata; mutable std::mutex resource_mutex; - String read_resource_name_from_config; // specified in disk config.xml read_resource element - String write_resource_name_from_config; // specified in disk config.xml write_resource element - String read_resource_name_from_sql; // described by CREATE RESOURCE query with READ DISK clause - String write_resource_name_from_sql; // described by CREATE RESOURCE query with WRITE DISK clause - String read_resource_name_from_sql_any; // described by CREATE RESOURCE query with READ ANY DISK clause - String write_resource_name_from_sql_any; // described by CREATE RESOURCE query with WRITE ANY DISK clause - scope_guard resource_changes_subscription; + String read_resource_name; + String write_resource_name; std::unique_ptr metadata_helper; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fbf0cbd0eb7..b8e178e402b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -67,6 +67,7 @@ #include #include #include +#include #include #include #include @@ -91,8 +92,6 @@ #include #include #include -#include -#include #include #include #include @@ -371,9 +370,6 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag user_defined_sql_objects_storage_initialized; mutable std::unique_ptr user_defined_sql_objects_storage; - mutable OnceFlag workload_entity_storage_initialized; - mutable std::unique_ptr workload_entity_storage; - #if USE_NLP mutable OnceFlag synonyms_extensions_initialized; mutable std::optional synonyms_extensions; @@ -715,7 +711,6 @@ struct ContextSharedPart : boost::noncopyable SHUTDOWN(log, "dictionaries loader", external_dictionaries_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "UDFs loader", external_user_defined_executable_functions_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "another UDFs storage", user_defined_sql_objects_storage, stopWatching()); - SHUTDOWN(log, "workload entity storage", workload_entity_storage, stopWatching()); LOG_TRACE(log, "Shutting down named sessions"); Session::shutdownNamedSessions(); @@ -747,7 +742,6 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_external_dictionaries_loader; std::unique_ptr delete_external_user_defined_executable_functions_loader; std::unique_ptr delete_user_defined_sql_objects_storage; - std::unique_ptr delete_workload_entity_storage; std::unique_ptr delete_buffer_flush_schedule_pool; std::unique_ptr delete_schedule_pool; std::unique_ptr delete_distributed_schedule_pool; @@ -832,7 +826,6 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader = std::move(external_dictionaries_loader); delete_external_user_defined_executable_functions_loader = std::move(external_user_defined_executable_functions_loader); delete_user_defined_sql_objects_storage = std::move(user_defined_sql_objects_storage); - delete_workload_entity_storage = std::move(workload_entity_storage); delete_buffer_flush_schedule_pool = std::move(buffer_flush_schedule_pool); delete_schedule_pool = std::move(schedule_pool); delete_distributed_schedule_pool = std::move(distributed_schedule_pool); @@ -851,7 +844,6 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader.reset(); delete_external_user_defined_executable_functions_loader.reset(); delete_user_defined_sql_objects_storage.reset(); - delete_workload_entity_storage.reset(); delete_ddl_worker.reset(); delete_buffer_flush_schedule_pool.reset(); delete_schedule_pool.reset(); @@ -1776,7 +1768,7 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { callOnce(shared->resource_manager_initialized, [&] { - shared->resource_manager = createResourceManager(getGlobalContext()); + shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); }); return shared->resource_manager; @@ -3023,16 +3015,6 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } -IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const -{ - callOnce(shared->workload_entity_storage_initialized, [&] { - shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); - }); - - std::lock_guard lock(shared->mutex); - return *shared->workload_entity_storage; -} - #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e8ccc31f597..c62c16098e5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -76,7 +76,6 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; class IUserDefinedSQLObjectsStorage; -class IWorkloadEntityStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; @@ -894,8 +893,6 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); - IWorkloadEntityStorage & getWorkloadEntityStorage() const; - #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; Lemmatizers & getLemmatizers() const; diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp deleted file mode 100644 index c6eca7a90d8..00000000000 --- a/src/Interpreters/InterpreterCreateResourceQuery.cpp +++ /dev/null @@ -1,68 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_QUERY; -} - -BlockIO InterpreterCreateResourceQuery::execute() -{ - ASTCreateResourceQuery & create_resource_query = query_ptr->as(); - - AccessRightsElements access_rights_elements; - access_rights_elements.emplace_back(AccessType::CREATE_RESOURCE); - - if (create_resource_query.or_replace) - access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); - - auto current_context = getContext(); - - if (!create_resource_query.cluster.empty()) - { - if (current_context->getWorkloadEntityStorage().isReplicated()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); - - DDLQueryOnClusterParams params; - params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); - } - - current_context->checkAccess(access_rights_elements); - - auto resource_name = create_resource_query.getResourceName(); - bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; - bool replace_if_exists = create_resource_query.or_replace; - - current_context->getWorkloadEntityStorage().storeEntity( - current_context, - WorkloadEntityType::Resource, - resource_name, - query_ptr, - throw_if_exists, - replace_if_exists, - current_context->getSettingsRef()); - - return {}; -} - -void registerInterpreterCreateResourceQuery(InterpreterFactory & factory) -{ - auto create_fn = [] (const InterpreterFactory::Arguments & args) - { - return std::make_unique(args.query, args.context); - }; - factory.registerInterpreter("InterpreterCreateResourceQuery", create_fn); -} - -} diff --git a/src/Interpreters/InterpreterCreateResourceQuery.h b/src/Interpreters/InterpreterCreateResourceQuery.h deleted file mode 100644 index 4bd427e5e8f..00000000000 --- a/src/Interpreters/InterpreterCreateResourceQuery.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; - -class InterpreterCreateResourceQuery : public IInterpreter, WithMutableContext -{ -public: - InterpreterCreateResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_) - { - } - - BlockIO execute() override; - -private: - ASTPtr query_ptr; -}; - -} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp deleted file mode 100644 index 41d0f52c685..00000000000 --- a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp +++ /dev/null @@ -1,68 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_QUERY; -} - -BlockIO InterpreterCreateWorkloadQuery::execute() -{ - ASTCreateWorkloadQuery & create_workload_query = query_ptr->as(); - - AccessRightsElements access_rights_elements; - access_rights_elements.emplace_back(AccessType::CREATE_WORKLOAD); - - if (create_workload_query.or_replace) - access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); - - auto current_context = getContext(); - - if (!create_workload_query.cluster.empty()) - { - if (current_context->getWorkloadEntityStorage().isReplicated()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); - - DDLQueryOnClusterParams params; - params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); - } - - current_context->checkAccess(access_rights_elements); - - auto workload_name = create_workload_query.getWorkloadName(); - bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; - bool replace_if_exists = create_workload_query.or_replace; - - current_context->getWorkloadEntityStorage().storeEntity( - current_context, - WorkloadEntityType::Workload, - workload_name, - query_ptr, - throw_if_exists, - replace_if_exists, - current_context->getSettingsRef()); - - return {}; -} - -void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory) -{ - auto create_fn = [] (const InterpreterFactory::Arguments & args) - { - return std::make_unique(args.query, args.context); - }; - factory.registerInterpreter("InterpreterCreateWorkloadQuery", create_fn); -} - -} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.h b/src/Interpreters/InterpreterCreateWorkloadQuery.h deleted file mode 100644 index 319388fb64c..00000000000 --- a/src/Interpreters/InterpreterCreateWorkloadQuery.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; - -class InterpreterCreateWorkloadQuery : public IInterpreter, WithMutableContext -{ -public: - InterpreterCreateWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) - : WithMutableContext(context_), query_ptr(query_ptr_) - { - } - - BlockIO execute() override; - -private: - ASTPtr query_ptr; -}; - -} diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp deleted file mode 100644 index 848a74fda23..00000000000 --- a/src/Interpreters/InterpreterDropResourceQuery.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_QUERY; -} - -BlockIO InterpreterDropResourceQuery::execute() -{ - ASTDropResourceQuery & drop_resource_query = query_ptr->as(); - - AccessRightsElements access_rights_elements; - access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); - - auto current_context = getContext(); - - if (!drop_resource_query.cluster.empty()) - { - if (current_context->getWorkloadEntityStorage().isReplicated()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); - - DDLQueryOnClusterParams params; - params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); - } - - current_context->checkAccess(access_rights_elements); - - bool throw_if_not_exists = !drop_resource_query.if_exists; - - current_context->getWorkloadEntityStorage().removeEntity( - current_context, - WorkloadEntityType::Resource, - drop_resource_query.resource_name, - throw_if_not_exists); - - return {}; -} - -void registerInterpreterDropResourceQuery(InterpreterFactory & factory) -{ - auto create_fn = [] (const InterpreterFactory::Arguments & args) - { - return std::make_unique(args.query, args.context); - }; - factory.registerInterpreter("InterpreterDropResourceQuery", create_fn); -} - -} diff --git a/src/Interpreters/InterpreterDropResourceQuery.h b/src/Interpreters/InterpreterDropResourceQuery.h deleted file mode 100644 index 588f26fb88c..00000000000 --- a/src/Interpreters/InterpreterDropResourceQuery.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class Context; - -class InterpreterDropResourceQuery : public IInterpreter, WithMutableContext -{ -public: - InterpreterDropResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} - - BlockIO execute() override; - -private: - ASTPtr query_ptr; -}; - -} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp deleted file mode 100644 index bbaa2beb4cd..00000000000 --- a/src/Interpreters/InterpreterDropWorkloadQuery.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_QUERY; -} - -BlockIO InterpreterDropWorkloadQuery::execute() -{ - ASTDropWorkloadQuery & drop_workload_query = query_ptr->as(); - - AccessRightsElements access_rights_elements; - access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); - - auto current_context = getContext(); - - if (!drop_workload_query.cluster.empty()) - { - if (current_context->getWorkloadEntityStorage().isReplicated()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); - - DDLQueryOnClusterParams params; - params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); - } - - current_context->checkAccess(access_rights_elements); - - bool throw_if_not_exists = !drop_workload_query.if_exists; - - current_context->getWorkloadEntityStorage().removeEntity( - current_context, - WorkloadEntityType::Workload, - drop_workload_query.workload_name, - throw_if_not_exists); - - return {}; -} - -void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory) -{ - auto create_fn = [] (const InterpreterFactory::Arguments & args) - { - return std::make_unique(args.query, args.context); - }; - factory.registerInterpreter("InterpreterDropWorkloadQuery", create_fn); -} - -} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.h b/src/Interpreters/InterpreterDropWorkloadQuery.h deleted file mode 100644 index 1297c95e949..00000000000 --- a/src/Interpreters/InterpreterDropWorkloadQuery.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class Context; - -class InterpreterDropWorkloadQuery : public IInterpreter, WithMutableContext -{ -public: - InterpreterDropWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} - - BlockIO execute() override; - -private: - ASTPtr query_ptr; -}; - -} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 729a7b86312..cfc95124895 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -3,13 +3,9 @@ #include #include #include -#include -#include #include #include #include -#include -#include #include #include #include @@ -336,22 +332,6 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { interpreter_name = "InterpreterDropFunctionQuery"; } - else if (query->as()) - { - interpreter_name = "InterpreterCreateWorkloadQuery"; - } - else if (query->as()) - { - interpreter_name = "InterpreterDropWorkloadQuery"; - } - else if (query->as()) - { - interpreter_name = "InterpreterCreateResourceQuery"; - } - else if (query->as()) - { - interpreter_name = "InterpreterDropResourceQuery"; - } else if (query->as()) { interpreter_name = "InterpreterCreateIndexQuery"; diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp index 838b3a669da..481d0597a85 100644 --- a/src/Interpreters/registerInterpreters.cpp +++ b/src/Interpreters/registerInterpreters.cpp @@ -52,10 +52,6 @@ void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); -void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory); -void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory); -void registerInterpreterCreateResourceQuery(InterpreterFactory & factory); -void registerInterpreterDropResourceQuery(InterpreterFactory & factory); void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); void registerInterpreterDropIndexQuery(InterpreterFactory & factory); @@ -115,10 +111,6 @@ void registerInterpreters() registerInterpreterTransactionControlQuery(factory); registerInterpreterCreateFunctionQuery(factory); registerInterpreterDropFunctionQuery(factory); - registerInterpreterCreateWorkloadQuery(factory); - registerInterpreterDropWorkloadQuery(factory); - registerInterpreterCreateResourceQuery(factory); - registerInterpreterDropResourceQuery(factory); registerInterpreterCreateIndexQuery(factory); registerInterpreterCreateNamedCollectionQuery(factory); registerInterpreterDropIndexQuery(factory); diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp deleted file mode 100644 index 3e40d76ba1b..00000000000 --- a/src/Parsers/ASTCreateResourceQuery.cpp +++ /dev/null @@ -1,83 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -ASTPtr ASTCreateResourceQuery::clone() const -{ - auto res = std::make_shared(*this); - res->children.clear(); - - res->resource_name = resource_name->clone(); - res->children.push_back(res->resource_name); - - res->operations = operations; - - return res; -} - -void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const -{ - format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; - - if (or_replace) - format.ostr << "OR REPLACE "; - - format.ostr << "RESOURCE "; - - if (if_not_exists) - format.ostr << "IF NOT EXISTS "; - - format.ostr << (format.hilite ? hilite_none : ""); - - format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : ""); - - formatOnCluster(format); - - format.ostr << " ("; - - bool first = true; - for (const auto & operation : operations) - { - if (!first) - format.ostr << ", "; - else - first = false; - - switch (operation.mode) - { - case AccessMode::Read: - { - format.ostr << (format.hilite ? hilite_keyword : "") << "READ "; - break; - } - case AccessMode::Write: - { - format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE "; - break; - } - } - if (operation.disk) - { - format.ostr << "DISK " << (format.hilite ? hilite_none : ""); - format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(*operation.disk) << (format.hilite ? hilite_none : ""); - } - else - format.ostr << "ANY DISK" << (format.hilite ? hilite_none : ""); - } - - format.ostr << ")"; -} - -String ASTCreateResourceQuery::getResourceName() const -{ - String name; - tryGetIdentifierNameInto(resource_name, name); - return name; -} - -} diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h deleted file mode 100644 index 51933a375f8..00000000000 --- a/src/Parsers/ASTCreateResourceQuery.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster -{ -public: - enum class AccessMode - { - Read, - Write - }; - struct Operation - { - AccessMode mode; - std::optional disk; // Applies to all disks if not set - - friend bool operator ==(const Operation & lhs, const Operation & rhs) { return lhs.mode == rhs.mode && lhs.disk == rhs.disk; } - friend bool operator !=(const Operation & lhs, const Operation & rhs) { return !(lhs == rhs); } - }; - - using Operations = std::vector; - - ASTPtr resource_name; - Operations operations; /// List of operations that require this resource - - bool or_replace = false; - bool if_not_exists = false; - - String getID(char delim) const override { return "CreateResourceQuery" + (delim + getResourceName()); } - - ASTPtr clone() const override; - - void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; - - ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } - - String getResourceName() const; - - QueryKind getQueryKind() const override { return QueryKind::Create; } -}; - -} diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp deleted file mode 100644 index 972ce733651..00000000000 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ /dev/null @@ -1,95 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -ASTPtr ASTCreateWorkloadQuery::clone() const -{ - auto res = std::make_shared(*this); - res->children.clear(); - - res->workload_name = workload_name->clone(); - res->children.push_back(res->workload_name); - - if (workload_parent) - { - res->workload_parent = workload_parent->clone(); - res->children.push_back(res->workload_parent); - } - - res->changes = changes; - - return res; -} - -void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const -{ - format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; - - if (or_replace) - format.ostr << "OR REPLACE "; - - format.ostr << "WORKLOAD "; - - if (if_not_exists) - format.ostr << "IF NOT EXISTS "; - - format.ostr << (format.hilite ? hilite_none : ""); - - format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : ""); - - formatOnCluster(format); - - if (hasParent()) - { - format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : ""); - format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); - } - - if (!changes.empty()) - { - format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; - - bool first = true; - - for (const auto & change : changes) - { - if (!first) - format.ostr << ", "; - else - first = false; - format.ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value); - if (!change.resource.empty()) - { - format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' '; - format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : ""); - } - } - } -} - -String ASTCreateWorkloadQuery::getWorkloadName() const -{ - String name; - tryGetIdentifierNameInto(workload_name, name); - return name; -} - -bool ASTCreateWorkloadQuery::hasParent() const -{ - return workload_parent != nullptr; -} - -String ASTCreateWorkloadQuery::getWorkloadParent() const -{ - String name; - tryGetIdentifierNameInto(workload_parent, name); - return name; -} - -} diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h deleted file mode 100644 index 8a4cecc001e..00000000000 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster -{ -public: - ASTPtr workload_name; - ASTPtr workload_parent; - - /// Special version of settings that support optional `FOR resource` clause - struct SettingChange - { - String name; - Field value; - String resource; - - SettingChange() = default; - SettingChange(std::string_view name_, const Field & value_, std::string_view resource_) : name(name_), value(value_), resource(resource_) {} - SettingChange(std::string_view name_, Field && value_, std::string_view resource_) : name(name_), value(std::move(value_)), resource(resource_) {} - - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.resource == rhs.resource); } - friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } - }; - - using SettingsChanges = std::vector; - SettingsChanges changes; - - bool or_replace = false; - bool if_not_exists = false; - - String getID(char delim) const override { return "CreateWorkloadQuery" + (delim + getWorkloadName()); } - - ASTPtr clone() const override; - - void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; - - ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } - - String getWorkloadName() const; - bool hasParent() const; - String getWorkloadParent() const; - - QueryKind getQueryKind() const override { return QueryKind::Create; } -}; - -} diff --git a/src/Parsers/ASTDropResourceQuery.cpp b/src/Parsers/ASTDropResourceQuery.cpp deleted file mode 100644 index 753ac4e30e7..00000000000 --- a/src/Parsers/ASTDropResourceQuery.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -ASTPtr ASTDropResourceQuery::clone() const -{ - return std::make_shared(*this); -} - -void ASTDropResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const -{ - settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE "; - - if (if_exists) - settings.ostr << "IF EXISTS "; - - settings.ostr << (settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : ""); - formatOnCluster(settings); -} - -} diff --git a/src/Parsers/ASTDropResourceQuery.h b/src/Parsers/ASTDropResourceQuery.h deleted file mode 100644 index e1534ea454a..00000000000 --- a/src/Parsers/ASTDropResourceQuery.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class ASTDropResourceQuery : public IAST, public ASTQueryWithOnCluster -{ -public: - String resource_name; - - bool if_exists = false; - - String getID(char) const override { return "DropResourceQuery"; } - - ASTPtr clone() const override; - - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; - - ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } - - QueryKind getQueryKind() const override { return QueryKind::Drop; } -}; - -} diff --git a/src/Parsers/ASTDropWorkloadQuery.cpp b/src/Parsers/ASTDropWorkloadQuery.cpp deleted file mode 100644 index 3192223c4b3..00000000000 --- a/src/Parsers/ASTDropWorkloadQuery.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -ASTPtr ASTDropWorkloadQuery::clone() const -{ - return std::make_shared(*this); -} - -void ASTDropWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const -{ - settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD "; - - if (if_exists) - settings.ostr << "IF EXISTS "; - - settings.ostr << (settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : ""); - formatOnCluster(settings); -} - -} diff --git a/src/Parsers/ASTDropWorkloadQuery.h b/src/Parsers/ASTDropWorkloadQuery.h deleted file mode 100644 index 99c3a011447..00000000000 --- a/src/Parsers/ASTDropWorkloadQuery.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class ASTDropWorkloadQuery : public IAST, public ASTQueryWithOnCluster -{ -public: - String workload_name; - - bool if_exists = false; - - String getID(char) const override { return "DropWorkloadQuery"; } - - ASTPtr clone() const override; - - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; - - ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } - - QueryKind getQueryKind() const override { return QueryKind::Drop; } -}; - -} diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index dd0ba91d428..83b7eb71d64 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -392,7 +392,6 @@ namespace DB MR_MACROS(RANDOMIZE_FOR, "RANDOMIZE FOR") \ MR_MACROS(RANDOMIZED, "RANDOMIZED") \ MR_MACROS(RANGE, "RANGE") \ - MR_MACROS(READ, "READ") \ MR_MACROS(READONLY, "READONLY") \ MR_MACROS(REALM, "REALM") \ MR_MACROS(RECOMPRESS, "RECOMPRESS") \ @@ -412,7 +411,6 @@ namespace DB MR_MACROS(REPLACE, "REPLACE") \ MR_MACROS(RESET_SETTING, "RESET SETTING") \ MR_MACROS(RESET_AUTHENTICATION_METHODS_TO_NEW, "RESET AUTHENTICATION METHODS TO NEW") \ - MR_MACROS(RESOURCE, "RESOURCE") \ MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \ MR_MACROS(RESTORE, "RESTORE") \ MR_MACROS(RESTRICT, "RESTRICT") \ @@ -525,7 +523,6 @@ namespace DB MR_MACROS(WHEN, "WHEN") \ MR_MACROS(WHERE, "WHERE") \ MR_MACROS(WINDOW, "WINDOW") \ - MR_MACROS(WORKLOAD, "WORKLOAD") \ MR_MACROS(QUALIFY, "QUALIFY") \ MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \ MR_MACROS(WITH_CHECK, "WITH CHECK") \ @@ -538,7 +535,6 @@ namespace DB MR_MACROS(WITH, "WITH") \ MR_MACROS(RECURSIVE, "RECURSIVE") \ MR_MACROS(WK, "WK") \ - MR_MACROS(WRITE, "WRITE") \ MR_MACROS(WRITABLE, "WRITABLE") \ MR_MACROS(WW, "WW") \ MR_MACROS(YEAR, "YEAR") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp deleted file mode 100644 index 68c157df175..00000000000 --- a/src/Parsers/ParserCreateResourceQuery.cpp +++ /dev/null @@ -1,144 +0,0 @@ -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -bool parseOneOperation(ASTCreateResourceQuery::Operation & operation, IParser::Pos & pos, Expected & expected) -{ - ParserIdentifier disk_name_p; - - ASTCreateResourceQuery::AccessMode mode; - ASTPtr node; - std::optional disk; - - if (ParserKeyword(Keyword::WRITE).ignore(pos, expected)) - mode = ASTCreateResourceQuery::AccessMode::Write; - else if (ParserKeyword(Keyword::READ).ignore(pos, expected)) - mode = ASTCreateResourceQuery::AccessMode::Read; - else - return false; - - if (ParserKeyword(Keyword::ANY).ignore(pos, expected)) - { - if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) - return false; - } - else - { - if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) - return false; - - if (!disk_name_p.parse(pos, node, expected)) - return false; - - disk.emplace(); - if (!tryGetIdentifierNameInto(node, *disk)) - return false; - } - - operation.mode = mode; - operation.disk = std::move(disk); - - return true; -} - -bool parseOperations(IParser::Pos & pos, Expected & expected, ASTCreateResourceQuery::Operations & operations) -{ - return IParserBase::wrapParseImpl(pos, [&] - { - ParserToken s_open(TokenType::OpeningRoundBracket); - ParserToken s_close(TokenType::ClosingRoundBracket); - - if (!s_open.ignore(pos, expected)) - return false; - - ASTCreateResourceQuery::Operations res_operations; - - auto parse_operation = [&] - { - ASTCreateResourceQuery::Operation operation; - if (!parseOneOperation(operation, pos, expected)) - return false; - res_operations.push_back(std::move(operation)); - return true; - }; - - if (!ParserList::parseUtil(pos, expected, parse_operation, false)) - return false; - - if (!s_close.ignore(pos, expected)) - return false; - - operations = std::move(res_operations); - return true; - }); -} - -} - -bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword s_create(Keyword::CREATE); - ParserKeyword s_resource(Keyword::RESOURCE); - ParserKeyword s_or_replace(Keyword::OR_REPLACE); - ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); - ParserKeyword s_on(Keyword::ON); - ParserIdentifier resource_name_p; - - ASTPtr resource_name; - - String cluster_str; - bool or_replace = false; - bool if_not_exists = false; - - if (!s_create.ignore(pos, expected)) - return false; - - if (s_or_replace.ignore(pos, expected)) - or_replace = true; - - if (!s_resource.ignore(pos, expected)) - return false; - - if (!or_replace && s_if_not_exists.ignore(pos, expected)) - if_not_exists = true; - - if (!resource_name_p.parse(pos, resource_name, expected)) - return false; - - if (s_on.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - - ASTCreateResourceQuery::Operations operations; - if (!parseOperations(pos, expected, operations)) - return false; - - auto create_resource_query = std::make_shared(); - node = create_resource_query; - - create_resource_query->resource_name = resource_name; - create_resource_query->children.push_back(resource_name); - - create_resource_query->or_replace = or_replace; - create_resource_query->if_not_exists = if_not_exists; - create_resource_query->cluster = std::move(cluster_str); - - create_resource_query->operations = std::move(operations); - - return true; -} - -} diff --git a/src/Parsers/ParserCreateResourceQuery.h b/src/Parsers/ParserCreateResourceQuery.h deleted file mode 100644 index 1b7c9fc4a7f..00000000000 --- a/src/Parsers/ParserCreateResourceQuery.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include "IParserBase.h" - -namespace DB -{ - -/// CREATE RESOURCE cache_io (WRITE DISK s3diskWithCache, READ DISK s3diskWithCache) -class ParserCreateResourceQuery : public IParserBase -{ -protected: - const char * getName() const override { return "CREATE RESOURCE query"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -} diff --git a/src/Parsers/ParserCreateWorkloadEntity.cpp b/src/Parsers/ParserCreateWorkloadEntity.cpp deleted file mode 100644 index 013210a6d87..00000000000 --- a/src/Parsers/ParserCreateWorkloadEntity.cpp +++ /dev/null @@ -1,16 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -bool ParserCreateWorkloadEntity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserCreateWorkloadQuery create_workload_p; - ParserCreateResourceQuery create_resource_p; - - return create_workload_p.parse(pos, node, expected) || create_resource_p.parse(pos, node, expected); -} - -} diff --git a/src/Parsers/ParserCreateWorkloadEntity.h b/src/Parsers/ParserCreateWorkloadEntity.h deleted file mode 100644 index 1e7b78b3ccc..00000000000 --- a/src/Parsers/ParserCreateWorkloadEntity.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/// Special parser for the CREATE WORKLOAD and CREATE RESOURCE queries. -class ParserCreateWorkloadEntity : public IParserBase -{ -protected: - const char * getName() const override { return "CREATE workload entity query"; } - - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -} diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp deleted file mode 100644 index 9caf474741c..00000000000 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ /dev/null @@ -1,155 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -namespace -{ - -bool parseWorkloadSetting( - ASTCreateWorkloadQuery::SettingChange & change, IParser::Pos & pos, Expected & expected) -{ - ParserIdentifier name_p; - ParserLiteral value_p; - ParserToken s_eq(TokenType::Equals); - ParserIdentifier resource_name_p; - - ASTPtr name_node; - ASTPtr value_node; - ASTPtr resource_name_node; - - String name; - String resource_name; - - if (!name_p.parse(pos, name_node, expected)) - return false; - tryGetIdentifierNameInto(name_node, name); - - if (!s_eq.ignore(pos, expected)) - return false; - - if (!value_p.parse(pos, value_node, expected)) - return false; - - if (ParserKeyword(Keyword::FOR).ignore(pos, expected)) - { - if (!resource_name_p.parse(pos, resource_name_node, expected)) - return false; - tryGetIdentifierNameInto(resource_name_node, resource_name); - } - - change.name = std::move(name); - change.value = value_node->as().value; - change.resource = std::move(resource_name); - - return true; -} - -bool parseSettings(IParser::Pos & pos, Expected & expected, ASTCreateWorkloadQuery::SettingsChanges & changes) -{ - return IParserBase::wrapParseImpl(pos, [&] - { - if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) - return false; - - ASTCreateWorkloadQuery::SettingsChanges res_changes; - - auto parse_setting = [&] - { - ASTCreateWorkloadQuery::SettingChange change; - if (!parseWorkloadSetting(change, pos, expected)) - return false; - res_changes.push_back(std::move(change)); - return true; - }; - - if (!ParserList::parseUtil(pos, expected, parse_setting, false)) - return false; - - changes = std::move(res_changes); - return true; - }); -} - -} - -bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword s_create(Keyword::CREATE); - ParserKeyword s_workload(Keyword::WORKLOAD); - ParserKeyword s_or_replace(Keyword::OR_REPLACE); - ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); - ParserIdentifier workload_name_p; - ParserKeyword s_on(Keyword::ON); - ParserKeyword s_in(Keyword::IN); - - ASTPtr workload_name; - ASTPtr workload_parent; - - String cluster_str; - bool or_replace = false; - bool if_not_exists = false; - - if (!s_create.ignore(pos, expected)) - return false; - - if (s_or_replace.ignore(pos, expected)) - or_replace = true; - - if (!s_workload.ignore(pos, expected)) - return false; - - if (!or_replace && s_if_not_exists.ignore(pos, expected)) - if_not_exists = true; - - if (!workload_name_p.parse(pos, workload_name, expected)) - return false; - - if (s_on.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - - if (s_in.ignore(pos, expected)) - { - if (!workload_name_p.parse(pos, workload_parent, expected)) - return false; - } - - ASTCreateWorkloadQuery::SettingsChanges changes; - parseSettings(pos, expected, changes); - - auto create_workload_query = std::make_shared(); - node = create_workload_query; - - create_workload_query->workload_name = workload_name; - create_workload_query->children.push_back(workload_name); - - if (workload_parent) - { - create_workload_query->workload_parent = workload_parent; - create_workload_query->children.push_back(workload_parent); - } - - create_workload_query->or_replace = or_replace; - create_workload_query->if_not_exists = if_not_exists; - create_workload_query->cluster = std::move(cluster_str); - create_workload_query->changes = std::move(changes); - - - return true; -} - -} diff --git a/src/Parsers/ParserCreateWorkloadQuery.h b/src/Parsers/ParserCreateWorkloadQuery.h deleted file mode 100644 index 62c89affeda..00000000000 --- a/src/Parsers/ParserCreateWorkloadQuery.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include "IParserBase.h" - -namespace DB -{ - -/// CREATE WORKLOAD production IN all SETTINGS weight = 3, max_speed = '1G' FOR network_read, max_speed = '2G' FOR network_write -class ParserCreateWorkloadQuery : public IParserBase -{ -protected: - const char * getName() const override { return "CREATE WORKLOAD query"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - -} diff --git a/src/Parsers/ParserDropResourceQuery.cpp b/src/Parsers/ParserDropResourceQuery.cpp deleted file mode 100644 index 6c078281828..00000000000 --- a/src/Parsers/ParserDropResourceQuery.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -bool ParserDropResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword s_drop(Keyword::DROP); - ParserKeyword s_resource(Keyword::RESOURCE); - ParserKeyword s_if_exists(Keyword::IF_EXISTS); - ParserKeyword s_on(Keyword::ON); - ParserIdentifier resource_name_p; - - String cluster_str; - bool if_exists = false; - - ASTPtr resource_name; - - if (!s_drop.ignore(pos, expected)) - return false; - - if (!s_resource.ignore(pos, expected)) - return false; - - if (s_if_exists.ignore(pos, expected)) - if_exists = true; - - if (!resource_name_p.parse(pos, resource_name, expected)) - return false; - - if (s_on.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - - auto drop_resource_query = std::make_shared(); - drop_resource_query->if_exists = if_exists; - drop_resource_query->cluster = std::move(cluster_str); - - node = drop_resource_query; - - drop_resource_query->resource_name = resource_name->as().name(); - - return true; -} - -} diff --git a/src/Parsers/ParserDropResourceQuery.h b/src/Parsers/ParserDropResourceQuery.h deleted file mode 100644 index 651603d1e90..00000000000 --- a/src/Parsers/ParserDropResourceQuery.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include "IParserBase.h" - -namespace DB -{ -/// DROP RESOURCE resource1 -class ParserDropResourceQuery : public IParserBase -{ -protected: - const char * getName() const override { return "DROP RESOURCE query"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; -} diff --git a/src/Parsers/ParserDropWorkloadQuery.cpp b/src/Parsers/ParserDropWorkloadQuery.cpp deleted file mode 100644 index edc82c8f30a..00000000000 --- a/src/Parsers/ParserDropWorkloadQuery.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -bool ParserDropWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) -{ - ParserKeyword s_drop(Keyword::DROP); - ParserKeyword s_workload(Keyword::WORKLOAD); - ParserKeyword s_if_exists(Keyword::IF_EXISTS); - ParserKeyword s_on(Keyword::ON); - ParserIdentifier workload_name_p; - - String cluster_str; - bool if_exists = false; - - ASTPtr workload_name; - - if (!s_drop.ignore(pos, expected)) - return false; - - if (!s_workload.ignore(pos, expected)) - return false; - - if (s_if_exists.ignore(pos, expected)) - if_exists = true; - - if (!workload_name_p.parse(pos, workload_name, expected)) - return false; - - if (s_on.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - - auto drop_workload_query = std::make_shared(); - drop_workload_query->if_exists = if_exists; - drop_workload_query->cluster = std::move(cluster_str); - - node = drop_workload_query; - - drop_workload_query->workload_name = workload_name->as().name(); - - return true; -} - -} diff --git a/src/Parsers/ParserDropWorkloadQuery.h b/src/Parsers/ParserDropWorkloadQuery.h deleted file mode 100644 index af060caf303..00000000000 --- a/src/Parsers/ParserDropWorkloadQuery.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include "IParserBase.h" - -namespace DB -{ -/// DROP WORKLOAD workload1 -class ParserDropWorkloadQuery : public IParserBase -{ -protected: - const char * getName() const override { return "DROP WORKLOAD query"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; -} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 4ed6e4267f4..d5645298ecf 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,12 +1,8 @@ #include #include -#include -#include #include #include #include -#include -#include #include #include #include @@ -55,10 +51,6 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; - ParserCreateWorkloadQuery create_workload_p; - ParserDropWorkloadQuery drop_workload_p; - ParserCreateResourceQuery create_resource_p; - ParserDropResourceQuery drop_resource_p; ParserCreateNamedCollectionQuery create_named_collection_p; ParserDropNamedCollectionQuery drop_named_collection_p; ParserAlterNamedCollectionQuery alter_named_collection_p; @@ -90,10 +82,6 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) - || create_workload_p.parse(pos, node, expected) - || drop_workload_p.parse(pos, node, expected) - || create_resource_p.parse(pos, node, expected) - || drop_resource_p.parse(pos, node, expected) || create_named_collection_p.parse(pos, node, expected) || drop_named_collection_p.parse(pos, node, expected) || alter_named_collection_p.parse(pos, node, expected) diff --git a/src/Storages/System/StorageSystemResources.cpp b/src/Storages/System/StorageSystemResources.cpp deleted file mode 100644 index 2f948b8e057..00000000000 --- a/src/Storages/System/StorageSystemResources.cpp +++ /dev/null @@ -1,71 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -ColumnsDescription StorageSystemResources::getColumnsDescription() -{ - return ColumnsDescription - { - {"name", std::make_shared(), "The name of the resource."}, - {"read_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for read operations."}, - {"write_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for write operations."}, - {"create_query", std::make_shared(), "CREATE query of the resource."}, - }; -} - -void StorageSystemResources::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const -{ - const auto & storage = context->getWorkloadEntityStorage(); - const auto & resource_names = storage.getAllEntityNames(WorkloadEntityType::Resource); - for (const auto & resource_name : resource_names) - { - auto ast = storage.get(resource_name); - auto & resource = typeid_cast(*ast); - res_columns[0]->insert(resource_name); - { - Array read_disks; - Array write_disks; - for (const auto & [mode, disk] : resource.operations) - { - switch (mode) - { - case DB::ASTCreateResourceQuery::AccessMode::Read: - { - read_disks.emplace_back(disk ? *disk : "ANY"); - break; - } - case DB::ASTCreateResourceQuery::AccessMode::Write: - { - write_disks.emplace_back(disk ? *disk : "ANY"); - break; - } - } - } - res_columns[1]->insert(read_disks); - res_columns[2]->insert(write_disks); - } - res_columns[3]->insert(queryToString(ast)); - } -} - -void StorageSystemResources::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) -{ - // TODO(serxa): add backup for resources - // storage.backup(backup_entries_collector, data_path_in_backup); -} - -void StorageSystemResources::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) -{ - // TODO(serxa): add restore for resources - // storage.restore(restorer, data_path_in_backup); -} - -} diff --git a/src/Storages/System/StorageSystemResources.h b/src/Storages/System/StorageSystemResources.h deleted file mode 100644 index 42bbcd09aa4..00000000000 --- a/src/Storages/System/StorageSystemResources.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; - - -/// Implements `resources` system table, which allows you to get a list of all RESOURCEs -class StorageSystemResources final : public IStorageSystemOneBlock -{ -public: - std::string getName() const override { return "SystemResources"; } - - static ColumnsDescription getColumnsDescription(); - - void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; - void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; - -protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; - - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; -}; - -} diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 8784ba084ce..b42c807d6fc 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -84,12 +84,12 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { - context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, ISchedulerNode * node) + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) { size_t i = 0; res_columns[i++]->insert(resource); res_columns[i++]->insert(path); - res_columns[i++]->insert(node->getTypeName()); + res_columns[i++]->insert(type); res_columns[i++]->insert(node->info.weight); res_columns[i++]->insert(node->info.priority.value); res_columns[i++]->insert(node->isActive()); @@ -118,23 +118,23 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c if (auto * parent = dynamic_cast(node->parent)) { - if (auto value = parent->getChildVRuntime(node)) + if (auto value = parent->getChildVRuntime(node.get())) vruntime = *value; } - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) system_vruntime = ptr->getSystemVRuntime(); - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) budget = ptr->getBudget(); - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) is_satisfied = ptr->isSatisfied(); - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) { std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); std::tie(max_requests, max_cost) = ptr->getLimits(); } - if (auto * ptr = dynamic_cast(node)) + if (auto * ptr = dynamic_cast(node.get())) { std::tie(max_speed, max_burst) = ptr->getParams(); throttling_us = ptr->getThrottlingDuration().count() / 1000; diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp deleted file mode 100644 index ebb7e693e26..00000000000 --- a/src/Storages/System/StorageSystemWorkloads.cpp +++ /dev/null @@ -1,48 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -ColumnsDescription StorageSystemWorkloads::getColumnsDescription() -{ - return ColumnsDescription - { - {"name", std::make_shared(), "The name of the workload."}, - {"parent", std::make_shared(), "The name of the parent workload."}, - {"create_query", std::make_shared(), "CREATE query of the workload."}, - }; -} - -void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const -{ - const auto & storage = context->getWorkloadEntityStorage(); - const auto & workload_names = storage.getAllEntityNames(WorkloadEntityType::Workload); - for (const auto & workload_name : workload_names) - { - auto ast = storage.get(workload_name); - auto & workload = typeid_cast(*ast); - res_columns[0]->insert(workload_name); - res_columns[1]->insert(workload.getWorkloadParent()); - res_columns[2]->insert(queryToString(ast)); - } -} - -void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) -{ - // TODO(serxa): add backup for workloads - // storage.backup(backup_entries_collector, data_path_in_backup); -} - -void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) -{ - // TODO(serxa): add restore for workloads - // storage.restore(restorer, data_path_in_backup); -} - -} diff --git a/src/Storages/System/StorageSystemWorkloads.h b/src/Storages/System/StorageSystemWorkloads.h deleted file mode 100644 index 9d4770a02b8..00000000000 --- a/src/Storages/System/StorageSystemWorkloads.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class Context; - - -/// Implements `workloads` system table, which allows you to get a list of all workloads -class StorageSystemWorkloads final : public IStorageSystemOneBlock -{ -public: - std::string getName() const override { return "SystemWorkloads"; } - - static ColumnsDescription getColumnsDescription(); - - void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; - void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; - -protected: - using IStorageSystemOneBlock::IStorageSystemOneBlock; - - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; -}; - -} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 0bd3369ff32..70dcec884a6 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -23,8 +23,6 @@ #include #include #include -#include -#include #include #include #include @@ -232,8 +230,6 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach>(context, system_database, "azure_queue_settings", "Contains a list of settings of AzureQueue tables."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); - attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); - attach(context, system_database, "resources", "Contains a list of all currently existing resources."); if (has_zookeeper) { diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 9498044c836..823a00a05de 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -1,5 +1,4 @@ - /clickhouse/workload/definitions.sql @@ -13,15 +12,6 @@ network_read network_write - - s3 - http://minio1:9001/root/data/ - minio - minio123 - 33554432 - 10 - 10 - @@ -31,13 +21,6 @@ - - -

- s3_no_resource -
- - diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e4ef83759e4..050281b2e3a 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -2,7 +2,6 @@ # pylint: disable=redefined-outer-name # pylint: disable=line-too-long -import random import threading import time @@ -10,7 +9,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) @@ -25,21 +23,6 @@ node = cluster.add_instance( "configs/workloads.xml.default", ], with_minio=True, - with_zookeeper=True, -) - -node2 = cluster.add_instance( - "node2", - stay_alive=True, - main_configs=[ - "configs/storage_configuration.xml", - "configs/resources.xml", - "configs/resources.xml.default", - "configs/workloads.xml", - "configs/workloads.xml.default", - ], - with_minio=True, - with_zookeeper=True, ) @@ -72,22 +55,6 @@ def set_default_configs(): yield -@pytest.fixture(scope="function", autouse=True) -def clear_workloads_and_resources(): - node.query( - f""" - drop workload if exists production; - drop workload if exists development; - drop workload if exists admin; - drop workload if exists all; - drop resource if exists io_write; - drop resource if exists io_read; - drop resource if exists io; - """ - ) - yield - - def update_workloads_config(**settings): xml = "" for name in settings: @@ -603,364 +570,3 @@ def test_mutation_workload_change(): assert reads_before < reads_after assert writes_before < writes_after - - -def test_create_workload(): - node.query( - f""" - create resource io_write (write disk s3_no_resource); - create resource io_read (read disk s3_no_resource); - create workload all settings max_cost = 1000000 for io_write, max_cost = 2000000 for io_read; - create workload admin in all settings priority = 0; - create workload production in all settings priority = 1, weight = 9; - create workload development in all settings priority = 1, weight = 1; - """ - ) - - def do_checks(): - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/admin/%' and type='fifo'" - ) - == "2\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/admin' and type='unified' and priority=0" - ) - == "2\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/production/%' and type='fifo'" - ) - == "2\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/production' and type='unified' and weight=9" - ) - == "2\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/development/%' and type='fifo'" - ) - == "2\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_write' and max_cost=1000000" - ) - == "1\n" - ) - assert ( - node.query( - f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_read' and max_cost=2000000" - ) - == "1\n" - ) - - do_checks() - node.restart_clickhouse() # Check that workloads persist - do_checks() - - -def test_workload_hierarchy_changes(): - node.query("create resource io_write (write disk s3_no_resource);") - node.query("create resource io_read (read disk s3_no_resource);") - queries = [ - "create workload all;", - "create workload X in all settings priority = 0;", - "create workload Y in all settings priority = 1;", - "create workload A1 in X settings priority = -1;", - "create workload B1 in X settings priority = 1;", - "create workload C1 in Y settings priority = -1;", - "create workload D1 in Y settings priority = 1;", - "create workload A2 in X settings priority = -1;", - "create workload B2 in X settings priority = 1;", - "create workload C2 in Y settings priority = -1;", - "create workload D2 in Y settings priority = 1;", - "drop workload A1;", - "drop workload A2;", - "drop workload B1;", - "drop workload B2;", - "drop workload C1;", - "drop workload C2;", - "drop workload D1;", - "drop workload D2;", - "create workload Z in all;", - "create workload A1 in Z settings priority = -1;", - "create workload A2 in Z settings priority = -1;", - "create workload A3 in Z settings priority = -1;", - "create workload B1 in Z settings priority = 1;", - "create workload B2 in Z settings priority = 1;", - "create workload B3 in Z settings priority = 1;", - "create workload C1 in X settings priority = -1;", - "create workload C2 in X settings priority = -1;", - "create workload C3 in X settings priority = -1;", - "create workload D1 in X settings priority = 1;", - "create workload D2 in X settings priority = 1;", - "create workload D3 in X settings priority = 1;", - "drop workload A1;", - "drop workload B1;", - "drop workload C1;", - "drop workload D1;", - "drop workload A2;", - "drop workload B2;", - "drop workload C2;", - "drop workload D2;", - "drop workload A3;", - "drop workload B3;", - "drop workload C3;", - "drop workload D3;", - "drop workload X;", - "drop workload Y;", - "drop workload Z;", - "drop workload all;", - ] - for iteration in range(3): - split_idx = random.randint(1, len(queries) - 2) - for query_idx in range(0, split_idx): - node.query(queries[query_idx]) - node.query( - "create resource io_test (write disk non_existent_disk, read disk non_existent_disk);" - ) - node.query("drop resource io_test;") - for query_idx in range(split_idx, len(queries)): - node.query(queries[query_idx]) - - -def test_resource_read_and_write(): - node.query( - f""" - drop table if exists data; - create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; - """ - ) - - node.query( - f""" - create resource io_write (write disk s3_no_resource); - create resource io_read (read disk s3_no_resource); - create workload all settings max_cost = 1000000; - create workload admin in all settings priority = 0; - create workload production in all settings priority = 1, weight = 9; - create workload development in all settings priority = 1, weight = 1; - """ - ) - - def write_query(workload): - try: - node.query( - f"insert into data select * from numbers(1e5) settings workload='{workload}'" - ) - except QueryRuntimeException: - pass - - thread1 = threading.Thread(target=write_query, args=["development"]) - thread2 = threading.Thread(target=write_query, args=["production"]) - thread3 = threading.Thread(target=write_query, args=["admin"]) - - thread1.start() - thread2.start() - thread3.start() - - thread3.join() - thread2.join() - thread1.join() - - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/admin/%' and type='fifo'" - ) - == "1\n" - ) - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/development/%' and type='fifo'" - ) - == "1\n" - ) - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/production/%' and type='fifo'" - ) - == "1\n" - ) - - def read_query(workload): - try: - node.query(f"select sum(key*key) from data settings workload='{workload}'") - except QueryRuntimeException: - pass - - thread1 = threading.Thread(target=read_query, args=["development"]) - thread2 = threading.Thread(target=read_query, args=["production"]) - thread3 = threading.Thread(target=read_query, args=["admin"]) - - thread1.start() - thread2.start() - thread3.start() - - thread3.join() - thread2.join() - thread1.join() - - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/admin/%' and type='fifo'" - ) - == "1\n" - ) - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/development/%' and type='fifo'" - ) - == "1\n" - ) - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/production/%' and type='fifo'" - ) - == "1\n" - ) - - -def test_resource_any_disk(): - node.query( - f""" - drop table if exists data; - create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; - """ - ) - - node.query( - f""" - create resource io (write any disk, read any disk); - create workload all settings max_cost = 1000000; - """ - ) - - node.query(f"insert into data select * from numbers(1e5) settings workload='all'") - - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'" - ) - == "1\n" - ) - - node.query(f"select sum(key*key) from data settings workload='all'") - - assert ( - node.query( - f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'" - ) - == "1\n" - ) - - -def test_workload_entity_keeper_storage(): - node.query("create resource io_write (write disk s3_no_resource);") - node.query("create resource io_read (read disk s3_no_resource);") - queries = [ - "create workload all;", - "create workload X in all settings priority = 0;", - "create workload Y in all settings priority = 1;", - "create workload A1 in X settings priority = -1;", - "create workload B1 in X settings priority = 1;", - "create workload C1 in Y settings priority = -1;", - "create workload D1 in Y settings priority = 1;", - "create workload A2 in X settings priority = -1;", - "create workload B2 in X settings priority = 1;", - "create workload C2 in Y settings priority = -1;", - "create workload D2 in Y settings priority = 1;", - "drop workload A1;", - "drop workload A2;", - "drop workload B1;", - "drop workload B2;", - "drop workload C1;", - "drop workload C2;", - "drop workload D1;", - "drop workload D2;", - "create workload Z in all;", - "create workload A1 in Z settings priority = -1;", - "create workload A2 in Z settings priority = -1;", - "create workload A3 in Z settings priority = -1;", - "create workload B1 in Z settings priority = 1;", - "create workload B2 in Z settings priority = 1;", - "create workload B3 in Z settings priority = 1;", - "create workload C1 in X settings priority = -1;", - "create workload C2 in X settings priority = -1;", - "create workload C3 in X settings priority = -1;", - "create workload D1 in X settings priority = 1;", - "create workload D2 in X settings priority = 1;", - "create workload D3 in X settings priority = 1;", - "drop workload A1;", - "drop workload B1;", - "drop workload C1;", - "drop workload D1;", - "drop workload A2;", - "drop workload B2;", - "drop workload C2;", - "drop workload D2;", - "drop workload A3;", - "drop workload B3;", - "drop workload C3;", - "drop workload D3;", - "drop workload X;", - "drop workload Y;", - "drop workload Z;", - "drop workload all;", - ] - - def check_consistency(): - checks = [ - "select name, create_query from system.workloads order by all", - "select name, create_query from system.resources order by all", - "select resource, path, type, weight, priority, max_requests, max_cost, max_speed, max_burst from system.scheduler where resource not in ['network_read', 'network_write'] order by all", - ] - attempts = 10 - value1 = "" - value2 = "" - error_query = "" - for attempt in range(attempts): - for query in checks: - value1 = node.query(query) - value2 = node2.query(query) - if value1 != value2: - error_query = query - break # error - else: - break # success - time.sleep(0.5) - else: - raise Exception( - f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" - ) - - for iteration in range(3): - split_idx_1 = random.randint(1, len(queries) - 3) - split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node2) - for query_idx in range(0, split_idx_1): - node.query(queries[query_idx]) - - check_consistency() - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node2) - for query_idx in range(split_idx_1, split_idx_2): - node.query(queries[query_idx]) - - check_consistency() - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node2) - for query_idx in range(split_idx_2, len(queries)): - node.query(queries[query_idx]) - - check_consistency() diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 85ffee8e44d..10cedc36020 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -59,8 +59,6 @@ CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE ARBITRARY TEMPORARY TABLE CREATE ARBITRARY TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE -CREATE WORKLOAD [] GLOBAL CREATE -CREATE RESOURCE [] GLOBAL CREATE CREATE NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN CREATE [] \N ALL DROP DATABASE [] DATABASE DROP @@ -68,8 +66,6 @@ DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP -DROP WORKLOAD [] GLOBAL DROP -DROP RESOURCE [] GLOBAL DROP DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN DROP [] \N ALL UNDROP TABLE [] TABLE ALL diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.reference b/tests/queries/0_stateless/03232_resource_create_and_drop.reference deleted file mode 100644 index 2a1045d314c..00000000000 --- a/tests/queries/0_stateless/03232_resource_create_and_drop.reference +++ /dev/null @@ -1,5 +0,0 @@ -03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) -03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) -03232_resource_2 ['03232_disk_2'] [] CREATE RESOURCE `03232_resource_2` (READ DISK `03232_disk_2`) -03232_resource_3 [] ['03232_disk_2'] CREATE RESOURCE `03232_resource_3` (WRITE DISK `03232_disk_2`) -03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql deleted file mode 100644 index ceebd557a51..00000000000 --- a/tests/queries/0_stateless/03232_resource_create_and_drop.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-parallel --- Do not run this test in parallel because creating the same resource twice will fail -CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); -SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; -CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); -CREATE RESOURCE 03232_resource_3 (WRITE DISK 03232_disk_2); -SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; -DROP RESOURCE IF EXISTS 03232_resource_2; -DROP RESOURCE 03232_resource_3; -SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; -DROP RESOURCE 03232_resource_1; diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference deleted file mode 100644 index 923e8652a35..00000000000 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.reference +++ /dev/null @@ -1,5 +0,0 @@ -all CREATE WORKLOAD `all` -all CREATE WORKLOAD `all` -development all CREATE WORKLOAD development IN `all` -production all CREATE WORKLOAD production IN `all` -all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql deleted file mode 100644 index 1d8f97baf4c..00000000000 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.sql +++ /dev/null @@ -1,11 +0,0 @@ --- Tags: no-parallel --- Do not run this test in parallel because `all` workload might affect other queries execution process -CREATE OR REPLACE WORKLOAD all; -SELECT name, parent, create_query FROM system.workloads ORDER BY name; -CREATE WORKLOAD IF NOT EXISTS production IN all; -CREATE WORKLOAD development IN all; -SELECT name, parent, create_query FROM system.workloads ORDER BY name; -DROP WORKLOAD IF EXISTS production; -DROP WORKLOAD development; -SELECT name, parent, create_query FROM system.workloads ORDER BY name; -DROP WORKLOAD all; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.reference b/tests/queries/0_stateless/03232_workloads_and_resources.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql deleted file mode 100644 index a3e46166396..00000000000 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ /dev/null @@ -1,68 +0,0 @@ --- Tags: no-parallel --- Do not run this test in parallel because `all` workload might affect other queries execution process - --- Test simple resource and workload hierarchy creation -create resource 03232_write (write disk 03232_fake_disk); -create resource 03232_read (read disk 03232_fake_disk); -create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; -create workload admin in all settings priority = 0; -create workload production in all settings priority = 1, weight = 9; -create workload development in all settings priority = 1, weight = 1; - --- Test that illegal actions are not allowed -create workload another_root; -- {serverError BAD_ARGUMENTS} -create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} -drop workload all; -- {serverError BAD_ARGUMENTS} -create workload invalid in 03232_write; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} -create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} -create workload invalid in all settings weight = -1; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} -create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} -create or replace workload all in production; -- {serverError BAD_ARGUMENTS} - --- Test CREATE OR REPLACE WORKLOAD -create or replace workload all settings max_requests = 200 for 03232_write, max_requests = 100 for 03232_read; -create or replace workload admin in all settings priority = 1; -create or replace workload admin in all settings priority = 2; -create or replace workload admin in all settings priority = 0; -create or replace workload production in all settings priority = 1, weight = 90; -create or replace workload production in all settings priority = 0, weight = 9; -create or replace workload production in all settings priority = 2, weight = 9; -create or replace workload development in all settings priority = 1; -create or replace workload development in all settings priority = 0; -create or replace workload development in all settings priority = 2; - --- Test CREATE OR REPLACE RESOURCE -create or replace resource 03232_write (write disk 03232_fake_disk_2); -create or replace resource 03232_read (read disk 03232_fake_disk_2); - --- Test update settings with CREATE OR REPLACE WORKLOAD -create or replace workload production in all settings priority = 1, weight = 9, max_requests = 100; -create or replace workload development in all settings priority = 1, weight = 1, max_requests = 10; -create or replace workload production in all settings priority = 1, weight = 9, max_cost = 100000; -create or replace workload development in all settings priority = 1, weight = 1, max_cost = 10000; -create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000; -create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000; -create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000, max_burst = 10000000; -create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000, max_burst = 1000000; -create or replace workload all settings max_cost = 1000000, max_speed = 100000 for 03232_write, max_speed = 200000 for 03232_read; -create or replace workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; -create or replace workload production in all settings priority = 1, weight = 9; -create or replace workload development in all settings priority = 1, weight = 1; - --- Test change parent with CREATE OR REPLACE WORKLOAD -create or replace workload development in production settings priority = 1, weight = 1; -create or replace workload development in admin settings priority = 1, weight = 1; -create or replace workload development in all settings priority = 1, weight = 1; - --- Clean up -drop workload if exists production; -drop workload if exists development; -drop workload if exists admin; -drop workload if exists all; -drop resource if exists 03232_write; -drop resource if exists 03232_read; From 5e2355b1231774c7f3525c296df0e56ecb3d9c9f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 30 Oct 2024 13:01:20 +0100 Subject: [PATCH 629/816] better --- src/Planner/findParallelReplicasQuery.cpp | 23 ++++++++++++------- src/Processors/QueryPlan/SortingStep.h | 2 ++ ...3254_parallel_replicas_join_with_totals.sh | 2 ++ 3 files changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 66c7c6440c4..8a806045111 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -174,6 +174,14 @@ const QueryNode * findQueryForParallelReplicas( struct Frame { const QueryPlan::Node * node = nullptr; + /// Below we will check subqueries from `stack` to find outtermost subquery that could be executed remotely. + /// Currently traversal algorithm considers only steps with 0 or 1 children and JOIN specifically. + /// When we found some step that requires finalization on the initiator (e.g. GROUP BY) there are two options: + /// 1. If plan looks like a single path (e.g. AggregatingStep -> ExpressionStep -> Reading) we can execute + /// current subquery as a whole with replicas. + /// 2. If we were inside JOIN we cannot offload the whole subquery to replicas because at least one side + /// of the JOIN needs to be finalized on the initiator. + /// So this flag is used to track what subquery to return once we hit a step that needs finalization. bool inside_join = false; }; @@ -203,19 +211,21 @@ const QueryNode * findQueryForParallelReplicas( if (children.empty()) { - /// Found a source step. This should be possible only in the first iteration. - break; + /// Found a source step. } else if (children.size() == 1) { const auto * expression = typeid_cast(step); const auto * filter = typeid_cast(step); - const auto * sorting = typeid_cast(step); const auto * creating_sets = typeid_cast(step); - bool allowed_creating_sets = settings[Setting::parallel_replicas_allow_in_with_subquery] && creating_sets; + const bool allowed_creating_sets = settings[Setting::parallel_replicas_allow_in_with_subquery] && creating_sets; - if (!expression && !filter && !allowed_creating_sets && !(sorting && sorting->isSortingForMergeJoin())) + const auto * sorting = typeid_cast(step); + /// Sorting for merge join is supposed to be done locally before join itself, so it doesn't need finalization. + const bool allowed_sorting = sorting && sorting->isSortingForMergeJoin(); + + if (!expression && !filter && !allowed_creating_sets && !allowed_sorting) { can_distribute_full_node = false; currently_inside_join = inside_join; @@ -236,8 +246,6 @@ const QueryNode * findQueryForParallelReplicas( } } - /// Current node contains steps like GROUP BY / DISTINCT - /// Will try to execute query up to WithMergableStage if (!can_distribute_full_node) { /// Current query node does not contain subqueries. @@ -245,7 +253,6 @@ const QueryNode * findQueryForParallelReplicas( if (!res) return nullptr; - /// todo return currently_inside_join ? res : subquery_node; } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 9af591d603a..be2e4b0149c 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -127,6 +127,8 @@ private: const SortDescription result_description; SortDescription partition_by_description; + + /// See `findQueryForParallelReplicas` bool is_sorting_for_merge_join = false; UInt64 limit; diff --git a/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh index d3780d12ae0..365d7abed7a 100755 --- a/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh +++ b/tests/queries/0_stateless/03254_parallel_replicas_join_with_totals.sh @@ -20,6 +20,8 @@ INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); for enable_parallel_replicas in {0..1}; do ${CLICKHOUSE_CLIENT} --query=" + --- Old analyzer uses different code path and it produces wrong result in this case. + set enable_analyzer=1; set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; SELECT * From 9d0cc298ebb4263119f2b844c1c9843187dc9e70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:04:33 +0100 Subject: [PATCH 630/816] Adjust test for old analyzer --- .../0_stateless/03257_scalar_in_format_table_expression.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql index 1d74b0c3775..ec89c9874e9 100644 --- a/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql +++ b/tests/queries/0_stateless/03257_scalar_in_format_table_expression.sql @@ -32,9 +32,11 @@ $$ -- https://github.com/ClickHouse/ClickHouse/issues/70177 -- Resolution of the scalar subquery should work ok (already did, adding a test just for safety) +-- Disabled for the old analyzer since it incorrectly passes 's' to format, instead of resolving s and passing that WITH (SELECT sum(number)::String as s FROM numbers(4)) as s SELECT *, s -FROM format(TSVRaw, s); +FROM format(TSVRaw, s) +SETTINGS enable_analyzer=1; SELECT count() FROM format(TSVRaw, ( @@ -76,7 +78,7 @@ FROM format(TSVRaw, ( )), toLowCardinality('some long string')) RESPECT NULLS, '\n'), 'LowCardinality(String)') FROM numbers(10000) )) -FORMAT TSVRaw; -- { serverError UNKNOWN_IDENTIFIER } +FORMAT TSVRaw; -- { serverError UNKNOWN_IDENTIFIER, ILLEGAL_TYPE_OF_ARGUMENT } -- Same but for table function numbers SELECT 1 FROM numbers((SELECT DEFAULT)); -- { serverError UNKNOWN_IDENTIFIER } From fae5b1170910d8e2b6cd0bf7e12b9a72cbb9bb67 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 30 Oct 2024 12:06:30 +0000 Subject: [PATCH 631/816] Fix #69010 --- src/Interpreters/Cache/QueryCache.cpp | 37 +++++++++++++++++-- .../02494_query_cache_system_tables.sql | 8 +++- 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index c766c5209fc..cfd7608b6c6 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -89,11 +89,40 @@ struct HasSystemTablesMatcher { database_table = identifier->name(); } - /// Handle SELECT [...] FROM clusterAllReplicas(, '') - else if (const auto * literal = node->as()) + /// SELECT [...] FROM clusterAllReplicas(, '
') + /// This SQL syntax is quite common but we need to be careful. A naive attempt to cast 'node' to an ASTLiteral will be too general + /// and introduce false positives in queries like + /// 'SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;' + /// Therefore, make sure we are really in `clusterAllReplicas`. EXPLAIN AST for + /// 'SELECT * FROM clusterAllReplicas('default', system.one) SETTINGS use_query_cache = 1' + /// returns: + /// [...] + /// Function clusterAllReplicas (children 1) + /// ExpressionList (children 2) + /// Literal 'test_shard_localhost' + /// Literal 'system.one' + /// [...] + else if (const auto * function = node->as()) { - const auto & value = literal->value; - database_table = toString(value); + if (function->name == "clusterAllReplicas") + { + const ASTs & function_children = function->children; + if (!function_children.empty()) + { + if (const auto * expression_list = function_children[0]->as()) + { + const ASTs & expression_list_children = expression_list->children; + if (!expression_list_children.empty()) + { + if (const auto * literal = expression_list_children[1]->as()) + { + const auto & value = literal->value; + database_table = toString(value); + } + } + } + } + } } Tokens tokens(database_table.c_str(), database_table.c_str() + database_table.size(), /*max_query_size*/ 2048, /*skip_insignificant*/ true); diff --git a/tests/queries/0_stateless/02494_query_cache_system_tables.sql b/tests/queries/0_stateless/02494_query_cache_system_tables.sql index 7c9f01c4e91..12eaec0f8bc 100644 --- a/tests/queries/0_stateless/02494_query_cache_system_tables.sql +++ b/tests/queries/0_stateless/02494_query_cache_system_tables.sql @@ -44,9 +44,16 @@ SELECT * SETTINGS use_query_cache = 1; SELECT * FROM information_schema.tables SETTINGS use_query_cache = 1; -- { serverError QUERY_CACHE_USED_WITH_SYSTEM_TABLE } SELECT * FROM INFORMATION_SCHEMA.TABLES SETTINGS use_query_cache = 1; -- { serverError QUERY_CACHE_USED_WITH_SYSTEM_TABLE } +-- Issue #69010: A system table name appears as a literal. That's okay and must not throw. +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (uid Int16, name String) ENGINE = Memory; +SELECT * FROM tab WHERE name = 'system.one' SETTINGS use_query_cache = true; +DROP TABLE tab; + -- System tables can be "hidden" inside e.g. table functions SELECT * FROM clusterAllReplicas('test_shard_localhost', system.one) SETTINGS use_query_cache = 1; -- {serverError QUERY_CACHE_USED_WITH_SYSTEM_TABLE } SELECT * FROM clusterAllReplicas('test_shard_localhost', 'system.one') SETTINGS use_query_cache = 1; -- {serverError QUERY_CACHE_USED_WITH_SYSTEM_TABLE } +-- Note how in the previous query ^^ 'system.one' is also a literal. ClusterAllReplicas gets special handling. -- Criminal edge case that a user creates a table named "system". The query cache must not reject queries against it. DROP TABLE IF EXISTS system; @@ -60,5 +67,4 @@ CREATE TABLE system.system (c UInt64) ENGINE = Memory; SElECT * FROM system.system SETTINGS use_query_cache = 1; -- { serverError QUERY_CACHE_USED_WITH_SYSTEM_TABLE } DROP TABLE system.system; --- Cleanup SYSTEM DROP QUERY CACHE; From bbbb81f43dfa09cc1727b8596685a6acfe57ea9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:23:48 +0100 Subject: [PATCH 632/816] Improvements based on review --- src/Core/BaseSettings.cpp | 2 +- src/Core/Settings.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/BaseSettings.cpp b/src/Core/BaseSettings.cpp index 9d55179a5db..2cce94f9d0a 100644 --- a/src/Core/BaseSettings.cpp +++ b/src/Core/BaseSettings.cpp @@ -41,7 +41,7 @@ UInt64 BaseSettingsHelpers::readFlags(ReadBuffer & in) SettingsTierType BaseSettingsHelpers::getTier(UInt64 flags) { - int8_t tier = (flags & Flags::TIER); + int8_t tier = static_cast(flags & Flags::TIER); if (tier > SettingsTierType::BETA) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown tier value: '{}'", tier); return SettingsTierType{tier}; diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4159758fe76..aa9b7fd817b 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -38,7 +38,9 @@ namespace ErrorCodes * Note: as an alternative, we could implement settings to be completely dynamic in the form of the map: String -> Field, * but we are not going to do it, because settings are used everywhere as static struct fields. * - * `flags` can be either 0 or IMPORTANT + a Tier (PRODUCTION | BETA | EXPERIMENTAL) + * `flags` can include a Tier (BETA | EXPERIMENTAL) and an optional bitwise AND with IMPORTANT. + * The default (0) means a PRODUCTION ready setting + * * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions. * Tiers: * EXPERIMENTAL: The feature is in active development stage. Mostly for developers or for ClickHouse enthusiasts. @@ -5824,8 +5826,6 @@ Experimental data deduplication for SELECT queries based on part UUIDs \ /* ####################################################### */ \ /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ - /* ## ADD PRODUCTION / BETA FEATURES BEFORE THIS BLOCK ## */ \ - /* ####################################################### */ \ /* ####################################################### */ \ // End of COMMON_SETTINGS From 0dcb2b9c2c61674be298b706498763e8fcae7018 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 12:24:39 +0000 Subject: [PATCH 633/816] try another approach --- src/Interpreters/FillingRow.cpp | 315 +++++++++++++++--- src/Interpreters/FillingRow.h | 18 +- .../Transforms/FillingTransform.cpp | 92 +++-- 3 files changed, 348 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 8c5f102bcd6..caf6ad9e3ba 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -1,4 +1,7 @@ +#include #include +#include "Common/Logger.h" +#include "Common/logger_useful.h" #include #include @@ -95,108 +98,326 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, Field next_value = shifted_value; descr.step_func(next_value, step_len); - if (less(next_value, to, getDirection(0))) + // if (less(next_value, to, getDirection(0))) + // { + // shifted_value = std::move(next_value); + // step_len *= 2; + // } + // else + // { + // step_len /= 2; + // } + + if (less(to, next_value, getDirection(0))) { - shifted_value = std::move(next_value); - step_len *= 2; + step_len /= 2; } else { - step_len /= 2; + shifted_value = std::move(next_value); + step_len *= 2; } } return shifted_value; } -std::pair FillingRow::next(const FillingRow & to_row, bool long_jump) +Field findMin(Field a, Field b, Field c, int dir) { + auto logger = getLogger("FillingRow"); + LOG_DEBUG(logger, "a: {} b: {} c: {}", a.dump(), b.dump(), c.dump()); + + if (a.isNull() || (!b.isNull() && less(b, a, dir))) + a = b; + + if (a.isNull() || (!c.isNull() && less(c, a, dir))) + a = c; + + return a; +} + +std::pair FillingRow::next(const FillingRow & next_original_row) +{ + auto logger = getLogger("FillingRow"); + const size_t row_size = size(); size_t pos = 0; /// Find position we need to increment for generating next row. for (; pos < row_size; ++pos) - if (!row[pos].isNull() && !to_row.row[pos].isNull() && !equals(row[pos], to_row.row[pos])) - break; + { + if (row[pos].isNull()) + continue; - if (pos == row_size || less(to_row.row[pos], row[pos], getDirection(pos))) + const auto & descr = getFillDescription(pos); + auto min_constr = findMin(next_original_row[pos], staleness_border[pos], descr.fill_to, getDirection(pos)); + LOG_DEBUG(logger, "min_constr: {}", min_constr); + + if (!min_constr.isNull() && !equals(row[pos], min_constr)) + break; + } + + LOG_DEBUG(logger, "pos: {}", pos); + + if (pos == row_size) return {false, false}; - /// If we have any 'fill_to' value at position greater than 'pos', - /// we need to generate rows up to 'fill_to' value. + const auto & pos_descr = getFillDescription(pos); + + if (!next_original_row[pos].isNull() && less(next_original_row[pos], row[pos], getDirection(pos))) + return {false, false}; + + if (!staleness_border[pos].isNull() && !less(row[pos], staleness_border[pos], getDirection(pos))) + return {false, false}; + + if (!pos_descr.fill_to.isNull() && !less(row[pos], pos_descr.fill_to, getDirection(pos))) + return {false, false}; + + /// If we have any 'fill_to' value at position greater than 'pos' or configured staleness, + /// we need to generate rows up to one of this borders. for (size_t i = row_size - 1; i > pos; --i) { auto & fill_column_desc = getFillDescription(i); - if (fill_column_desc.fill_to.isNull() || row[i].isNull()) + if (row[i].isNull()) continue; - auto next_value = doJump(fill_column_desc, i); - if (next_value.has_value() && !equals(next_value.value(), fill_column_desc.fill_to)) - { - row[i] = std::move(next_value.value()); - initFromDefaults(i + 1); - return {true, true}; - } + if (fill_column_desc.fill_to.isNull() && staleness_border[i].isNull()) + continue; + + Field next_value = row[i]; + fill_column_desc.step_func(next_value, 1); + + if (!staleness_border[i].isNull() && !less(next_value, staleness_border[i], getDirection(i))) + continue; + + if (!fill_column_desc.fill_to.isNull() && !less(next_value, fill_column_desc.fill_to, getDirection(i))) + continue; + + row[i] = next_value; + initWithFrom(i + 1); + return {true, true}; } - auto & fill_column_desc = getFillDescription(pos); - std::optional next_value; + auto next_value = row[pos]; + getFillDescription(pos).step_func(next_value, 1); - if (long_jump) - { - next_value = doLongJump(fill_column_desc, pos, to_row[pos]); - - if (!next_value.has_value()) - return {false, false}; - - /// We need value >= to_row[pos] - fill_column_desc.step_func(next_value.value(), 1); - } - else - { - next_value = doJump(fill_column_desc, pos); - } - - if (!next_value.has_value() || less(to_row.row[pos], next_value.value(), getDirection(pos)) || equals(next_value.value(), getFillDescription(pos).fill_to)) + if (!next_original_row[pos].isNull() && less(next_original_row[pos], next_value, getDirection(pos))) return {false, false}; - row[pos] = std::move(next_value.value()); - if (equals(row[pos], to_row.row[pos])) + if (!staleness_border[pos].isNull() && !less(next_value, staleness_border[pos], getDirection(pos))) + return {false, false}; + + if (!pos_descr.fill_to.isNull() && !less(next_value, pos_descr.fill_to, getDirection(pos))) + return {false, false}; + + row[pos] = next_value; + if (equals(row[pos], next_original_row[pos])) { bool is_less = false; for (size_t i = pos + 1; i < row_size; ++i) { - const auto & fill_from = getFillDescription(i).fill_from; - if (!fill_from.isNull()) - row[i] = fill_from; + const auto & descr = getFillDescription(i); + if (!descr.fill_from.isNull()) + row[i] = descr.fill_from; else - row[i] = to_row.row[i]; - is_less |= less(row[i], to_row.row[i], getDirection(i)); + row[i] = next_original_row[i]; + + is_less |= ( + (next_original_row[i].isNull() || less(row[i], next_original_row[i], getDirection(i))) && + (staleness_border[i].isNull() || less(row[i], staleness_border[i], getDirection(i))) && + (descr.fill_to.isNull() || less(row[i], descr.fill_to, getDirection(i))) + ); } return {is_less, true}; } - initFromDefaults(pos + 1); + initWithFrom(pos + 1); return {true, true}; } -void FillingRow::initFromDefaults(size_t from_pos) +bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed) +{ + auto logger = getLogger("FillingRow::shift"); + LOG_DEBUG(logger, "next_original_row: {}, current: {}", next_original_row.dump(), dump()); + + for (size_t pos = 0; pos < size(); ++pos) + { + if (row[pos].isNull() || next_original_row[pos].isNull() || equals(row[pos], next_original_row[pos])) + continue; + + if (less(next_original_row[pos], row[pos], getDirection(pos))) + return false; + + std::optional next_value = doLongJump(getFillDescription(pos), pos, next_original_row[pos]); + + if (!next_value.has_value()) + { + LOG_DEBUG(logger, "next value: {}", "None"); + continue; + } + else + { + LOG_DEBUG(logger, "next value: {}", next_value->dump()); + } + + row[pos] = std::move(next_value.value()); + + if (equals(row[pos], next_original_row[pos])) + { + bool is_less = false; + for (size_t i = pos + 1; i < size(); ++i) + { + const auto & descr = getFillDescription(i); + if (!descr.fill_from.isNull()) + row[i] = descr.fill_from; + else + row[i] = next_original_row[i]; + + is_less |= ( + (next_original_row[i].isNull() || less(row[i], next_original_row[i], getDirection(i))) && + (staleness_border[i].isNull() || less(row[i], staleness_border[i], getDirection(i))) && + (descr.fill_to.isNull() || less(row[i], descr.fill_to, getDirection(i))) + ); + } + + LOG_DEBUG(logger, "is less: {}", is_less); + + value_changed = true; + return is_less; + } + else + { + // getFillDescription(pos).step_func(row[pos], 1); + initWithTo(/*from_pos=*/pos + 1); + + value_changed = false; + return false; + } + } + + return false; +} + +bool FillingRow::isConstraintComplete(size_t pos) const +{ + auto logger = getLogger("FillingRow::isConstraintComplete"); + + if (row[pos].isNull()) + { + LOG_DEBUG(logger, "disabled"); + return true; /// disabled + } + + const auto & descr = getFillDescription(pos); + int direction = getDirection(pos); + + if (!descr.fill_to.isNull() && !less(row[pos], descr.fill_to, direction)) + { + LOG_DEBUG(logger, "fill to: {}, row: {}, direction: {}", descr.fill_to.dump(), row[pos].dump(), direction); + return false; + } + + if (!staleness_border[pos].isNull() && !less(row[pos], staleness_border[pos], direction)) + { + LOG_DEBUG(logger, "staleness border: {}, row: {}, direction: {}", staleness_border[pos].dump(), row[pos].dump(), direction); + return false; + } + + return true; +} + +bool FillingRow::isConstraintsComplete() const +{ + for (size_t pos = 0; pos < size(); ++pos) + { + if (isConstraintComplete(pos)) + return true; + } + + return false; +} + +bool FillingRow::isLessStaleness() const +{ + auto logger = getLogger("FillingRow::isLessStaleness"); + + for (size_t pos = 0; pos < size(); ++pos) + { + LOG_DEBUG(logger, "staleness border: {}, row: {}", staleness_border[pos].dump(), row[pos].dump()); + + if (row[pos].isNull() || staleness_border[pos].isNull()) + continue; + + if (less(row[pos], staleness_border[pos], getDirection(pos))) + return true; + } + + return false; +} + +bool FillingRow::isStalenessConfigured() const +{ + for (size_t pos = 0; pos < size(); ++pos) + if (!getFillDescription(pos).fill_staleness.isNull()) + return true; + + return false; +} + +bool FillingRow::isLessFillTo() const +{ + auto logger = getLogger("FillingRow::isLessFillTo"); + + for (size_t pos = 0; pos < size(); ++pos) + { + const auto & descr = getFillDescription(pos); + + LOG_DEBUG(logger, "fill to: {}, row: {}", descr.fill_to.dump(), row[pos].dump()); + + if (row[pos].isNull() || descr.fill_to.isNull()) + continue; + + if (less(row[pos], descr.fill_to, getDirection(pos))) + return true; + } + + return false; +} + +bool FillingRow::isFillToConfigured() const +{ + for (size_t pos = 0; pos < size(); ++pos) + if (!getFillDescription(pos).fill_to.isNull()) + return true; + + return false; +} + + +void FillingRow::initWithFrom(size_t from_pos) { for (size_t i = from_pos; i < sort_description.size(); ++i) row[i] = getFillDescription(i).fill_from; } +void FillingRow::initWithTo(size_t from_pos) +{ + for (size_t i = from_pos; i < sort_description.size(); ++i) + row[i] = getFillDescription(i).fill_to; +} + void FillingRow::initStalenessRow(const Columns& base_row, size_t row_ind) { for (size_t i = 0; i < size(); ++i) { - staleness_border[i] = (*base_row[i])[row_ind]; - const auto& descr = getFillDescription(i); if (!descr.fill_staleness.isNull()) + { + staleness_border[i] = (*base_row[i])[row_ind]; descr.staleness_step_func(staleness_border[i], 1); + } } } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index dc787173191..a5e622e4c6e 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -25,9 +25,22 @@ public: /// Return pair of boolean /// apply - true if filling values should be inserted into result set /// value_changed - true if filling row value was changed - std::pair next(const FillingRow & to_row, bool long_jump); + std::pair next(const FillingRow & next_original_row); - void initFromDefaults(size_t from_pos = 0); + /// Returns true if need to generate some prefix for to_row + bool shift(const FillingRow & next_original_row, bool& value_changed); + + bool isConstraintComplete(size_t pos) const; + bool isConstraintsComplete() const; + + bool isLessStaleness() const; + bool isStalenessConfigured() const; + + bool isLessFillTo() const; + bool isFillToConfigured() const; + + void initWithFrom(size_t from_pos = 0); + void initWithTo(size_t from_pos = 0); void initStalenessRow(const Columns& base_row, size_t row_ind); Field & operator[](size_t index) { return row[index]; } @@ -39,6 +52,7 @@ public: bool isNull() const; int getDirection(size_t index) const { return sort_description[index].direction; } + Field getStalenessBorder(size_t index) const { return staleness_border[index]; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } const FillColumnDescription & getFillDescription(size_t index) const { return sort_description[index].fill_description; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 46a670394a5..a3a185929dc 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -11,13 +11,14 @@ #include #include #include +#include "Interpreters/FillingRow.h" #include namespace DB { -constexpr bool debug_logging_enabled = false; +constexpr bool debug_logging_enabled = true; template void logDebug(String key, const T & value, const char * separator = " : ") @@ -507,18 +508,39 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) - next_row[i] = filling_row.getFillDescription(i).fill_to; + next_row[i] = Field{}; logDebug("generateSuffixIfNeeded next_row updated", next_row); - if (filling_row >= next_row) + // if (!filling_row.isFillToConfigured() && !filling_row.isStalenessConfigured()) + // { + // logDebug("generateSuffixIfNeeded", "no other constraints, will not generate suffix"); + // return false; + // } + + // logDebug("filling_row.isLessFillTo()", filling_row.isLessFillTo()); + // logDebug("filling_row.isLessStaleness()", filling_row.isLessStaleness()); + + // if (filling_row.isFillToConfigured() && !filling_row.isLessFillTo()) + // { + // logDebug("generateSuffixIfNeeded", "not less than fill to, will not generate suffix"); + // return false; + // } + + // if (filling_row.isStalenessConfigured() && !filling_row.isLessStaleness()) + // { + // logDebug("generateSuffixIfNeeded", "not less than staleness border, will not generate suffix"); + // return false; + // } + + if (!filling_row.isConstraintsComplete()) { - logDebug("generateSuffixIfNeeded", "no need to generate suffix"); + logDebug("generateSuffixIfNeeded", "will not generate suffix"); return false; } Block interpolate_block; - if (should_insert_first && filling_row < next_row) + if (should_insert_first) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); @@ -533,7 +555,7 @@ bool FillingTransform::generateSuffixIfNeeded( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/false); + const auto [apply, changed] = filling_row.next(next_row); filling_row_changed = changed; if (!apply) break; @@ -615,7 +637,7 @@ void FillingTransform::transformRange( if (!fill_from.isNull() && !equals(current_value, fill_from)) { - filling_row.initFromDefaults(i); + filling_row.initWithFrom(i); filling_row_inserted = false; if (less(fill_from, current_value, filling_row.getDirection(i))) { @@ -642,24 +664,14 @@ void FillingTransform::transformRange( logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) - { - const auto current_value = (*input_fill_columns[i])[row_ind]; - const auto & fill_to = filling_row.getFillDescription(i).fill_to; + next_row[i] = (*input_fill_columns[i])[row_ind]; - logDebug("current value", current_value.dump()); - logDebug("fill to", fill_to.dump()); - - if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) - next_row[i] = current_value; - else - next_row[i] = fill_to; - } logDebug("next_row updated", next_row); /// The condition is true when filling row is initialized by value(s) in FILL FROM, /// and there are row(s) in current range with value(s) < then in the filling row. /// It can happen only once for a range. - if (should_insert_first && filling_row < next_row) + if (should_insert_first && filling_row < next_row && filling_row.isConstraintsComplete()) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); @@ -669,7 +681,7 @@ void FillingTransform::transformRange( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/false); + const auto [apply, changed] = filling_row.next(next_row); filling_row_changed = changed; if (!apply) break; @@ -679,12 +691,36 @@ void FillingTransform::transformRange( copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } - const auto [apply, changed] = filling_row.next(next_row, /*long_jump=*/true); - logDebug("long jump apply", apply); - logDebug("long jump changed", changed); + { + filling_row.initStalenessRow(input_fill_columns, row_ind); - if (changed) - filling_row_changed = true; + bool shift_apply = filling_row.shift(next_row, filling_row_changed); + logDebug("shift_apply", shift_apply); + logDebug("filling_row_changed", filling_row_changed); + + while (shift_apply) + { + logDebug("after shift", filling_row); + + while (true) + { + logDebug("filling_row in prefix", filling_row); + + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + + const auto [apply, changed] = filling_row.next(next_row); + logDebug("filling_row in prefix", filling_row); + + filling_row_changed = changed; + if (!apply) + break; + } + + shift_apply = filling_row.shift(next_row, filling_row_changed); + } + } /// new valid filling row was generated but not inserted, will use it during suffix generation if (filling_row_changed) @@ -697,8 +733,8 @@ void FillingTransform::transformRange( copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); - /// Init next staleness interval with current row, because we have already made the long jump to it - filling_row.initStalenessRow(input_fill_columns, row_ind); + // /// Init next staleness interval with current row, because we have already made the long jump to it + // filling_row.initStalenessRow(input_fill_columns, row_ind); } /// save sort prefix of last row in the range, it's used to generate suffix @@ -744,7 +780,7 @@ void FillingTransform::transform(Chunk & chunk) /// if no data was processed, then need to initialize filling_row if (last_row.empty()) { - filling_row.initFromDefaults(); + filling_row.initWithFrom(); filling_row_inserted = false; } From 4364be72f1983fc8306eb5e4e209c71d64a0e71a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:27:12 +0100 Subject: [PATCH 634/816] Mark merge_selector_algorithm as experimental --- src/Core/Settings.cpp | 3 ++- src/Storages/MergeTree/MergeTreeSettings.cpp | 8 +++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index aa9b7fd817b..1c392d2c547 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5905,13 +5905,14 @@ Experimental data deduplication for SELECT queries based on part UUIDs /** The section above is for obsolete settings. Do not add anything there. */ #endif /// __CLION_IDE__ - #define LIST_OF_SETTINGS(M, ALIAS) \ COMMON_SETTINGS(M, ALIAS) \ OBSOLETE_SETTINGS(M, ALIAS) \ FORMAT_FACTORY_SETTINGS(M, ALIAS) \ OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \ +// clang-format on + DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS) IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 36e146f4624..38c8f389fbe 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -30,10 +30,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +// clang-format off + /** These settings represent fine tunes for internal details of MergeTree storages * and should not be changed by the user without a reason. */ - #define MERGE_TREE_SETTINGS(DECLARE, ALIAS) \ DECLARE(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \ DECLARE(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \ @@ -98,7 +99,7 @@ namespace ErrorCodes DECLARE(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ DECLARE(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ DECLARE(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", 0) \ + DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \ \ /** Inserts settings. */ \ DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ @@ -276,8 +277,9 @@ namespace ErrorCodes MERGE_TREE_SETTINGS(M, ALIAS) \ OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) -DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) +// clang-format on +DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) /** Settings for the MergeTree family of engines. * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). From d064c3f317dbfc8b9079547bb851807b81740194 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 13:30:33 +0100 Subject: [PATCH 635/816] Make cloud sync title shorter --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index fc67959013b..8757332dcb0 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -241,7 +241,7 @@ class StatusNames(metaclass=WithIter): # mergeable status MERGEABLE = "Mergeable Check" # status of a sync pr - SYNC = "Cloud fork sync (only for ClickHouse Inc. employees)" + SYNC = "CH Inc sync" # PR formatting check status PR_CHECK = "PR Check" From 98f358baa3cac9813ed071067686af56653792c5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 30 Oct 2024 13:42:27 +0100 Subject: [PATCH 636/816] add test --- ...eplicas_join_algo_and_analyzer_4.reference | 29 ++++++ ...allel_replicas_join_algo_and_analyzer_4.sh | 93 +++++++++++++++++++ 2 files changed, 122 insertions(+) create mode 100644 tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference create mode 100755 tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference new file mode 100644 index 00000000000..9fc156b5fb0 --- /dev/null +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference @@ -0,0 +1,29 @@ +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` GROUP BY `__table1`.`item_id` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP BY `__table1`.`item_id` +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` ALL LEFT JOIN (SELECT `__table4`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table4`) AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh new file mode 100755 index 00000000000..a588fa47c2d --- /dev/null +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh @@ -0,0 +1,93 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query=" +CREATE TABLE t +( + item_id UInt64, + price_sold Float32, + date Date +) +ENGINE = MergeTree +ORDER BY item_id; + +CREATE TABLE t1 +( + item_id UInt64, + price_sold Float32, + date Date +) +ENGINE = MergeTree +ORDER BY item_id; + +INSERT INTO t SELECT number, number % 10, toDate(number) FROM numbers(100000); +INSERT INTO t1 SELECT number, number % 10, toDate(number) FROM numbers(100000); +" + +query1=" + SELECT sum(item_id) + FROM + ( + SELECT item_id + FROM t + GROUP BY item_id + ) AS l + LEFT JOIN + ( + SELECT item_id + FROM t1 + ) AS r ON l.item_id = r.item_id +" + +query2=" + SELECT sum(item_id) + FROM + ( + SELECT item_id + FROM t + ) AS l + LEFT JOIN + ( + SELECT item_id + FROM t1 + GROUP BY item_id + ) AS r ON l.item_id = r.item_id +" + +query3=" + SELECT sum(item_id) + FROM + ( + SELECT item_id, price_sold + FROM t + ) AS l + LEFT JOIN + ( + SELECT item_id + FROM t1 + ) AS r ON l.item_id = r.item_id + GROUP BY price_sold + ORDER BY price_sold +" + +for query in "${query1}" "${query2}" "${query3}"; do + for enable_parallel_replicas in {0..1}; do + ${CLICKHOUSE_CLIENT} --query=" + set enable_analyzer=1; + set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; + + ${query}; + + SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') + FROM + ( + EXPLAIN actions=1 ${query} + ) + WHERE explain LIKE '%ParallelReplicas%'; + " + done +done From e76f66d865540f86e32ac415974cfcd9b35c6b65 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 30 Oct 2024 13:58:33 +0100 Subject: [PATCH 637/816] fix typo --- src/Planner/findParallelReplicasQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 8a806045111..fce86a6cda0 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -174,7 +174,7 @@ const QueryNode * findQueryForParallelReplicas( struct Frame { const QueryPlan::Node * node = nullptr; - /// Below we will check subqueries from `stack` to find outtermost subquery that could be executed remotely. + /// Below we will check subqueries from `stack` to find outermost subquery that could be executed remotely. /// Currently traversal algorithm considers only steps with 0 or 1 children and JOIN specifically. /// When we found some step that requires finalization on the initiator (e.g. GROUP BY) there are two options: /// 1. If plan looks like a single path (e.g. AggregatingStep -> ExpressionStep -> Reading) we can execute From 9ab5f16968cb1c89a8c47b5dae07ea050380327f Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 30 Oct 2024 14:10:12 +0100 Subject: [PATCH 638/816] Update test.py --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 999324b563a..336ca824a2d 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4193,7 +4193,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster, create_query_generator ], "expected": { "raw_message": "050102696405496E743634000000000000000007626C6F636B4E6F06537472696E67034241440476616C3106537472696E6702414D0476616C3207466C6F617433320000003F0476616C330555496E743801", - "error": "Cannot parse string \'BAD\' as UInt16", + "error": "Cannot parse string 'BAD' as UInt16", }, "printable": False, }, From 0e063673d5fe5c5c9a60aa82d3258c20c7816f22 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 30 Oct 2024 13:18:37 +0000 Subject: [PATCH 639/816] Fix potential out-of-bound access --- src/Interpreters/Cache/QueryCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index cfd7608b6c6..7dbee567c5b 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -112,7 +112,7 @@ struct HasSystemTablesMatcher if (const auto * expression_list = function_children[0]->as()) { const ASTs & expression_list_children = expression_list->children; - if (!expression_list_children.empty()) + if (expression_list_children.size() >= 2) { if (const auto * literal = expression_list_children[1]->as()) { From 0840f7854c9ff286623d2165b79cec72254cdc67 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 30 Oct 2024 13:40:27 +0000 Subject: [PATCH 640/816] Fix ifdefs in ObjectStorageObject table --- src/TableFunctions/TableFunctionObjectStorage.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 6d81269f2d7..12de08afad0 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -269,41 +269,43 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) } #endif + +#if USE_AWS_S3 #if USE_PARQUET void registerTableFunctionDeltaLake(TableFunctionFactory & factory) { -#if USE_AWS_S3 factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the DeltaLake table stored on object store.)", .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); -#endif } #endif void registerTableFunctionHudi(TableFunctionFactory & factory) { -#if USE_AWS_S3 factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Hudi table stored on object store.)", .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); -#endif } +#endif + void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AVRO registerTableFunctionIceberg(factory); #endif +#if USE_AWS_S3 #if USE_PARQUET registerTableFunctionDeltaLake(factory); #endif registerTableFunctionHudi(factory); +#endif } } From b9829c703fd4ceae38b5d195ae195c2321e17444 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 13:44:59 +0000 Subject: [PATCH 641/816] change constraints check --- src/Interpreters/FillingRow.cpp | 75 ++++++++++++------- src/Interpreters/FillingRow.h | 6 +- .../Transforms/FillingTransform.cpp | 2 +- 3 files changed, 53 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index caf6ad9e3ba..825b0b1488a 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -3,6 +3,7 @@ #include "Common/Logger.h" #include "Common/logger_useful.h" #include +#include "base/defines.h" #include @@ -122,6 +123,43 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, return shifted_value; } +bool FillingRow::hasSomeConstraints(size_t pos) const +{ + const auto & descr = getFillDescription(pos); + + if (!descr.fill_to.isNull()) + return true; + + if (!descr.fill_staleness.isNull()) + return true; + + return false; +} + +bool FillingRow::isConstraintsComplete(size_t pos) const +{ + auto logger = getLogger("FillingRow::isConstraintComplete"); + chassert(!row[pos].isNull()); + chassert(hasSomeConstraints(pos)); + + const auto & descr = getFillDescription(pos); + int direction = getDirection(pos); + + if (!descr.fill_to.isNull() && !less(row[pos], descr.fill_to, direction)) + { + LOG_DEBUG(logger, "fill to: {}, row: {}, direction: {}", descr.fill_to.dump(), row[pos].dump(), direction); + return false; + } + + if (!descr.fill_staleness.isNull() && !less(row[pos], staleness_border[pos], direction)) + { + LOG_DEBUG(logger, "staleness border: {}, row: {}, direction: {}", staleness_border[pos].dump(), row[pos].dump(), direction); + return false; + } + + return true; +} + Field findMin(Field a, Field b, Field c, int dir) { auto logger = getLogger("FillingRow"); @@ -300,43 +338,26 @@ bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed return false; } -bool FillingRow::isConstraintComplete(size_t pos) const +bool FillingRow::hasSomeConstraints() const { - auto logger = getLogger("FillingRow::isConstraintComplete"); + for (size_t pos = 0; pos < size(); ++pos) + if (hasSomeConstraints(pos)) + return true; - if (row[pos].isNull()) - { - LOG_DEBUG(logger, "disabled"); - return true; /// disabled - } - - const auto & descr = getFillDescription(pos); - int direction = getDirection(pos); - - if (!descr.fill_to.isNull() && !less(row[pos], descr.fill_to, direction)) - { - LOG_DEBUG(logger, "fill to: {}, row: {}, direction: {}", descr.fill_to.dump(), row[pos].dump(), direction); - return false; - } - - if (!staleness_border[pos].isNull() && !less(row[pos], staleness_border[pos], direction)) - { - LOG_DEBUG(logger, "staleness border: {}, row: {}, direction: {}", staleness_border[pos].dump(), row[pos].dump(), direction); - return false; - } - - return true; + return false; } bool FillingRow::isConstraintsComplete() const { for (size_t pos = 0; pos < size(); ++pos) { - if (isConstraintComplete(pos)) - return true; + if (row[pos].isNull() || !hasSomeConstraints(pos)) + continue; + + return isConstraintsComplete(pos); } - return false; + return true; } bool FillingRow::isLessStaleness() const diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index a5e622e4c6e..bd5a1b877a5 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -18,6 +18,9 @@ class FillingRow std::optional doJump(const FillColumnDescription & descr, size_t column_ind); std::optional doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to); + bool hasSomeConstraints(size_t pos) const; + bool isConstraintsComplete(size_t pos) const; + public: explicit FillingRow(const SortDescription & sort_description); @@ -30,7 +33,7 @@ public: /// Returns true if need to generate some prefix for to_row bool shift(const FillingRow & next_original_row, bool& value_changed); - bool isConstraintComplete(size_t pos) const; + bool hasSomeConstraints() const; bool isConstraintsComplete() const; bool isLessStaleness() const; @@ -52,7 +55,6 @@ public: bool isNull() const; int getDirection(size_t index) const { return sort_description[index].direction; } - Field getStalenessBorder(size_t index) const { return staleness_border[index]; } FillColumnDescription & getFillDescription(size_t index) { return sort_description[index].fill_description; } const FillColumnDescription & getFillDescription(size_t index) const { return sort_description[index].fill_description; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index a3a185929dc..ce804c94d8e 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -533,7 +533,7 @@ bool FillingTransform::generateSuffixIfNeeded( // return false; // } - if (!filling_row.isConstraintsComplete()) + if (!filling_row.hasSomeConstraints() || !filling_row.isConstraintsComplete()) { logDebug("generateSuffixIfNeeded", "will not generate suffix"); return false; From 433523c6f29a55d28930ec86fe268edffc16738e Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 13:49:42 +0000 Subject: [PATCH 642/816] update test --- .../03266_with_fill_staleness.reference | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.reference b/tests/queries/0_stateless/03266_with_fill_staleness.reference index 6b090443359..25d7b7c3f24 100644 --- a/tests/queries/0_stateless/03266_with_fill_staleness.reference +++ b/tests/queries/0_stateless/03266_with_fill_staleness.reference @@ -50,6 +50,8 @@ staleness 3 seconds 2016-06-15 23:00:21 20 2016-06-15 23:00:22 20 2016-06-15 23:00:25 25 original +2016-06-15 23:00:26 25 +2016-06-15 23:00:27 25 descending order 2016-06-15 23:00:25 25 original 2016-06-15 23:00:24 25 @@ -62,6 +64,7 @@ descending order 2016-06-15 23:00:05 5 original 2016-06-15 23:00:04 5 2016-06-15 23:00:00 0 original +2016-06-15 22:59:59 0 staleness with to and step 2016-06-15 23:00:00 0 original 2016-06-15 23:00:03 0 @@ -86,33 +89,41 @@ staleness with another regular with fill 2016-06-15 23:00:01 1970-01-01 01:00:00 0 2016-06-15 23:00:01 1970-01-01 01:00:01 0 2016-06-15 23:00:01 1970-01-01 01:00:02 0 +2016-06-15 23:00:05 1970-01-01 01:00:00 0 +2016-06-15 23:00:05 1970-01-01 01:00:01 0 +2016-06-15 23:00:05 1970-01-01 01:00:02 0 2016-06-15 23:00:05 2016-06-15 23:00:05 5 original -2016-06-15 23:00:05 1970-01-01 01:00:01 5 -2016-06-15 23:00:05 1970-01-01 01:00:02 5 2016-06-15 23:00:06 1970-01-01 01:00:00 5 2016-06-15 23:00:06 1970-01-01 01:00:01 5 2016-06-15 23:00:06 1970-01-01 01:00:02 5 +2016-06-15 23:00:10 1970-01-01 01:00:00 5 +2016-06-15 23:00:10 1970-01-01 01:00:01 5 +2016-06-15 23:00:10 1970-01-01 01:00:02 5 2016-06-15 23:00:10 2016-06-15 23:00:10 10 original -2016-06-15 23:00:10 1970-01-01 01:00:01 10 -2016-06-15 23:00:10 1970-01-01 01:00:02 10 2016-06-15 23:00:11 1970-01-01 01:00:00 10 2016-06-15 23:00:11 1970-01-01 01:00:01 10 2016-06-15 23:00:11 1970-01-01 01:00:02 10 +2016-06-15 23:00:15 1970-01-01 01:00:00 10 +2016-06-15 23:00:15 1970-01-01 01:00:01 10 +2016-06-15 23:00:15 1970-01-01 01:00:02 10 2016-06-15 23:00:15 2016-06-15 23:00:15 15 original -2016-06-15 23:00:15 1970-01-01 01:00:01 15 -2016-06-15 23:00:15 1970-01-01 01:00:02 15 2016-06-15 23:00:16 1970-01-01 01:00:00 15 2016-06-15 23:00:16 1970-01-01 01:00:01 15 2016-06-15 23:00:16 1970-01-01 01:00:02 15 +2016-06-15 23:00:20 1970-01-01 01:00:00 15 +2016-06-15 23:00:20 1970-01-01 01:00:01 15 +2016-06-15 23:00:20 1970-01-01 01:00:02 15 2016-06-15 23:00:20 2016-06-15 23:00:20 20 original -2016-06-15 23:00:20 1970-01-01 01:00:01 20 -2016-06-15 23:00:20 1970-01-01 01:00:02 20 2016-06-15 23:00:21 1970-01-01 01:00:00 20 2016-06-15 23:00:21 1970-01-01 01:00:01 20 2016-06-15 23:00:21 1970-01-01 01:00:02 20 +2016-06-15 23:00:25 1970-01-01 01:00:00 20 +2016-06-15 23:00:25 1970-01-01 01:00:01 20 +2016-06-15 23:00:25 1970-01-01 01:00:02 20 2016-06-15 23:00:25 2016-06-15 23:00:25 25 original -2016-06-15 23:00:25 1970-01-01 01:00:01 25 -2016-06-15 23:00:25 1970-01-01 01:00:02 25 +2016-06-15 23:00:26 1970-01-01 01:00:00 25 +2016-06-15 23:00:26 1970-01-01 01:00:01 25 +2016-06-15 23:00:26 1970-01-01 01:00:02 25 double staleness 2016-06-15 23:00:00 2016-06-15 23:00:00 0 original 2016-06-15 23:00:00 2016-06-15 23:00:02 0 @@ -137,3 +148,4 @@ double staleness 2016-06-15 23:00:25 2016-06-15 23:00:25 25 original 2016-06-15 23:00:25 2016-06-15 23:00:27 25 2016-06-15 23:00:25 2016-06-15 23:00:29 25 +2016-06-15 23:00:26 1970-01-01 01:00:00 25 From 96f073f49a182daad247e77d1686505048e63b88 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Oct 2024 14:53:25 +0100 Subject: [PATCH 643/816] Add ReadBufferFromAzureBlobStorage and AsynchronousBoundedReadBuffer --- src/Disks/IO/AsynchronousBoundedReadBuffer.cpp | 2 +- tests/docker_scripts/stress_tests.lib | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index b24b95af85c..c405d296e60 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -365,7 +365,7 @@ AsynchronousBoundedReadBuffer::~AsynchronousBoundedReadBuffer() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index b0d6cf6d532..5c346a2d17f 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -266,9 +266,9 @@ function check_logs_for_critical_errors() # ignore: # - a.myext which is used in 02724_database_s3.sh and does not exist # - "DistributedCacheTCPHandler" and "caller id: None:DistribCache" because they happen inside distributed cache server - # - "ReadBufferFromDistributedCache" exception printed internally by ReadBufferFromDistributedCache, exception will be rethrown and handled correctly - # - "ReadBufferFromS3" exception printed internally by ReadBufferFromS3, exception will be rethrown and handled correctly - rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v -e "a.myext" -e "DistributedCacheTCPHandler" -e "ReadBufferFromDistributedCache" -e "ReadBufferFromS3" -e "caller id: None:DistribCache" > /test_output/no_such_key_errors.txt \ + # - "ReadBufferFromDistributedCache", "AsynchronousBoundedReadBuffer", "ReadBufferFromS3", "ReadBufferFromAzureBlobStorage" + # exceptions printed internally by a buffer, exception will be rethrown and handled correctly + rg --text "Code: 499.*The specified key does not exist" /var/log/clickhouse-server/clickhouse-server*.log | grep -v -e "a.myext" -e "DistributedCacheTCPHandler" -e "ReadBufferFromDistributedCache" -e "ReadBufferFromS3" -e "ReadBufferFromAzureBlobStorage" -e "AsynchronousBoundedReadBuffer" -e "caller id: None:DistribCache" > /test_output/no_such_key_errors.txt \ && echo -e "S3_ERROR No such key thrown (see clickhouse-server.log or no_such_key_errors.txt)$FAIL$(trim_server_logs no_such_key_errors.txt)" >> /test_output/test_results.tsv \ || echo -e "No lost s3 keys$OK" >> /test_output/test_results.tsv From 60840cb05fc2f948745e92d00b0e15cbd7a8923a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Oct 2024 14:55:15 +0100 Subject: [PATCH 644/816] Fix memory usage in remote read when enable_filesystem_cache=1, but cached disk absent --- src/Disks/IO/AsynchronousBoundedReadBuffer.cpp | 6 ++++-- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 2 ++ src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 2 ++ src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 16 +++------------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 5 ++--- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 18 ++++++++++++++++-- src/IO/ReadBufferFromFileBase.h | 2 ++ .../StorageObjectStorageSource.cpp | 10 +++++++++- 8 files changed, 40 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index b24b95af85c..77b03cdd1f7 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -46,11 +46,13 @@ AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer( ImplPtr impl_, IAsynchronousReader & reader_, const ReadSettings & settings_, + size_t buffer_size_, AsyncReadCountersPtr async_read_counters_, FilesystemReadPrefetchesLogPtr prefetches_log_) : ReadBufferFromFileBase(0, nullptr, 0) , impl(std::move(impl_)) , read_settings(settings_) + , buffer_size(buffer_size_) , reader(reader_) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , current_reader_id(getRandomASCIIString(8)) @@ -112,7 +114,7 @@ void AsynchronousBoundedReadBuffer::prefetch(Priority priority) last_prefetch_info.submit_time = std::chrono::system_clock::now(); last_prefetch_info.priority = priority; - prefetch_buffer.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize())); + prefetch_buffer.resize(buffer_size); prefetch_future = readAsync(prefetch_buffer.data(), prefetch_buffer.size(), priority); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } @@ -211,7 +213,7 @@ bool AsynchronousBoundedReadBuffer::nextImpl() } else { - memory.resize(chooseBufferSizeForRemoteReading(read_settings, impl->getFileSize())); + memory.resize(buffer_size); { ProfileEventTimeIncrement watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds); diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index 3dc8fcc39cb..7664cc4d386 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -27,6 +27,7 @@ public: ImplPtr impl_, IAsynchronousReader & reader_, const ReadSettings & settings_, + size_t buffer_size_, AsyncReadCountersPtr async_read_counters_ = nullptr, FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr); @@ -53,6 +54,7 @@ public: private: const ImplPtr impl; const ReadSettings read_settings; + const size_t buffer_size; IAsynchronousReader & reader; size_t file_offset_of_buffer_end = 0; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 119fa166214..4881b6a309d 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -41,6 +41,8 @@ public: ~CachedOnDiskReadBufferFromFile() override; + bool isCached() const override { return true; } + bool nextImpl() override; off_t seek(off_t off, int whence) override; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 8e4ec6f3dfb..8d3b9366261 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -18,24 +18,14 @@ namespace ErrorCodes extern const int CANNOT_SEEK_THROUGH_FILE; } -size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size) -{ - /// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task. - if (!settings.enable_filesystem_cache && !settings.read_through_distributed_cache) - return settings.remote_fs_buffer_size; - - /// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file. - return std::min(std::max(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size); -} - ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_, std::shared_ptr cache_log_, - bool use_external_buffer_) - : ReadBufferFromFileBase(use_external_buffer_ ? 0 : chooseBufferSizeForRemoteReading( - settings_, getTotalSize(blobs_to_read_)), nullptr, 0) + bool use_external_buffer_, + size_t buffer_size) + : ReadBufferFromFileBase(use_external_buffer_ ? 0 : buffer_size, nullptr, 0) , settings(settings_) , blobs_to_read(blobs_to_read_) , read_buffer_creator(std::move(read_buffer_creator_)) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 27f94a3e552..c5f1966dc38 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -28,7 +28,8 @@ public: const StoredObjects & blobs_to_read_, const ReadSettings & settings_, std::shared_ptr cache_log_, - bool use_external_buffer_); + bool use_external_buffer_, + size_t buffer_size); ~ReadBufferFromRemoteFSGather() override; @@ -84,6 +85,4 @@ private: LoggerPtr log; }; - -size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index fbab25490c1..bd7ffeb5a00 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -532,19 +532,33 @@ std::unique_ptr DiskObjectStorage::readFile( return impl; }; + /// Avoid cache fragmentation by choosing bigger buffer size. + bool prefer_bigger_buffer_size = object_storage->supportsCache() && read_settings.enable_filesystem_cache; + size_t buffer_size = prefer_bigger_buffer_size + ? std::max(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) + : settings.remote_fs_buffer_size; + + size_t total_objects_size = getTotalSize(storage_objects); + if (total_objects_size) + buffer_size = std::min(buffer_size, total_objects_size); + const bool use_async_buffer = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; auto impl = std::make_unique( std::move(read_buffer_creator), storage_objects, read_settings, global_context->getFilesystemCacheLog(), - /* use_external_buffer */use_async_buffer); + /* use_external_buffer */use_async_buffer, + /* buffer_size */use_async_buffer ? 0 : buffer_size); if (use_async_buffer) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( - std::move(impl), reader, read_settings, + std::move(impl), + reader, + read_settings, + buffer_size, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index c98dcd5a93e..c59a5c152b6 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -60,6 +60,8 @@ public: /// file offset and what getPosition() returns. virtual bool isRegularLocalFile(size_t * /*out_view_offsee*/) { return false; } + virtual bool isCached() const { return false; } + protected: std::optional file_size; ProfileCallback profile_callback; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 52b0f00f71a..90871b8c0ad 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -517,9 +517,17 @@ std::unique_ptr StorageObjectStorageSource::createReadBu LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + bool prefer_bigger_buffer_size = impl->isCached(); + size_t buffer_size = prefer_bigger_buffer_size + ? std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) + : read_settings.remote_fs_buffer_size; + auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); impl = std::make_unique( - std::move(impl), reader, modified_read_settings, + std::move(impl), + reader, + modified_read_settings, + buffer_size, context_->getAsyncReadCounters(), context_->getFilesystemReadPrefetchesLog()); From 81c58d9406a0194f83bc800f5f7c0cc502f13b10 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Oct 2024 15:15:41 +0100 Subject: [PATCH 645/816] Better check --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index bd7ffeb5a00..d677623ab57 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -538,7 +538,7 @@ std::unique_ptr DiskObjectStorage::readFile( ? std::max(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) : settings.remote_fs_buffer_size; - size_t total_objects_size = getTotalSize(storage_objects); + size_t total_objects_size = file_size ? *file_size : getTotalSize(storage_objects); if (total_objects_size) buffer_size = std::min(buffer_size, total_objects_size); From e5fe7a0f52625d3460ca04a21982a1af24e0adcd Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 14:35:29 +0000 Subject: [PATCH 646/816] add more tests --- .../0_stateless/03266_with_fill_staleness.sql | 1 + .../03266_with_fill_staleness_cases.reference | 67 +++++++++++++++++++ .../03266_with_fill_staleness_cases.sql | 25 +++++++ 3 files changed, 93 insertions(+) create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness_cases.reference create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness_cases.sql diff --git a/tests/queries/0_stateless/03266_with_fill_staleness.sql b/tests/queries/0_stateless/03266_with_fill_staleness.sql index fff702ffd83..de47d8287ad 100644 --- a/tests/queries/0_stateless/03266_with_fill_staleness.sql +++ b/tests/queries/0_stateless/03266_with_fill_staleness.sql @@ -1,4 +1,5 @@ SET session_timezone='Europe/Amsterdam'; +SET enable_analyzer=1; DROP TABLE IF EXISTS with_fill_staleness; CREATE TABLE with_fill_staleness (a DateTime, b DateTime, c UInt64) ENGINE = MergeTree ORDER BY a; diff --git a/tests/queries/0_stateless/03266_with_fill_staleness_cases.reference b/tests/queries/0_stateless/03266_with_fill_staleness_cases.reference new file mode 100644 index 00000000000..bf8e5bbe331 --- /dev/null +++ b/tests/queries/0_stateless/03266_with_fill_staleness_cases.reference @@ -0,0 +1,67 @@ +test-1 +0 5 10 original +0 5 13 +0 5 16 +0 5 19 +0 5 22 +0 7 0 +7 8 15 original +7 8 18 +7 8 21 +7 8 24 +7 10 0 +14 10 20 original +14 10 23 +14 12 0 +test-2-1 +1 0 original +1 1 +1 2 +1 3 +1 4 original +1 5 +1 6 +1 7 +1 8 original +1 9 +1 10 +1 11 +1 12 original +test-2-2 +1 0 original +1 1 +1 2 +1 3 +1 4 original +1 5 +1 6 +1 7 +1 8 original +1 9 +1 10 +1 11 +1 12 original +1 13 +1 14 +2 0 +3 0 +4 0 +test-3-1 +25 -10 +25 -8 +25 -6 +25 -4 +25 -2 +25 0 +25 2 +25 4 +25 6 +25 8 +25 10 +25 12 +25 14 +25 16 +25 17 original +28 -10 +30 18 original +31 -10 diff --git a/tests/queries/0_stateless/03266_with_fill_staleness_cases.sql b/tests/queries/0_stateless/03266_with_fill_staleness_cases.sql new file mode 100644 index 00000000000..9e28041c9a1 --- /dev/null +++ b/tests/queries/0_stateless/03266_with_fill_staleness_cases.sql @@ -0,0 +1,25 @@ +SET enable_analyzer=1; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int64, b Int64, c Int64) Engine=MergeTree ORDER BY a; +INSERT INTO test(a, b, c) VALUES (0, 5, 10), (7, 8, 15), (14, 10, 20); + +SELECT 'test-1'; +SELECT *, 'original' AS orig FROM test ORDER BY a, b WITH FILL TO 20 STEP 2 STALENESS 3, c WITH FILL TO 25 step 3; + +DROP TABLE IF EXISTS test2; +CREATE TABLE test2 (a Int64, b Int64) Engine=MergeTree ORDER BY a; +INSERT INTO test2(a, b) values (1, 0), (1, 4), (1, 8), (1, 12); + +SELECT 'test-2-1'; +SELECT *, 'original' AS orig FROM test2 ORDER BY a, b WITH FILL; + +SELECT 'test-2-2'; +SELECT *, 'original' AS orig FROM test2 ORDER BY a WITH FILL to 20 STALENESS 4, b WITH FILL TO 15 STALENESS 7; + +DROP TABLE IF EXISTS test2; +CREATE TABLE test3 (a Int64, b Int64) Engine=MergeTree ORDER BY a; +INSERT INTO test3(a, b) VALUES (25, 17), (30, 18); + +SELECT 'test-3-1'; +SELECT a, b, 'original' AS orig FROM test3 ORDER BY a WITH FILL TO 33 STEP 3, b WITH FILL FROM -10 STEP 2; From 2cda4dd9012059b6c287df7c615cef8e310b2d8e Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 14:46:56 +0000 Subject: [PATCH 647/816] cleanup --- src/Interpreters/FillingRow.cpp | 97 +------------------ src/Interpreters/FillingRow.h | 12 +-- .../Transforms/FillingTransform.cpp | 30 +----- 3 files changed, 11 insertions(+), 128 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 825b0b1488a..a87ca418b7b 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -68,25 +68,6 @@ bool FillingRow::isNull() const return true; } -std::optional FillingRow::doJump(const FillColumnDescription& descr, size_t column_ind) -{ - Field next_value = row[column_ind]; - descr.step_func(next_value, 1); - - if (!descr.fill_to.isNull() && less(descr.fill_to, next_value, getDirection(column_ind))) - return std::nullopt; - - if (!descr.fill_staleness.isNull()) - { - if (less(next_value, staleness_border[column_ind], getDirection(column_ind))) - return next_value; - else - return std::nullopt; - } - - return next_value; -} - std::optional FillingRow::doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to) { Field shifted_value = row[column_ind]; @@ -99,16 +80,6 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, Field next_value = shifted_value; descr.step_func(next_value, step_len); - // if (less(next_value, to, getDirection(0))) - // { - // shifted_value = std::move(next_value); - // step_len *= 2; - // } - // else - // { - // step_len /= 2; - // } - if (less(to, next_value, getDirection(0))) { step_len /= 2; @@ -233,7 +204,7 @@ std::pair FillingRow::next(const FillingRow & next_original_row) continue; row[i] = next_value; - initWithFrom(i + 1); + initUsingFrom(i + 1); return {true, true}; } @@ -271,7 +242,7 @@ std::pair FillingRow::next(const FillingRow & next_original_row) return {is_less, true}; } - initWithFrom(pos + 1); + initUsingFrom(pos + 1); return {true, true}; } @@ -327,8 +298,7 @@ bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed } else { - // getFillDescription(pos).step_func(row[pos], 1); - initWithTo(/*from_pos=*/pos + 1); + initUsingTo(/*from_pos=*/pos + 1); value_changed = false; return false; @@ -360,70 +330,13 @@ bool FillingRow::isConstraintsComplete() const return true; } -bool FillingRow::isLessStaleness() const -{ - auto logger = getLogger("FillingRow::isLessStaleness"); - - for (size_t pos = 0; pos < size(); ++pos) - { - LOG_DEBUG(logger, "staleness border: {}, row: {}", staleness_border[pos].dump(), row[pos].dump()); - - if (row[pos].isNull() || staleness_border[pos].isNull()) - continue; - - if (less(row[pos], staleness_border[pos], getDirection(pos))) - return true; - } - - return false; -} - -bool FillingRow::isStalenessConfigured() const -{ - for (size_t pos = 0; pos < size(); ++pos) - if (!getFillDescription(pos).fill_staleness.isNull()) - return true; - - return false; -} - -bool FillingRow::isLessFillTo() const -{ - auto logger = getLogger("FillingRow::isLessFillTo"); - - for (size_t pos = 0; pos < size(); ++pos) - { - const auto & descr = getFillDescription(pos); - - LOG_DEBUG(logger, "fill to: {}, row: {}", descr.fill_to.dump(), row[pos].dump()); - - if (row[pos].isNull() || descr.fill_to.isNull()) - continue; - - if (less(row[pos], descr.fill_to, getDirection(pos))) - return true; - } - - return false; -} - -bool FillingRow::isFillToConfigured() const -{ - for (size_t pos = 0; pos < size(); ++pos) - if (!getFillDescription(pos).fill_to.isNull()) - return true; - - return false; -} - - -void FillingRow::initWithFrom(size_t from_pos) +void FillingRow::initUsingFrom(size_t from_pos) { for (size_t i = from_pos; i < sort_description.size(); ++i) row[i] = getFillDescription(i).fill_from; } -void FillingRow::initWithTo(size_t from_pos) +void FillingRow::initUsingTo(size_t from_pos) { for (size_t i = from_pos; i < sort_description.size(); ++i) row[i] = getFillDescription(i).fill_to; diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index bd5a1b877a5..d33e3f95541 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -15,7 +15,7 @@ bool equals(const Field & lhs, const Field & rhs); */ class FillingRow { - std::optional doJump(const FillColumnDescription & descr, size_t column_ind); + /// finds last value <= to std::optional doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to); bool hasSomeConstraints(size_t pos) const; @@ -36,14 +36,8 @@ public: bool hasSomeConstraints() const; bool isConstraintsComplete() const; - bool isLessStaleness() const; - bool isStalenessConfigured() const; - - bool isLessFillTo() const; - bool isFillToConfigured() const; - - void initWithFrom(size_t from_pos = 0); - void initWithTo(size_t from_pos = 0); + void initUsingFrom(size_t from_pos = 0); + void initUsingTo(size_t from_pos = 0); void initStalenessRow(const Columns& base_row, size_t row_ind); Field & operator[](size_t index) { return row[index]; } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index ce804c94d8e..40650b485f8 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -21,7 +21,7 @@ namespace DB constexpr bool debug_logging_enabled = true; template -void logDebug(String key, const T & value, const char * separator = " : ") +static void logDebug(String key, const T & value, const char * separator = " : ") { if constexpr (debug_logging_enabled) { @@ -512,27 +512,6 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded next_row updated", next_row); - // if (!filling_row.isFillToConfigured() && !filling_row.isStalenessConfigured()) - // { - // logDebug("generateSuffixIfNeeded", "no other constraints, will not generate suffix"); - // return false; - // } - - // logDebug("filling_row.isLessFillTo()", filling_row.isLessFillTo()); - // logDebug("filling_row.isLessStaleness()", filling_row.isLessStaleness()); - - // if (filling_row.isFillToConfigured() && !filling_row.isLessFillTo()) - // { - // logDebug("generateSuffixIfNeeded", "not less than fill to, will not generate suffix"); - // return false; - // } - - // if (filling_row.isStalenessConfigured() && !filling_row.isLessStaleness()) - // { - // logDebug("generateSuffixIfNeeded", "not less than staleness border, will not generate suffix"); - // return false; - // } - if (!filling_row.hasSomeConstraints() || !filling_row.isConstraintsComplete()) { logDebug("generateSuffixIfNeeded", "will not generate suffix"); @@ -637,7 +616,7 @@ void FillingTransform::transformRange( if (!fill_from.isNull() && !equals(current_value, fill_from)) { - filling_row.initWithFrom(i); + filling_row.initUsingFrom(i); filling_row_inserted = false; if (less(fill_from, current_value, filling_row.getDirection(i))) { @@ -732,9 +711,6 @@ void FillingTransform::transformRange( copyRowFromColumns(res_interpolate_columns, input_interpolate_columns, row_ind); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); copyRowFromColumns(res_other_columns, input_other_columns, row_ind); - - // /// Init next staleness interval with current row, because we have already made the long jump to it - // filling_row.initStalenessRow(input_fill_columns, row_ind); } /// save sort prefix of last row in the range, it's used to generate suffix @@ -780,7 +756,7 @@ void FillingTransform::transform(Chunk & chunk) /// if no data was processed, then need to initialize filling_row if (last_row.empty()) { - filling_row.initWithFrom(); + filling_row.initUsingFrom(); filling_row_inserted = false; } From 3099eae4794a4ec669306e5abc790b01f1fd18bf Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Oct 2024 15:56:20 +0100 Subject: [PATCH 648/816] Fix build --- src/Disks/tests/gtest_asynchronous_bounded_read_buffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/tests/gtest_asynchronous_bounded_read_buffer.cpp b/src/Disks/tests/gtest_asynchronous_bounded_read_buffer.cpp index 63a39fe39c7..11b4fc3118d 100644 --- a/src/Disks/tests/gtest_asynchronous_bounded_read_buffer.cpp +++ b/src/Disks/tests/gtest_asynchronous_bounded_read_buffer.cpp @@ -51,7 +51,7 @@ TEST_F(AsynchronousBoundedReadBufferTest, setReadUntilPosition) for (bool with_prefetch : {false, true}) { - AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {}); + AsynchronousBoundedReadBuffer read_buffer(createReadBufferFromFileBase(file_path, {}), remote_fs_reader, {}, DBMS_DEFAULT_BUFFER_SIZE); read_buffer.setReadUntilPosition(20); auto try_read = [&](size_t count) From 2760f283a1c54351c3103e4a9ce6556b06009149 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 30 Oct 2024 16:18:35 +0100 Subject: [PATCH 649/816] Revert "Revert "SQL syntax for workload and resource management"" --- .gitignore | 1 + .../settings.md | 28 + docs/en/operations/system-tables/resources.md | 37 + docs/en/operations/system-tables/workloads.md | 40 + docs/en/operations/workload-scheduling.md | 53 ++ programs/server/Server.cpp | 5 +- programs/server/config.xml | 4 + src/Access/Common/AccessType.h | 4 + src/Access/ContextAccess.cpp | 6 +- src/CMakeLists.txt | 1 + src/Common/Priority.h | 5 +- src/Common/Scheduler/IResourceManager.h | 8 +- src/Common/Scheduler/ISchedulerConstraint.h | 29 +- src/Common/Scheduler/ISchedulerNode.h | 63 +- src/Common/Scheduler/ISchedulerQueue.h | 9 + .../Scheduler/Nodes/ClassifiersConfig.cpp | 9 +- .../Scheduler/Nodes/ClassifiersConfig.h | 1 + ...eManager.cpp => CustomResourceManager.cpp} | 45 +- ...ourceManager.h => CustomResourceManager.h} | 12 +- src/Common/Scheduler/Nodes/FairPolicy.h | 19 +- src/Common/Scheduler/Nodes/FifoQueue.h | 35 +- .../Scheduler/Nodes/IOResourceManager.cpp | 532 ++++++++++++ .../Scheduler/Nodes/IOResourceManager.h | 281 +++++++ src/Common/Scheduler/Nodes/PriorityPolicy.h | 19 +- .../Scheduler/Nodes/SemaphoreConstraint.h | 73 +- .../Scheduler/Nodes/ThrottlerConstraint.h | 56 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 606 ++++++++++++++ .../Nodes/registerResourceManagers.cpp | 15 - .../Nodes/registerResourceManagers.h | 8 - .../Scheduler/Nodes/tests/ResourceTest.h | 209 ++++- ....cpp => gtest_custom_resource_manager.cpp} | 28 +- .../Nodes/tests/gtest_event_queue.cpp | 6 + .../Nodes/tests/gtest_io_resource_manager.cpp | 335 ++++++++ .../Nodes/tests/gtest_resource_class_fair.cpp | 15 +- .../tests/gtest_resource_class_priority.cpp | 13 +- .../Nodes/tests/gtest_resource_scheduler.cpp | 26 +- .../tests/gtest_throttler_constraint.cpp | 34 +- .../tests/gtest_unified_scheduler_node.cpp | 748 +++++++++++++++++ src/Common/Scheduler/ResourceGuard.h | 20 + src/Common/Scheduler/ResourceManagerFactory.h | 55 -- src/Common/Scheduler/ResourceRequest.cpp | 25 +- src/Common/Scheduler/ResourceRequest.h | 27 +- src/Common/Scheduler/SchedulerRoot.h | 37 +- src/Common/Scheduler/SchedulingSettings.cpp | 130 +++ src/Common/Scheduler/SchedulingSettings.h | 39 + .../Workload/IWorkloadEntityStorage.h | 91 +++ .../Workload/WorkloadEntityDiskStorage.cpp | 287 +++++++ .../Workload/WorkloadEntityDiskStorage.h | 44 + .../Workload/WorkloadEntityKeeperStorage.cpp | 273 +++++++ .../Workload/WorkloadEntityKeeperStorage.h | 71 ++ .../Workload/WorkloadEntityStorageBase.cpp | 773 ++++++++++++++++++ .../Workload/WorkloadEntityStorageBase.h | 126 +++ .../Workload/createWorkloadEntityStorage.cpp | 45 + .../Workload/createWorkloadEntityStorage.h | 11 + .../Scheduler/createResourceManager.cpp | 104 +++ src/Common/Scheduler/createResourceManager.h | 11 + .../ObjectStorages/DiskObjectStorage.cpp | 127 ++- src/Disks/ObjectStorages/DiskObjectStorage.h | 13 +- src/Interpreters/Context.cpp | 22 +- src/Interpreters/Context.h | 3 + .../InterpreterCreateResourceQuery.cpp | 68 ++ .../InterpreterCreateResourceQuery.h | 25 + .../InterpreterCreateWorkloadQuery.cpp | 68 ++ .../InterpreterCreateWorkloadQuery.h | 25 + .../InterpreterDropResourceQuery.cpp | 60 ++ .../InterpreterDropResourceQuery.h | 21 + .../InterpreterDropWorkloadQuery.cpp | 60 ++ .../InterpreterDropWorkloadQuery.h | 21 + src/Interpreters/InterpreterFactory.cpp | 20 + src/Interpreters/registerInterpreters.cpp | 8 + src/Parsers/ASTCreateResourceQuery.cpp | 83 ++ src/Parsers/ASTCreateResourceQuery.h | 48 ++ src/Parsers/ASTCreateWorkloadQuery.cpp | 95 +++ src/Parsers/ASTCreateWorkloadQuery.h | 53 ++ src/Parsers/ASTDropResourceQuery.cpp | 25 + src/Parsers/ASTDropResourceQuery.h | 28 + src/Parsers/ASTDropWorkloadQuery.cpp | 25 + src/Parsers/ASTDropWorkloadQuery.h | 28 + src/Parsers/CommonParsers.h | 4 + src/Parsers/ParserCreateResourceQuery.cpp | 144 ++++ src/Parsers/ParserCreateResourceQuery.h | 16 + src/Parsers/ParserCreateWorkloadEntity.cpp | 16 + src/Parsers/ParserCreateWorkloadEntity.h | 17 + src/Parsers/ParserCreateWorkloadQuery.cpp | 155 ++++ src/Parsers/ParserCreateWorkloadQuery.h | 16 + src/Parsers/ParserDropResourceQuery.cpp | 52 ++ src/Parsers/ParserDropResourceQuery.h | 14 + src/Parsers/ParserDropWorkloadQuery.cpp | 52 ++ src/Parsers/ParserDropWorkloadQuery.h | 14 + src/Parsers/ParserQuery.cpp | 12 + .../System/StorageSystemResources.cpp | 71 ++ src/Storages/System/StorageSystemResources.h | 29 + .../System/StorageSystemScheduler.cpp | 18 +- .../System/StorageSystemWorkloads.cpp | 48 ++ src/Storages/System/StorageSystemWorkloads.h | 29 + src/Storages/System/attachSystemTables.cpp | 4 + .../configs/storage_configuration.xml | 17 + tests/integration/test_scheduler/test.py | 394 +++++++++ .../01271_show_privileges.reference | 4 + .../03232_resource_create_and_drop.reference | 5 + .../03232_resource_create_and_drop.sql | 11 + .../03232_workload_create_and_drop.reference | 5 + .../03232_workload_create_and_drop.sql | 11 + .../03232_workloads_and_resources.reference | 0 .../03232_workloads_and_resources.sql | 68 ++ 105 files changed, 7403 insertions(+), 336 deletions(-) create mode 100644 docs/en/operations/system-tables/resources.md create mode 100644 docs/en/operations/system-tables/workloads.md rename src/Common/Scheduler/Nodes/{DynamicResourceManager.cpp => CustomResourceManager.cpp} (84%) rename src/Common/Scheduler/Nodes/{DynamicResourceManager.h => CustomResourceManager.h} (86%) create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.cpp create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.h create mode 100644 src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.cpp delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.h rename src/Common/Scheduler/Nodes/tests/{gtest_dynamic_resource_manager.cpp => gtest_custom_resource_manager.cpp} (82%) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp delete mode 100644 src/Common/Scheduler/ResourceManagerFactory.h create mode 100644 src/Common/Scheduler/SchedulingSettings.cpp create mode 100644 src/Common/Scheduler/SchedulingSettings.h create mode 100644 src/Common/Scheduler/Workload/IWorkloadEntityStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.h create mode 100644 src/Common/Scheduler/createResourceManager.cpp create mode 100644 src/Common/Scheduler/createResourceManager.h create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.h create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.h create mode 100644 src/Interpreters/InterpreterDropResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterDropResourceQuery.h create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.h create mode 100644 src/Parsers/ASTCreateResourceQuery.cpp create mode 100644 src/Parsers/ASTCreateResourceQuery.h create mode 100644 src/Parsers/ASTCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ASTCreateWorkloadQuery.h create mode 100644 src/Parsers/ASTDropResourceQuery.cpp create mode 100644 src/Parsers/ASTDropResourceQuery.h create mode 100644 src/Parsers/ASTDropWorkloadQuery.cpp create mode 100644 src/Parsers/ASTDropWorkloadQuery.h create mode 100644 src/Parsers/ParserCreateResourceQuery.cpp create mode 100644 src/Parsers/ParserCreateResourceQuery.h create mode 100644 src/Parsers/ParserCreateWorkloadEntity.cpp create mode 100644 src/Parsers/ParserCreateWorkloadEntity.h create mode 100644 src/Parsers/ParserCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ParserCreateWorkloadQuery.h create mode 100644 src/Parsers/ParserDropResourceQuery.cpp create mode 100644 src/Parsers/ParserDropResourceQuery.h create mode 100644 src/Parsers/ParserDropWorkloadQuery.cpp create mode 100644 src/Parsers/ParserDropWorkloadQuery.h create mode 100644 src/Storages/System/StorageSystemResources.cpp create mode 100644 src/Storages/System/StorageSystemResources.h create mode 100644 src/Storages/System/StorageSystemWorkloads.cpp create mode 100644 src/Storages/System/StorageSystemWorkloads.h create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.sql create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.sql create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.reference create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.sql diff --git a/.gitignore b/.gitignore index 4bc162c1b0f..8a745655cbf 100644 --- a/.gitignore +++ b/.gitignore @@ -159,6 +159,7 @@ website/package-lock.json /programs/server/store /programs/server/uuid /programs/server/coordination +/programs/server/workload # temporary test files tests/queries/0_stateless/test_* diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 76d6f5388e3..02fa5a8ca58 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3224,6 +3224,34 @@ Default value: "default" **See Also** - [Workload Scheduling](/docs/en/operations/workload-scheduling.md) +## workload_path {#workload_path} + +The directory used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. By default `/workload/` folder under server working directory is used. + +**Example** + +``` xml +/var/lib/clickhouse/workload/ +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_zookeeper_path](#workload_zookeeper_path) + +## workload_zookeeper_path {#workload_zookeeper_path} + +The path to a ZooKeeper node, which is used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. For consistency all SQL definitions are stored as a value of this single znode. By default ZooKeeper is not used and definitions are stored on [disk](#workload_path). + +**Example** + +``` xml +/clickhouse/workload/definitions.sql +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_path](#workload_path) + ## max_authentication_methods_per_user {#max_authentication_methods_per_user} The maximum number of authentication methods a user can be created with or altered to. diff --git a/docs/en/operations/system-tables/resources.md b/docs/en/operations/system-tables/resources.md new file mode 100644 index 00000000000..6329f05f610 --- /dev/null +++ b/docs/en/operations/system-tables/resources.md @@ -0,0 +1,37 @@ +--- +slug: /en/operations/system-tables/resources +--- +# resources + +Contains information for [resources](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every resource. + +Example: + +``` sql +SELECT * +FROM system.resources +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: io_read +read_disks: ['s3'] +write_disks: [] +create_query: CREATE RESOURCE io_read (READ DISK s3) + +Row 2: +────── +name: io_write +read_disks: [] +write_disks: ['s3'] +create_query: CREATE RESOURCE io_write (WRITE DISK s3) +``` + +Columns: + +- `name` (`String`) - Resource name. +- `read_disks` (`Array(String)`) - The array of disk names that uses this resource for read operations. +- `write_disks` (`Array(String)`) - The array of disk names that uses this resource for write operations. +- `create_query` (`String`) - The definition of the resource. diff --git a/docs/en/operations/system-tables/workloads.md b/docs/en/operations/system-tables/workloads.md new file mode 100644 index 00000000000..d9c62372044 --- /dev/null +++ b/docs/en/operations/system-tables/workloads.md @@ -0,0 +1,40 @@ +--- +slug: /en/operations/system-tables/workloads +--- +# workloads + +Contains information for [workloads](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every workload. + +Example: + +``` sql +SELECT * +FROM system.workloads +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: production +parent: all +create_query: CREATE WORKLOAD production IN `all` SETTINGS weight = 9 + +Row 2: +────── +name: development +parent: all +create_query: CREATE WORKLOAD development IN `all` + +Row 3: +────── +name: all +parent: +create_query: CREATE WORKLOAD `all` +``` + +Columns: + +- `name` (`String`) - Workload name. +- `parent` (`String`) - Parent workload name. +- `create_query` (`String`) - The definition of the workload. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 08629492ec6..a43bea7a5b1 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -43,6 +43,20 @@ Example: ``` +An alternative way to express which disks are used by a resource is SQL syntax: + +```sql +CREATE RESOURCE resource_name (WRITE DISK disk1, READ DISK disk2) +``` + +Resource could be used for any number of disk for READ or WRITE or both for READ and WRITE. There a syntax allowing to use a resource for all the disks: + +```sql +CREATE RESOURCE all_io (READ ANY DISK, WRITE ANY DISK); +``` + +Note that server configuration options have priority over SQL way to define resources. + ## Workload markup {#workload_markup} Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. @@ -153,9 +167,48 @@ Example: ``` +## Workload hierarchy (SQL only) {#workloads} + +Defining resources and classifiers in XML could be challenging. ClickHouse provides SQL syntax that is much more convenient. All resources that were created with `CREATE RESOURCE` share the same structure of the hierarchy, but could differ in some aspects. Every workload created with `CREATE WORKLOAD` maintains a few automatically created scheduling nodes for every resource. A child workload can be created inside another parent workload. Here is the example that defines exactly the same hierarchy as XML configuration above: + +```sql +CREATE RESOURCE network_write (WRITE DISK s3) +CREATE RESOURCE network_read (READ DISK s3) +CREATE WORKLOAD all SETTINGS max_requests = 100 +CREATE WORKLOAD development IN all +CREATE WORKLOAD production IN all SETTINGS weight = 3 +``` + +The name of a leaf workload without children could be used in query settings `SETTINGS workload = 'name'`. Note that workload classifiers are also created automatically when using SQL syntax. + +To customize workload the following settings could be used: +* `priority` - sibling workloads are served according to static priority values (lower value means higher priority). +* `weight` - sibling workloads having the same static priority share resources according to weights. +* `max_requests` - the limit on the number of concurrent resource requests in this workload. +* `max_cost` - the limit on the total inflight bytes count of concurrent resource requests in this workload. +* `max_speed` - the limit on byte processing rate of this workload (the limit is independent for every resource). +* `max_burst` - maximum number of bytes that could be processed by the workload without being throttled (for every resource independently). + +Note that workload settings are translated into a proper set of scheduling nodes. For more details, see the description of the scheduling node [types and options](#hierarchy). + +There is no way to specify different hierarchies of workloads for different resources. But there is a way to specify different workload setting value for a specific resource: + +```sql +CREATE OR REPLACE WORKLOAD all SETTINGS max_requests = 100, max_speed = 1000000 FOR network_read, max_speed = 2000000 FOR network_write +``` + +Also note that workload or resource could not be dropped if it is referenced from another workload. To update a definition of a workload use `CREATE OR REPLACE WORKLOAD` query. + +## Workloads and resources storage {#workload_entity_storage} +Definitions of all workloads and resources in the form of `CREATE WORKLOAD` and `CREATE RESOURCE` queries are stored persistently either on disk at `workload_path` or in ZooKeeper at `workload_zookeeper_path`. ZooKeeper storage is recommended to achieve consistency between nodes. Alternatively `ON CLUSTER` clause could be used along with disk storage. + ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) + - [system.workloads](/docs/en/operations/system-tables/workloads.md) + - [system.resources](/docs/en/operations/system-tables/resources.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting + - [workload_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_path) global server setting + - [workload_zookeeper_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_zookeeper_path) global server setting diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d061d134e69..826100f68e2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -86,7 +86,7 @@ #include #include #include -#include +#include #include #include #include "MetricsTransmitter.h" @@ -920,7 +920,6 @@ try registerFormats(); registerRemoteFileMetadatas(); registerSchedulerNodes(); - registerResourceManagers(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); @@ -2253,6 +2252,8 @@ try database_catalog.assertDatabaseExists(default_database); /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); + /// Load WORKLOADs and RESOURCEs. + global_context->getWorkloadEntityStorage().loadEntities(); global_context->getRefreshSet().setRefreshesStopped(false); } diff --git a/programs/server/config.xml b/programs/server/config.xml index 15649b5c95d..9807f8c0d5a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1399,6 +1399,10 @@ If not specified they will be stored locally. --> + + + diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 010d11e533a..242dfcd8c35 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -99,6 +99,8 @@ enum class AccessType : uint8_t M(CREATE_ARBITRARY_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables with arbitrary table engine */\ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ + M(CREATE_WORKLOAD, "", GLOBAL, CREATE) /* allows to execute CREATE WORKLOAD */ \ + M(CREATE_RESOURCE, "", GLOBAL, CREATE) /* allows to execute CREATE RESOURCE */ \ M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ @@ -108,6 +110,8 @@ enum class AccessType : uint8_t implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ + M(DROP_WORKLOAD, "", GLOBAL, DROP) /* allows to execute DROP WORKLOAD */\ + M(DROP_RESOURCE, "", GLOBAL, DROP) /* allows to execute DROP RESOURCE */\ M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 949fd37e403..a5d0451714b 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -701,15 +701,17 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; + const AccessFlags workload_ddl = AccessType::CREATE_WORKLOAD | AccessType::DROP_WORKLOAD; + const AccessFlags resource_ddl = AccessType::CREATE_RESOURCE | AccessType::DROP_RESOURCE; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; const AccessFlags table_and_dictionary_and_function_ddl = table_ddl | dictionary_ddl | function_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; - const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; + const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | workload_ddl | resource_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; - const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; + const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl | workload_ddl | resource_ddl; const AccessFlags introspection_flags = AccessType::INTROSPECTION; }; static const PrecalculatedFlags precalc; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 39499cc577d..3627d760d4c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -136,6 +136,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) +add_headers_and_sources(dbms Common/Scheduler/Workload) if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) diff --git a/src/Common/Priority.h b/src/Common/Priority.h index 8952fe4dd5a..f0e5787ae91 100644 --- a/src/Common/Priority.h +++ b/src/Common/Priority.h @@ -6,6 +6,7 @@ /// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize struct Priority { - Int64 value = 0; /// Note that lower value means higher priority. - constexpr operator Int64() const { return value; } /// NOLINT + using Value = Int64; + Value value = 0; /// Note that lower value means higher priority. + constexpr operator Value() const { return value; } /// NOLINT }; diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index 8a7077ac3d5..c6f41346e11 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -26,6 +26,9 @@ class IClassifier : private boost::noncopyable public: virtual ~IClassifier() = default; + /// Returns true iff resource access is allowed by this classifier + virtual bool has(const String & resource_name) = 0; + /// Returns ResourceLink that should be used to access resource. /// Returned link is valid until classifier destruction. virtual ResourceLink get(const String & resource_name) = 0; @@ -46,12 +49,15 @@ public: /// Initialize or reconfigure manager. virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0; + /// Returns true iff given resource is controlled through this manager. + virtual bool hasResource(const String & resource_name) const = 0; + /// Obtain a classifier instance required to get access to resources. /// Note that it holds resource configuration, so should be destructed when query is done. virtual ClassifierPtr acquire(const String & classifier_name) = 0; /// For introspection, see `system.scheduler` table - using VisitorFunc = std::function; + using VisitorFunc = std::function; virtual void forEachNode(VisitorFunc visitor) = 0; }; diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index a976206de74..3bee9c1b424 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -15,8 +15,7 @@ namespace DB * When constraint is again satisfied, scheduleActivation() is called from finishRequest(). * * Derived class behaviour requirements: - * - dequeueRequest() must fill `request->constraint` iff it is nullptr; - * - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`. + * - dequeueRequest() must call `request->addConstraint()`. */ class ISchedulerConstraint : public ISchedulerNode { @@ -25,34 +24,16 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + /// Resource consumption by `request` is finished. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; - void setParent(ISchedulerNode * parent_) override - { - ISchedulerNode::setParent(parent_); - - // Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint - for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent) - { - if (auto * constraint = dynamic_cast(node)) - { - parent_constraint = constraint; - break; - } - } - } - /// For introspection of current state (true = satisfied, false = violated) virtual bool isSatisfied() = 0; - -protected: - // Reference to nearest parent that is also derived from ISchedulerConstraint. - // Request can traverse through multiple constraints while being dequeue from hierarchy, - // while finishing request should traverse the same chain in reverse order. - // NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest() - ISchedulerConstraint * parent_constraint = nullptr; }; } diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 0705c4f0a35..5e1239de274 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -57,7 +57,13 @@ struct SchedulerNodeInfo SchedulerNodeInfo() = default; - explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + explicit SchedulerNodeInfo(double weight_, Priority priority_ = {}) + { + setWeight(weight_); + setPriority(priority_); + } + + explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config, const String & config_prefix = {}) { setWeight(config.getDouble(config_prefix + ".weight", weight)); setPriority(config.getInt64(config_prefix + ".priority", priority)); @@ -68,7 +74,7 @@ struct SchedulerNodeInfo if (value <= 0 || !isfinite(value)) throw Exception( ErrorCodes::INVALID_SCHEDULER_NODE, - "Negative and non-finite node weights are not allowed: {}", + "Zero, negative and non-finite node weights are not allowed: {}", value); weight = value; } @@ -78,6 +84,11 @@ struct SchedulerNodeInfo priority.value = value; } + void setPriority(Priority value) + { + priority = value; + } + // To check if configuration update required bool equals(const SchedulerNodeInfo & o) const { @@ -123,7 +134,14 @@ public: , info(config, config_prefix) {} - virtual ~ISchedulerNode() = default; + ISchedulerNode(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : event_queue(event_queue_) + , info(info_) + {} + + virtual ~ISchedulerNode(); + + virtual const String & getTypeName() const = 0; /// Checks if two nodes configuration is equal virtual bool equals(ISchedulerNode * other) @@ -134,10 +152,11 @@ public: /// Attach new child virtual void attachChild(const std::shared_ptr & child) = 0; - /// Detach and destroy child + /// Detach child + /// NOTE: child might be destroyed if the only reference was stored in parent virtual void removeChild(ISchedulerNode * child) = 0; - /// Get attached child by name + /// Get attached child by name (for tests only) virtual ISchedulerNode * getChild(const String & child_name) = 0; /// Activation of child due to the first pending request @@ -147,7 +166,7 @@ public: /// Returns true iff node is active virtual bool isActive() = 0; - /// Returns number of active children + /// Returns number of active children (for introspection only). virtual size_t activeChildren() = 0; /// Returns the first request to be executed as the first component of resulting pair. @@ -155,10 +174,10 @@ public: virtual std::pair dequeueRequest() = 0; /// Returns full path string using names of every parent - String getPath() + String getPath() const { String result; - ISchedulerNode * ptr = this; + const ISchedulerNode * ptr = this; while (ptr->parent) { result = "/" + ptr->basename + result; @@ -168,10 +187,7 @@ public: } /// Attach to a parent (used by attachChild) - virtual void setParent(ISchedulerNode * parent_) - { - parent = parent_; - } + void setParent(ISchedulerNode * parent_); protected: /// Notify parents about the first pending request or constraint becoming satisfied. @@ -307,6 +323,15 @@ public: pending.notify_one(); } + /// Removes an activation from queue + void cancelActivation(ISchedulerNode * node) + { + std::unique_lock lock{mutex}; + if (node->is_linked()) + activations.erase(activations.iterator_to(*node)); + node->activation_event_id = 0; + } + /// Process single event if it exists /// Note that postponing constraint are ignored, use it to empty the queue including postponed events on shutdown /// Returns `true` iff event has been processed @@ -471,6 +496,20 @@ private: std::atomic manual_time{TimePoint()}; // for tests only }; +inline ISchedulerNode::~ISchedulerNode() +{ + // Make sure there is no dangling reference in activations queue + event_queue->cancelActivation(this); +} + +inline void ISchedulerNode::setParent(ISchedulerNode * parent_) +{ + parent = parent_; + // Avoid activation of a detached node + if (parent == nullptr) + event_queue->cancelActivation(this); +} + inline void ISchedulerNode::scheduleActivation() { if (likely(parent)) diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index b7a51870a24..6c77cee6b9d 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -21,6 +21,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + // Wrapper for `enqueueRequest()` that should be used to account for available resource budget // Returns `estimated_cost` that should be passed later to `adjustBudget()` [[ nodiscard ]] ResourceCost enqueueRequestUsingBudget(ResourceRequest * request) @@ -47,6 +51,11 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual bool cancelRequest(ResourceRequest * request) = 0; + /// Fails all the resource requests in queue and marks this queue as not usable. + /// Afterwards any new request will be failed on `enqueueRequest()`. + /// NOTE: This is done for queues that are about to be destructed. + virtual void purgeQueue() = 0; + /// For introspection ResourceCost getBudget() const { diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 3be61801149..455d0880aa6 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int RESOURCE_NOT_FOUND; -} - ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { Poco::Util::AbstractConfiguration::Keys keys; @@ -31,9 +26,11 @@ ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & c const ClassifierDescription & ClassifiersConfig::get(const String & classifier_name) { + static ClassifierDescription empty; if (auto it = classifiers.find(classifier_name); it != classifiers.end()) return it->second; - throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name); + else + return empty; } } diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.h b/src/Common/Scheduler/Nodes/ClassifiersConfig.h index 186c49943ad..62db719568b 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.h +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.h @@ -10,6 +10,7 @@ namespace DB /// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class") struct ClassifierDescription : std::unordered_map { + ClassifierDescription() = default; ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); }; diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp similarity index 84% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.cpp rename to src/Common/Scheduler/Nodes/CustomResourceManager.cpp index 5bf884fc3df..b9ab89ee2b8 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp @@ -1,7 +1,6 @@ -#include +#include #include -#include #include #include @@ -21,7 +20,7 @@ namespace ErrorCodes extern const int INVALID_SCHEDULER_NODE; } -DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) +CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) : classifiers(config) { Poco::Util::AbstractConfiguration::Keys keys; @@ -35,7 +34,7 @@ DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util: } } -DynamicResourceManager::State::Resource::Resource( +CustomResourceManager::State::Resource::Resource( const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, @@ -92,7 +91,7 @@ DynamicResourceManager::State::Resource::Resource( throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name); } -DynamicResourceManager::State::Resource::~Resource() +CustomResourceManager::State::Resource::~Resource() { // NOTE: we should rely on `attached_to` and cannot use `parent`, // NOTE: because `parent` can be `nullptr` in case attachment is still in event queue @@ -106,14 +105,14 @@ DynamicResourceManager::State::Resource::~Resource() } } -DynamicResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +CustomResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) : type(config.getString(config_prefix + ".type", "fifo")) , ptr(SchedulerNodeFactory::instance().get(type, event_queue, config, config_prefix)) { ptr->basename = name; } -bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManager::State::Resource & o) const +bool CustomResourceManager::State::Resource::equals(const CustomResourceManager::State::Resource & o) const { if (nodes.size() != o.nodes.size()) return false; @@ -130,14 +129,14 @@ bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManage return true; } -bool DynamicResourceManager::State::Node::equals(const DynamicResourceManager::State::Node & o) const +bool CustomResourceManager::State::Node::equals(const CustomResourceManager::State::Node & o) const { if (type != o.type) return false; return ptr->equals(o.ptr.get()); } -DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::StatePtr & state_, const String & classifier_name) +CustomResourceManager::Classifier::Classifier(const CustomResourceManager::StatePtr & state_, const String & classifier_name) : state(state_) { // State is immutable, but nodes are mutable and thread-safe @@ -162,20 +161,25 @@ DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::Sta } } -ResourceLink DynamicResourceManager::Classifier::get(const String & resource_name) +bool CustomResourceManager::Classifier::has(const String & resource_name) +{ + return resources.contains(resource_name); +} + +ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) return iter->second; throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); } -DynamicResourceManager::DynamicResourceManager() +CustomResourceManager::CustomResourceManager() : state(new State()) { scheduler.start(); } -void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { StatePtr new_state = std::make_shared(scheduler.event_queue, config); @@ -217,7 +221,13 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } -ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) +bool CustomResourceManager::hasResource(const String & resource_name) const +{ + std::lock_guard lock{mutex}; + return state->resources.contains(resource_name); +} + +ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state StatePtr state_ref; @@ -229,7 +239,7 @@ ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) return std::make_shared(state_ref, classifier_name); } -void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { // Acquire a reference to the current state StatePtr state_ref; @@ -244,7 +254,7 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { for (auto & [name, resource] : state_ref->resources) for (auto & [path, node] : resource->nodes) - visitor(name, path, node.type, node.ptr); + visitor(name, path, node.ptr.get()); promise.set_value(); }); @@ -252,9 +262,4 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) future.get(); } -void registerDynamicResourceManager(ResourceManagerFactory & factory) -{ - factory.registerMethod("dynamic"); -} - } diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.h b/src/Common/Scheduler/Nodes/CustomResourceManager.h similarity index 86% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.h rename to src/Common/Scheduler/Nodes/CustomResourceManager.h index 4b0a3a48b61..900a9c4e50b 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.h +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.h @@ -10,7 +10,9 @@ namespace DB { /* - * Implementation of `IResourceManager` supporting arbitrary dynamic hierarchy of scheduler nodes. + * Implementation of `IResourceManager` supporting arbitrary hierarchy of scheduler nodes. + * Scheduling hierarchies for every resource is described through server xml or yaml configuration. + * Configuration could be changed dynamically without server restart. * All resources are controlled by single root `SchedulerRoot`. * * State of manager is set of resources attached to the scheduler. States are referenced by classifiers. @@ -24,11 +26,12 @@ namespace DB * violation will apply to fairness. Old version exists as long as there is at least one classifier * instance referencing it. Classifiers are typically attached to queries and will be destructed with them. */ -class DynamicResourceManager : public IResourceManager +class CustomResourceManager : public IResourceManager { public: - DynamicResourceManager(); + CustomResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; @@ -79,6 +82,7 @@ private: { public: Classifier(const StatePtr & state_, const String & classifier_name); + bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; private: std::unordered_map resources; // accessible resources by names @@ -86,7 +90,7 @@ private: }; SchedulerRoot scheduler; - std::mutex mutex; + mutable std::mutex mutex; StatePtr state; }; diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index 246642ff2fd..a865711c460 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -28,7 +28,7 @@ namespace ErrorCodes * of a child is set to vruntime of "start" of the last request. This guarantees immediate processing * of at least single request of newly activated children and thus best isolation and scheduling latency. */ -class FairPolicy : public ISchedulerNode +class FairPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item @@ -48,6 +48,23 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + FairPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + + ~FairPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + + const String & getTypeName() const override + { + static String type_name("fair"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 90f8fffe665..9502fae1a45 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -23,13 +23,28 @@ namespace ErrorCodes /* * FIFO queue to hold pending resource requests */ -class FifoQueue : public ISchedulerQueue +class FifoQueue final : public ISchedulerQueue { public: FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) : ISchedulerQueue(event_queue_, config, config_prefix) {} + FifoQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerQueue(event_queue_, info_) + {} + + ~FifoQueue() override + { + purgeQueue(); + } + + const String & getTypeName() const override + { + static String type_name("fifo"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -42,6 +57,8 @@ public: void enqueueRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue is about to be destructed"); queue_cost += request->cost; bool was_empty = requests.empty(); requests.push_back(*request); @@ -66,6 +83,8 @@ public: bool cancelRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + return false; // Any request should already be failed or executed if (request->is_linked()) { // It's impossible to check that `request` is indeed inserted to this queue and not another queue. @@ -88,6 +107,19 @@ public: return false; } + void purgeQueue() override + { + std::lock_guard lock(mutex); + is_not_usable = true; + while (!requests.empty()) + { + ResourceRequest * request = &requests.front(); + requests.pop_front(); + request->failed(std::make_exception_ptr( + Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request is about to be destructed"))); + } + } + bool isActive() override { std::lock_guard lock(mutex); @@ -131,6 +163,7 @@ private: std::mutex mutex; Int64 queue_cost = 0; boost::intrusive::list requests; + bool is_not_usable = false; }; } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp new file mode 100644 index 00000000000..e2042a29a80 --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -0,0 +1,532 @@ +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int RESOURCE_NOT_FOUND; + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +namespace +{ + String getEntityName(const ASTPtr & ast) + { + if (auto * create = typeid_cast(ast.get())) + return create->getWorkloadName(); + if (auto * create = typeid_cast(ast.get())) + return create->getResourceName(); + return "unknown-workload-entity"; + } +} + +IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resource_name) +{ + auto * create = assert_cast(ast.get()); + name = create->getWorkloadName(); + parent = create->getWorkloadParent(); + settings.updateFromChanges(create->changes, resource_name); +} + +IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) + : resource_entity(resource_entity_) + , resource_name(getEntityName(resource_entity)) +{ + scheduler.start(); +} + +IOResourceManager::Resource::~Resource() +{ + scheduler.stop(); +} + +void IOResourceManager::Resource::createNode(const NodeInfo & info) +{ + if (info.name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", + resource_name); + + if (info.name == info.parent) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", + info.name, resource_name); + + if (node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", + info.name, resource_name); + + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + if (info.parent.empty() && root_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", + info.name, root_node->basename, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = std::make_shared(scheduler.event_queue, info.settings); + node->basename = info.name; + if (!info.parent.empty()) + node_for_workload[info.parent]->attachUnifiedChild(node); + else + { + root_node = node; + scheduler.attachChild(root_node); + } + node_for_workload[info.name] = node; + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::deleteNode(const NodeInfo & info) +{ + if (!node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for removing workload '{}' does not exist in resource '{}'", + info.name, resource_name); + + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for removing workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + auto node = node_for_workload[info.name]; + + if (node->hasUnifiedChildren()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", + info.name, resource_name); + + executeInSchedulerThread([&] + { + if (!info.parent.empty()) + node_for_workload[info.parent]->detachUnifiedChild(node); + else + { + chassert(node == root_node); + scheduler.removeChild(root_node.get()); + root_node.reset(); + } + + node_for_workload.erase(info.name); + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const NodeInfo & new_info) +{ + if (old_info.name != new_info.name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", + old_info.name, new_info.name, resource_name); + + if (old_info.parent != new_info.parent && (old_info.parent.empty() || new_info.parent.empty())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", + old_info.name, old_info.parent, new_info.parent, resource_name); + + if (!node_for_workload.contains(old_info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for updating workload '{}' does not exist in resource '{}'", + old_info.name, resource_name); + + if (!old_info.parent.empty() && !node_for_workload.contains(old_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Old parent node '{}' for updating workload '{}' does not exist in resource '{}'", + old_info.parent, old_info.name, resource_name); + + if (!new_info.parent.empty() && !node_for_workload.contains(new_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "New parent node '{}' for updating workload '{}' does not exist in resource '{}'", + new_info.parent, new_info.name, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = node_for_workload[old_info.name]; + bool detached = false; + if (UnifiedSchedulerNode::updateRequiresDetach(old_info.parent, new_info.parent, old_info.settings, new_info.settings)) + { + if (!old_info.parent.empty()) + node_for_workload[old_info.parent]->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_info.settings); + + if (detached) + { + if (!new_info.parent.empty()) + node_for_workload[new_info.parent]->attachUnifiedChild(node); + } + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateCurrentVersion() +{ + auto previous_version = current_version; + + // Create a full list of constraints and queues in the current hierarchy + current_version = std::make_shared(); + if (root_node) + root_node->addRawPointerNodes(current_version->nodes); + + // See details in version control section of description in IOResourceManager.h + if (previous_version) + { + previous_version->newer_version = current_version; + previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it + } +} + +IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_) + : resource_manager(resource_manager_) + , workload_entity(workload_entity_) +{ + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->createNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } +} + +IOResourceManager::Workload::~Workload() +{ + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->deleteNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } +} + +void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) +{ + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); + workload_entity = new_entity; + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } +} + +String IOResourceManager::Workload::getParent() const +{ + return assert_cast(workload_entity.get())->getWorkloadParent(); +} + +IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) + : storage(storage_) + , log{getLogger("IOResourceManager")} +{ + subscription = storage.getAllEntitiesAndSubscribe( + [this] (const std::vector & events) + { + for (const auto & [entity_type, entity_name, entity] : events) + { + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + break; + } + case WorkloadEntityType::Resource: + { + if (entity) + createOrUpdateResource(entity_name, entity); + else + deleteResource(entity_name); + break; + } + case WorkloadEntityType::MAX: break; + } + } + }); +} + +IOResourceManager::~IOResourceManager() +{ + subscription.reset(); + resources.clear(); + workloads.clear(); +} + +void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) +{ + // No-op +} + +void IOResourceManager::createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + workload_iter->second->updateWorkload(ast); + else + workloads.emplace(workload_name, std::make_shared(this, ast)); +} + +void IOResourceManager::deleteWorkload(const String & workload_name) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + { + // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent + workloads.erase(workload_iter); + } + else // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete workload that doesn't exist: {}", workload_name); +} + +void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + resource_iter->second->updateResource(ast); + else + { + // Add all workloads into the new resource + auto resource = std::make_shared(ast); + for (Workload * workload : topologicallySortedWorkloads()) + resource->createNode(NodeInfo(workload->workload_entity, resource_name)); + + // Attach the resource + resources.emplace(resource_name, resource); + } +} + +void IOResourceManager::deleteResource(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + { + resources.erase(resource_iter); + } + else // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete resource that doesn't exist: {}", resource_name); +} + +IOResourceManager::Classifier::~Classifier() +{ + // Detach classifier from all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(attachments.size()); + for (auto & [resource_name, attachment] : attachments) + { + futures.emplace_back(attachment.resource->detachClassifier(std::move(attachment.version))); + attachment.link.reset(); // Just in case because it is not valid any longer + } + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // There should not be any exceptions because it just destruct few objects, but let's rethrow just in case + for (auto & future : futures) + future.get(); + + // This unreferences and probably destroys `Resource` objects. + // NOTE: We cannot do it in the scheduler threads (because thread cannot join itself). + attachments.clear(); +} + +std::future IOResourceManager::Resource::detachClassifier(VersionPtr && version) +{ + auto detach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semanticss + auto future = detach_promise->get_future(); + scheduler.event_queue->enqueue([detached_version = std::move(version), promise = std::move(detach_promise)] mutable + { + try + { + // Unreferences and probably destroys the version and scheduler nodes it owns. + // The main reason from moving destruction into the scheduler thread is to + // free memory in the same thread it was allocated to avoid memtrackers drift. + detached_version.reset(); + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +bool IOResourceManager::Classifier::has(const String & resource_name) +{ + std::unique_lock lock{mutex}; + return attachments.contains(resource_name); +} + +ResourceLink IOResourceManager::Classifier::get(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto iter = attachments.find(resource_name); iter != attachments.end()) + return iter->second.link; + else + throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Access denied to resource '{}'", resource_name); +} + +void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link) +{ + std::unique_lock lock{mutex}; + chassert(!attachments.contains(resource->getName())); + attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; +} + +void IOResourceManager::Resource::updateResource(const ASTPtr & new_resource_entity) +{ + chassert(getEntityName(new_resource_entity) == resource_name); + resource_entity = new_resource_entity; +} + +std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) +{ + auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics + auto future = attach_promise->get_future(); + scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] + { + try + { + if (auto iter = node_for_workload.find(workload_name); iter != node_for_workload.end()) + { + auto queue = iter->second->getQueue(); + if (!queue) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to use workload '{}' that have children for resource '{}'", + workload_name, resource_name); + classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); + } + else + { + // This resource does not have specified workload. It is either unknown or managed by another resource manager. + // We leave this resource not attached to the classifier. Access denied will be thrown later on `classifier->get(resource_name)` + } + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +bool IOResourceManager::hasResource(const String & resource_name) const +{ + std::unique_lock lock{mutex}; + return resources.contains(resource_name); +} + +ClassifierPtr IOResourceManager::acquire(const String & workload_name) +{ + auto classifier = std::make_shared(); + + // Attach classifier to all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(resources.size()); + for (auto & [resource_name, resource] : resources) + futures.emplace_back(resource->attachClassifier(*classifier, workload_name)); + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // Rethrow exceptions if any + for (auto & future : futures) + future.get(); + + return classifier; +} + +void IOResourceManager::Resource::forEachResourceNode(IResourceManager::VisitorFunc & visitor) +{ + executeInSchedulerThread([&, this] + { + for (auto & [path, node] : node_for_workload) + { + node->forEachSchedulerNode([&] (ISchedulerNode * scheduler_node) + { + visitor(resource_name, scheduler_node->getPath(), scheduler_node); + }); + } + }); +} + +void IOResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +{ + // Copy resource to avoid holding mutex for a long time + std::unordered_map resources_copy; + { + std::unique_lock lock{mutex}; + resources_copy = resources; + } + + /// Run tasks one by one to avoid concurrent calls to visitor + for (auto & [resource_name, resource] : resources_copy) + resource->forEachResourceNode(visitor); +} + +void IOResourceManager::topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads) +{ + if (visited.contains(workload)) + return; + visited.insert(workload); + + // Recurse into parent (if any) + String parent = workload->getParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + chassert(parent_iter != workloads.end()); // validations check that all parents exist + topologicallySortedWorkloadsImpl(parent_iter->second.get(), visited, sorted_workloads); + } + + sorted_workloads.push_back(workload); +} + +std::vector IOResourceManager::topologicallySortedWorkloads() +{ + std::vector sorted_workloads; + std::unordered_set visited; + for (auto & [workload_name, workload] : workloads) + topologicallySortedWorkloadsImpl(workload.get(), visited, sorted_workloads); + return sorted_workloads; +} + +} diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h new file mode 100644 index 00000000000..cfd8a234b37 --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -0,0 +1,281 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +/* + * Implementation of `IResourceManager` that creates hierarchy of scheduler nodes according to + * workload entities (WORKLOADs and RESOURCEs). It subscribes for updates in IWorkloadEntityStorage and + * creates hierarchy of UnifiedSchedulerNode identical to the hierarchy of WORKLOADs. + * For every RESOURCE an independent hierarchy of scheduler nodes is created. + * + * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. + * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). + * After DROP RESOURCE parts of hierarchy might be kept alive while at least one query uses it. + * + * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having + * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. + * + * Classifiers are used (1) to access IO resources and (2) to keep shared ownership of scheduling nodes. + * This allows `ResourceRequest` and `ResourceLink` to hold raw pointers as long as + * `ClassifierPtr` is acquired and held. + * + * === RESOURCE ARCHITECTURE === + * Let's consider how a single resource is implemented. Every workload is represented by corresponding UnifiedSchedulerNode. + * Every UnifiedSchedulerNode manages its own subtree of ISchedulerNode objects (see details in UnifiedSchedulerNode.h) + * UnifiedSchedulerNode for workload w/o children has a queue, which provide a ResourceLink for consumption. + * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. + * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). + * + * Here is an example of SQL and corresponding hierarchy of scheduler nodes: + * CREATE RESOURCE my_io_resource (...) + * CREATE WORKLOAD all + * CREATE WORKLOAD production PARENT all + * CREATE WORKLOAD development PARENT all + * + * root - SchedulerRoot (with scheduler thread and EventQueue) + * | + * all - UnifiedSchedulerNode + * | + * p0_fair - FairPolicy (part of parent UnifiedSchedulerNode internal structure) + * / \ + * production development - UnifiedSchedulerNode + * | | + * queue queue - FifoQueue (part of parent UnifiedSchedulerNode internal structure) + * + * === UPDATING WORKLOADS === + * Workload may be created, updated or deleted. + * Updating a child of a workload might lead to updating other workloads: + * 1. Workload itself: it's structure depend on settings of children workloads + * (e.g. fifo node of a leaf workload is remove when the first child is added; + * and a fair node is inserted after the first two children are added). + * 2. Other children: for them path to root might be changed (e.g. intermediate priority node is inserted) + * + * === VERSION CONTROL === + * Versions are created on hierarchy updates and hold ownership of nodes that are used through raw pointers. + * Classifier reference version of every resource it use. Older version reference newer version. + * Here is a diagram explaining version control based on Version objects (for 1 resource): + * + * [nodes] [nodes] [nodes] + * ^ ^ ^ + * | | | + * version1 --> version2 -...-> versionN + * ^ ^ ^ + * | | | + * old_classifier new_classifier current_version + * + * Previous version should hold reference to a newer version. It is required for proper handling of updates. + * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). + * It may move a queue to a new position in the hierarchy or create/destroy constraints, thus resource requests + * created by old classifier may reference constraints of newer versions through `request->constraints` which + * is filled during dequeueRequest(). + * + * === THREADS === + * scheduler thread: + * - one thread per resource + * - uses event_queue (per resource) for processing w/o holding mutex for every scheduler node + * - handle resource requests + * - node activations + * - scheduler hierarchy updates + * query thread: + * - multiple independent threads + * - send resource requests + * - acquire and release classifiers (via scheduler event queues) + * control thread: + * - modify workload and resources through subscription + * + * === SYNCHRONIZATION === + * List of related sync primitives and their roles: + * IOResourceManager::mutex + * - protects resource manager data structures - resource and workloads + * - serialize control thread actions + * IOResourceManager::Resource::scheduler->event_queue + * - serializes scheduler hierarchy events + * - events are created in control and query threads + * - all events are processed by specific scheduler thread + * - hierarchy-wide actions: requests dequeueing, activations propagation and nodes updates. + * - resource version control management + * FifoQueue::mutex and SemaphoreContraint::mutex + * - serializes query and scheduler threads on specific node accesses + * - resource request processing: enqueueRequest(), dequeueRequest() and finishRequest() + */ +class IOResourceManager : public IResourceManager +{ +public: + explicit IOResourceManager(IWorkloadEntityStorage & storage_); + ~IOResourceManager() override; + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; + ClassifierPtr acquire(const String & workload_name) override; + void forEachNode(VisitorFunc visitor) override; + +private: + // Forward declarations + struct NodeInfo; + struct Version; + class Resource; + struct Workload; + class Classifier; + + friend struct Workload; + + using VersionPtr = std::shared_ptr; + using ResourcePtr = std::shared_ptr; + using WorkloadPtr = std::shared_ptr; + + /// Helper for parsing workload AST for a specific resource + struct NodeInfo + { + String name; // Workload name + String parent; // Name of parent workload + SchedulingSettings settings; // Settings specific for a given resource + + NodeInfo(const ASTPtr & ast, const String & resource_name); + }; + + /// Ownership control for scheduler nodes, which could be referenced by raw pointers + struct Version + { + std::vector nodes; + VersionPtr newer_version; + }; + + /// Holds a thread and hierarchy of unified scheduler nodes for specific RESOURCE + class Resource : public std::enable_shared_from_this, boost::noncopyable + { + public: + explicit Resource(const ASTPtr & resource_entity_); + ~Resource(); + + const String & getName() const { return resource_name; } + + /// Hierarchy management + void createNode(const NodeInfo & info); + void deleteNode(const NodeInfo & info); + void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); + + /// Updates resource entity + void updateResource(const ASTPtr & new_resource_entity); + + /// Updates a classifier to contain a reference for specified workload + std::future attachClassifier(Classifier & classifier, const String & workload_name); + + /// Remove classifier reference. This destroys scheduler nodes in proper scheduler thread + std::future detachClassifier(VersionPtr && version); + + /// Introspection + void forEachResourceNode(IOResourceManager::VisitorFunc & visitor); + + private: + void updateCurrentVersion(); + + template + void executeInSchedulerThread(Task && task) + { + std::promise promise; + auto future = promise.get_future(); + scheduler.event_queue->enqueue([&] + { + try + { + task(); + promise.set_value(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } + }); + future.get(); // Blocks until execution is done in the scheduler thread + } + + ASTPtr resource_entity; + const String resource_name; + SchedulerRoot scheduler; + + // TODO(serxa): consider using resource_manager->mutex + scheduler thread for updates and mutex only for reading to avoid slow acquire/release of classifier + /// These field should be accessed only by the scheduler thread + std::unordered_map node_for_workload; + UnifiedSchedulerNodePtr root_node; + VersionPtr current_version; + }; + + struct Workload : boost::noncopyable + { + IOResourceManager * resource_manager; + ASTPtr workload_entity; + + Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_); + ~Workload(); + + void updateWorkload(const ASTPtr & new_entity); + String getParent() const; + }; + + class Classifier : public IClassifier + { + public: + ~Classifier() override; + + /// Implements IClassifier interface + /// NOTE: It is called from query threads (possibly multiple) + bool has(const String & resource_name) override; + ResourceLink get(const String & resource_name) override; + + /// Attaches/detaches a specific resource + /// NOTE: It is called from scheduler threads (possibly multiple) + void attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link); + void detach(const ResourcePtr & resource); + + private: + IOResourceManager * resource_manager; + std::mutex mutex; + struct Attachment + { + ResourcePtr resource; + VersionPtr version; + ResourceLink link; + }; + std::unordered_map attachments; // TSA_GUARDED_BY(mutex); + }; + + void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); + void deleteWorkload(const String & workload_name); + void createOrUpdateResource(const String & resource_name, const ASTPtr & ast); + void deleteResource(const String & resource_name); + + // Topological sorting of workloads + void topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads); + std::vector topologicallySortedWorkloads(); + + IWorkloadEntityStorage & storage; + scope_guard subscription; + + mutable std::mutex mutex; + std::unordered_map workloads; // TSA_GUARDED_BY(mutex); + std::unordered_map resources; // TSA_GUARDED_BY(mutex); + + LoggerPtr log; +}; + +} diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index b170ab0dbee..cfbe242c13e 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -19,7 +19,7 @@ namespace ErrorCodes * Scheduler node that implements priority scheduling policy. * Requests are scheduled in order of priorities. */ -class PriorityPolicy : public ISchedulerNode +class PriorityPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item @@ -39,6 +39,23 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + explicit PriorityPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & node_info) + : ISchedulerNode(event_queue_, node_info) + {} + + ~PriorityPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + + const String & getTypeName() const override + { + static String type_name("priority"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index fe1b03b74bd..e223100a646 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/Scheduler/ISchedulerNode.h" #include #include @@ -13,7 +14,7 @@ namespace DB * Limited concurrency constraint. * Blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost` */ -class SemaphoreConstraint : public ISchedulerConstraint +class SemaphoreConstraint final : public ISchedulerConstraint { static constexpr Int64 default_max_requests = std::numeric_limits::max(); static constexpr Int64 default_max_cost = std::numeric_limits::max(); @@ -24,6 +25,25 @@ public: , max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost))) {} + SemaphoreConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, Int64 max_requests_, Int64 max_cost_) + : ISchedulerConstraint(event_queue_, info_) + , max_requests(max_requests_) + , max_cost(max_cost_) + {} + + ~SemaphoreConstraint() override + { + // We need to clear `parent` in child to avoid dangling references + if (child) + removeChild(child.get()); + } + + const String & getTypeName() const override + { + static String type_name("inflight_limit"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -68,15 +88,14 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; - - // Update state on request arrival std::unique_lock lock(mutex); - requests++; - cost += request->cost; + if (request->addConstraint(this)) + { + // Update state on request arrival + requests++; + cost += request->cost; + } + child_active = child_now_active; if (!active()) busy_periods++; @@ -86,10 +105,6 @@ public: void finishRequest(ResourceRequest * request) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // Update state on request departure std::unique_lock lock(mutex); bool was_active = active(); @@ -109,6 +124,32 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation or deactivation + void updateConstraints(const SchedulerNodePtr & self, Int64 new_max_requests, UInt64 new_max_cost) + { + std::unique_lock lock(mutex); + bool was_active = active(); + max_requests = new_max_requests; + max_cost = new_max_cost; + + if (parent) + { + // Activate on transition from inactive state + if (!was_active && active()) + parent->activateChild(this); + // Deactivate on transition into inactive state + else if (was_active && !active()) + { + // Node deactivation is usually done in dequeueRequest(), but we do not want to + // do extra call to active() on every request just to make sure there was no update(). + // There is no interface method to do deactivation, so we do the following trick. + parent->removeChild(this); + parent->attachChild(self); // This call is the only reason we have `recursive_mutex` + } + } + } + bool isActive() override { std::unique_lock lock(mutex); @@ -150,10 +191,10 @@ private: return satisfied() && child_active; } - const Int64 max_requests = default_max_requests; - const Int64 max_cost = default_max_cost; + Int64 max_requests = default_max_requests; + Int64 max_cost = default_max_cost; - std::mutex mutex; + std::recursive_mutex mutex; Int64 requests = 0; Int64 cost = 0; bool child_active = false; diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index b279cbe972b..a2594b7ff2e 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -3,8 +3,6 @@ #include #include -#include -#include #include @@ -15,7 +13,7 @@ namespace DB * Limited throughput constraint. Blocks if token-bucket constraint is violated: * i.e. more than `max_burst + duration * max_speed` cost units (aka tokens) dequeued from this node in last `duration` seconds. */ -class ThrottlerConstraint : public ISchedulerConstraint +class ThrottlerConstraint final : public ISchedulerConstraint { public: static constexpr double default_burst_seconds = 1.0; @@ -28,10 +26,28 @@ public: , tokens(max_burst) {} + ThrottlerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, double max_speed_, double max_burst_) + : ISchedulerConstraint(event_queue_, info_) + , max_speed(max_speed_) + , max_burst(max_burst_) + , last_update(event_queue_->now()) + , tokens(max_burst) + {} + ~ThrottlerConstraint() override { // We should cancel event on destruction to avoid dangling references from event queue event_queue->cancelPostponed(postponed); + + // We need to clear `parent` in child to avoid dangling reference + if (child) + removeChild(child.get()); + } + + const String & getTypeName() const override + { + static String type_name("bandwidth_limit"); + return type_name; } bool equals(ISchedulerNode * other) override @@ -78,10 +94,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + // We don't do `request->addConstraint(this)` because `finishRequest()` is no-op updateBucket(request->cost); @@ -92,12 +105,8 @@ public: return {request, active()}; } - void finishRequest(ResourceRequest * request) override + void finishRequest(ResourceRequest *) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // NOTE: Token-bucket constraint does not require any action when consumption ends } @@ -108,6 +117,21 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation + void updateConstraints(double new_max_speed, double new_max_burst) + { + event_queue->cancelPostponed(postponed); + postponed = EventQueue::not_postponed; + bool was_active = active(); + updateBucket(0, true); // To apply previous params for duration since `last_update` + max_speed = new_max_speed; + max_burst = new_max_burst; + updateBucket(0, false); // To postpone (if needed) using new params + if (!was_active && active() && parent) + parent->activateChild(this); + } + bool isActive() override { return active(); @@ -150,7 +174,7 @@ private: parent->activateChild(this); } - void updateBucket(ResourceCost use = 0) + void updateBucket(ResourceCost use = 0, bool do_not_postpone = false) { auto now = event_queue->now(); if (max_speed > 0.0) @@ -160,7 +184,7 @@ private: tokens -= use; // This is done outside min() to avoid passing large requests w/o token consumption after long idle period // Postpone activation until there is positive amount of tokens - if (tokens < 0.0) + if (!do_not_postpone && tokens < 0.0) { auto delay_ns = std::chrono::nanoseconds(static_cast(-tokens / max_speed * 1e9)); if (postponed == EventQueue::not_postponed) @@ -184,8 +208,8 @@ private: return satisfied() && child_active; } - const double max_speed{0}; /// in tokens per second - const double max_burst{0}; /// in tokens + double max_speed{0}; /// in tokens per second + double max_burst{0}; /// in tokens EventQueue::TimePoint last_update; UInt64 postponed = EventQueue::not_postponed; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h new file mode 100644 index 00000000000..84923c49c62 --- /dev/null +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -0,0 +1,606 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +class UnifiedSchedulerNode; +using UnifiedSchedulerNodePtr = std::shared_ptr; + +/* + * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. + * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes + * are also present. This approach is easiers for manipulations in runtime than using multiple types of nodes. + * + * Unified node is capable of updating its internal structure based on: + * 1. Number of children (fifo if =0 or fairness/priority if >0). + * 2. Priorities of its children (for subtree structure). + * 3. `SchedulingSettings` associated with unified node (for throttler and semaphore constraints). + * + * In general, unified node has "internal" subtree with the following structure: + * + * THIS <-- UnifiedSchedulerNode object + * | + * THROTTLER <-- [Optional] Throttling scheduling constraint + * | + * [If no children]------ SEMAPHORE <-- [Optional] Semaphore constraint + * | | + * FIFO PRIORITY <-- [Optional] Scheduling policy distinguishing priorities + * .-------' '-------. + * FAIRNESS[p1] ... FAIRNESS[pN] <-- [Optional] Policies for fairness if priorities are equal + * / \ / \ + * CHILD[p1,w1] ... CHILD[p1,wM] CHILD[pN,w1] ... CHILD[pN,wM] <-- Unified children (UnifiedSchedulerNode objects) + * + * NOTE: to distinguish different kinds of children we use the following terms: + * - immediate child: child of unified object (THROTTLER); + * - unified child: leaf of this "internal" subtree (CHILD[p,w]); + * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) + */ +class UnifiedSchedulerNode final : public ISchedulerNode +{ +private: + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, const SchedulerNodePtr & new_parent) + { + reparent(node, new_parent.get()); + } + + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) + { + chassert(node); + chassert(new_parent); + if (new_parent == node->parent) + return; + if (node->parent) + node->parent->removeChild(node.get()); + new_parent->attachChild(node); + } + + /// Helper function for managing a parent of a node + static void detach(const SchedulerNodePtr & node) + { + if (node->parent) + node->parent->removeChild(node.get()); + } + + /// A branch of the tree for a specific priority value + struct FairnessBranch + { + SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached + std::unordered_map children; // basename -> child + + bool empty() const { return children.empty(); } + + SchedulerNodePtr getRoot() + { + chassert(!children.empty()); + if (root) + return root; + chassert(children.size() == 1); + return children.begin()->second; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) + throw Exception( + ErrorCodes::INVALID_SCHEDULER_NODE, + "Can't add another child with the same path: {}", + it->second->getPath()); + + if (children.size() == 2) + { + // Insert fair node if we have just added the second child + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->info.setPriority(child->info.priority); + root->basename = fmt::format("p{}_fair", child->info.priority.value); + for (auto & [_, node] : children) + reparent(node, root); + return root; // New root has been created + } + else if (children.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue *, const UnifiedSchedulerNodePtr & child) + { + auto it = children.find(child->basename); + if (it == children.end()) + return {}; // unknown child + + detach(child); + children.erase(it); + if (children.size() == 1) + { + // Remove fair if the only child has left + chassert(root); + detach(root); + root.reset(); + return children.begin()->second; // The last child is a new root now + } + else if (children.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children have left) + } + }; + + /// Handles all the children nodes with intermediate fair and/or priority nodes + struct ChildrenBranch + { + SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached + std::unordered_map branches; /// Branches for different priority values + + // Returns true iff there are no unified children attached + bool empty() const { return branches.empty(); } + + SchedulerNodePtr getRoot() + { + chassert(!branches.empty()); + if (root) + return root; + return branches.begin()->second.getRoot(); // There should be exactly one child-branch + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + auto [it, new_branch] = branches.try_emplace(child->info.priority); + auto & child_branch = it->second; + auto branch_root = child_branch.attachUnifiedChild(event_queue_, child); + if (!new_branch) + { + if (branch_root) + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } + return {}; + } + else + { + chassert(branch_root); + if (branches.size() == 2) + { + // Insert priority node if we have just added the second branch + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->basename = "prio"; + for (auto & [_, branch] : branches) + reparent(branch.getRoot(), root); + return root; // New root has been created + } + else if (branches.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + auto it = branches.find(child->info.priority); + if (it == branches.end()) + return {}; // unknown child + + auto & child_branch = it->second; + auto branch_root = child_branch.detachUnifiedChild(event_queue_, child); + if (child_branch.empty()) + { + branches.erase(it); + if (branches.size() == 1) + { + // Remove priority node if the only child-branch has left + chassert(root); + detach(root); + root.reset(); + return branches.begin()->second.getRoot(); // The last child-branch is a new root now + } + else if (branches.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children-branches have left) + } + if (branch_root) + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } + return {}; // Root is the same + } + }; + + /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. + struct QueueOrChildrenBranch + { + SchedulerNodePtr queue; /// FifoQueue node is used if there are no children + ChildrenBranch branch; /// Used if there is at least one child + + SchedulerNodePtr getRoot() + { + if (queue) + return queue; + else + return branch.getRoot(); + } + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) + { + createQueue(event_queue_); + return queue; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (queue) + removeQueue(); + return branch.attachUnifiedChild(event_queue_, child); + } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (queue) + return {}; // No-op, it already has no children + auto branch_root = branch.detachUnifiedChild(event_queue_, child); + if (branch.empty()) + { + createQueue(event_queue_); + return queue; + } + return branch_root; + } + + private: + void createQueue(EventQueue * event_queue_) + { + queue = std::make_shared(event_queue_, SchedulerNodeInfo{}); + queue->basename = "fifo"; + } + + void removeQueue() + { + // This unified node will not be able to process resource requests any longer + // All remaining resource requests are be aborted on queue destruction + detach(queue); + std::static_pointer_cast(queue)->purgeQueue(); + queue.reset(); + } + }; + + /// Handles all the nodes under this unified node + /// Specifically handles constraints with `QueueOrChildrenBranch` under it + struct ConstraintsBranch + { + SchedulerNodePtr throttler; + SchedulerNodePtr semaphore; + QueueOrChildrenBranch branch; + SchedulingSettings settings; + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_, const SchedulingSettings & settings_) + { + settings = settings_; + SchedulerNodePtr node = branch.initialize(event_queue_); + if (settings.hasSemaphore()) + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_requests, settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + if (settings.hasThrottler()) + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_speed, settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + return node; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) + { + // If both semaphore and throttler exist we should reparent to the farthest from the root + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (auto branch_root = branch.detachUnifiedChild(event_queue_, child)) + { + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } + + /// Updates constraint-related nodes. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr updateSchedulingSettings(EventQueue * event_queue_, const SchedulingSettings & new_settings) + { + SchedulerNodePtr node = branch.getRoot(); + + if (!settings.hasSemaphore() && new_settings.hasSemaphore()) // Add semaphore + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_requests, new_settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + else if (settings.hasSemaphore() && !new_settings.hasSemaphore()) // Remove semaphore + { + detach(semaphore); + semaphore.reset(); + } + else if (settings.hasSemaphore() && new_settings.hasSemaphore()) // Update semaphore + { + static_cast(*semaphore).updateConstraints(semaphore, new_settings.max_requests, new_settings.max_cost); + node = semaphore; + } + + if (!settings.hasThrottler() && new_settings.hasThrottler()) // Add throttler + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_speed, new_settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + else if (settings.hasThrottler() && !new_settings.hasThrottler()) // Remove throttler + { + detach(throttler); + throttler.reset(); + } + else if (settings.hasThrottler() && new_settings.hasThrottler()) // Update throttler + { + static_cast(*throttler).updateConstraints(new_settings.max_speed, new_settings.max_burst); + node = throttler; + } + + settings = new_settings; + return node; + } + }; + +public: + explicit UnifiedSchedulerNode(EventQueue * event_queue_, const SchedulingSettings & settings) + : ISchedulerNode(event_queue_, SchedulerNodeInfo(settings.weight, settings.priority)) + { + immediate_child = impl.initialize(event_queue, settings); + reparent(immediate_child, this); + } + + ~UnifiedSchedulerNode() override + { + // We need to clear `parent` in child to avoid dangling references + if (immediate_child) + removeChild(immediate_child.get()); + } + + /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes + /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children + void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) + { + if (auto new_child = impl.attachUnifiedChild(event_queue, child)) + reparent(new_child, this); + } + + /// Detaches unified child and update all the intermediate nodes. + /// Detached child could be safely attached to another parent. + /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children + void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) + { + if (auto new_child = impl.detachUnifiedChild(event_queue, child)) + reparent(new_child, this); + } + + static bool updateRequiresDetach(const String & old_parent, const String & new_parent, const SchedulingSettings & old_settings, const SchedulingSettings & new_settings) + { + return old_parent != new_parent || old_settings.priority != new_settings.priority; + } + + /// Updates scheduling settings. Set of constraints might change. + /// NOTE: Caller is responsible for detaching and attaching if `updateRequiresDetach` returns true + void updateSchedulingSettings(const SchedulingSettings & new_settings) + { + info.setPriority(new_settings.priority); + info.setWeight(new_settings.weight); + if (auto new_child = impl.updateSchedulingSettings(event_queue, new_settings)) + reparent(new_child, this); + } + + const SchedulingSettings & getSettings() const + { + return impl.settings; + } + + /// Returns the queue to be used for resource requests or `nullptr` if it has unified children + std::shared_ptr getQueue() const + { + return static_pointer_cast(impl.branch.queue); + } + + /// Collects nodes that could be accessed with raw pointers by resource requests (queue and constraints) + /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints + /// for that queue might change in future, and `request->constraints` might reference nodes not in + /// the initial set of nodes returned by `addRawPointerNodes()`. To avoid destruction of such additional nodes + /// classifier must (indirectly) hold nodes return by `addRawPointerNodes()` for all future versions of + /// all unified nodes. Such a version control is done by `IOResourceManager`. + void addRawPointerNodes(std::vector & nodes) + { + // NOTE: `impl.throttler` could be skipped, because ThrottlerConstraint does not call `request->addConstraint()` + if (impl.semaphore) + nodes.push_back(impl.semaphore); + if (impl.branch.queue) + nodes.push_back(impl.branch.queue); + for (auto & [_, branch] : impl.branch.branch.branches) + { + for (auto & [_, child] : branch.children) + child->addRawPointerNodes(nodes); + } + } + + bool hasUnifiedChildren() const + { + return impl.branch.queue == nullptr; + } + + /// Introspection. Calls a visitor for self and every internal node. Do not recurse into unified children. + void forEachSchedulerNode(std::function visitor) + { + visitor(this); + if (impl.throttler) + visitor(impl.throttler.get()); + if (impl.semaphore) + visitor(impl.semaphore.get()); + if (impl.branch.queue) + visitor(impl.branch.queue.get()); + if (impl.branch.branch.root) // priority + visitor(impl.branch.branch.root.get()); + for (auto & [_, branch] : impl.branch.branch.branches) + { + if (branch.root) // fairness + visitor(branch.root.get()); + } + } + +protected: // Hide all the ISchedulerNode interface methods as an implementation details + const String & getTypeName() const override + { + static String type_name("unified"); + return type_name; + } + + bool equals(ISchedulerNode *) override + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "UnifiedSchedulerNode should not be used with CustomResourceManager"); + } + + /// Attaches an immediate child (used through `reparent()`) + void attachChild(const SchedulerNodePtr & child_) override + { + immediate_child = child_; + immediate_child->setParent(this); + + // Activate if required + if (immediate_child->isActive()) + activateChild(immediate_child.get()); + } + + /// Removes an immediate child (used through `reparent()`) + void removeChild(ISchedulerNode * child) override + { + if (immediate_child.get() == child) + { + child_active = false; // deactivate + immediate_child->setParent(nullptr); // detach + immediate_child.reset(); + } + } + + ISchedulerNode * getChild(const String & child_name) override + { + if (immediate_child->basename == child_name) + return immediate_child.get(); + else + return nullptr; + } + + std::pair dequeueRequest() override + { + auto [request, child_now_active] = immediate_child->dequeueRequest(); + if (!request) + return {nullptr, false}; + + child_active = child_now_active; + if (!child_active) + busy_periods++; + incrementDequeued(request->cost); + return {request, child_active}; + } + + bool isActive() override + { + return child_active; + } + + /// Shows number of immediate active children (for introspection) + size_t activeChildren() override + { + return child_active; + } + + /// Activate an immediate child + void activateChild(ISchedulerNode * child) override + { + if (child == immediate_child.get()) + if (!std::exchange(child_active, true) && parent) + parent->activateChild(this); + } + +private: + ConstraintsBranch impl; + SchedulerNodePtr immediate_child; // An immediate child (actually the root of the whole subtree) + bool child_active = false; +}; + +} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp b/src/Common/Scheduler/Nodes/registerResourceManagers.cpp deleted file mode 100644 index c5d5ba5b981..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDynamicResourceManager(ResourceManagerFactory &); - -void registerResourceManagers() -{ - auto & factory = ResourceManagerFactory::instance(); - registerDynamicResourceManager(factory); -} - -} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.h b/src/Common/Scheduler/Nodes/registerResourceManagers.h deleted file mode 100644 index 243b25a9587..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.h +++ /dev/null @@ -1,8 +0,0 @@ -#pragma once - -namespace DB -{ - -void registerResourceManagers(); - -} diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index c787a686a09..927f87d5aa6 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,5 +1,8 @@ #pragma once +#include + +#include #include #include #include @@ -7,26 +10,35 @@ #include #include #include +#include #include -#include #include #include #include +#include +#include +#include #include #include #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + struct ResourceTestBase { ResourceTestBase() { - [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }(); + [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); return true; }(); } template @@ -37,10 +49,16 @@ struct ResourceTestBase Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)}; String config_prefix = "node"; + return add(event_queue, root_node, path, std::ref(*config), config_prefix); + } + + template + static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) + { if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); - root_node.reset(new TClass(event_queue, *config, config_prefix)); + root_node.reset(new TClass(event_queue, std::forward(args)...)); return static_cast(root_node.get()); } @@ -65,73 +83,114 @@ struct ResourceTestBase } EXPECT_TRUE(!child_name.empty()); // wrong path - SchedulerNodePtr node = std::make_shared(event_queue, *config, config_prefix); + SchedulerNodePtr node = std::make_shared(event_queue, std::forward(args)...); node->basename = child_name; parent->attachChild(node); return static_cast(node.get()); } }; - -struct ConstraintTest : public SemaphoreConstraint -{ - explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : SemaphoreConstraint(event_queue_, config, config_prefix) - {} - - std::pair dequeueRequest() override - { - auto [request, active] = SemaphoreConstraint::dequeueRequest(); - if (request) - { - std::unique_lock lock(mutex); - requests.insert(request); - } - return {request, active}; - } - - void finishRequest(ResourceRequest * request) override - { - { - std::unique_lock lock(mutex); - requests.erase(request); - } - SemaphoreConstraint::finishRequest(request); - } - - std::mutex mutex; - std::set requests; -}; - class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest { + ResourceTestClass * test; String name; - Request(ResourceCost cost_, const String & name_) + Request(ResourceTestClass * test_, ResourceCost cost_, const String & name_) : ResourceRequest(cost_) + , test(test_) , name(name_) {} void execute() override { } + + void failed(const std::exception_ptr &) override + { + test->failed_cost += cost; + delete this; + } }; public: + ~ResourceTestClass() + { + if (root_node) + dequeue(); // Just to avoid any leaks of `Request` object + } + template void add(const String & path, const String & xml = {}) { ResourceTestBase::add(&event_queue, root_node, path, xml); } + template + void addCustom(const String & path, Args... args) + { + ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); + } + + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const SchedulingSettings & settings = {}) + { + return createUnifiedNode(basename, {}, settings); + } + + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const UnifiedSchedulerNodePtr & parent, const SchedulingSettings & settings = {}) + { + auto node = std::make_shared(&event_queue, settings); + node->basename = basename; + if (parent) + { + parent->attachUnifiedChild(node); + } + else + { + EXPECT_TRUE(root_node.get() == nullptr); + root_node = node; + } + return node; + } + + // Updates the parent and/or scheduling settings for a specidfied `node`. + // Unit test implementation must make sure that all needed queues and constraints are not going to be destroyed. + // Normally it is the responsibility of IOResourceManager, but we do not use it here, so manual version control is required. + // (see IOResourceManager::Resource::updateCurrentVersion() fo details) + void updateUnifiedNode(const UnifiedSchedulerNodePtr & node, const UnifiedSchedulerNodePtr & old_parent, const UnifiedSchedulerNodePtr & new_parent, const SchedulingSettings & new_settings) + { + EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported + bool detached = false; + if (UnifiedSchedulerNode::updateRequiresDetach( + old_parent ? old_parent->basename : "", + new_parent ? new_parent->basename : "", + node->getSettings(), + new_settings)) + { + if (old_parent) + old_parent->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_settings); + + if (detached && new_parent) + new_parent->attachUnifiedChild(node); + } + + + void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) + { + enqueueImpl(node->getQueue().get(), costs, node->basename); + } + void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first ISchedulerNode * node = root_node.get(); size_t pos = 1; - while (pos < path.length()) + while (node && pos < path.length()) { size_t slash = path.find('/', pos); if (slash != String::npos) @@ -146,13 +205,17 @@ public: pos = String::npos; } } - ISchedulerQueue * queue = dynamic_cast(node); - ASSERT_TRUE(queue != nullptr); // not a queue + if (node) + enqueueImpl(dynamic_cast(node), costs); + } + void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs, const String & name = {}) + { + ASSERT_TRUE(queue != nullptr); // not a queue + if (!queue) + return; // to make clang-analyzer-core.NonNullParamChecker happy for (ResourceCost cost : costs) - { - queue->enqueueRequest(new Request(cost, queue->basename)); - } + queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues } @@ -208,6 +271,12 @@ public: consumed_cost[name] -= value; } + void failed(ResourceCost value) + { + EXPECT_EQ(failed_cost, value); + failed_cost -= value; + } + void processEvents() { while (event_queue.tryProcess()) {} @@ -217,8 +286,11 @@ private: EventQueue event_queue; SchedulerNodePtr root_node; std::unordered_map consumed_cost; + ResourceCost failed_cost = 0; }; +enum EnqueueOnlyEnum { EnqueueOnly }; + template struct ResourceTestManager : public ResourceTestBase { @@ -230,16 +302,49 @@ struct ResourceTestManager : public ResourceTestBase struct Guard : public ResourceGuard { ResourceTestManager & t; + ResourceCost cost; - Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) + /// Works like regular ResourceGuard, ready for consumption after constructor + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) , t(t_) + , cost(cost_) { t.onEnqueue(link); + waitExecute(); + } + + /// Just enqueue resource request, do not block (needed for tests to sync). Call `waitExecuted()` afterwards + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) + , t(t_) + , cost(cost_) + { + t.onEnqueue(link); + } + + /// Waits for ResourceRequest::execute() to be called for enqueued request + void waitExecute() + { lock(); t.onExecute(link); consume(cost); } + + /// Waits for ResourceRequest::failure() to be called for enqueued request + void waitFailed(const String & pattern) + { + try + { + lock(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::RESOURCE_ACCESS_DENIED); + ASSERT_TRUE(e.message().contains(pattern)); + } + } }; struct TItem @@ -264,10 +369,24 @@ struct ResourceTestManager : public ResourceTestBase , busy_period(thread_count) {} + enum DoNotInitManagerEnum { DoNotInitManager }; + + explicit ResourceTestManager(size_t thread_count, DoNotInitManagerEnum) + : busy_period(thread_count) + {} + ~ResourceTestManager() + { + wait(); + } + + void wait() { for (auto & thread : threads) - thread.join(); + { + if (thread.joinable()) + thread.join(); + } } void update(const String & xml) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp similarity index 82% rename from src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp rename to src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp index 3328196cced..37432128606 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp @@ -2,15 +2,15 @@ #include -#include +#include #include using namespace DB; -using ResourceTest = ResourceTestManager; +using ResourceTest = ResourceTestManager; using TestGuard = ResourceTest::Guard; -TEST(SchedulerDynamicResourceManager, Smoke) +TEST(SchedulerCustomResourceManager, Smoke) { ResourceTest t; @@ -31,25 +31,25 @@ TEST(SchedulerDynamicResourceManager, Smoke) )CONFIG"); - ClassifierPtr cA = t.manager->acquire("A"); - ClassifierPtr cB = t.manager->acquire("B"); + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); - gA.lock(); - gA.consume(1); - gA.unlock(); + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); - ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); - gB.unlock(); + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); - ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); - gB.consume(2); + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); } } -TEST(SchedulerDynamicResourceManager, Fairness) +TEST(SchedulerCustomResourceManager, Fairness) { // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). // Requests from A use `value = 1` and from B `value = -1` is used. diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp index 07798f78080..9989215ba7b 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -13,6 +13,12 @@ public: , log(log_) {} + const String & getTypeName() const override + { + static String type_name("fake"); + return type_name; + } + void attachChild(const SchedulerNodePtr & child) override { log += " +" + child->basename; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp new file mode 100644 index 00000000000..2bac69185d3 --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -0,0 +1,335 @@ +#include + +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +class WorkloadEntityTestStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityTestStorage() + : WorkloadEntityStorageBase(Context::getGlobalContextInstance()) + {} + + void loadEntities() override {} + + void executeQuery(const String & query) + { + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; + + auto parse = [&] (IParser & parser) + { + String error; + const char * end = query.data(); + return tryParseQuery( + parser, + end, + query.data() + query.size(), + error, + false, + "", + false, + 0, + DBMS_DEFAULT_MAX_PARSER_DEPTH, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, + true); + }; + + if (ASTPtr create_workload = parse(create_workload_p)) + { + auto & parsed = create_workload->as(); + auto workload_name = parsed.getWorkloadName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Workload, + workload_name, + create_workload, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr create_resource = parse(create_resource_p)) + { + auto & parsed = create_resource->as(); + auto resource_name = parsed.getResourceName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Resource, + resource_name, + create_resource, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr drop_workload = parse(drop_workload_p)) + { + auto & parsed = drop_workload->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Workload, + parsed.workload_name, + throw_if_not_exists); + } + else if (ASTPtr drop_resource = parse(drop_resource_p)) + { + auto & parsed = drop_resource->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Resource, + parsed.resource_name, + throw_if_not_exists); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query in WorkloadEntityTestStorage: {}", query); + } + +private: + WorkloadEntityStorageBase::OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override + { + UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); + return OperationResult::Ok; + } + + WorkloadEntityStorageBase::OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override + { + UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); + return OperationResult::Ok; + } +}; + +struct ResourceTest : ResourceTestManager +{ + WorkloadEntityTestStorage storage; + + explicit ResourceTest(size_t thread_count = 1) + : ResourceTestManager(thread_count, DoNotInitManager) + { + manager = std::make_shared(storage); + } + + void query(const String & query_str) + { + storage.executeQuery(query_str); + } + + template + void async(const String & workload, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + func2(classifier); + }); + } + + template + void async(const String & workload, const String & resource, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + ResourceLink link = classifier->get(resource); + func2(link); + }); + } +}; + +using TestGuard = ResourceTest::Guard; + +TEST(SchedulerIOResourceManager, Smoke) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); + t.query("CREATE WORKLOAD A in all"); + t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); + + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); + + for (int i = 0; i < 10; i++) + { + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); + + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); + + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); + } +} + +TEST(SchedulerIOResourceManager, Fairness) +{ + // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). + // Requests from A use `value = 1` and from B `value = -1` is used. + std::atomic unfairness = 0; + auto fairness_diff = [&] (Int64 value) + { + Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; + EXPECT_NEAR(cur_unfairness, 0, 1); + }; + + constexpr size_t threads_per_queue = 2; + int requests_per_thread = 100; + ResourceTest t(2 * threads_per_queue + 1); + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD A IN all"); + t.query("CREATE WORKLOAD B IN all"); + t.query("CREATE WORKLOAD leader IN all"); + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("A"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(1); + } + }); + } + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("B"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(-1); + } + }); + } + + ClassifierPtr c = t.manager->acquire("leader"); + ResourceLink link = c->get("res1"); + t.blockResource(link); + + t.wait(); // Wait for threads to finish before destructing locals +} + +TEST(SchedulerIOResourceManager, DropNotEmptyQueue) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(3); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // 2nd resource request is enqueued + g.waitFailed("is about to be destructed"); + }); + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); + + t.wait(); // Wait for threads to finish before destructing locals +} + +TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + static constexpr int queue_size = 100; + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(2 + queue_size); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + for (int i = 0; i < queue_size; i++) + { + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // many resource requests are enqueued + g.waitFailed("is about to be destructed"); + }); + } + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); + + t.wait(); // Wait for threads to finish before destructing locals +} diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp index 16cce309c2a..d859693eba5 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_fair.cpp @@ -8,18 +8,17 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerFairPolicy, Factory) +TEST(SchedulerFairPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", /* event_queue = */ nullptr, *cfg, ""); + EventQueue event_queue; + SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", &event_queue, *cfg, ""); EXPECT_TRUE(dynamic_cast(fair.get()) != nullptr); } -TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) +TEST(SchedulerFairPolicy, FairnessWeights) { ResourceTest t; @@ -43,7 +42,7 @@ TEST(DISABLED_SchedulerFairPolicy, FairnessWeights) t.consumed("B", 20); } -TEST(DISABLED_SchedulerFairPolicy, Activation) +TEST(SchedulerFairPolicy, Activation) { ResourceTest t; @@ -79,7 +78,7 @@ TEST(DISABLED_SchedulerFairPolicy, Activation) t.consumed("B", 10); } -TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) +TEST(SchedulerFairPolicy, FairnessMaxMin) { ResourceTest t; @@ -103,7 +102,7 @@ TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin) t.consumed("A", 20); } -TEST(DISABLED_SchedulerFairPolicy, HierarchicalFairness) +TEST(SchedulerFairPolicy, HierarchicalFairness) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp index d3d38aae048..ab248209635 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_class_priority.cpp @@ -8,18 +8,17 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerPriorityPolicy, Factory) +TEST(SchedulerPriorityPolicy, Factory) { ResourceTest t; Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration(); - SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", /* event_queue = */ nullptr, *cfg, ""); + EventQueue event_queue; + SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", &event_queue, *cfg, ""); EXPECT_TRUE(dynamic_cast(prio.get()) != nullptr); } -TEST(DISABLED_SchedulerPriorityPolicy, Priorities) +TEST(SchedulerPriorityPolicy, Priorities) { ResourceTest t; @@ -53,7 +52,7 @@ TEST(DISABLED_SchedulerPriorityPolicy, Priorities) t.consumed("C", 0); } -TEST(DISABLED_SchedulerPriorityPolicy, Activation) +TEST(SchedulerPriorityPolicy, Activation) { ResourceTest t; @@ -94,7 +93,7 @@ TEST(DISABLED_SchedulerPriorityPolicy, Activation) t.consumed("C", 0); } -TEST(DISABLED_SchedulerPriorityPolicy, SinglePriority) +TEST(SchedulerPriorityPolicy, SinglePriority) { ResourceTest t; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ddfe0cfbc6f..85d35fab0a6 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,5 +1,6 @@ #include +#include #include #include @@ -101,6 +102,11 @@ struct MyRequest : public ResourceRequest if (on_execute) on_execute(); } + + void failed(const std::exception_ptr &) override + { + FAIL(); + } }; TEST(SchedulerRoot, Smoke) @@ -108,14 +114,14 @@ TEST(SchedulerRoot, Smoke) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); r1.registerResource(); ResourceHolder r2(t); - auto * fc2 = r2.add("/", "1"); + auto * fc2 = r2.add("/", "1"); r2.add("/prio"); auto c = r2.addQueue("/prio/C", "-1"); auto d = r2.addQueue("/prio/D", "-2"); @@ -123,25 +129,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } } @@ -151,7 +157,7 @@ TEST(SchedulerRoot, Budget) ResourceTest t; ResourceHolder r1(t); - r1.add("/", "1"); + r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", ""); r1.registerResource(); @@ -176,7 +182,7 @@ TEST(SchedulerRoot, Cancel) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); @@ -189,7 +195,7 @@ TEST(SchedulerRoot, Cancel) MyRequest request(1,[&] { sync.arrive_and_wait(); // (A) - EXPECT_TRUE(fc1->requests.contains(&request)); + EXPECT_TRUE(fc1->getInflights().first == 1); sync.arrive_and_wait(); // (B) request.finish(); destruct_sync.arrive_and_wait(); // (C) @@ -214,5 +220,5 @@ TEST(SchedulerRoot, Cancel) consumer1.join(); consumer2.join(); - EXPECT_TRUE(fc1->requests.empty()); + EXPECT_TRUE(fc1->getInflights().first == 0); } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 2bc24cdb292..585bb738b27 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -10,9 +10,7 @@ using namespace DB; using ResourceTest = ResourceTestClass; -/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678 - -TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) +TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -42,7 +40,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint) t.consumed("A", 10); } -TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) +TEST(SchedulerThrottlerConstraint, Unlimited) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -59,7 +57,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited) } } -TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) +TEST(SchedulerThrottlerConstraint, Pacing) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -79,7 +77,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, Pacing) } } -TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) +TEST(SchedulerThrottlerConstraint, BucketFilling) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -113,7 +111,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling) t.consumed("A", 3); } -TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) +TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -141,7 +139,7 @@ TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits) } } -TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) +TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness) { ResourceTest t; EventQueue::TimePoint start = std::chrono::system_clock::now(); @@ -160,22 +158,22 @@ TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) t.enqueue("/fair/B", {req_cost}); } - double shareA = 0.1; - double shareB = 0.9; + double share_a = 0.1; + double share_b = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); - auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - double consumedA = 0; - double consumedB = 0; + double consumed_a = 0; + double consumed_b = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); - t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); - consumedA = arrival_curve * shareA; - consumedB = arrival_curve * shareB; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp new file mode 100644 index 00000000000..b5bcc07f71a --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -0,0 +1,748 @@ +#include +#include + +#include +#include +#include + +#include +#include +#include + +using namespace DB; + +using ResourceTest = ResourceTestClass; + +TEST(SchedulerUnifiedNode, Smoke) +{ + ResourceTest t; + + t.addCustom("/", SchedulingSettings{}); + + t.enqueue("/fifo", {10, 10}); + t.dequeue(2); + t.consumed("fifo", 20); +} + +TEST(SchedulerUnifiedNode, FairnessWeight) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(); + t.consumed("A", 60); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, FairnessActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + auto c = t.createUnifiedNode("C", all); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 10); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(4); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 10); + + t.enqueue(b, {10, 10}); + t.dequeue(1); + t.consumed("B", 10); + + t.enqueue(c, {10, 10}); + t.dequeue(1); + t.consumed("C", 10); + + t.dequeue(2); // A B or B A + t.consumed("A", 10); + t.consumed("B", 10); +} + +TEST(SchedulerUnifiedNode, FairnessMaxMin) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + + t.enqueue(a, {10, 10}); // make sure A is never empty + + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + + t.dequeue(2); + t.consumed("A", 20); +} + +TEST(SchedulerUnifiedNode, FairnessHierarchical) +{ + ResourceTest t; + + + auto all = t.createUnifiedNode("all"); + auto x = t.createUnifiedNode("X", all); + auto y = t.createUnifiedNode("Y", all); + auto a = t.createUnifiedNode("A", x); + auto b = t.createUnifiedNode("B", x); + auto c = t.createUnifiedNode("C", y); + auto d = t.createUnifiedNode("D", y); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("B", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("D", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("D", 40); + } +} + +TEST(SchedulerUnifiedNode, Priority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10}); + t.enqueue(b, {10, 10, 10}); + t.enqueue(c, {10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 0); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.dequeue(); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 0); +} + +TEST(SchedulerUnifiedNode, PriorityActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); + + t.enqueue(b, {10, 10, 10}); + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.enqueue(c, {10, 10}); + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); +} + +TEST(SchedulerUnifiedNode, List) +{ + ResourceTest t; + + std::list list; + list.push_back(t.createUnifiedNode("all")); + + for (int length = 1; length < 5; length++) + { + String name = fmt::format("L{}", length); + list.push_back(t.createUnifiedNode(name, list.back())); + + for (int i = 0; i < 3; i++) + { + t.enqueue(list.back(), {10, 10}); + t.dequeue(1); + t.consumed(name, 10); + + for (int j = 0; j < 3; j++) + { + t.enqueue(list.back(), {10, 10, 10}); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + } + + t.dequeue(1); + t.consumed(name, 10); + } + } +} + +TEST(SchedulerUnifiedNode, ThrottlerLeakyBucket) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(3)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(4)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, ThrottlerPacing) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + // Zero burst allows you to send one request of any `size` and than throttle for `size/max_speed` seconds. + // Useful if outgoing traffic should be "paced", i.e. have the least possible burstiness. + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 1.0, .max_burst = 0.0}); + + t.enqueue(all, {1, 2, 3, 1, 2, 1}); + int output[] = {1, 2, 0, 3, 0, 0, 1, 2, 0, 1, 0}; + for (int i = 0; i < std::size(output); i++) + { + t.process(start + std::chrono::seconds(i)); + t.consumed("all", output[i]); + } +} + +TEST(SchedulerUnifiedNode, ThrottlerBucketFilling) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 60); // 5 sec * 10 tokens/sec = 50 tokens + 1 extra request to go below zero + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 40); // Consume rest + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {95, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 101); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} + +TEST(SchedulerUnifiedNode, ThrottlerAndFairness) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + auto a = t.createUnifiedNode("A", all, {.weight = 10.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 90.0, .priority = Priority{}}); + + ResourceCost req_cost = 1; + ResourceCost total_cost = 2000; + for (int i = 0; i < total_cost / req_cost; i++) + { + t.enqueue(a, {req_cost}); + t.enqueue(b, {req_cost}); + } + + double share_a = 0.1; + double share_b = 0.9; + + // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); + + double consumed_a = 0; + double consumed_b = 0; + for (int seconds = 0; seconds < 100; seconds++) + { + t.process(start + std::chrono::seconds(seconds)); + double arrival_curve = 100.0 + 10.0 * seconds + req_cost; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; + } +} + +TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + // This will destroy the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} + +TEST(SchedulerUnifiedNode, ResourceGuardException) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + std::thread consumer([queue = all->getQueue()] + { + ResourceLink link{.queue = queue.get()}; + bool caught = false; + try + { + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link); + } + catch (...) + { + caught = true; + } + ASSERT_TRUE(caught); + }); + + // This will destroy the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + consumer.join(); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} + +TEST(SchedulerUnifiedNode, UpdateWeight) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{}}); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, UpdatePriority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-1}}); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); +} + +TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto x = t.createUnifiedNode("X", a, {}); + auto y = t.createUnifiedNode("Y", b, {}); + + t.enqueue(x, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("X", 20); + t.consumed("Y", 0); + + t.updateUnifiedNode(x, a, b, {}); + + t.dequeue(2); + t.consumed("X", 10); + t.consumed("Y", 10); + + t.updateUnifiedNode(y, b, a, {}); + + t.dequeue(2); + t.consumed("X", 0); + t.consumed("Y", 20); + + t.updateUnifiedNode(y, a, all, {}); + t.updateUnifiedNode(x, b, all, {}); + + t.dequeue(4); + t.consumed("X", 20); + t.consumed("Y", 20); +} + +TEST(SchedulerUnifiedNode, UpdatePriorityOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto x1 = t.createUnifiedNode("X1", a, {}); + auto y1 = t.createUnifiedNode("Y1", b, {}); + auto x2 = t.createUnifiedNode("X2", a, {}); + auto y2 = t.createUnifiedNode("Y2", b, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{2}}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{1}}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} + +TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto c = t.createUnifiedNode("C", a, {}); + auto d = t.createUnifiedNode("D", b, {}); + auto x1 = t.createUnifiedNode("X1", c, {}); + auto y1 = t.createUnifiedNode("Y1", d, {}); + auto x2 = t.createUnifiedNode("X2", c, {}); + auto y2 = t.createUnifiedNode("Y2", d, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(c, a, b, {}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(d, b, a, {}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxSpeed) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 1.0, .max_burst = 20.0}); + + t.process(start + std::chrono::seconds(12)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(22)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxBurst) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 0); // There was nothing to consume + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 30.0}); + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 40); // min(30 tokens, 5 sec * 10 tokens/sec) = 30 tokens + 1 extra request to go below zero + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 60); // Consume rest + + t.process(start + std::chrono::seconds(150)); + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 100.0, .max_burst = 200.0}); + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {195, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 201); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index cf97f7acf93..ba3532598af 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -12,6 +12,7 @@ #include #include +#include #include @@ -34,6 +35,11 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor @@ -109,12 +115,25 @@ public: dequeued_cv.notify_one(); } + // This function is executed inside scheduler thread and wakes thread that issued this `request`. + // That thread will throw an exception. + void failed(const std::exception_ptr & ptr) override + { + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + exception = ptr; + dequeued_cv.notify_one(); + } + void wait() { CurrentMetrics::Increment scheduled(metrics->scheduled_count); auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); + if (exception) + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Resource request failed: {}", getExceptionMessage(exception, /* with_stacktrace = */ false)); } void finish(ResourceCost real_cost_, ResourceLink link_) @@ -151,6 +170,7 @@ public: std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; + std::exception_ptr exception; }; /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) diff --git a/src/Common/Scheduler/ResourceManagerFactory.h b/src/Common/Scheduler/ResourceManagerFactory.h deleted file mode 100644 index 52f271e51b1..00000000000 --- a/src/Common/Scheduler/ResourceManagerFactory.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include -#include - -#include - -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SCHEDULER_NODE; -} - -class ResourceManagerFactory : private boost::noncopyable -{ -public: - static ResourceManagerFactory & instance() - { - static ResourceManagerFactory ret; - return ret; - } - - ResourceManagerPtr get(const String & name) - { - std::lock_guard lock{mutex}; - if (auto iter = methods.find(name); iter != methods.end()) - return iter->second(); - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name); - } - - template - void registerMethod(const String & name) - { - std::lock_guard lock{mutex}; - methods[name] = [] () - { - return std::make_shared(); - }; - } - -private: - std::mutex mutex; - using Method = std::function; - std::unordered_map methods; -}; - -} diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 26e8084cdfa..674c7650adf 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -1,13 +1,34 @@ #include #include +#include + +#include + namespace DB { void ResourceRequest::finish() { - if (constraint) - constraint->finishRequest(this); + // Iterate over constraints in reverse order + for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints)) + { + if (constraint) + constraint->finishRequest(this); + } +} + +bool ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +{ + for (auto & constraint : constraints) + { + if (!constraint) + { + constraint = new_constraint; + return true; + } + } + return false; } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index 7b6a5af0fe6..bb9bfbfc8fd 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -2,7 +2,9 @@ #include #include +#include #include +#include namespace DB { @@ -15,6 +17,9 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); +/// Max number of constraints for a request to pass though (depth of constraints chain) +constexpr size_t ResourceMaxConstraints = 8; + /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -39,8 +44,7 @@ constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); * * Request can also be canceled before (3) using ISchedulerQueue::cancelRequest(). * Returning false means it is too late for request to be canceled. It should be processed in a regular way. - * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen - * and step (6) MUST be omitted. + * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen. */ class ResourceRequest : public boost::intrusive::list_base_hook<> { @@ -49,9 +53,10 @@ public: /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; - /// Scheduler node to be notified on consumption finish - /// Auto-filled during request enqueue/dequeue - ISchedulerConstraint * constraint; + /// Scheduler nodes to be notified on consumption finish + /// Auto-filled during request dequeue + /// Vector is not used to avoid allocations in the scheduler thread + std::array constraints; explicit ResourceRequest(ResourceCost cost_ = 1) { @@ -62,7 +67,8 @@ public: void reset(ResourceCost cost_) { cost = cost_; - constraint = nullptr; + for (auto & constraint : constraints) + constraint = nullptr; // Note that list_base_hook should be reset independently (by intrusive list) } @@ -74,11 +80,18 @@ public: /// (e.g. setting an std::promise or creating a job in a thread pool) virtual void execute() = 0; + /// Callback to trigger an error in case if resource is unavailable. + virtual void failed(const std::exception_ptr & ptr) = 0; + /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// WARNING: this function MUST not be called if request was canceled. + /// It is okay to call finish() even for failed and canceled requests (it will be no-op) void finish(); + + /// Is called from the scheduler thread to fill `constraints` chain + /// Returns `true` iff constraint was added successfully + bool addConstraint(ISchedulerConstraint * new_constraint); }; } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 6a3c3962eb1..451f29f33f2 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -28,27 +28,27 @@ namespace ErrorCodes * Resource scheduler root node with a dedicated thread. * Immediate children correspond to different resources. */ -class SchedulerRoot : public ISchedulerNode +class SchedulerRoot final : public ISchedulerNode { private: - struct TResource + struct Resource { SchedulerNodePtr root; // Intrusive cyclic list of active resources - TResource * next = nullptr; - TResource * prev = nullptr; + Resource * next = nullptr; + Resource * prev = nullptr; - explicit TResource(const SchedulerNodePtr & root_) + explicit Resource(const SchedulerNodePtr & root_) : root(root_) { root->info.parent.ptr = this; } // Get pointer stored by ctor in info - static TResource * get(SchedulerNodeInfo & info) + static Resource * get(SchedulerNodeInfo & info) { - return reinterpret_cast(info.parent.ptr); + return reinterpret_cast(info.parent.ptr); } }; @@ -60,6 +60,8 @@ public: ~SchedulerRoot() override { stop(); + while (!children.empty()) + removeChild(children.begin()->first); } /// Runs separate scheduler thread @@ -95,6 +97,12 @@ public: } } + const String & getTypeName() const override + { + static String type_name("scheduler"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) @@ -179,16 +187,11 @@ public: void activateChild(ISchedulerNode * child) override { - activate(TResource::get(child->info)); - } - - void setParent(ISchedulerNode *) override - { - abort(); // scheduler must be the root and this function should not be called + activate(Resource::get(child->info)); } private: - void activate(TResource * value) + void activate(Resource * value) { assert(value->next == nullptr && value->prev == nullptr); if (current == nullptr) // No active children @@ -206,7 +209,7 @@ private: } } - void deactivate(TResource * value) + void deactivate(Resource * value) { if (value->next == nullptr) return; // Already deactivated @@ -251,8 +254,8 @@ private: request->execute(); } - TResource * current = nullptr; // round-robin pointer - std::unordered_map children; // resources by pointer + Resource * current = nullptr; // round-robin pointer + std::unordered_map children; // resources by pointer std::atomic stop_flag = false; EventQueue events; ThreadFromGlobalPool scheduler; diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp new file mode 100644 index 00000000000..60319cdd54c --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -0,0 +1,130 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) +{ + struct { + std::optional new_weight; + std::optional new_priority; + std::optional new_max_speed; + std::optional new_max_burst; + std::optional new_max_requests; + std::optional new_max_cost; + + static Float64 getNotNegativeFloat64(const String & name, const Field & field) + { + { + UInt64 val; + if (field.tryGet(val)) + return static_cast(val); // We dont mind slight loss of precision + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return static_cast(val); // We dont mind slight loss of precision + } + } + + return field.safeGet(); + } + + static Int64 getNotNegativeInt64(const String & name, const Field & field) + { + { + UInt64 val; + if (field.tryGet(val)) + { + // Saturate on overflow + if (val > static_cast(std::numeric_limits::max())) + val = std::numeric_limits::max(); + return static_cast(val); + } + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return val; + } + } + + return field.safeGet(); + } + + void read(const String & name, const Field & value) + { + if (name == "weight") + new_weight = getNotNegativeFloat64(name, value); + else if (name == "priority") + new_priority = Priority{value.safeGet()}; + else if (name == "max_speed") + new_max_speed = getNotNegativeFloat64(name, value); + else if (name == "max_burst") + new_max_burst = getNotNegativeFloat64(name, value); + else if (name == "max_requests") + new_max_requests = getNotNegativeInt64(name, value); + else if (name == "max_cost") + new_max_cost = getNotNegativeInt64(name, value); + } + } regular, specific; + + // Read changed setting values + for (const auto & [name, value, resource] : changes) + { + if (resource.empty()) + regular.read(name, value); + else if (resource == resource_name) + specific.read(name, value); + } + + auto get_value = [] (const std::optional & specific_new, const std::optional & regular_new, T & old) + { + if (specific_new) + return *specific_new; + if (regular_new) + return *regular_new; + return old; + }; + + // Validate that we could use values read in a scheduler node + { + SchedulerNodeInfo validating_node( + get_value(specific.new_weight, regular.new_weight, weight), + get_value(specific.new_priority, regular.new_priority, priority)); + } + + // Commit new values. + // Previous values are left intentionally for ALTER query to be able to skip not mentioned setting values + weight = get_value(specific.new_weight, regular.new_weight, weight); + priority = get_value(specific.new_priority, regular.new_priority, priority); + if (specific.new_max_speed || regular.new_max_speed) + { + max_speed = get_value(specific.new_max_speed, regular.new_max_speed, max_speed); + // We always set max_burst if max_speed is changed. + // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed + max_burst = default_burst_seconds * max_speed; + } + max_burst = get_value(specific.new_max_burst, regular.new_max_burst, max_burst); + max_requests = get_value(specific.new_max_requests, regular.new_max_requests, max_requests); + max_cost = get_value(specific.new_max_cost, regular.new_max_cost, max_cost); +} + +} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h new file mode 100644 index 00000000000..6db3ef0dce9 --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -0,0 +1,39 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +struct SchedulingSettings +{ + /// Priority and weight among siblings + Float64 weight = 1.0; + Priority priority; + + /// Throttling constraints. + /// Up to 2 independent throttlers: one for average speed and one for peek speed. + static constexpr Float64 default_burst_seconds = 1.0; + Float64 max_speed = 0; // Zero means unlimited + Float64 max_burst = 0; // default is `default_burst_seconds * max_speed` + + /// Limits total number of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_requests = std::numeric_limits::max(); + Int64 max_requests = default_max_requests; + + /// Limits total cost of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_cost = std::numeric_limits::max(); + Int64 max_cost = default_max_cost; + + bool hasThrottler() const { return max_speed != 0; } + bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } + + void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name = {}); +}; + +} diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h new file mode 100644 index 00000000000..adb3a808eea --- /dev/null +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include + +#include + +#include + + +namespace DB +{ + +class IAST; +struct Settings; + +enum class WorkloadEntityType : uint8_t +{ + Workload, + Resource, + + MAX +}; + +/// Interface for a storage of workload entities (WORKLOAD and RESOURCE). +class IWorkloadEntityStorage +{ +public: + virtual ~IWorkloadEntityStorage() = default; + + /// Whether this storage can replicate entities to another node. + virtual bool isReplicated() const { return false; } + virtual String getReplicationID() const { return ""; } + + /// Loads all entities. Can be called once - if entities are already loaded the function does nothing. + virtual void loadEntities() = 0; + + /// Get entity by name. If no entity stored with entity_name throws exception. + virtual ASTPtr get(const String & entity_name) const = 0; + + /// Get entity by name. If no entity stored with entity_name return nullptr. + virtual ASTPtr tryGet(const String & entity_name) const = 0; + + /// Check if entity with entity_name is stored. + virtual bool has(const String & entity_name) const = 0; + + /// Get all entity names. + virtual std::vector getAllEntityNames() const = 0; + + /// Get all entity names of specified type. + virtual std::vector getAllEntityNames(WorkloadEntityType entity_type) const = 0; + + /// Get all entities. + virtual std::vector> getAllEntities() const = 0; + + /// Check whether any entity have been stored. + virtual bool empty() const = 0; + + /// Stops watching. + virtual void stopWatching() {} + + /// Stores an entity. + virtual bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + /// Removes an entity. + virtual bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; + + struct Event + { + WorkloadEntityType type; + String name; + ASTPtr entity; /// new or changed entity, null if removed + }; + using OnChangedHandler = std::function &)>; + + /// Gets all current entries, pass them through `handler` and subscribes for all later changes. + virtual scope_guard getAllEntitiesAndSubscribe(const OnChangedHandler & handler) = 0; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp new file mode 100644 index 00000000000..1bff672c150 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -0,0 +1,287 @@ +#include + +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include +#include + +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + +namespace Setting +{ + extern const SettingsUInt64 max_parser_backtracks; + extern const SettingsUInt64 max_parser_depth; + extern const SettingsBool fsync_metadata; +} + +namespace ErrorCodes +{ + extern const int DIRECTORY_DOESNT_EXIST; + extern const int BAD_ARGUMENTS; +} + + +namespace +{ + constexpr std::string_view workload_prefix = "workload_"; + constexpr std::string_view resource_prefix = "resource_"; + constexpr std::string_view sql_suffix = ".sql"; + + /// Converts a path to an absolute path and append it with a separator. + String makeDirectoryPathCanonical(const String & directory_path) + { + auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path); + if (canonical_directory_path.has_filename()) + canonical_directory_path += std::filesystem::path::preferred_separator; + return canonical_directory_path; + } +} + +WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) + : WorkloadEntityStorageBase(global_context_) + , dir_path{makeDirectoryPathCanonical(dir_path_)} +{ + log = getLogger("WorkloadEntityDiskStorage"); +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name) +{ + return tryLoadEntity(entity_type, entity_name, getFilePath(entity_type, entity_name), /* check_file_exists= */ true); +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & path, bool check_file_exists) +{ + LOG_DEBUG(log, "Loading workload entity {} from file {}", backQuote(entity_name), path); + + try + { + if (check_file_exists && !fs::exists(path)) + return nullptr; + + /// There is .sql file with workload entity creation statement. + ReadBufferFromFile in(path); + + String entity_create_query; + readStringUntilEOF(entity_create_query, in); + + auto parse = [&] (auto parser) + { + return parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); + }; + + switch (entity_type) + { + case WorkloadEntityType::Workload: return parse(ParserCreateWorkloadQuery()); + case WorkloadEntityType::Resource: return parse(ParserCreateResourceQuery()); + case WorkloadEntityType::MAX: return nullptr; + } + } + catch (...) + { + tryLogCurrentException(log, fmt::format("while loading workload entity {} from path {}", backQuote(entity_name), path)); + return nullptr; /// Failed to load this entity, will ignore it + } +} + + +void WorkloadEntityDiskStorage::loadEntities() +{ + if (!entities_loaded) + loadEntitiesImpl(); +} + + +void WorkloadEntityDiskStorage::loadEntitiesImpl() +{ + LOG_INFO(log, "Loading workload entities from {}", dir_path); + + if (!std::filesystem::exists(dir_path)) + { + LOG_DEBUG(log, "The directory for workload entities ({}) does not exist: nothing to load", dir_path); + return; + } + + std::vector> entities_name_and_queries; + + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + if (it->isDirectory()) + continue; + + const String & file_name = it.name(); + + if (file_name.starts_with(workload_prefix) && file_name.ends_with(sql_suffix)) + { + String name = unescapeForFileName(file_name.substr( + workload_prefix.size(), + file_name.size() - workload_prefix.size() - sql_suffix.size())); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Workload, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + + if (file_name.starts_with(resource_prefix) && file_name.ends_with(sql_suffix)) + { + String name = unescapeForFileName(file_name.substr( + resource_prefix.size(), + file_name.size() - resource_prefix.size() - sql_suffix.size())); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Resource, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + } + + setAllEntities(entities_name_and_queries); + entities_loaded = true; + + LOG_DEBUG(log, "Workload entities loaded"); +} + + +void WorkloadEntityDiskStorage::createDirectory() +{ + std::error_code create_dir_error_code; + fs::create_directories(dir_path, create_dir_error_code); + if (!fs::exists(dir_path) || !fs::is_directory(dir_path) || create_dir_error_code) + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "Couldn't create directory {} reason: '{}'", + dir_path, create_dir_error_code.message()); +} + + +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + createDirectory(); + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Storing workload entity {} to file {}", backQuote(entity_name), file_path); + + if (fs::exists(file_path)) + { + if (throw_if_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return OperationResult::Failed; + } + + + String temp_file_path = file_path + ".tmp"; + + try + { + WriteBufferFromFile out(temp_file_path); + formatAST(*create_entity_query, out, false); + writeChar('\n', out); + out.next(); + if (settings[Setting::fsync_metadata]) + out.sync(); + out.close(); + + if (replace_if_exists) + fs::rename(temp_file_path, file_path); + else + renameNoReplace(temp_file_path, file_path); + } + catch (...) + { + fs::remove(temp_file_path); + throw; + } + + LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); + return OperationResult::Ok; +} + + +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Removing workload entity {} stored in file {}", backQuote(entity_name), file_path); + + bool existed = fs::remove(file_path); + + if (!existed) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); + else + return OperationResult::Failed; + } + + LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); + return OperationResult::Ok; +} + + +String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, const String & entity_name) const +{ + String file_path; + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + file_path = dir_path + "workload_" + escapeForFileName(entity_name) + ".sql"; + break; + } + case WorkloadEntityType::Resource: + { + file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; + break; + } + case WorkloadEntityType::MAX: break; + } + return file_path; +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h new file mode 100644 index 00000000000..cb3fb600182 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Loads workload entities from a specified folder. +class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); + void loadEntities() override; + +private: + OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void createDirectory(); + void loadEntitiesImpl(); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & file_path, bool check_file_exists); + String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; + + String dir_path; + std::atomic entities_loaded = false; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp new file mode 100644 index 00000000000..4b60a7ec57e --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -0,0 +1,273 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace Setting +{ +extern const SettingsUInt64 max_parser_backtracks; +extern const SettingsUInt64 max_parser_depth; +} + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; +} + +WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( + const ContextPtr & global_context_, const String & zookeeper_path_) + : WorkloadEntityStorageBase(global_context_) + , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} + , zookeeper_path{zookeeper_path_} + , watch{std::make_shared()} +{ + log = getLogger("WorkloadEntityKeeperStorage"); + if (zookeeper_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); + + if (zookeeper_path.back() == '/') + zookeeper_path.pop_back(); + + /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. + if (zookeeper_path.front() != '/') + zookeeper_path = "/" + zookeeper_path; +} + +WorkloadEntityKeeperStorage::~WorkloadEntityKeeperStorage() +{ + SCOPE_EXIT_SAFE(stopWatchingThread()); +} + +void WorkloadEntityKeeperStorage::startWatchingThread() +{ + if (!watching_flag.exchange(true)) + watching_thread = ThreadFromGlobalPool(&WorkloadEntityKeeperStorage::processWatchQueue, this); +} + +void WorkloadEntityKeeperStorage::stopWatchingThread() +{ + if (watching_flag.exchange(false)) + { + watch->cv.notify_one(); + if (watching_thread.joinable()) + watching_thread.join(); + } +} + +zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() +{ + auto [zookeeper, session_status] = zookeeper_getter.getZooKeeper(); + + if (session_status == zkutil::ZooKeeperCachingGetter::SessionStatus::New) + { + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + + createRootNodes(zookeeper); + + auto lock = getLock(); + refreshEntities(zookeeper); + } + + return zookeeper; +} + +void WorkloadEntityKeeperStorage::loadEntities() +{ + /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. + /// However the watching thread must be started anyway in case the connection will be established later. + try + { + auto lock = getLock(); + refreshEntities(getZooKeeper()); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load workload entities"); + } + startWatchingThread(); +} + + +void WorkloadEntityKeeperStorage::processWatchQueue() +{ + LOG_DEBUG(log, "Started watching thread"); + setThreadName("WrkldEntWatch"); + + UInt64 handled = 0; + while (watching_flag) + { + try + { + /// Re-initialize ZooKeeper session if expired + getZooKeeper(); + + { + std::unique_lock lock{watch->mutex}; + if (!watch->cv.wait_for(lock, std::chrono::seconds(10), [&] { return !watching_flag || handled != watch->triggered; })) + continue; + handled = watch->triggered; + } + + auto lock = getLock(); + refreshEntities(getZooKeeper()); + } + catch (...) + { + tryLogCurrentException(log, "Will try to restart watching thread after error"); + zookeeper_getter.resetCache(); + sleepForSeconds(5); + } + } + + LOG_DEBUG(log, "Stopped watching thread"); +} + + +void WorkloadEntityKeeperStorage::stopWatching() +{ + stopWatchingThread(); +} + +void WorkloadEntityKeeperStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper) +{ + zookeeper->createAncestors(zookeeper_path); + // If node does not exist we consider it to be equal to empty node: no workload entities + zookeeper->createIfNotExists(zookeeper_path, ""); +} + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool /*throw_if_exists*/, + bool /*replace_if_exists*/, + const Settings &) +{ + LOG_DEBUG(log, "Storing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, create_entity_query}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} stored", backQuote(entity_name)); + + return OperationResult::Ok; +} + + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool /*throw_if_not_exists*/) +{ + LOG_DEBUG(log, "Removing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, {}}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} removed", backQuote(entity_name)); + + return OperationResult::Ok; +} + +std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) +{ + const auto data_watcher = [my_watch = watch](const Coordination::WatchResponse & response) + { + if (response.type == Coordination::Event::CHANGED) + { + std::unique_lock lock{my_watch->mutex}; + my_watch->triggered++; + my_watch->cv.notify_one(); + } + }; + + Coordination::Stat stat; + String data; + bool exists = zookeeper->tryGetWatch(zookeeper_path, data, &stat, data_watcher); + if (!exists) + { + createRootNodes(zookeeper); + data = zookeeper->getWatch(zookeeper_path, &stat, data_watcher); + } + return {data, stat.version}; +} + +void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) +{ + auto [data, version] = getDataAndSetWatch(zookeeper); + if (version == current_version) + return; + + LOG_DEBUG(log, "Refreshing workload entities from keeper"); + ASTs queries; + ParserCreateWorkloadEntity parser; + const char * begin = data.data(); /// begin of current query + const char * pos = begin; /// parser moves pos from begin to the end of current query + const char * end = begin + data.size(); + while (pos < end) + { + queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS)); + while (isWhitespaceASCII(*pos) || *pos == ';') + ++pos; + } + + /// Read and parse all SQL entities from data we just read from ZooKeeper + std::vector> new_entities; + for (const auto & query : queries) + { + LOG_TRACE(log, "Read keeper entity definition: {}", serializeAST(*query)); + if (auto * create_workload_query = query->as()) + new_entities.emplace_back(create_workload_query->getWorkloadName(), query); + else if (auto * create_resource_query = query->as()) + new_entities.emplace_back(create_resource_query->getResourceName(), query); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity query in keeper storage: {}", query->getID()); + } + + setAllEntities(new_entities); + current_version = version; + + LOG_DEBUG(log, "Workload entities refreshing is done"); +} + +} + diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h new file mode 100644 index 00000000000..25dcd6d8c9a --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -0,0 +1,71 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +/// Loads RESOURCE and WORKLOAD sql objects from Keeper. +class WorkloadEntityKeeperStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityKeeperStorage(const ContextPtr & global_context_, const String & zookeeper_path_); + ~WorkloadEntityKeeperStorage() override; + + bool isReplicated() const override { return true; } + String getReplicationID() const override { return zookeeper_path; } + + void loadEntities() override; + void stopWatching() override; + +private: + OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void processWatchQueue(); + + zkutil::ZooKeeperPtr getZooKeeper(); + + void startWatchingThread(); + void stopWatchingThread(); + + void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); + std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); + void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); + + zkutil::ZooKeeperCachingGetter zookeeper_getter; + String zookeeper_path; + Int32 current_version = 0; + + ThreadFromGlobalPool watching_thread; + std::atomic watching_flag = false; + + struct WatchEvent + { + std::mutex mutex; + std::condition_variable cv; + UInt64 triggered = 0; + }; + std::shared_ptr watch; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp new file mode 100644 index 00000000000..1b7a559698c --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -0,0 +1,773 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; +} + +namespace +{ + +/// Removes details from a CREATE query to be used as workload entity definition +ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) +{ + auto ptr = create_query.clone(); + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } + return ptr; +} + +/// Returns a type of a workload entity `ptr` +WorkloadEntityType getEntityType(const ASTPtr & ptr) +{ + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Workload; + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Resource; + chassert(false); + return WorkloadEntityType::MAX; +} + +bool entityEquals(const ASTPtr & lhs, const ASTPtr & rhs) +{ + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + { + return std::forward_as_tuple(a->getWorkloadName(), a->getWorkloadParent(), a->changes) + == std::forward_as_tuple(b->getWorkloadName(), b->getWorkloadParent(), b->changes); + } + } + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + return std::forward_as_tuple(a->getResourceName(), a->operations) + == std::forward_as_tuple(b->getResourceName(), b->operations); + } + return false; +} + +/// Workload entities could reference each other. +/// This enum defines all possible reference types +enum class ReferenceType +{ + Parent, // Source workload references target workload as a parent + ForResource // Source workload references target resource in its `SETTINGS x = y FOR resource` clause +}; + +/// Runs a `func` callback for every reference from `source` to `target`. +/// This function is the source of truth defining what `target` references are stored in a workload `source_entity` +void forEachReference( + const ASTPtr & source_entity, + std::function func) +{ + if (auto * res = typeid_cast(source_entity.get())) + { + // Parent reference + String parent = res->getWorkloadParent(); + if (!parent.empty()) + func(parent, res->getWorkloadName(), ReferenceType::Parent); + + // References to RESOURCEs mentioned in SETTINGS clause after FOR keyword + std::unordered_set resources; + for (const auto & [name, value, resource] : res->changes) + { + if (!resource.empty()) + resources.insert(resource); + } + for (const String & resource : resources) + func(resource, res->getWorkloadName(), ReferenceType::ForResource); + } + if (auto * res = typeid_cast(source_entity.get())) + { + // RESOURCE has no references to be validated, we allow mentioned disks to be created later + } +} + +/// Helper for recursive DFS +void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + // Recurse into parent (if any) + String parent = typeid_cast(ast.get())->getWorkloadParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + if (parent_iter == workloads.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload metadata inconsistency: Workload '{}' parent '{}' does not exist. This must be fixed manually.", name, parent); + topologicallySortedWorkloadsImpl(parent, parent_iter->second, workloads, visited, sorted_workloads); + } + + sorted_workloads.emplace_back(name, ast); +} + +/// Returns pairs {worload_name, create_workload_ast} in order that respect child-parent relation (parent first, then children) +std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) +{ + std::vector> sorted_workloads; + std::unordered_set visited; + for (const auto & [name, ast] : workloads) + topologicallySortedWorkloadsImpl(name, ast, workloads, visited, sorted_workloads); + return sorted_workloads; +} + +/// Helper for recursive DFS +void topologicallySortedDependenciesImpl( + const String & name, + const std::unordered_map> & dependencies, + std::unordered_set & visited, + std::vector & result) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + if (auto it = dependencies.find(name); it != dependencies.end()) + { + for (const String & dep : it->second) + topologicallySortedDependenciesImpl(dep, dependencies, visited, result); + } + + result.emplace_back(name); +} + +/// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) +std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) +{ + std::unordered_set visited; // Set to track visited nodes + std::vector result; // Result to store nodes in topologically sorted order + + // Perform DFS for each node in the graph + for (const auto & [name, _] : dependencies) + topologicallySortedDependenciesImpl(name, dependencies, visited, result); + + return result; +} + +/// Represents a change of a workload entity (WORKLOAD or RESOURCE) +struct EntityChange +{ + String name; /// Name of entity + ASTPtr before; /// Entity before change (CREATE if not set) + ASTPtr after; /// Entity after change (DROP if not set) + + std::vector toEvents() const + { + if (!after) + return {{getEntityType(before), name, {}}}; + else if (!before) + return {{getEntityType(after), name, after}}; + else + { + auto type_before = getEntityType(before); + auto type_after = getEntityType(after); + // If type changed, we have to remove an old entity and add a new one + if (type_before != type_after) + return {{type_before, name, {}}, {type_after, name, after}}; + else + return {{type_after, name, after}}; + } + } +}; + +/// Returns `changes` ordered for execution. +/// Every intemediate state during execution will be consistent (i.e. all references will be valid) +/// NOTE: It does not validate changes, any problem will be detected during execution. +/// NOTE: There will be no error if valid order does not exist. +std::vector topologicallySortedChanges(const std::vector & changes) +{ + // Construct map from entity name into entity change + std::unordered_map change_by_name; + for (const auto & change : changes) + change_by_name[change.name] = &change; + + // Construct references maps (before changes and after changes) + std::unordered_map> old_sources; // Key is target. Value is set of names of source entities. + std::unordered_map> new_targets; // Key is source. Value is set of names of target entities. + for (const auto & change : changes) + { + if (change.before) + { + forEachReference(change.before, + [&] (const String & target, const String & source, ReferenceType) + { + old_sources[target].insert(source); + }); + } + if (change.after) + { + forEachReference(change.after, + [&] (const String & target, const String & source, ReferenceType) + { + new_targets[source].insert(target); + }); + } + } + + // There are consistency rules that regulate order in which changes must be applied (see below). + // Construct DAG of dependencies between changes. + std::unordered_map> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. + for (const auto & change : changes) + { + dependencies.emplace(change.name, std::unordered_set{}); // Make sure we create nodes that have no dependencies + for (const auto & event : change.toEvents()) + { + if (!event.entity) // DROP + { + // Rule 1: Entity can only be removed after all existing references to it are removed as well. + for (const String & source : old_sources[event.name]) + { + if (change_by_name.contains(source)) + dependencies[event.name].insert(source); + } + } + else // CREATE || CREATE OR REPLACE + { + // Rule 2: Entity can only be created after all entities it references are created as well. + for (const String & target : new_targets[event.name]) + { + if (auto it = change_by_name.find(target); it != change_by_name.end()) + { + const EntityChange & target_change = *it->second; + // If target is creating, it should be created first. + // (But if target is updating, there is no dependency). + if (!target_change.before) + dependencies[event.name].insert(target); + } + } + } + } + } + + // Topological sort of changes to respect consistency rules + std::vector result; + for (const String & name : topologicallySortedDependencies(dependencies)) + result.push_back(*change_by_name[name]); + + return result; +} + +} + +WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) + : handlers(std::make_shared()) + , global_context(std::move(global_context_)) + , log{getLogger("WorkloadEntityStorage")} // could be overridden in derived class +{} + +ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const +{ + if (auto result = tryGet(entity_name)) + return result; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "The workload entity name '{}' is not saved", + entity_name); +} + +ASTPtr WorkloadEntityStorageBase::tryGet(const String & entity_name) const +{ + std::lock_guard lock(mutex); + + auto it = entities.find(entity_name); + if (it == entities.end()) + return nullptr; + + return it->second; +} + +bool WorkloadEntityStorageBase::has(const String & entity_name) const +{ + return tryGet(entity_name) != nullptr; +} + +std::vector WorkloadEntityStorageBase::getAllEntityNames() const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + entity_names.reserve(entities.size()); + + for (const auto & [name, _] : entities) + entity_names.emplace_back(name); + + return entity_names; +} + +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + for (const auto & [name, entity] : entities) + { + if (getEntityType(entity) == entity_type) + entity_names.emplace_back(name); + } + + return entity_names; +} + +bool WorkloadEntityStorageBase::empty() const +{ + std::lock_guard lock(mutex); + return entities.empty(); +} + +bool WorkloadEntityStorageBase::storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + if (entity_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); + + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); + auto * workload = typeid_cast(create_entity_query.get()); + auto * resource = typeid_cast(create_entity_query.get()); + + while (true) + { + std::unique_lock lock{mutex}; + + ASTPtr old_entity; // entity to be REPLACED + if (auto it = entities.find(entity_name); it != entities.end()) + { + if (throw_if_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + else + old_entity = it->second; + } + + // Validate CREATE OR REPLACE + if (old_entity) + { + auto * old_workload = typeid_cast(old_entity.get()); + auto * old_resource = typeid_cast(old_entity.get()); + if (workload && !old_workload) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); + if (resource && !old_resource) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); + if (workload && !old_workload->hasParent() && workload->hasParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); + } + + // Validate workload + if (workload) + { + if (!workload->hasParent()) + { + if (!root_name.empty() && root_name != workload->getWorkloadName()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); + } + + SchedulingSettings validator; + validator.updateFromChanges(workload->changes); + } + + forEachReference(create_entity_query, + [this, workload] (const String & target, const String & source, ReferenceType type) + { + if (auto it = entities.find(target); it == entities.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + + switch (type) + { + case ReferenceType::Parent: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); + break; + } + case ReferenceType::ForResource: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); + + // Validate that we could parse the settings for specific resource + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + break; + } + } + + // Detect reference cycles. + // The only way to create a cycle is to add an edge that will be a part of a new cycle. + // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. + if (isIndirectlyReferenced(source, target)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); + }); + + auto result = storeEntityImpl( + current_context, + entity_type, + entity_name, + create_entity_query, + throw_if_exists, + replace_if_exists, + settings); + + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, create_entity_query}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; + } +} + +bool WorkloadEntityStorageBase::removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + while (true) + { + std::unique_lock lock(mutex); + auto it = entities.find(entity_name); + if (it == entities.end()) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + if (auto reference_it = references.find(entity_name); reference_it != references.end()) + { + String names; + for (const String & name : reference_it->second) + names += " " + name; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); + } + + auto result = removeEntityImpl( + current_context, + entity_type, + entity_name, + throw_if_not_exists); + + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, {}}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; + } +} + +scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) +{ + scope_guard result; + + std::vector current_state; + { + std::lock_guard lock{mutex}; + current_state = orderEntities(entities); + + std::lock_guard lock2{handlers->mutex}; + handlers->list.push_back(handler); + auto handler_it = std::prev(handlers->list.end()); + result = [my_handlers = handlers, handler_it] + { + std::lock_guard lock3{my_handlers->mutex}; + my_handlers->list.erase(handler_it); + }; + } + + // When you subscribe you get all the entities back to your handler immediately if already loaded, or later when loaded + handler(current_state); + + return result; +} + +void WorkloadEntityStorageBase::unlockAndNotify( + std::unique_lock & lock, + std::vector tx) +{ + if (tx.empty()) + return; + + std::vector current_handlers; + { + std::lock_guard handlers_lock{handlers->mutex}; + boost::range::copy(handlers->list, std::back_inserter(current_handlers)); + } + + lock.unlock(); + + for (const auto & handler : current_handlers) + { + try + { + handler(tx); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +std::unique_lock WorkloadEntityStorageBase::getLock() const +{ + return std::unique_lock{mutex}; +} + +void WorkloadEntityStorageBase::setAllEntities(const std::vector> & raw_new_entities) +{ + std::unordered_map new_entities; + for (const auto & [entity_name, create_query] : raw_new_entities) + new_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); + + std::unique_lock lock(mutex); + + // Fill vector of `changes` based on difference between current `entities` and `new_entities` + std::vector changes; + for (const auto & [entity_name, entity] : entities) + { + if (auto it = new_entities.find(entity_name); it != new_entities.end()) + { + if (!entityEquals(entity, it->second)) + { + changes.emplace_back(entity_name, entity, it->second); // Update entities that are present in both `new_entities` and `entities` + LOG_TRACE(log, "Entity {} was updated", entity_name); + } + else + LOG_TRACE(log, "Entity {} is the same", entity_name); + } + else + { + changes.emplace_back(entity_name, entity, ASTPtr{}); // Remove entities that are not present in `new_entities` + LOG_TRACE(log, "Entity {} was dropped", entity_name); + } + } + for (const auto & [entity_name, entity] : new_entities) + { + if (!entities.contains(entity_name)) + { + changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` + LOG_TRACE(log, "Entity {} was created", entity_name); + } + } + + // Sort `changes` to respect consistency of references and apply them one by one. + std::vector tx; + for (const auto & change : topologicallySortedChanges(changes)) + { + for (const auto & event : change.toEvents()) + { + // TODO(serxa): do validation and throw LOGICAL_ERROR if failed + applyEvent(lock, event); + tx.push_back(event); + } + } + + // Notify subscribers + unlockAndNotify(lock, tx); +} + +void WorkloadEntityStorageBase::applyEvent( + std::unique_lock &, + const Event & event) +{ + if (event.entity) // CREATE || CREATE OR REPLACE + { + LOG_DEBUG(log, "Create or replace workload entity: {}", serializeAST(*event.entity)); + + auto * workload = typeid_cast(event.entity.get()); + + // Validate workload + if (workload && !workload->hasParent()) + root_name = workload->getWorkloadName(); + + // Remove references of a replaced entity (only for CREATE OR REPLACE) + if (auto it = entities.find(event.name); it != entities.end()) + removeReferences(it->second); + + // Insert references of created entity + insertReferences(event.entity); + + // Store in memory + entities[event.name] = event.entity; + } + else // DROP + { + auto it = entities.find(event.name); + chassert(it != entities.end()); + + LOG_DEBUG(log, "Drop workload entity: {}", event.name); + + if (event.name == root_name) + root_name.clear(); + + // Clean up references + removeReferences(it->second); + + // Remove from memory + entities.erase(it); + } +} + +std::vector> WorkloadEntityStorageBase::getAllEntities() const +{ + std::lock_guard lock{mutex}; + std::vector> all_entities; + all_entities.reserve(entities.size()); + std::copy(entities.begin(), entities.end(), std::back_inserter(all_entities)); + return all_entities; +} + +bool WorkloadEntityStorageBase::isIndirectlyReferenced(const String & target, const String & source) +{ + std::queue bfs; + std::unordered_set visited; + visited.insert(target); + bfs.push(target); + while (!bfs.empty()) + { + String current = bfs.front(); + bfs.pop(); + if (current == source) + return true; + if (auto it = references.find(current); it != references.end()) + { + for (const String & node : it->second) + { + if (visited.contains(node)) + continue; + visited.insert(node); + bfs.push(node); + } + } + } + return false; +} + +void WorkloadEntityStorageBase::insertReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].insert(source); + }); +} + +void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].erase(source); + if (references[target].empty()) + references.erase(target); + }); +} + +std::vector WorkloadEntityStorageBase::orderEntities( + const std::unordered_map & all_entities, + std::optional change) +{ + std::vector result; + + std::unordered_map workloads; + for (const auto & [entity_name, ast] : all_entities) + { + if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this workload if it is removed or updated + workloads.emplace(entity_name, ast); + } + else if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this resource if it is removed or updated + // Resources should go first because workloads could reference them + result.emplace_back(WorkloadEntityType::Resource, entity_name, ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + } + + // Introduce new entity described by `change` + if (change && change->entity) + { + if (change->type == WorkloadEntityType::Workload) + workloads.emplace(change->name, change->entity); + else if (change->type == WorkloadEntityType::Resource) + result.emplace_back(WorkloadEntityType::Resource, change->name, change->entity); + } + + // Workloads should go in an order such that children are enlisted only after its parent + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + result.emplace_back(WorkloadEntityType::Workload, entity_name, ast); + + return result; +} + +String WorkloadEntityStorageBase::serializeAllEntities(std::optional change) +{ + std::unique_lock lock; + auto ordered_entities = orderEntities(entities, change); + WriteBufferFromOwnString buf; + for (const auto & event : ordered_entities) + { + formatAST(*event.entity, buf, false, true); + buf.write(";\n", 2); + } + return buf.str(); +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h new file mode 100644 index 00000000000..d57bf8201b3 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -0,0 +1,126 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + +#include + +namespace DB +{ + +class WorkloadEntityStorageBase : public IWorkloadEntityStorage +{ +public: + explicit WorkloadEntityStorageBase(ContextPtr global_context_); + ASTPtr get(const String & entity_name) const override; + + ASTPtr tryGet(const String & entity_name) const override; + + bool has(const String & entity_name) const override; + + std::vector getAllEntityNames() const override; + std::vector getAllEntityNames(WorkloadEntityType entity_type) const override; + + std::vector> getAllEntities() const override; + + bool empty() const override; + + bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + scope_guard getAllEntitiesAndSubscribe( + const OnChangedHandler & handler) override; + +protected: + enum class OperationResult + { + Ok, + Failed, + Retry + }; + + virtual OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + virtual OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; + + std::unique_lock getLock() const; + + /// Replace current `entities` with `new_entities` and notifies subscribers. + /// Note that subscribers will be notified with a sequence of events. + /// It is guaranteed that all itermediate states (between every pair of consecutive events) + /// will be consistent (all references between entities will be valid) + void setAllEntities(const std::vector> & new_entities); + + /// Serialize `entities` stored in memory plus one optional `change` into multiline string + String serializeAllEntities(std::optional change = {}); + +private: + /// Change state in memory + void applyEvent(std::unique_lock & lock, const Event & event); + + /// Notify subscribers about changes describe by vector of events `tx` + void unlockAndNotify(std::unique_lock & lock, std::vector tx); + + /// Return true iff `references` has a path from `source` to `target` + bool isIndirectlyReferenced(const String & target, const String & source); + + /// Adds references that are described by `entity` to `references` + void insertReferences(const ASTPtr & entity); + + /// Removes references that are described by `entity` from `references` + void removeReferences(const ASTPtr & entity); + + /// Returns an ordered vector of `entities` + std::vector orderEntities( + const std::unordered_map & all_entities, + std::optional change = {}); + + struct Handlers + { + std::mutex mutex; + std::list list; + }; + /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. + std::shared_ptr handlers; + + mutable std::recursive_mutex mutex; + std::unordered_map entities; /// Maps entity name into CREATE entity query + + // Validation + std::unordered_map> references; /// Keep track of references between entities. Key is target. Value is set of sources + String root_name; /// current root workload name + +protected: + ContextPtr global_context; + LoggerPtr log; +}; + +} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp new file mode 100644 index 00000000000..5dc1265e31d --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int INVALID_CONFIG_PARAMETER; +} + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context) +{ + const String zookeeper_path_key = "workload_zookeeper_path"; + const String disk_path_key = "workload_path"; + + const auto & config = global_context->getConfigRef(); + if (config.has(zookeeper_path_key)) + { + if (config.has(disk_path_key)) + { + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "'{}' and '{}' must not be both specified in the config", + zookeeper_path_key, + disk_path_key); + } + return std::make_unique(global_context, config.getString(zookeeper_path_key)); + } + + String default_path = fs::path{global_context->getPath()} / "workload" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); +} + +} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h new file mode 100644 index 00000000000..936e1275010 --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context); + +} diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp new file mode 100644 index 00000000000..fd9743dbf72 --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + +class ResourceManagerDispatcher : public IResourceManager +{ +private: + class Classifier : public IClassifier + { + public: + void addClassifier(const ClassifierPtr & classifier) + { + classifiers.push_back(classifier); + } + + bool has(const String & resource_name) override + { + for (const auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return true; + } + return false; + } + + ResourceLink get(const String & resource_name) override + { + for (auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return classifier->get(resource_name); + } + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); + } + private: + std::vector classifiers; // should be constant after initialization to avoid races + }; + +public: + void addManager(const ResourceManagerPtr & manager) + { + managers.push_back(manager); + } + + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override + { + for (auto & manager : managers) + manager->updateConfiguration(config); + } + + bool hasResource(const String & resource_name) const override + { + for (const auto & manager : managers) + { + if (manager->hasResource(resource_name)) + return true; + } + return false; + } + + ClassifierPtr acquire(const String & workload_name) override + { + auto classifier = std::make_shared(); + for (const auto & manager : managers) + classifier->addClassifier(manager->acquire(workload_name)); + return classifier; + } + + void forEachNode(VisitorFunc visitor) override + { + for (const auto & manager : managers) + manager->forEachNode(visitor); + } + +private: + std::vector managers; // Should be constant after initialization to avoid races +}; + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) +{ + auto dispatcher = std::make_shared(); + + // NOTE: if the same resource is described by both managers, then manager added earlier will be used. + dispatcher->addManager(std::make_shared()); + dispatcher->addManager(std::make_shared(global_context->getWorkloadEntityStorage())); + + return dispatcher; +} + +} diff --git a/src/Common/Scheduler/createResourceManager.h b/src/Common/Scheduler/createResourceManager.h new file mode 100644 index 00000000000..d80a17f3bff --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context); + +} diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index fbab25490c1..cc8a873c544 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -18,7 +18,8 @@ #include #include #include - +#include +#include namespace DB { @@ -71,8 +72,8 @@ DiskObjectStorage::DiskObjectStorage( , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) - , read_resource_name(config.getString(config_prefix + ".read_resource", "")) - , write_resource_name(config.getString(config_prefix + ".write_resource", "")) + , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) + , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) { data_source_description = DataSourceDescription{ @@ -83,6 +84,98 @@ DiskObjectStorage::DiskObjectStorage( .is_encrypted = false, .is_cached = object_storage->supportsCache(), }; + resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe( + [this] (const std::vector & events) + { + std::unique_lock lock{resource_mutex}; + + // Sets of matching resource names. Required to resolve possible conflicts in deterministic way + std::set new_read_resource_name_from_sql; + std::set new_write_resource_name_from_sql; + std::set new_read_resource_name_from_sql_any; + std::set new_write_resource_name_from_sql_any; + + // Current state + if (!read_resource_name_from_sql.empty()) + new_read_resource_name_from_sql.insert(read_resource_name_from_sql); + if (!write_resource_name_from_sql.empty()) + new_write_resource_name_from_sql.insert(write_resource_name_from_sql); + if (!read_resource_name_from_sql_any.empty()) + new_read_resource_name_from_sql_any.insert(read_resource_name_from_sql_any); + if (!write_resource_name_from_sql_any.empty()) + new_write_resource_name_from_sql_any.insert(write_resource_name_from_sql_any); + + // Process all updates in specified order + for (const auto & [entity_type, resource_name, resource] : events) + { + if (entity_type == WorkloadEntityType::Resource) + { + if (resource) // CREATE RESOURCE + { + auto * create = typeid_cast(resource.get()); + chassert(create); + for (const auto & [mode, disk] : create->operations) + { + if (!disk) + { + switch (mode) + { + case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql_any.insert(resource_name); break; + case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql_any.insert(resource_name); break; + } + } + else if (*disk == name) + { + switch (mode) + { + case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql.insert(resource_name); break; + case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql.insert(resource_name); break; + } + } + } + } + else // DROP RESOURCE + { + new_read_resource_name_from_sql.erase(resource_name); + new_write_resource_name_from_sql.erase(resource_name); + new_read_resource_name_from_sql_any.erase(resource_name); + new_write_resource_name_from_sql_any.erase(resource_name); + } + } + } + + String old_read_resource = getReadResourceNameNoLock(); + String old_write_resource = getWriteResourceNameNoLock(); + + // Apply changes + if (!new_read_resource_name_from_sql_any.empty()) + read_resource_name_from_sql_any = *new_read_resource_name_from_sql_any.begin(); + else + read_resource_name_from_sql_any.clear(); + + if (!new_write_resource_name_from_sql_any.empty()) + write_resource_name_from_sql_any = *new_write_resource_name_from_sql_any.begin(); + else + write_resource_name_from_sql_any.clear(); + + if (!new_read_resource_name_from_sql.empty()) + read_resource_name_from_sql = *new_read_resource_name_from_sql.begin(); + else + read_resource_name_from_sql.clear(); + + if (!new_write_resource_name_from_sql.empty()) + write_resource_name_from_sql = *new_write_resource_name_from_sql.begin(); + else + write_resource_name_from_sql.clear(); + + String new_read_resource = getReadResourceNameNoLock(); + String new_write_resource = getWriteResourceNameNoLock(); + + if (old_read_resource != new_read_resource) + LOG_INFO(log, "Using resource '{}' instead of '{}' for READ", new_read_resource, old_read_resource); + if (old_write_resource != new_write_resource) + LOG_INFO(log, "Using resource '{}' instead of '{}' for WRITE", new_write_resource, old_write_resource); + }); } StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const @@ -480,13 +573,29 @@ static inline Settings updateIOSchedulingSettings(const Settings & settings, con String DiskObjectStorage::getReadResourceName() const { std::unique_lock lock(resource_mutex); - return read_resource_name; + return getReadResourceNameNoLock(); } String DiskObjectStorage::getWriteResourceName() const { std::unique_lock lock(resource_mutex); - return write_resource_name; + return getWriteResourceNameNoLock(); +} + +String DiskObjectStorage::getReadResourceNameNoLock() const +{ + if (read_resource_name_from_config.empty()) + return read_resource_name_from_sql.empty() ? read_resource_name_from_sql_any : read_resource_name_from_sql; + else + return read_resource_name_from_config; +} + +String DiskObjectStorage::getWriteResourceNameNoLock() const +{ + if (write_resource_name_from_config.empty()) + return write_resource_name_from_sql.empty() ? write_resource_name_from_sql_any : write_resource_name_from_sql; + else + return write_resource_name_from_config; } std::unique_ptr DiskObjectStorage::readFile( @@ -607,10 +716,10 @@ void DiskObjectStorage::applyNewSettings( { std::unique_lock lock(resource_mutex); - if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name) - read_resource_name = new_read_resource_name; - if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name) - write_resource_name = new_write_resource_name; + if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name_from_config) + read_resource_name_from_config = new_read_resource_name; + if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name_from_config) + write_resource_name_from_config = new_write_resource_name; } IDisk::applyNewSettings(config, context_, config_prefix, disk_map); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index b4cdf620555..6657ee352c9 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -6,6 +6,8 @@ #include #include +#include + #include "config.h" @@ -228,6 +230,8 @@ private: String getReadResourceName() const; String getWriteResourceName() const; + String getReadResourceNameNoLock() const; + String getWriteResourceNameNoLock() const; const String object_key_prefix; LoggerPtr log; @@ -246,8 +250,13 @@ private: const bool send_metadata; mutable std::mutex resource_mutex; - String read_resource_name; - String write_resource_name; + String read_resource_name_from_config; // specified in disk config.xml read_resource element + String write_resource_name_from_config; // specified in disk config.xml write_resource element + String read_resource_name_from_sql; // described by CREATE RESOURCE query with READ DISK clause + String write_resource_name_from_sql; // described by CREATE RESOURCE query with WRITE DISK clause + String read_resource_name_from_sql_any; // described by CREATE RESOURCE query with READ ANY DISK clause + String write_resource_name_from_sql_any; // described by CREATE RESOURCE query with WRITE ANY DISK clause + scope_guard resource_changes_subscription; std::unique_ptr metadata_helper; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b8e178e402b..fbf0cbd0eb7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -67,7 +67,6 @@ #include #include #include -#include #include #include #include @@ -92,6 +91,8 @@ #include #include #include +#include +#include #include #include #include @@ -370,6 +371,9 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag user_defined_sql_objects_storage_initialized; mutable std::unique_ptr user_defined_sql_objects_storage; + mutable OnceFlag workload_entity_storage_initialized; + mutable std::unique_ptr workload_entity_storage; + #if USE_NLP mutable OnceFlag synonyms_extensions_initialized; mutable std::optional synonyms_extensions; @@ -711,6 +715,7 @@ struct ContextSharedPart : boost::noncopyable SHUTDOWN(log, "dictionaries loader", external_dictionaries_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "UDFs loader", external_user_defined_executable_functions_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "another UDFs storage", user_defined_sql_objects_storage, stopWatching()); + SHUTDOWN(log, "workload entity storage", workload_entity_storage, stopWatching()); LOG_TRACE(log, "Shutting down named sessions"); Session::shutdownNamedSessions(); @@ -742,6 +747,7 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_external_dictionaries_loader; std::unique_ptr delete_external_user_defined_executable_functions_loader; std::unique_ptr delete_user_defined_sql_objects_storage; + std::unique_ptr delete_workload_entity_storage; std::unique_ptr delete_buffer_flush_schedule_pool; std::unique_ptr delete_schedule_pool; std::unique_ptr delete_distributed_schedule_pool; @@ -826,6 +832,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader = std::move(external_dictionaries_loader); delete_external_user_defined_executable_functions_loader = std::move(external_user_defined_executable_functions_loader); delete_user_defined_sql_objects_storage = std::move(user_defined_sql_objects_storage); + delete_workload_entity_storage = std::move(workload_entity_storage); delete_buffer_flush_schedule_pool = std::move(buffer_flush_schedule_pool); delete_schedule_pool = std::move(schedule_pool); delete_distributed_schedule_pool = std::move(distributed_schedule_pool); @@ -844,6 +851,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader.reset(); delete_external_user_defined_executable_functions_loader.reset(); delete_user_defined_sql_objects_storage.reset(); + delete_workload_entity_storage.reset(); delete_ddl_worker.reset(); delete_buffer_flush_schedule_pool.reset(); delete_schedule_pool.reset(); @@ -1768,7 +1776,7 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { callOnce(shared->resource_manager_initialized, [&] { - shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); + shared->resource_manager = createResourceManager(getGlobalContext()); }); return shared->resource_manager; @@ -3015,6 +3023,16 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } +IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const +{ + callOnce(shared->workload_entity_storage_initialized, [&] { + shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); + }); + + std::lock_guard lock(shared->mutex); + return *shared->workload_entity_storage; +} + #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c62c16098e5..e8ccc31f597 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -76,6 +76,7 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; class IUserDefinedSQLObjectsStorage; +class IWorkloadEntityStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; @@ -893,6 +894,8 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); + IWorkloadEntityStorage & getWorkloadEntityStorage() const; + #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; Lemmatizers & getLemmatizers() const; diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp new file mode 100644 index 00000000000..c6eca7a90d8 --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.cpp @@ -0,0 +1,68 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateResourceQuery::execute() +{ + ASTCreateResourceQuery & create_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_RESOURCE); + + if (create_resource_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!create_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto resource_name = create_resource_query.getResourceName(); + bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; + bool replace_if_exists = create_resource_query.or_replace; + + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Resource, + resource_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); + + return {}; +} + +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateResourceQuery.h b/src/Interpreters/InterpreterCreateResourceQuery.h new file mode 100644 index 00000000000..4bd427e5e8f --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..41d0f52c685 --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp @@ -0,0 +1,68 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateWorkloadQuery::execute() +{ + ASTCreateWorkloadQuery & create_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_WORKLOAD); + + if (create_workload_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!create_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto workload_name = create_workload_query.getWorkloadName(); + bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; + bool replace_if_exists = create_workload_query.or_replace; + + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Workload, + workload_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); + + return {}; +} + +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.h b/src/Interpreters/InterpreterCreateWorkloadQuery.h new file mode 100644 index 00000000000..319388fb64c --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp new file mode 100644 index 00000000000..848a74fda23 --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.cpp @@ -0,0 +1,60 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropResourceQuery::execute() +{ + ASTDropResourceQuery & drop_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!drop_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + bool throw_if_not_exists = !drop_resource_query.if_exists; + + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Resource, + drop_resource_query.resource_name, + throw_if_not_exists); + + return {}; +} + +void registerInterpreterDropResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropResourceQuery.h b/src/Interpreters/InterpreterDropResourceQuery.h new file mode 100644 index 00000000000..588f26fb88c --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp new file mode 100644 index 00000000000..bbaa2beb4cd --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.cpp @@ -0,0 +1,60 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropWorkloadQuery::execute() +{ + ASTDropWorkloadQuery & drop_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!drop_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + bool throw_if_not_exists = !drop_workload_query.if_exists; + + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Workload, + drop_workload_query.workload_name, + throw_if_not_exists); + + return {}; +} + +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.h b/src/Interpreters/InterpreterDropWorkloadQuery.h new file mode 100644 index 00000000000..1297c95e949 --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index cfc95124895..729a7b86312 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -3,9 +3,13 @@ #include #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -332,6 +336,22 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { interpreter_name = "InterpreterDropFunctionQuery"; } + else if (query->as()) + { + interpreter_name = "InterpreterCreateWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterCreateResourceQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropResourceQuery"; + } else if (query->as()) { interpreter_name = "InterpreterCreateIndexQuery"; diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp index 481d0597a85..838b3a669da 100644 --- a/src/Interpreters/registerInterpreters.cpp +++ b/src/Interpreters/registerInterpreters.cpp @@ -52,6 +52,10 @@ void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory); +void registerInterpreterDropResourceQuery(InterpreterFactory & factory); void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); void registerInterpreterDropIndexQuery(InterpreterFactory & factory); @@ -111,6 +115,10 @@ void registerInterpreters() registerInterpreterTransactionControlQuery(factory); registerInterpreterCreateFunctionQuery(factory); registerInterpreterDropFunctionQuery(factory); + registerInterpreterCreateWorkloadQuery(factory); + registerInterpreterDropWorkloadQuery(factory); + registerInterpreterCreateResourceQuery(factory); + registerInterpreterDropResourceQuery(factory); registerInterpreterCreateIndexQuery(factory); registerInterpreterCreateNamedCollectionQuery(factory); registerInterpreterDropIndexQuery(factory); diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp new file mode 100644 index 00000000000..3e40d76ba1b --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.cpp @@ -0,0 +1,83 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateResourceQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->resource_name = resource_name->clone(); + res->children.push_back(res->resource_name); + + res->operations = operations; + + return res; +} + +void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const +{ + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + format.ostr << "OR REPLACE "; + + format.ostr << "RESOURCE "; + + if (if_not_exists) + format.ostr << "IF NOT EXISTS "; + + format.ostr << (format.hilite ? hilite_none : ""); + + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : ""); + + formatOnCluster(format); + + format.ostr << " ("; + + bool first = true; + for (const auto & operation : operations) + { + if (!first) + format.ostr << ", "; + else + first = false; + + switch (operation.mode) + { + case AccessMode::Read: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "READ "; + break; + } + case AccessMode::Write: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE "; + break; + } + } + if (operation.disk) + { + format.ostr << "DISK " << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(*operation.disk) << (format.hilite ? hilite_none : ""); + } + else + format.ostr << "ANY DISK" << (format.hilite ? hilite_none : ""); + } + + format.ostr << ")"; +} + +String ASTCreateResourceQuery::getResourceName() const +{ + String name; + tryGetIdentifierNameInto(resource_name, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h new file mode 100644 index 00000000000..51933a375f8 --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + enum class AccessMode + { + Read, + Write + }; + struct Operation + { + AccessMode mode; + std::optional disk; // Applies to all disks if not set + + friend bool operator ==(const Operation & lhs, const Operation & rhs) { return lhs.mode == rhs.mode && lhs.disk == rhs.disk; } + friend bool operator !=(const Operation & lhs, const Operation & rhs) { return !(lhs == rhs); } + }; + + using Operations = std::vector; + + ASTPtr resource_name; + Operations operations; /// List of operations that require this resource + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateResourceQuery" + (delim + getResourceName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getResourceName() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..972ce733651 --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateWorkloadQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->workload_name = workload_name->clone(); + res->children.push_back(res->workload_name); + + if (workload_parent) + { + res->workload_parent = workload_parent->clone(); + res->children.push_back(res->workload_parent); + } + + res->changes = changes; + + return res; +} + +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const +{ + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + format.ostr << "OR REPLACE "; + + format.ostr << "WORKLOAD "; + + if (if_not_exists) + format.ostr << "IF NOT EXISTS "; + + format.ostr << (format.hilite ? hilite_none : ""); + + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : ""); + + formatOnCluster(format); + + if (hasParent()) + { + format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); + } + + if (!changes.empty()) + { + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; + + bool first = true; + + for (const auto & change : changes) + { + if (!first) + format.ostr << ", "; + else + first = false; + format.ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value); + if (!change.resource.empty()) + { + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' '; + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : ""); + } + } + } +} + +String ASTCreateWorkloadQuery::getWorkloadName() const +{ + String name; + tryGetIdentifierNameInto(workload_name, name); + return name; +} + +bool ASTCreateWorkloadQuery::hasParent() const +{ + return workload_parent != nullptr; +} + +String ASTCreateWorkloadQuery::getWorkloadParent() const +{ + String name; + tryGetIdentifierNameInto(workload_parent, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h new file mode 100644 index 00000000000..8a4cecc001e --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + ASTPtr workload_name; + ASTPtr workload_parent; + + /// Special version of settings that support optional `FOR resource` clause + struct SettingChange + { + String name; + Field value; + String resource; + + SettingChange() = default; + SettingChange(std::string_view name_, const Field & value_, std::string_view resource_) : name(name_), value(value_), resource(resource_) {} + SettingChange(std::string_view name_, Field && value_, std::string_view resource_) : name(name_), value(std::move(value_)), resource(resource_) {} + + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.resource == rhs.resource); } + friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } + }; + + using SettingsChanges = std::vector; + SettingsChanges changes; + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateWorkloadQuery" + (delim + getWorkloadName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getWorkloadName() const; + bool hasParent() const; + String getWorkloadParent() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/ASTDropResourceQuery.cpp b/src/Parsers/ASTDropResourceQuery.cpp new file mode 100644 index 00000000000..753ac4e30e7 --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropResourceQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropResourceQuery.h b/src/Parsers/ASTDropResourceQuery.h new file mode 100644 index 00000000000..e1534ea454a --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String resource_name; + + bool if_exists = false; + + String getID(char) const override { return "DropResourceQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.cpp b/src/Parsers/ASTDropWorkloadQuery.cpp new file mode 100644 index 00000000000..3192223c4b3 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropWorkloadQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.h b/src/Parsers/ASTDropWorkloadQuery.h new file mode 100644 index 00000000000..99c3a011447 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String workload_name; + + bool if_exists = false; + + String getID(char) const override { return "DropWorkloadQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 83b7eb71d64..dd0ba91d428 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -392,6 +392,7 @@ namespace DB MR_MACROS(RANDOMIZE_FOR, "RANDOMIZE FOR") \ MR_MACROS(RANDOMIZED, "RANDOMIZED") \ MR_MACROS(RANGE, "RANGE") \ + MR_MACROS(READ, "READ") \ MR_MACROS(READONLY, "READONLY") \ MR_MACROS(REALM, "REALM") \ MR_MACROS(RECOMPRESS, "RECOMPRESS") \ @@ -411,6 +412,7 @@ namespace DB MR_MACROS(REPLACE, "REPLACE") \ MR_MACROS(RESET_SETTING, "RESET SETTING") \ MR_MACROS(RESET_AUTHENTICATION_METHODS_TO_NEW, "RESET AUTHENTICATION METHODS TO NEW") \ + MR_MACROS(RESOURCE, "RESOURCE") \ MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \ MR_MACROS(RESTORE, "RESTORE") \ MR_MACROS(RESTRICT, "RESTRICT") \ @@ -523,6 +525,7 @@ namespace DB MR_MACROS(WHEN, "WHEN") \ MR_MACROS(WHERE, "WHERE") \ MR_MACROS(WINDOW, "WINDOW") \ + MR_MACROS(WORKLOAD, "WORKLOAD") \ MR_MACROS(QUALIFY, "QUALIFY") \ MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \ MR_MACROS(WITH_CHECK, "WITH CHECK") \ @@ -535,6 +538,7 @@ namespace DB MR_MACROS(WITH, "WITH") \ MR_MACROS(RECURSIVE, "RECURSIVE") \ MR_MACROS(WK, "WK") \ + MR_MACROS(WRITE, "WRITE") \ MR_MACROS(WRITABLE, "WRITABLE") \ MR_MACROS(WW, "WW") \ MR_MACROS(YEAR, "YEAR") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp new file mode 100644 index 00000000000..68c157df175 --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.cpp @@ -0,0 +1,144 @@ +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +bool parseOneOperation(ASTCreateResourceQuery::Operation & operation, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier disk_name_p; + + ASTCreateResourceQuery::AccessMode mode; + ASTPtr node; + std::optional disk; + + if (ParserKeyword(Keyword::WRITE).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Write; + else if (ParserKeyword(Keyword::READ).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Read; + else + return false; + + if (ParserKeyword(Keyword::ANY).ignore(pos, expected)) + { + if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) + return false; + } + else + { + if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) + return false; + + if (!disk_name_p.parse(pos, node, expected)) + return false; + + disk.emplace(); + if (!tryGetIdentifierNameInto(node, *disk)) + return false; + } + + operation.mode = mode; + operation.disk = std::move(disk); + + return true; +} + +bool parseOperations(IParser::Pos & pos, Expected & expected, ASTCreateResourceQuery::Operations & operations) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + ParserToken s_open(TokenType::OpeningRoundBracket); + ParserToken s_close(TokenType::ClosingRoundBracket); + + if (!s_open.ignore(pos, expected)) + return false; + + ASTCreateResourceQuery::Operations res_operations; + + auto parse_operation = [&] + { + ASTCreateResourceQuery::Operation operation; + if (!parseOneOperation(operation, pos, expected)) + return false; + res_operations.push_back(std::move(operation)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_operation, false)) + return false; + + if (!s_close.ignore(pos, expected)) + return false; + + operations = std::move(res_operations); + return true; + }); +} + +} + +bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + + ASTPtr resource_name; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + ASTCreateResourceQuery::Operations operations; + if (!parseOperations(pos, expected, operations)) + return false; + + auto create_resource_query = std::make_shared(); + node = create_resource_query; + + create_resource_query->resource_name = resource_name; + create_resource_query->children.push_back(resource_name); + + create_resource_query->or_replace = or_replace; + create_resource_query->if_not_exists = if_not_exists; + create_resource_query->cluster = std::move(cluster_str); + + create_resource_query->operations = std::move(operations); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateResourceQuery.h b/src/Parsers/ParserCreateResourceQuery.h new file mode 100644 index 00000000000..1b7c9fc4a7f --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE RESOURCE cache_io (WRITE DISK s3diskWithCache, READ DISK s3diskWithCache) +class ParserCreateResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserCreateWorkloadEntity.cpp b/src/Parsers/ParserCreateWorkloadEntity.cpp new file mode 100644 index 00000000000..013210a6d87 --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +bool ParserCreateWorkloadEntity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserCreateWorkloadQuery create_workload_p; + ParserCreateResourceQuery create_resource_p; + + return create_workload_p.parse(pos, node, expected) || create_resource_p.parse(pos, node, expected); +} + +} diff --git a/src/Parsers/ParserCreateWorkloadEntity.h b/src/Parsers/ParserCreateWorkloadEntity.h new file mode 100644 index 00000000000..1e7b78b3ccc --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +/// Special parser for the CREATE WORKLOAD and CREATE RESOURCE queries. +class ParserCreateWorkloadEntity : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE workload entity query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..9caf474741c --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -0,0 +1,155 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +bool parseWorkloadSetting( + ASTCreateWorkloadQuery::SettingChange & change, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier name_p; + ParserLiteral value_p; + ParserToken s_eq(TokenType::Equals); + ParserIdentifier resource_name_p; + + ASTPtr name_node; + ASTPtr value_node; + ASTPtr resource_name_node; + + String name; + String resource_name; + + if (!name_p.parse(pos, name_node, expected)) + return false; + tryGetIdentifierNameInto(name_node, name); + + if (!s_eq.ignore(pos, expected)) + return false; + + if (!value_p.parse(pos, value_node, expected)) + return false; + + if (ParserKeyword(Keyword::FOR).ignore(pos, expected)) + { + if (!resource_name_p.parse(pos, resource_name_node, expected)) + return false; + tryGetIdentifierNameInto(resource_name_node, resource_name); + } + + change.name = std::move(name); + change.value = value_node->as().value; + change.resource = std::move(resource_name); + + return true; +} + +bool parseSettings(IParser::Pos & pos, Expected & expected, ASTCreateWorkloadQuery::SettingsChanges & changes) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) + return false; + + ASTCreateWorkloadQuery::SettingsChanges res_changes; + + auto parse_setting = [&] + { + ASTCreateWorkloadQuery::SettingChange change; + if (!parseWorkloadSetting(change, pos, expected)) + return false; + res_changes.push_back(std::move(change)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_setting, false)) + return false; + + changes = std::move(res_changes); + return true; + }); +} + +} + +bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserIdentifier workload_name_p; + ParserKeyword s_on(Keyword::ON); + ParserKeyword s_in(Keyword::IN); + + ASTPtr workload_name; + ASTPtr workload_parent; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (s_in.ignore(pos, expected)) + { + if (!workload_name_p.parse(pos, workload_parent, expected)) + return false; + } + + ASTCreateWorkloadQuery::SettingsChanges changes; + parseSettings(pos, expected, changes); + + auto create_workload_query = std::make_shared(); + node = create_workload_query; + + create_workload_query->workload_name = workload_name; + create_workload_query->children.push_back(workload_name); + + if (workload_parent) + { + create_workload_query->workload_parent = workload_parent; + create_workload_query->children.push_back(workload_parent); + } + + create_workload_query->or_replace = or_replace; + create_workload_query->if_not_exists = if_not_exists; + create_workload_query->cluster = std::move(cluster_str); + create_workload_query->changes = std::move(changes); + + + return true; +} + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.h b/src/Parsers/ParserCreateWorkloadQuery.h new file mode 100644 index 00000000000..62c89affeda --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE WORKLOAD production IN all SETTINGS weight = 3, max_speed = '1G' FOR network_read, max_speed = '2G' FOR network_write +class ParserCreateWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserDropResourceQuery.cpp b/src/Parsers/ParserDropResourceQuery.cpp new file mode 100644 index 00000000000..6c078281828 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr resource_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_resource_query = std::make_shared(); + drop_resource_query->if_exists = if_exists; + drop_resource_query->cluster = std::move(cluster_str); + + node = drop_resource_query; + + drop_resource_query->resource_name = resource_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropResourceQuery.h b/src/Parsers/ParserDropResourceQuery.h new file mode 100644 index 00000000000..651603d1e90 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP RESOURCE resource1 +class ParserDropResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/ParserDropWorkloadQuery.cpp b/src/Parsers/ParserDropWorkloadQuery.cpp new file mode 100644 index 00000000000..edc82c8f30a --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier workload_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr workload_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_workload_query = std::make_shared(); + drop_workload_query->if_exists = if_exists; + drop_workload_query->cluster = std::move(cluster_str); + + node = drop_workload_query; + + drop_workload_query->workload_name = workload_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropWorkloadQuery.h b/src/Parsers/ParserDropWorkloadQuery.h new file mode 100644 index 00000000000..af060caf303 --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP WORKLOAD workload1 +class ParserDropWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index d5645298ecf..4ed6e4267f4 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,8 +1,12 @@ #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -51,6 +55,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; ParserCreateNamedCollectionQuery create_named_collection_p; ParserDropNamedCollectionQuery drop_named_collection_p; ParserAlterNamedCollectionQuery alter_named_collection_p; @@ -82,6 +90,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) + || create_workload_p.parse(pos, node, expected) + || drop_workload_p.parse(pos, node, expected) + || create_resource_p.parse(pos, node, expected) + || drop_resource_p.parse(pos, node, expected) || create_named_collection_p.parse(pos, node, expected) || drop_named_collection_p.parse(pos, node, expected) || alter_named_collection_p.parse(pos, node, expected) diff --git a/src/Storages/System/StorageSystemResources.cpp b/src/Storages/System/StorageSystemResources.cpp new file mode 100644 index 00000000000..2f948b8e057 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.cpp @@ -0,0 +1,71 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemResources::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the resource."}, + {"read_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for read operations."}, + {"write_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for write operations."}, + {"create_query", std::make_shared(), "CREATE query of the resource."}, + }; +} + +void StorageSystemResources::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & resource_names = storage.getAllEntityNames(WorkloadEntityType::Resource); + for (const auto & resource_name : resource_names) + { + auto ast = storage.get(resource_name); + auto & resource = typeid_cast(*ast); + res_columns[0]->insert(resource_name); + { + Array read_disks; + Array write_disks; + for (const auto & [mode, disk] : resource.operations) + { + switch (mode) + { + case DB::ASTCreateResourceQuery::AccessMode::Read: + { + read_disks.emplace_back(disk ? *disk : "ANY"); + break; + } + case DB::ASTCreateResourceQuery::AccessMode::Write: + { + write_disks.emplace_back(disk ? *disk : "ANY"); + break; + } + } + } + res_columns[1]->insert(read_disks); + res_columns[2]->insert(write_disks); + } + res_columns[3]->insert(queryToString(ast)); + } +} + +void StorageSystemResources::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemResources::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemResources.h b/src/Storages/System/StorageSystemResources.h new file mode 100644 index 00000000000..42bbcd09aa4 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `resources` system table, which allows you to get a list of all RESOURCEs +class StorageSystemResources final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemResources"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index b42c807d6fc..8784ba084ce 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -84,12 +84,12 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { - context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, ISchedulerNode * node) { size_t i = 0; res_columns[i++]->insert(resource); res_columns[i++]->insert(path); - res_columns[i++]->insert(type); + res_columns[i++]->insert(node->getTypeName()); res_columns[i++]->insert(node->info.weight); res_columns[i++]->insert(node->info.priority.value); res_columns[i++]->insert(node->isActive()); @@ -118,23 +118,23 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c if (auto * parent = dynamic_cast(node->parent)) { - if (auto value = parent->getChildVRuntime(node.get())) + if (auto value = parent->getChildVRuntime(node)) vruntime = *value; } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) system_vruntime = ptr->getSystemVRuntime(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) budget = ptr->getBudget(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) is_satisfied = ptr->isSatisfied(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); std::tie(max_requests, max_cost) = ptr->getLimits(); } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(max_speed, max_burst) = ptr->getParams(); throttling_us = ptr->getThrottlingDuration().count() / 1000; diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp new file mode 100644 index 00000000000..ebb7e693e26 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemWorkloads::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the workload."}, + {"parent", std::make_shared(), "The name of the parent workload."}, + {"create_query", std::make_shared(), "CREATE query of the workload."}, + }; +} + +void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & workload_names = storage.getAllEntityNames(WorkloadEntityType::Workload); + for (const auto & workload_name : workload_names) + { + auto ast = storage.get(workload_name); + auto & workload = typeid_cast(*ast); + res_columns[0]->insert(workload_name); + res_columns[1]->insert(workload.getWorkloadParent()); + res_columns[2]->insert(queryToString(ast)); + } +} + +void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for workloads + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for workloads + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemWorkloads.h b/src/Storages/System/StorageSystemWorkloads.h new file mode 100644 index 00000000000..9d4770a02b8 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `workloads` system table, which allows you to get a list of all workloads +class StorageSystemWorkloads final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemWorkloads"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 70dcec884a6..0bd3369ff32 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include #include #include @@ -230,6 +232,8 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach>(context, system_database, "azure_queue_settings", "Contains a list of settings of AzureQueue tables."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); + attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); + attach(context, system_database, "resources", "Contains a list of all currently existing resources."); if (has_zookeeper) { diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 823a00a05de..9498044c836 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -1,4 +1,5 @@ + /clickhouse/workload/definitions.sql @@ -12,6 +13,15 @@ network_read network_write + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + @@ -21,6 +31,13 @@ + + +
+ s3_no_resource +
+
+
diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 050281b2e3a..e4ef83759e4 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -2,6 +2,7 @@ # pylint: disable=redefined-outer-name # pylint: disable=line-too-long +import random import threading import time @@ -9,6 +10,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) @@ -23,6 +25,21 @@ node = cluster.add_instance( "configs/workloads.xml.default", ], with_minio=True, + with_zookeeper=True, +) + +node2 = cluster.add_instance( + "node2", + stay_alive=True, + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], + with_minio=True, + with_zookeeper=True, ) @@ -55,6 +72,22 @@ def set_default_configs(): yield +@pytest.fixture(scope="function", autouse=True) +def clear_workloads_and_resources(): + node.query( + f""" + drop workload if exists production; + drop workload if exists development; + drop workload if exists admin; + drop workload if exists all; + drop resource if exists io_write; + drop resource if exists io_read; + drop resource if exists io; + """ + ) + yield + + def update_workloads_config(**settings): xml = "" for name in settings: @@ -570,3 +603,364 @@ def test_mutation_workload_change(): assert reads_before < reads_after assert writes_before < writes_after + + +def test_create_workload(): + node.query( + f""" + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); + create workload all settings max_cost = 1000000 for io_write, max_cost = 2000000 for io_read; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def do_checks(): + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin' and type='unified' and priority=0" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production' and type='unified' and weight=9" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/development/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_write' and max_cost=1000000" + ) + == "1\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_read' and max_cost=2000000" + ) + == "1\n" + ) + + do_checks() + node.restart_clickhouse() # Check that workloads persist + do_checks() + + +def test_workload_hierarchy_changes(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + for iteration in range(3): + split_idx = random.randint(1, len(queries) - 2) + for query_idx in range(0, split_idx): + node.query(queries[query_idx]) + node.query( + "create resource io_test (write disk non_existent_disk, read disk non_existent_disk);" + ) + node.query("drop resource io_test;") + for query_idx in range(split_idx, len(queries)): + node.query(queries[query_idx]) + + +def test_resource_read_and_write(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; + """ + ) + + node.query( + f""" + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); + create workload all settings max_cost = 1000000; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def write_query(workload): + try: + node.query( + f"insert into data select * from numbers(1e5) settings workload='{workload}'" + ) + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=write_query, args=["development"]) + thread2 = threading.Thread(target=write_query, args=["production"]) + thread3 = threading.Thread(target=write_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) + + def read_query(workload): + try: + node.query(f"select sum(key*key) from data settings workload='{workload}'") + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=read_query, args=["development"]) + thread2 = threading.Thread(target=read_query, args=["production"]) + thread3 = threading.Thread(target=read_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) + + +def test_resource_any_disk(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; + """ + ) + + node.query( + f""" + create resource io (write any disk, read any disk); + create workload all settings max_cost = 1000000; + """ + ) + + node.query(f"insert into data select * from numbers(1e5) settings workload='all'") + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'" + ) + == "1\n" + ) + + node.query(f"select sum(key*key) from data settings workload='all'") + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'" + ) + == "1\n" + ) + + +def test_workload_entity_keeper_storage(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + + def check_consistency(): + checks = [ + "select name, create_query from system.workloads order by all", + "select name, create_query from system.resources order by all", + "select resource, path, type, weight, priority, max_requests, max_cost, max_speed, max_burst from system.scheduler where resource not in ['network_read', 'network_write'] order by all", + ] + attempts = 10 + value1 = "" + value2 = "" + error_query = "" + for attempt in range(attempts): + for query in checks: + value1 = node.query(query) + value2 = node2.query(query) + if value1 != value2: + error_query = query + break # error + else: + break # success + time.sleep(0.5) + else: + raise Exception( + f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" + ) + + for iteration in range(3): + split_idx_1 = random.randint(1, len(queries) - 3) + split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(0, split_idx_1): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_1, split_idx_2): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_2, len(queries)): + node.query(queries[query_idx]) + + check_consistency() diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 10cedc36020..85ffee8e44d 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -59,6 +59,8 @@ CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE ARBITRARY TEMPORARY TABLE CREATE ARBITRARY TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE +CREATE WORKLOAD [] GLOBAL CREATE +CREATE RESOURCE [] GLOBAL CREATE CREATE NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN CREATE [] \N ALL DROP DATABASE [] DATABASE DROP @@ -66,6 +68,8 @@ DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP +DROP WORKLOAD [] GLOBAL DROP +DROP RESOURCE [] GLOBAL DROP DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN DROP [] \N ALL UNDROP TABLE [] TABLE ALL diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.reference b/tests/queries/0_stateless/03232_resource_create_and_drop.reference new file mode 100644 index 00000000000..2a1045d314c --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.reference @@ -0,0 +1,5 @@ +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_2 ['03232_disk_2'] [] CREATE RESOURCE `03232_resource_2` (READ DISK `03232_disk_2`) +03232_resource_3 [] ['03232_disk_2'] CREATE RESOURCE `03232_resource_3` (WRITE DISK `03232_disk_2`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql new file mode 100644 index 00000000000..ceebd557a51 --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because creating the same resource twice will fail +CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); +CREATE RESOURCE 03232_resource_3 (WRITE DISK 03232_disk_2); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE IF EXISTS 03232_resource_2; +DROP RESOURCE 03232_resource_3; +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE 03232_resource_1; diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference new file mode 100644 index 00000000000..923e8652a35 --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -0,0 +1,5 @@ +all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` +development all CREATE WORKLOAD development IN `all` +production all CREATE WORKLOAD production IN `all` +all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql new file mode 100644 index 00000000000..1d8f97baf4c --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +CREATE OR REPLACE WORKLOAD all; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; +CREATE WORKLOAD IF NOT EXISTS production IN all; +CREATE WORKLOAD development IN all; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; +DROP WORKLOAD IF EXISTS production; +DROP WORKLOAD development; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; +DROP WORKLOAD all; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.reference b/tests/queries/0_stateless/03232_workloads_and_resources.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql new file mode 100644 index 00000000000..a3e46166396 --- /dev/null +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -0,0 +1,68 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process + +-- Test simple resource and workload hierarchy creation +create resource 03232_write (write disk 03232_fake_disk); +create resource 03232_read (read disk 03232_fake_disk); +create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create workload admin in all settings priority = 0; +create workload production in all settings priority = 1, weight = 9; +create workload development in all settings priority = 1, weight = 1; + +-- Test that illegal actions are not allowed +create workload another_root; -- {serverError BAD_ARGUMENTS} +create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} +drop workload all; -- {serverError BAD_ARGUMENTS} +create workload invalid in 03232_write; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} +create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} +create workload invalid in all settings weight = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} +create or replace workload all in production; -- {serverError BAD_ARGUMENTS} + +-- Test CREATE OR REPLACE WORKLOAD +create or replace workload all settings max_requests = 200 for 03232_write, max_requests = 100 for 03232_read; +create or replace workload admin in all settings priority = 1; +create or replace workload admin in all settings priority = 2; +create or replace workload admin in all settings priority = 0; +create or replace workload production in all settings priority = 1, weight = 90; +create or replace workload production in all settings priority = 0, weight = 9; +create or replace workload production in all settings priority = 2, weight = 9; +create or replace workload development in all settings priority = 1; +create or replace workload development in all settings priority = 0; +create or replace workload development in all settings priority = 2; + +-- Test CREATE OR REPLACE RESOURCE +create or replace resource 03232_write (write disk 03232_fake_disk_2); +create or replace resource 03232_read (read disk 03232_fake_disk_2); + +-- Test update settings with CREATE OR REPLACE WORKLOAD +create or replace workload production in all settings priority = 1, weight = 9, max_requests = 100; +create or replace workload development in all settings priority = 1, weight = 1, max_requests = 10; +create or replace workload production in all settings priority = 1, weight = 9, max_cost = 100000; +create or replace workload development in all settings priority = 1, weight = 1, max_cost = 10000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000, max_burst = 10000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000, max_burst = 1000000; +create or replace workload all settings max_cost = 1000000, max_speed = 100000 for 03232_write, max_speed = 200000 for 03232_read; +create or replace workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create or replace workload production in all settings priority = 1, weight = 9; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Test change parent with CREATE OR REPLACE WORKLOAD +create or replace workload development in production settings priority = 1, weight = 1; +create or replace workload development in admin settings priority = 1, weight = 1; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Clean up +drop workload if exists production; +drop workload if exists development; +drop workload if exists admin; +drop workload if exists all; +drop resource if exists 03232_write; +drop resource if exists 03232_read; From 3a41e79eb8fe5f9ad69cf2a65056db4e7901a09e Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Wed, 30 Oct 2024 15:20:07 +0000 Subject: [PATCH 650/816] Fix test --- tests/queries/0_stateless/01271_show_privileges.reference | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..930e92cda4e 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -183,6 +183,9 @@ HDFS [] GLOBAL SOURCES S3 [] GLOBAL SOURCES HIVE [] GLOBAL SOURCES AZURE [] GLOBAL SOURCES +KAFKA [] GLOBAL SOURCES +NATS [] GLOBAL SOURCES +RABBITMQ [] GLOBAL SOURCES SOURCES [] \N ALL CLUSTER [] GLOBAL ALL ALL ['ALL PRIVILEGES'] \N \N From 7af2e822e7eb486ae95319a09364ea36498bb49b Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 15:22:45 +0000 Subject: [PATCH 651/816] cleanup --- src/Interpreters/FillingRow.cpp | 37 +++++++++------- src/Interpreters/FillingRow.h | 6 +-- .../Transforms/FillingTransform.cpp | 44 ++++++------------- 3 files changed, 36 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index a87ca418b7b..df93ece2af4 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -1,10 +1,10 @@ #include -#include -#include "Common/Logger.h" -#include "Common/logger_useful.h" -#include -#include "base/defines.h" + #include +#include +#include +#include +#include namespace DB @@ -145,7 +145,7 @@ Field findMin(Field a, Field b, Field c, int dir) return a; } -std::pair FillingRow::next(const FillingRow & next_original_row) +bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) { auto logger = getLogger("FillingRow"); @@ -169,18 +169,18 @@ std::pair FillingRow::next(const FillingRow & next_original_row) LOG_DEBUG(logger, "pos: {}", pos); if (pos == row_size) - return {false, false}; + return false; const auto & pos_descr = getFillDescription(pos); if (!next_original_row[pos].isNull() && less(next_original_row[pos], row[pos], getDirection(pos))) - return {false, false}; + return false; if (!staleness_border[pos].isNull() && !less(row[pos], staleness_border[pos], getDirection(pos))) - return {false, false}; + return false; if (!pos_descr.fill_to.isNull() && !less(row[pos], pos_descr.fill_to, getDirection(pos))) - return {false, false}; + return false; /// If we have any 'fill_to' value at position greater than 'pos' or configured staleness, /// we need to generate rows up to one of this borders. @@ -205,20 +205,22 @@ std::pair FillingRow::next(const FillingRow & next_original_row) row[i] = next_value; initUsingFrom(i + 1); - return {true, true}; + + value_changed = true; + return true; } auto next_value = row[pos]; getFillDescription(pos).step_func(next_value, 1); if (!next_original_row[pos].isNull() && less(next_original_row[pos], next_value, getDirection(pos))) - return {false, false}; + return false; if (!staleness_border[pos].isNull() && !less(next_value, staleness_border[pos], getDirection(pos))) - return {false, false}; + return false; if (!pos_descr.fill_to.isNull() && !less(next_value, pos_descr.fill_to, getDirection(pos))) - return {false, false}; + return false; row[pos] = next_value; if (equals(row[pos], next_original_row[pos])) @@ -239,11 +241,14 @@ std::pair FillingRow::next(const FillingRow & next_original_row) ); } - return {is_less, true}; + value_changed = true; + return is_less; } initUsingFrom(pos + 1); - return {true, true}; + + value_changed = true; + return true; } bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed) diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index d33e3f95541..d4590d7b81c 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -25,10 +25,8 @@ public: explicit FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. - /// Return pair of boolean - /// apply - true if filling values should be inserted into result set - /// value_changed - true if filling row value was changed - std::pair next(const FillingRow & next_original_row); + /// Returns true if filling values should be inserted into result set + bool next(const FillingRow & next_original_row, bool& value_changed); /// Returns true if need to generate some prefix for to_row bool shift(const FillingRow & next_original_row, bool& value_changed); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 40650b485f8..f23ffec43de 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -11,7 +11,6 @@ #include #include #include -#include "Interpreters/FillingRow.h" #include @@ -534,9 +533,7 @@ bool FillingTransform::generateSuffixIfNeeded( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row); - filling_row_changed = changed; - if (!apply) + if (!filling_row.next(next_row, filling_row_changed)) break; interpolate(result_columns, interpolate_block); @@ -660,9 +657,7 @@ void FillingTransform::transformRange( bool filling_row_changed = false; while (true) { - const auto [apply, changed] = filling_row.next(next_row); - filling_row_changed = changed; - if (!apply) + if (!filling_row.next(next_row, filling_row_changed)) break; interpolate(result_columns, interpolate_block); @@ -670,35 +665,22 @@ void FillingTransform::transformRange( copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); } + /// Initialize staleness border for current row to generate it's prefix + filling_row.initStalenessRow(input_fill_columns, row_ind); + + while (filling_row.shift(next_row, filling_row_changed)) { - filling_row.initStalenessRow(input_fill_columns, row_ind); + logDebug("filling_row after shift", filling_row); - bool shift_apply = filling_row.shift(next_row, filling_row_changed); - logDebug("shift_apply", shift_apply); - logDebug("filling_row_changed", filling_row_changed); - - while (shift_apply) + do { - logDebug("after shift", filling_row); + logDebug("inserting prefix filling_row", filling_row); - while (true) - { - logDebug("filling_row in prefix", filling_row); + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); - copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - - const auto [apply, changed] = filling_row.next(next_row); - logDebug("filling_row in prefix", filling_row); - - filling_row_changed = changed; - if (!apply) - break; - } - - shift_apply = filling_row.shift(next_row, filling_row_changed); - } + } while (filling_row.next(next_row, filling_row_changed)); } /// new valid filling row was generated but not inserted, will use it during suffix generation From a819cfa709f3e100e9ae139a81f16eb99e98eec8 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 30 Oct 2024 16:50:40 +0100 Subject: [PATCH 652/816] Read ECS token from file --- src/IO/S3/Credentials.cpp | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index a3f671e76d9..91571432840 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include namespace DB { @@ -693,6 +695,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN"; + static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN_PATH"; static const char AWS_EC2_METADATA_DISABLED[] = "AWS_EC2_METADATA_DISABLED"; /// The only difference from DefaultAWSCredentialsProviderChain::DefaultAWSCredentialsProviderChain() @@ -750,7 +753,22 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( } else if (!absolute_uri.empty()) { - const auto token = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN); + auto token = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN); + const auto token_path = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH); + + if (!token_path.empty()) + { + LOG_INFO(logger, "The environment variable value {} is {}", AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH, token_path); + + String token_from_file; + + ReadBufferFromFile in(token_path); + readStringUntilEOF(token_from_file, in); + Poco::trimInPlace(token_from_file); + + token = token_from_file; + } + AddProvider(std::make_shared(absolute_uri.c_str(), token.c_str())); /// DO NOT log the value of the authorization token for security purposes. From 12e36c39fc823986e3aecb105773d18a9b4e601e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 30 Oct 2024 16:52:59 +0100 Subject: [PATCH 653/816] Sort headers --- src/IO/S3/Credentials.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 91571432840..cde9a7a3662 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -1,7 +1,7 @@ -#include -#include #include #include +#include +#include namespace DB { From ab5738b9f1e87cf8b49b3d74a3bbd05e53c39850 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 16:11:40 +0000 Subject: [PATCH 654/816] merge constraints --- src/Interpreters/FillingRow.cpp | 92 +++++++------------ src/Interpreters/FillingRow.h | 4 +- .../Transforms/FillingTransform.cpp | 4 +- 3 files changed, 37 insertions(+), 63 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index df93ece2af4..67827567e04 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -32,7 +32,10 @@ FillingRow::FillingRow(const SortDescription & sort_description_) : sort_description(sort_description_) { row.resize(sort_description.size()); - staleness_border.resize(sort_description.size()); + + constraints.reserve(sort_description.size()); + for (size_t i = 0; i < size(); ++i) + constraints.push_back(getFillDescription(i).fill_to); } bool FillingRow::operator<(const FillingRow & other) const @@ -96,53 +99,33 @@ std::optional FillingRow::doLongJump(const FillColumnDescription & descr, bool FillingRow::hasSomeConstraints(size_t pos) const { - const auto & descr = getFillDescription(pos); - - if (!descr.fill_to.isNull()) - return true; - - if (!descr.fill_staleness.isNull()) - return true; - - return false; + return !constraints[pos].isNull(); } bool FillingRow::isConstraintsComplete(size_t pos) const { - auto logger = getLogger("FillingRow::isConstraintComplete"); + auto logger = getLogger("FillingRow::isConstraintsComplete"); chassert(!row[pos].isNull()); chassert(hasSomeConstraints(pos)); - const auto & descr = getFillDescription(pos); int direction = getDirection(pos); + LOG_DEBUG(logger, "constraint: {}, row: {}, direction: {}", constraints[pos].dump(), row[pos].dump(), direction); - if (!descr.fill_to.isNull() && !less(row[pos], descr.fill_to, direction)) - { - LOG_DEBUG(logger, "fill to: {}, row: {}, direction: {}", descr.fill_to.dump(), row[pos].dump(), direction); - return false; - } - - if (!descr.fill_staleness.isNull() && !less(row[pos], staleness_border[pos], direction)) - { - LOG_DEBUG(logger, "staleness border: {}, row: {}, direction: {}", staleness_border[pos].dump(), row[pos].dump(), direction); - return false; - } - - return true; + return less(row[pos], constraints[pos], direction); } -Field findMin(Field a, Field b, Field c, int dir) +static const Field & findBorder(const Field & constraint, const Field & next_original, int direction) { - auto logger = getLogger("FillingRow"); - LOG_DEBUG(logger, "a: {} b: {} c: {}", a.dump(), b.dump(), c.dump()); + if (constraint.isNull()) + return next_original; - if (a.isNull() || (!b.isNull() && less(b, a, dir))) - a = b; + if (next_original.isNull()) + return constraint; - if (a.isNull() || (!c.isNull() && less(c, a, dir))) - a = c; + if (less(constraint, next_original, direction)) + return constraint; - return a; + return next_original; } bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) @@ -158,11 +141,10 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) if (row[pos].isNull()) continue; - const auto & descr = getFillDescription(pos); - auto min_constr = findMin(next_original_row[pos], staleness_border[pos], descr.fill_to, getDirection(pos)); - LOG_DEBUG(logger, "min_constr: {}", min_constr); + const Field & border = findBorder(constraints[pos], next_original_row[pos], getDirection(pos)); + LOG_DEBUG(logger, "border: {}", border); - if (!min_constr.isNull() && !equals(row[pos], min_constr)) + if (!border.isNull() && !equals(row[pos], border)) break; } @@ -171,15 +153,10 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) if (pos == row_size) return false; - const auto & pos_descr = getFillDescription(pos); - if (!next_original_row[pos].isNull() && less(next_original_row[pos], row[pos], getDirection(pos))) return false; - if (!staleness_border[pos].isNull() && !less(row[pos], staleness_border[pos], getDirection(pos))) - return false; - - if (!pos_descr.fill_to.isNull() && !less(row[pos], pos_descr.fill_to, getDirection(pos))) + if (!constraints[pos].isNull() && !less(row[pos], constraints[pos], getDirection(pos))) return false; /// If we have any 'fill_to' value at position greater than 'pos' or configured staleness, @@ -191,16 +168,13 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) if (row[i].isNull()) continue; - if (fill_column_desc.fill_to.isNull() && staleness_border[i].isNull()) + if (constraints[i].isNull()) continue; Field next_value = row[i]; fill_column_desc.step_func(next_value, 1); - if (!staleness_border[i].isNull() && !less(next_value, staleness_border[i], getDirection(i))) - continue; - - if (!fill_column_desc.fill_to.isNull() && !less(next_value, fill_column_desc.fill_to, getDirection(i))) + if (!less(next_value, constraints[i], getDirection(i))) continue; row[i] = next_value; @@ -216,10 +190,7 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) if (!next_original_row[pos].isNull() && less(next_original_row[pos], next_value, getDirection(pos))) return false; - if (!staleness_border[pos].isNull() && !less(next_value, staleness_border[pos], getDirection(pos))) - return false; - - if (!pos_descr.fill_to.isNull() && !less(next_value, pos_descr.fill_to, getDirection(pos))) + if (!constraints[pos].isNull() && !less(next_value, constraints[pos], getDirection(pos))) return false; row[pos] = next_value; @@ -236,8 +207,7 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) is_less |= ( (next_original_row[i].isNull() || less(row[i], next_original_row[i], getDirection(i))) && - (staleness_border[i].isNull() || less(row[i], staleness_border[i], getDirection(i))) && - (descr.fill_to.isNull() || less(row[i], descr.fill_to, getDirection(i))) + (constraints[i].isNull() || less(row[i], constraints[i], getDirection(i))) ); } @@ -291,8 +261,7 @@ bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed is_less |= ( (next_original_row[i].isNull() || less(row[i], next_original_row[i], getDirection(i))) && - (staleness_border[i].isNull() || less(row[i], staleness_border[i], getDirection(i))) && - (descr.fill_to.isNull() || less(row[i], descr.fill_to, getDirection(i))) + (constraints[i].isNull() || less(row[i], constraints[i], getDirection(i))) ); } @@ -347,15 +316,20 @@ void FillingRow::initUsingTo(size_t from_pos) row[i] = getFillDescription(i).fill_to; } -void FillingRow::initStalenessRow(const Columns& base_row, size_t row_ind) +void FillingRow::updateConstraintsWithStalenessRow(const Columns& base_row, size_t row_ind) { for (size_t i = 0; i < size(); ++i) { const auto& descr = getFillDescription(i); + constraints[i] = descr.fill_to; + if (!descr.fill_staleness.isNull()) { - staleness_border[i] = (*base_row[i])[row_ind]; - descr.staleness_step_func(staleness_border[i], 1); + Field staleness_border = (*base_row[i])[row_ind]; + descr.staleness_step_func(staleness_border, 1); + + if (constraints[i].isNull() || less(staleness_border, constraints[i], getDirection(i))) + constraints[i] = std::move(staleness_border); } } } diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index d4590d7b81c..edcaba02aa7 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -36,7 +36,7 @@ public: void initUsingFrom(size_t from_pos = 0); void initUsingTo(size_t from_pos = 0); - void initStalenessRow(const Columns& base_row, size_t row_ind); + void updateConstraintsWithStalenessRow(const Columns& base_row, size_t row_ind); Field & operator[](size_t index) { return row[index]; } const Field & operator[](size_t index) const { return row[index]; } @@ -54,7 +54,7 @@ public: private: Row row; - Row staleness_border; + Row constraints; SortDescription sort_description; }; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index f23ffec43de..407a79efb93 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -628,7 +628,7 @@ void FillingTransform::transformRange( } /// Init staleness first interval - filling_row.initStalenessRow(input_fill_columns, range_begin); + filling_row.updateConstraintsWithStalenessRow(input_fill_columns, range_begin); for (size_t row_ind = range_begin; row_ind < range_end; ++row_ind) { @@ -666,7 +666,7 @@ void FillingTransform::transformRange( } /// Initialize staleness border for current row to generate it's prefix - filling_row.initStalenessRow(input_fill_columns, row_ind); + filling_row.updateConstraintsWithStalenessRow(input_fill_columns, row_ind); while (filling_row.shift(next_row, filling_row_changed)) { From acdd9f37d210e4f51d24bbbdf1c34449c89a708c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 17:12:48 +0100 Subject: [PATCH 655/816] Fix tests --- tests/queries/0_stateless/01921_test_progress_bar.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 4199503ba4a..e686698ad9f 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,4 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\\.00 thousand rows, 8\\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\\.[\\w]{3} sec.") - client1.expect("Query peak memory usage: .*B" + end_of_block) + client1.expect("Peak memory usage: .*B" + end_of_block) From b54ae806fe8b6a06536a873ba18959611a4cc8d6 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Wed, 30 Oct 2024 09:16:14 -0700 Subject: [PATCH 656/816] Update README.md - Meetups update Add SF meetup on Dec 12 --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 9d55d1fe9da..dcaeda13acd 100644 --- a/README.md +++ b/README.md @@ -48,6 +48,7 @@ Upcoming meetups * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 * [Amsterdam Meetup](https://www.meetup.com/clickhouse-netherlands-user-group/events/303638814) - December 3 * [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9 +* [San Francisco Meetup](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/304286951/) - December 12 Recently completed meetups From 5b4d55dd3f0ff4393e81a7a36ad092eee46be2c6 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 16:41:02 +0000 Subject: [PATCH 657/816] move logs under flag --- src/Interpreters/FillingRow.cpp | 33 +++++++++---------- .../Transforms/FillingTransform.cpp | 2 +- 2 files changed, 16 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 67827567e04..deb4c765d31 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -10,6 +10,15 @@ namespace DB { +constexpr static bool debug_logging_enabled = true; + +template +static void logDebug(String fmt_str, Args&&... args) +{ + if constexpr (debug_logging_enabled) + LOG_DEBUG(getLogger("FillingRow"), "{}", fmt::format(fmt::runtime(fmt_str), std::forward(args)...)); +} + bool less(const Field & lhs, const Field & rhs, int direction) { if (direction == -1) @@ -104,12 +113,11 @@ bool FillingRow::hasSomeConstraints(size_t pos) const bool FillingRow::isConstraintsComplete(size_t pos) const { - auto logger = getLogger("FillingRow::isConstraintsComplete"); chassert(!row[pos].isNull()); chassert(hasSomeConstraints(pos)); int direction = getDirection(pos); - LOG_DEBUG(logger, "constraint: {}, row: {}, direction: {}", constraints[pos].dump(), row[pos].dump(), direction); + logDebug("constraint: {}, row: {}, direction: {}", constraints[pos].dump(), row[pos].dump(), direction); return less(row[pos], constraints[pos], direction); } @@ -130,7 +138,6 @@ static const Field & findBorder(const Field & constraint, const Field & next_ori bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) { - auto logger = getLogger("FillingRow"); const size_t row_size = size(); size_t pos = 0; @@ -142,13 +149,13 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) continue; const Field & border = findBorder(constraints[pos], next_original_row[pos], getDirection(pos)); - LOG_DEBUG(logger, "border: {}", border); + logDebug("border: {}", border); if (!border.isNull() && !equals(row[pos], border)) break; } - LOG_DEBUG(logger, "pos: {}", pos); + logDebug("pos: {}", pos); if (pos == row_size) return false; @@ -223,8 +230,7 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed) { - auto logger = getLogger("FillingRow::shift"); - LOG_DEBUG(logger, "next_original_row: {}, current: {}", next_original_row.dump(), dump()); + logDebug("next_original_row: {}, current: {}", next_original_row.dump(), dump()); for (size_t pos = 0; pos < size(); ++pos) { @@ -235,16 +241,7 @@ bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed return false; std::optional next_value = doLongJump(getFillDescription(pos), pos, next_original_row[pos]); - - if (!next_value.has_value()) - { - LOG_DEBUG(logger, "next value: {}", "None"); - continue; - } - else - { - LOG_DEBUG(logger, "next value: {}", next_value->dump()); - } + logDebug("jumped to next value: {}", next_value.value_or("Did not complete")); row[pos] = std::move(next_value.value()); @@ -265,7 +262,7 @@ bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed ); } - LOG_DEBUG(logger, "is less: {}", is_less); + logDebug("is less: {}", is_less); value_changed = true; return is_less; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 407a79efb93..81d93a6eadb 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -17,7 +17,7 @@ namespace DB { -constexpr bool debug_logging_enabled = true; +constexpr static bool debug_logging_enabled = true; template static void logDebug(String key, const T & value, const char * separator = " : ") From 82783fe020b83425590ab14949d5b5face7c9fd6 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 16:41:38 +0000 Subject: [PATCH 658/816] disable logs --- src/Interpreters/FillingRow.cpp | 2 +- src/Processors/Transforms/FillingTransform.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index deb4c765d31..3b40c2b6cdd 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -10,7 +10,7 @@ namespace DB { -constexpr static bool debug_logging_enabled = true; +constexpr static bool debug_logging_enabled = false; template static void logDebug(String fmt_str, Args&&... args) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 81d93a6eadb..dc0bafba3e3 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -17,7 +17,7 @@ namespace DB { -constexpr static bool debug_logging_enabled = true; +constexpr static bool debug_logging_enabled = false; template static void logDebug(String key, const T & value, const char * separator = " : ") From b6bd776355171896abb3ef95d2dfdb204799a4b1 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 17:09:35 +0000 Subject: [PATCH 659/816] cleanup --- src/Interpreters/FillingRow.cpp | 8 ++++---- src/Interpreters/FillingRow.h | 4 ++-- src/Processors/Transforms/FillingTransform.cpp | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 3b40c2b6cdd..98c18e9b2ae 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -13,7 +13,7 @@ namespace DB constexpr static bool debug_logging_enabled = false; template -static void logDebug(String fmt_str, Args&&... args) +inline static void logDebug(String fmt_str, Args&&... args) { if constexpr (debug_logging_enabled) LOG_DEBUG(getLogger("FillingRow"), "{}", fmt::format(fmt::runtime(fmt_str), std::forward(args)...)); @@ -111,7 +111,7 @@ bool FillingRow::hasSomeConstraints(size_t pos) const return !constraints[pos].isNull(); } -bool FillingRow::isConstraintsComplete(size_t pos) const +bool FillingRow::isConstraintsSatisfied(size_t pos) const { chassert(!row[pos].isNull()); chassert(hasSomeConstraints(pos)); @@ -288,14 +288,14 @@ bool FillingRow::hasSomeConstraints() const return false; } -bool FillingRow::isConstraintsComplete() const +bool FillingRow::isConstraintsSatisfied() const { for (size_t pos = 0; pos < size(); ++pos) { if (row[pos].isNull() || !hasSomeConstraints(pos)) continue; - return isConstraintsComplete(pos); + return isConstraintsSatisfied(pos); } return true; diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index edcaba02aa7..08d624a2405 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -19,7 +19,7 @@ class FillingRow std::optional doLongJump(const FillColumnDescription & descr, size_t column_ind, const Field & to); bool hasSomeConstraints(size_t pos) const; - bool isConstraintsComplete(size_t pos) const; + bool isConstraintsSatisfied(size_t pos) const; public: explicit FillingRow(const SortDescription & sort_description); @@ -32,7 +32,7 @@ public: bool shift(const FillingRow & next_original_row, bool& value_changed); bool hasSomeConstraints() const; - bool isConstraintsComplete() const; + bool isConstraintsSatisfied() const; void initUsingFrom(size_t from_pos = 0); void initUsingTo(size_t from_pos = 0); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index dc0bafba3e3..a5c6460db0a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -20,7 +20,7 @@ namespace DB constexpr static bool debug_logging_enabled = false; template -static void logDebug(String key, const T & value, const char * separator = " : ") +inline static void logDebug(String key, const T & value, const char * separator = " : ") { if constexpr (debug_logging_enabled) { @@ -511,7 +511,7 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded next_row updated", next_row); - if (!filling_row.hasSomeConstraints() || !filling_row.isConstraintsComplete()) + if (!filling_row.hasSomeConstraints() || !filling_row.isConstraintsSatisfied()) { logDebug("generateSuffixIfNeeded", "will not generate suffix"); return false; @@ -647,7 +647,7 @@ void FillingTransform::transformRange( /// The condition is true when filling row is initialized by value(s) in FILL FROM, /// and there are row(s) in current range with value(s) < then in the filling row. /// It can happen only once for a range. - if (should_insert_first && filling_row < next_row && filling_row.isConstraintsComplete()) + if (should_insert_first && filling_row < next_row && filling_row.isConstraintsSatisfied()) { interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); From c8b94a3c61330fb0649ee92ec69ffe6e6059860b Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 17:21:29 +0000 Subject: [PATCH 660/816] fix empty stream filling --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index a5c6460db0a..4a8965dcfaa 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -503,7 +503,7 @@ bool FillingTransform::generateSuffixIfNeeded( logDebug("generateSuffixIfNeeded next_row", next_row); /// Determines if we should insert filling row before start generating next rows - bool should_insert_first = (next_row < filling_row && !filling_row_inserted) || next_row.isNull(); + bool should_insert_first = (next_row < filling_row && !filling_row_inserted) || (next_row.isNull() && !filling_row.isNull()); logDebug("should_insert_first", should_insert_first); for (size_t i = 0, size = filling_row.size(); i < size; ++i) From a99428fcd9d10da6b6f6fea10d033b485e558b1c Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 17:25:06 +0000 Subject: [PATCH 661/816] add errors test --- .../0_stateless/03266_with_fill_staleness_errors.reference | 0 .../queries/0_stateless/03266_with_fill_staleness_errors.sql | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness_errors.reference create mode 100644 tests/queries/0_stateless/03266_with_fill_staleness_errors.sql diff --git a/tests/queries/0_stateless/03266_with_fill_staleness_errors.reference b/tests/queries/0_stateless/03266_with_fill_staleness_errors.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql b/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql new file mode 100644 index 00000000000..339747e4343 --- /dev/null +++ b/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql @@ -0,0 +1,5 @@ +SET enable_analyzer=1; + +SELECT 1 AS a, 2 AS b ORDER BY a, b WITH FILL FROM 0 TO 10 STALENESS 3; -- { serverError INVALID_WITH_FILL_EXPRESSION } +SELECT 1 AS a, 2 AS b ORDER BY a, b DESC WITH FILL FROM 0 TO 10 STALENESS 3; -- { serverError INVALID_WITH_FILL_EXPRESSION } +SELECT 1 AS a, 2 AS b ORDER BY a, b ASC WITH FILL FROM 0 TO 10 STALENESS -3; -- { serverError INVALID_WITH_FILL_EXPRESSION } From 10088a0947aaf16a3ce1664c422d66daea3324d2 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 17:26:31 +0000 Subject: [PATCH 662/816] extend fuzzer dict with staleness --- tests/fuzz/dictionaries/keywords.dict | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/dictionaries/keywords.dict b/tests/fuzz/dictionaries/keywords.dict index abaaf9e53b5..a37675ebcad 100644 --- a/tests/fuzz/dictionaries/keywords.dict +++ b/tests/fuzz/dictionaries/keywords.dict @@ -538,6 +538,7 @@ "WITH ADMIN OPTION" "WITH CHECK" "WITH FILL" +"STALENESS" "WITH GRANT OPTION" "WITH NAME" "WITH REPLACE OPTION" From e50176c62f18a95648c6b65627b17a095bdccbe5 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 30 Oct 2024 17:29:08 +0000 Subject: [PATCH 663/816] improve test --- .../queries/0_stateless/03266_with_fill_staleness_errors.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql b/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql index 339747e4343..fbfaf3743ca 100644 --- a/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql +++ b/tests/queries/0_stateless/03266_with_fill_staleness_errors.sql @@ -1,5 +1,5 @@ SET enable_analyzer=1; SELECT 1 AS a, 2 AS b ORDER BY a, b WITH FILL FROM 0 TO 10 STALENESS 3; -- { serverError INVALID_WITH_FILL_EXPRESSION } -SELECT 1 AS a, 2 AS b ORDER BY a, b DESC WITH FILL FROM 0 TO 10 STALENESS 3; -- { serverError INVALID_WITH_FILL_EXPRESSION } -SELECT 1 AS a, 2 AS b ORDER BY a, b ASC WITH FILL FROM 0 TO 10 STALENESS -3; -- { serverError INVALID_WITH_FILL_EXPRESSION } +SELECT 1 AS a, 2 AS b ORDER BY a, b DESC WITH FILL TO 10 STALENESS 3; -- { serverError INVALID_WITH_FILL_EXPRESSION } +SELECT 1 AS a, 2 AS b ORDER BY a, b ASC WITH FILL TO 10 STALENESS -3; -- { serverError INVALID_WITH_FILL_EXPRESSION } From 124736756f6b60f915c47e0844214f98590c8574 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 16:18:43 -0300 Subject: [PATCH 664/816] fix msan issue --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 18 +++---- .../Impl/Parquet/ParquetFilterCondition.cpp | 5 ++ .../Impl/Parquet/ParquetFilterCondition.h | 49 +++++++++++++++++++ 3 files changed, 62 insertions(+), 10 deletions(-) create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index fa38a24fd3c..b471989076b 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -296,16 +296,12 @@ void ParquetPlainValuesReader::readBatch( ); } -template <> -void ParquetBitPlainReader::readBatch( +template +void ParquetBitPlainReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) { - auto & column = *assert_cast(col_ptr.get()); - auto cursor = column.size(); - - auto & container = column.getData(); - - container.resize(cursor + num_values); + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData(*assert_cast(col_ptr.get()), cursor + num_values); def_level_reader->visitNullableValues( cursor, @@ -316,11 +312,11 @@ void ParquetBitPlainReader::readBatch( { uint8_t byte; bit_reader->GetValue(1, &byte); - container[nest_cursor] = byte; + column_data[nest_cursor] = byte; }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { - bit_reader->GetBatch(1, &container[nest_cursor], count); + bit_reader->GetBatch(1, &column_data[nest_cursor], count); } ); } @@ -592,6 +588,8 @@ template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader; template class ParquetPlainValuesReader; +template class ParquetBitPlainReader; + template class ParquetFixedLenPlainReader>; template class ParquetFixedLenPlainReader>; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp new file mode 100644 index 00000000000..27be594d3c2 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp @@ -0,0 +1,5 @@ +// +// Created by laptop on 10/29/24. +// + +#include "ParquetFilterCondition.h" diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h new file mode 100644 index 00000000000..a09eaa9ced0 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h @@ -0,0 +1,49 @@ +#pragma once + +#include + +#if USE_PARQUET + +#include + +namespace DB +{ + +class ParquetFilterCondition +{ + struct ConditionElement + { + enum Function + { + /// Atoms of a Boolean expression. + FUNCTION_EQUALS, + FUNCTION_NOT_EQUALS, + FUNCTION_IN, + FUNCTION_NOT_IN, + /// Can take any value. + FUNCTION_UNKNOWN, + /// Operators of the logical expression. + FUNCTION_NOT, + FUNCTION_AND, + FUNCTION_OR, + /// Constants + ALWAYS_FALSE, + ALWAYS_TRUE, + }; + + using ColumnPtr = IColumn::Ptr; + using HashesForColumns = std::vector>; + using KeyColumns = std::vector; + + Function function; + // each entry represents a list of hashes per column + // suppose there are three columns with 2 rows each + // hashes_per_column.size() == 3 and hashes_per_column[0].size() == 2 + HashesForColumns hashes_per_column; + KeyColumns key_columns; + }; +}; + +} + +#endif From f70053d925a0f0980a0f57e9787b4a642f28da1b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 30 Oct 2024 20:14:31 +0100 Subject: [PATCH 665/816] Adapt another test to new error message --- tests/integration/test_peak_memory_usage/test.py | 4 ++-- tests/queries/0_stateless/01383_log_broken_table.sh | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index b4f33b54bbf..51268dcf386 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -93,7 +93,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): client1.send( "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) - client1.expect("Query peak memory usage", timeout=60) + client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) @@ -112,7 +112,7 @@ def test_clickhouse_client_max_peak_memory_single_node(started_cluster): client1.send( "SELECT COUNT(*) FROM (SELECT number FROM numbers(1,300000) INTERSECT SELECT number FROM numbers(10000,1200000))" ) - client1.expect("Query peak memory usage", timeout=60) + client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) peak_memory_usage = get_memory_usage_from_client_output_and_close(client_output) diff --git a/tests/queries/0_stateless/01383_log_broken_table.sh b/tests/queries/0_stateless/01383_log_broken_table.sh index 997daf1bf2f..d3c5a2e9aad 100755 --- a/tests/queries/0_stateless/01383_log_broken_table.sh +++ b/tests/queries/0_stateless/01383_log_broken_table.sh @@ -24,7 +24,7 @@ function test_func() $CLICKHOUSE_CLIENT --query "INSERT INTO log SELECT number, number, number FROM numbers(1000000)" --max_memory_usage $MAX_MEM > "${CLICKHOUSE_TMP}"/insert_result 2>&1 RES=$? - grep -o -F 'Memory limit' "${CLICKHOUSE_TMP}"/insert_result || cat "${CLICKHOUSE_TMP}"/insert_result + grep -o -F 'emory limit' "${CLICKHOUSE_TMP}"/insert_result || cat "${CLICKHOUSE_TMP}"/insert_result $CLICKHOUSE_CLIENT --query "SELECT count(), sum(x + y + z) FROM log" > "${CLICKHOUSE_TMP}"/select_result 2>&1; @@ -36,9 +36,9 @@ function test_func() $CLICKHOUSE_CLIENT --query "DROP TABLE log"; } -test_func TinyLog | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' -test_func StripeLog | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' -test_func Log | grep -v -P '^(Memory limit|0\t0|[1-9]000000\t)' +test_func TinyLog | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' +test_func StripeLog | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' +test_func Log | grep -v -P '^(emory limit|0\t0|[1-9]000000\t)' rm "${CLICKHOUSE_TMP}/insert_result" rm "${CLICKHOUSE_TMP}/select_result" From 0cfbe95ca69d0bb52578c83570b34f4f40de92df Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 30 Oct 2024 21:20:11 +0100 Subject: [PATCH 666/816] Update 03258_multiple_array_joins.sql --- tests/queries/0_stateless/03258_multiple_array_joins.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03258_multiple_array_joins.sql b/tests/queries/0_stateless/03258_multiple_array_joins.sql index 5afe7725d3f..ddfac1da080 100644 --- a/tests/queries/0_stateless/03258_multiple_array_joins.sql +++ b/tests/queries/0_stateless/03258_multiple_array_joins.sql @@ -1,3 +1,4 @@ +SET enable_analyzer = 1; DROP TABLE IF EXISTS test_multiple_array_join; CREATE TABLE test_multiple_array_join ( From d24b029e45f5dcd2a57af8b3609c092327250632 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 29 Oct 2024 17:50:56 +0100 Subject: [PATCH 667/816] Add support for chrono data types to the "fmt" formatter. --- base/base/chrono_io.h | 47 ++++++++++++++++++++++++++++++++----------- 1 file changed, 35 insertions(+), 12 deletions(-) diff --git a/base/base/chrono_io.h b/base/base/chrono_io.h index 4ee8dec6634..d55aa11bc1d 100644 --- a/base/base/chrono_io.h +++ b/base/base/chrono_io.h @@ -4,6 +4,7 @@ #include #include #include +#include inline std::string to_string(const std::time_t & time) @@ -11,18 +12,6 @@ inline std::string to_string(const std::time_t & time) return cctz::format("%Y-%m-%d %H:%M:%S", std::chrono::system_clock::from_time_t(time), cctz::local_time_zone()); } -template -std::string to_string(const std::chrono::time_point & tp) -{ - // Don't use DateLUT because it shows weird characters for - // TimePoint::max(). I wish we could use C++20 format, but it's not - // there yet. - // return DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(tp)); - - auto in_time_t = std::chrono::system_clock::to_time_t(tp); - return to_string(in_time_t); -} - template > std::string to_string(const std::chrono::duration & duration) { @@ -33,6 +22,20 @@ std::string to_string(const std::chrono::duration & duration) return std::to_string(seconds_as_double.count()) + "s"; } +template +std::string to_string(const std::chrono::time_point & tp) +{ + // Don't use DateLUT because it shows weird characters for + // TimePoint::max(). I wish we could use C++20 format, but it's not + // there yet. + // return DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(tp)); + + if constexpr (std::is_same_v) + return to_string(std::chrono::system_clock::to_time_t(tp)); + else + return to_string(tp.time_since_epoch()); +} + template std::ostream & operator<<(std::ostream & o, const std::chrono::time_point & tp) { @@ -44,3 +47,23 @@ std::ostream & operator<<(std::ostream & o, const std::chrono::duration +struct fmt::formatter> : fmt::formatter +{ + template + auto format(const std::chrono::time_point & tp, FormatCtx & ctx) const + { + return fmt::formatter::format(::to_string(tp), ctx); + } +}; + +template +struct fmt::formatter> : fmt::formatter +{ + template + auto format(const std::chrono::duration & duration, FormatCtx & ctx) const + { + return fmt::formatter::format(::to_string(duration), ctx); + } +}; From 31402c5840a05a156ee6c5bb1942f42e27578052 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 29 Oct 2024 17:52:36 +0100 Subject: [PATCH 668/816] Add support for a custom cancellation exception to QueryStatus::cancelQuery(). --- src/Interpreters/ProcessList.cpp | 23 ++++++++++++++++++----- src/Interpreters/ProcessList.h | 9 ++++++++- 2 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 177468f1c8b..7a9b8566c77 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -447,12 +447,16 @@ void QueryStatus::ExecutorHolder::remove() executor = nullptr; } -CancellationCode QueryStatus::cancelQuery(bool) +CancellationCode QueryStatus::cancelQuery(bool /* kill */, std::exception_ptr exception) { - if (is_killed.load()) + if (is_killed.exchange(true)) return CancellationCode::CancelSent; - is_killed.store(true); + { + std::lock_guard lock{cancellation_exception_mutex}; + if (!cancellation_exception) + cancellation_exception = exception; + } std::vector executors_snapshot; @@ -486,7 +490,7 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) /// addPipelineExecutor() from the cancelQuery() context, and this will /// lead to deadlock. if (is_killed.load()) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + throwQueryWasCancelled(); std::lock_guard lock(executors_mutex); assert(!executors.contains(e)); @@ -512,11 +516,20 @@ void QueryStatus::removePipelineExecutor(PipelineExecutor * e) bool QueryStatus::checkTimeLimit() { if (is_killed.load()) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + throwQueryWasCancelled(); return limits.checkTimeLimit(watch, overflow_mode); } +void QueryStatus::throwQueryWasCancelled() const +{ + std::lock_guard lock{cancellation_exception_mutex}; + if (cancellation_exception) + std::rethrow_exception(cancellation_exception); + else + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); +} + bool QueryStatus::checkTimeLimitSoft() { if (is_killed.load()) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index b2583e74d9b..f171fe8f4d4 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -109,6 +109,9 @@ protected: /// KILL was send to the query std::atomic is_killed { false }; + std::exception_ptr cancellation_exception TSA_GUARDED_BY(cancellation_exception_mutex); + mutable std::mutex cancellation_exception_mutex; + /// All data to the client already had been sent. /// Including EndOfStream or Exception. std::atomic is_all_data_sent { false }; @@ -127,6 +130,8 @@ protected: /// A weak pointer is used here because it's a ProcessListEntry which owns this QueryStatus, and not vice versa. void setProcessListEntry(std::weak_ptr process_list_entry_); + [[noreturn]] void throwQueryWasCancelled() const; + mutable std::mutex executors_mutex; struct ExecutorHolder @@ -225,7 +230,9 @@ public: QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const; - CancellationCode cancelQuery(bool kill); + /// Cancels the current query. + /// Optional argument `exception` allows to set an exception which checkTimeLimit() will throw instead of "QUERY_WAS_CANCELLED". + CancellationCode cancelQuery(bool kill, std::exception_ptr exception = nullptr); bool isKilled() const { return is_killed; } From 8fea878834ca5d715284048a820a23b56dcd4f46 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 29 Oct 2024 17:56:55 +0100 Subject: [PATCH 669/816] Make configurable the number of retries used by ZooKeeper when connecting. --- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 4 ++++ src/Common/ZooKeeper/ZooKeeperArgs.h | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +++- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index cdc9a1afe4c..c488d829b9d 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -176,6 +176,10 @@ void ZooKeeperArgs::initFromKeeperSection(const Poco::Util::AbstractConfiguratio { connection_timeout_ms = config.getInt(config_name + "." + key); } + else if (key == "num_connection_retries") + { + num_connection_retries = config.getInt(config_name + "." + key); + } else if (key == "enable_fault_injections_during_startup") { enable_fault_injections_during_startup = config.getBool(config_name + "." + key); diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.h b/src/Common/ZooKeeper/ZooKeeperArgs.h index 3754c2f7aac..e790e578808 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.h +++ b/src/Common/ZooKeeper/ZooKeeperArgs.h @@ -39,6 +39,7 @@ struct ZooKeeperArgs String sessions_path = "/clickhouse/sessions"; String client_availability_zone; int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; + UInt64 num_connection_retries = 2; int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; bool enable_fault_injections_during_startup = false; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 173f37c3454..7b027f48d4b 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -440,7 +440,9 @@ void ZooKeeper::connect( if (nodes.empty()) throw Exception::fromMessage(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); - static constexpr size_t num_tries = 3; + /// We always have at least one attempt to connect. + size_t num_tries = args.num_connection_retries + 1; + bool connected = false; bool dns_error = false; From 982b67fb22b0bb0a508595624096cb23da4dc357 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 29 Oct 2024 20:20:08 +0100 Subject: [PATCH 670/816] Add support for zookeeper retries to executeDDLQueryOnCluster(). --- src/Common/ZooKeeper/ZooKeeperRetries.h | 12 ++++++++--- src/Databases/DatabaseReplicatedWorker.cpp | 3 +-- src/Databases/DatabaseReplicatedWorker.h | 2 +- src/Interpreters/DDLWorker.cpp | 21 ++++++++++++++++++- src/Interpreters/DDLWorker.h | 8 ++++++- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Interpreters/executeDDLQueryOnCluster.h | 3 +++ 7 files changed, 42 insertions(+), 9 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperRetries.h b/src/Common/ZooKeeper/ZooKeeperRetries.h index b5b03971385..acea521a7ce 100644 --- a/src/Common/ZooKeeper/ZooKeeperRetries.h +++ b/src/Common/ZooKeeper/ZooKeeperRetries.h @@ -15,14 +15,15 @@ namespace ErrorCodes struct ZooKeeperRetriesInfo { + ZooKeeperRetriesInfo() = default; ZooKeeperRetriesInfo(UInt64 max_retries_, UInt64 initial_backoff_ms_, UInt64 max_backoff_ms_) : max_retries(max_retries_), initial_backoff_ms(std::min(initial_backoff_ms_, max_backoff_ms_)), max_backoff_ms(max_backoff_ms_) { } - UInt64 max_retries; - UInt64 initial_backoff_ms; - UInt64 max_backoff_ms; + UInt64 max_retries = 0; /// "max_retries = 0" means only one attempt. + UInt64 initial_backoff_ms = 100; + UInt64 max_backoff_ms = 5000; }; class ZooKeeperRetriesControl @@ -220,6 +221,7 @@ private: return false; } + /// Check if the query was cancelled. if (process_list_element) process_list_element->checkTimeLimit(); @@ -228,6 +230,10 @@ private: sleepForMilliseconds(current_backoff_ms); current_backoff_ms = std::min(current_backoff_ms * 2, retries_info.max_backoff_ms); + /// Check if the query was cancelled again after sleeping. + if (process_list_element) + process_list_element->checkTimeLimit(); + return true; } diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 5d75dff391a..6a711c92332 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -199,13 +199,12 @@ void DatabaseReplicatedDDLWorker::initializeReplication() active_node_holder = zkutil::EphemeralNodeHolder::existing(active_path, *active_node_holder_zookeeper); } -String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) +String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry, const ZooKeeperRetriesInfo &, QueryStatusPtr) { auto zookeeper = getAndSetZooKeeper(); return enqueueQueryImpl(zookeeper, entry, database); } - bool DatabaseReplicatedDDLWorker::waitForReplicaToProcessAllEntries(UInt64 timeout_ms) { auto zookeeper = getAndSetZooKeeper(); diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h index b690854e249..d2385cbdba3 100644 --- a/src/Databases/DatabaseReplicatedWorker.h +++ b/src/Databases/DatabaseReplicatedWorker.h @@ -24,7 +24,7 @@ class DatabaseReplicatedDDLWorker : public DDLWorker public: DatabaseReplicatedDDLWorker(DatabaseReplicated * db, ContextPtr context_); - String enqueueQuery(DDLLogEntry & entry) override; + String enqueueQuery(DDLLogEntry & entry, const ZooKeeperRetriesInfo &, QueryStatusPtr) override; String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, ContextPtr query_context); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 1be1a0c9bb9..eaba46f5d48 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -1053,7 +1054,25 @@ void DDLWorker::createStatusDirs(const std::string & node_path, const ZooKeeperP } -String DDLWorker::enqueueQuery(DDLLogEntry & entry) +String DDLWorker::enqueueQuery(DDLLogEntry & entry, const ZooKeeperRetriesInfo & retries_info, QueryStatusPtr process_list_element) +{ + String node_path; + if (retries_info.max_retries > 0) + { + ZooKeeperRetriesControl retries_ctl{"DDLWorker::enqueueQuery", log, retries_info, process_list_element}; + retries_ctl.retryLoop([&]{ + node_path = enqueueQueryAttempt(entry); + }); + } + else + { + node_path = enqueueQueryAttempt(entry); + } + return node_path; +} + + +String DDLWorker::enqueueQueryAttempt(DDLLogEntry & entry) { if (entry.hosts.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty host list in a distributed DDL task"); diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index ee17714add9..a5f47a51bb3 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -48,6 +48,9 @@ struct DDLTaskBase; using DDLTaskPtr = std::unique_ptr; using ZooKeeperPtr = std::shared_ptr; class AccessRightsElements; +struct ZooKeeperRetriesInfo; +class QueryStatus; +using QueryStatusPtr = std::shared_ptr; class DDLWorker { @@ -65,7 +68,7 @@ public: virtual ~DDLWorker(); /// Pushes query into DDL queue, returns path to created node - virtual String enqueueQuery(DDLLogEntry & entry); + virtual String enqueueQuery(DDLLogEntry & entry, const ZooKeeperRetriesInfo & retries_info, QueryStatusPtr process_list_element); /// Host ID (name:port) for logging purposes /// Note that in each task hosts are identified individually by name:port from initiator server cluster config @@ -120,6 +123,9 @@ protected: mutable std::shared_mutex mtx; }; + /// Pushes query into DDL queue, returns path to created node + String enqueueQueryAttempt(DDLLogEntry & entry); + /// Iterates through queue tasks in ZooKeeper, runs execution of new tasks void scheduleTasks(bool reinitialized); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c0440c755ad..0b88d07148c 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -189,7 +189,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, entry.setSettingsIfRequired(context); entry.tracing_context = OpenTelemetry::CurrentContext(); entry.initial_query_id = context->getClientInfo().initial_query_id; - String node_path = ddl_worker.enqueueQuery(entry); + String node_path = ddl_worker.enqueueQuery(entry, params.retries_info, context->getProcessListElement()); return getDDLOnClusterStatus(node_path, ddl_worker.getReplicasDir(), entry, context); } diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index d015e8d8694..69e0c38834e 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -37,6 +37,9 @@ struct DDLQueryOnClusterParams /// Privileges which the current user should have to execute a query. AccessRightsElements access_to_check; + + /// Use retries when creating nodes "query-0000000000", "query-0000000001", "query-0000000002" in ZooKeeper. + ZooKeeperRetriesInfo retries_info; }; /// Pushes distributed DDL query to the queue. From f6b5d27c58895f2e39fe3c6b747170f50f524ad3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 29 Oct 2024 21:55:17 +0100 Subject: [PATCH 671/816] Rework coordination of hosts during BACKUP ON CLUSTER / RESTORE ON CLUSTER. Fix concurrency check, implement cancelling of distributed backups/restores. --- src/Backups/BackupConcurrencyCheck.cpp | 135 ++ src/Backups/BackupConcurrencyCheck.h | 55 + src/Backups/BackupCoordinationCleaner.cpp | 64 + src/Backups/BackupCoordinationCleaner.h | 40 + src/Backups/BackupCoordinationLocal.cpp | 38 +- src/Backups/BackupCoordinationLocal.h | 32 +- ...te.cpp => BackupCoordinationOnCluster.cpp} | 309 ++--- ...Remote.h => BackupCoordinationOnCluster.h} | 67 +- src/Backups/BackupCoordinationStage.h | 8 - src/Backups/BackupCoordinationStageSync.cpp | 1205 ++++++++++++++--- src/Backups/BackupCoordinationStageSync.h | 189 ++- src/Backups/BackupEntriesCollector.cpp | 17 +- src/Backups/BackupEntriesCollector.h | 4 - src/Backups/BackupIO.h | 5 + src/Backups/BackupIO_AzureBlobStorage.h | 1 + src/Backups/BackupIO_Disk.cpp | 28 +- src/Backups/BackupIO_Disk.h | 2 + src/Backups/BackupIO_File.cpp | 28 +- src/Backups/BackupIO_File.h | 2 + src/Backups/BackupIO_S3.h | 1 + src/Backups/BackupImpl.cpp | 82 +- src/Backups/BackupImpl.h | 6 +- src/Backups/BackupKeeperSettings.cpp | 58 + src/Backups/BackupKeeperSettings.h | 64 + src/Backups/BackupSettings.cpp | 11 + src/Backups/BackupSettings.h | 2 + src/Backups/BackupsWorker.cpp | 924 ++++++------- src/Backups/BackupsWorker.h | 47 +- src/Backups/IBackup.h | 9 +- src/Backups/IBackupCoordination.h | 36 +- src/Backups/IRestoreCoordination.h | 36 +- src/Backups/RestoreCoordinationLocal.cpp | 34 +- src/Backups/RestoreCoordinationLocal.h | 27 +- src/Backups/RestoreCoordinationOnCluster.cpp | 318 +++++ ...emote.h => RestoreCoordinationOnCluster.h} | 55 +- src/Backups/RestoreCoordinationRemote.cpp | 379 ------ src/Backups/RestorerFromBackup.cpp | 28 +- src/Backups/RestorerFromBackup.h | 5 +- src/Backups/WithRetries.cpp | 57 +- src/Backups/WithRetries.h | 32 +- src/Common/Exception.cpp | 4 +- src/Common/Exception.h | 2 +- src/Core/Settings.cpp | 37 +- src/Core/SettingsChangesHistory.cpp | 5 + src/Interpreters/InterpreterBackupQuery.cpp | 21 +- src/Storages/StorageKeeperMap.cpp | 14 +- tests/integration/helpers/cluster.py | 13 + tests/integration/helpers/config_manager.py | 65 + .../configs/faster_zk_disconnect_detect.xml | 12 + .../configs/lesser_timeouts.xml | 2 +- .../configs/shutdown_cancel_backups.xml | 3 + .../configs/slow_backups.xml | 7 + .../configs/zookeeper_retries.xml | 9 +- .../test_backup_restore_on_cluster/test.py | 2 +- .../test_cancel_backup.py | 780 +++++++++++ .../test_disallow_concurrency.py | 4 +- 56 files changed, 3849 insertions(+), 1571 deletions(-) create mode 100644 src/Backups/BackupConcurrencyCheck.cpp create mode 100644 src/Backups/BackupConcurrencyCheck.h create mode 100644 src/Backups/BackupCoordinationCleaner.cpp create mode 100644 src/Backups/BackupCoordinationCleaner.h rename src/Backups/{BackupCoordinationRemote.cpp => BackupCoordinationOnCluster.cpp} (73%) rename src/Backups/{BackupCoordinationRemote.h => BackupCoordinationOnCluster.h} (67%) create mode 100644 src/Backups/BackupKeeperSettings.cpp create mode 100644 src/Backups/BackupKeeperSettings.h create mode 100644 src/Backups/RestoreCoordinationOnCluster.cpp rename src/Backups/{RestoreCoordinationRemote.h => RestoreCoordinationOnCluster.h} (62%) delete mode 100644 src/Backups/RestoreCoordinationRemote.cpp create mode 100644 tests/integration/helpers/config_manager.py create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/faster_zk_disconnect_detect.xml create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/shutdown_cancel_backups.xml create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/slow_backups.xml create mode 100644 tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py diff --git a/src/Backups/BackupConcurrencyCheck.cpp b/src/Backups/BackupConcurrencyCheck.cpp new file mode 100644 index 00000000000..8b29ae41b53 --- /dev/null +++ b/src/Backups/BackupConcurrencyCheck.cpp @@ -0,0 +1,135 @@ +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CONCURRENT_ACCESS_NOT_SUPPORTED; +} + + +BackupConcurrencyCheck::BackupConcurrencyCheck( + const UUID & backup_or_restore_uuid_, + bool is_restore_, + bool on_cluster_, + bool allow_concurrency_, + BackupConcurrencyCounters & counters_) + : is_restore(is_restore_), backup_or_restore_uuid(backup_or_restore_uuid_), on_cluster(on_cluster_), counters(counters_) +{ + std::lock_guard lock{counters.mutex}; + + if (!allow_concurrency_) + { + bool found_concurrent_operation = false; + if (is_restore) + { + size_t num_local_restores = counters.local_restores; + size_t num_on_cluster_restores = counters.on_cluster_restores.size(); + if (on_cluster) + { + if (!counters.on_cluster_restores.contains(backup_or_restore_uuid)) + ++num_on_cluster_restores; + } + else + { + ++num_local_restores; + } + found_concurrent_operation = (num_local_restores + num_on_cluster_restores > 1); + } + else + { + size_t num_local_backups = counters.local_backups; + size_t num_on_cluster_backups = counters.on_cluster_backups.size(); + if (on_cluster) + { + if (!counters.on_cluster_backups.contains(backup_or_restore_uuid)) + ++num_on_cluster_backups; + } + else + { + ++num_local_backups; + } + found_concurrent_operation = (num_local_backups + num_on_cluster_backups > 1); + } + + if (found_concurrent_operation) + throwConcurrentOperationNotAllowed(is_restore); + } + + if (on_cluster) + { + if (is_restore) + ++counters.on_cluster_restores[backup_or_restore_uuid]; + else + ++counters.on_cluster_backups[backup_or_restore_uuid]; + } + else + { + if (is_restore) + ++counters.local_restores; + else + ++counters.local_backups; + } +} + + +BackupConcurrencyCheck::~BackupConcurrencyCheck() +{ + std::lock_guard lock{counters.mutex}; + + if (on_cluster) + { + if (is_restore) + { + auto it = counters.on_cluster_restores.find(backup_or_restore_uuid); + if (it != counters.on_cluster_restores.end()) + { + if (!--it->second) + counters.on_cluster_restores.erase(it); + } + } + else + { + auto it = counters.on_cluster_backups.find(backup_or_restore_uuid); + if (it != counters.on_cluster_backups.end()) + { + if (!--it->second) + counters.on_cluster_backups.erase(it); + } + } + } + else + { + if (is_restore) + --counters.local_restores; + else + --counters.local_backups; + } +} + + +void BackupConcurrencyCheck::throwConcurrentOperationNotAllowed(bool is_restore) +{ + throw Exception( + ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, + "Concurrent {} are not allowed, turn on setting '{}'", + is_restore ? "restores" : "backups", + is_restore ? "allow_concurrent_restores" : "allow_concurrent_backups"); +} + + +BackupConcurrencyCounters::BackupConcurrencyCounters() = default; + + +BackupConcurrencyCounters::~BackupConcurrencyCounters() +{ + if (local_backups > 0 || local_restores > 0 || !on_cluster_backups.empty() || !on_cluster_restores.empty()) + LOG_ERROR(getLogger(__PRETTY_FUNCTION__), "Some backups or restores are processing"); +} + +} diff --git a/src/Backups/BackupConcurrencyCheck.h b/src/Backups/BackupConcurrencyCheck.h new file mode 100644 index 00000000000..048a23a716a --- /dev/null +++ b/src/Backups/BackupConcurrencyCheck.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class BackupConcurrencyCounters; + +/// Local checker for concurrent BACKUP or RESTORE operations. +/// This class is used by implementations of IBackupCoordination and IRestoreCoordination +/// to throw an exception if concurrent backups or restores are not allowed. +class BackupConcurrencyCheck +{ +public: + /// Checks concurrency of a BACKUP operation or a RESTORE operation. + /// Keep a constructed instance of BackupConcurrencyCheck until the operation is done. + BackupConcurrencyCheck( + const UUID & backup_or_restore_uuid_, + bool is_restore_, + bool on_cluster_, + bool allow_concurrency_, + BackupConcurrencyCounters & counters_); + + ~BackupConcurrencyCheck(); + + [[noreturn]] static void throwConcurrentOperationNotAllowed(bool is_restore); + +private: + const bool is_restore; + const UUID backup_or_restore_uuid; + const bool on_cluster; + BackupConcurrencyCounters & counters; +}; + + +class BackupConcurrencyCounters +{ +public: + BackupConcurrencyCounters(); + ~BackupConcurrencyCounters(); + +private: + friend class BackupConcurrencyCheck; + size_t local_backups TSA_GUARDED_BY(mutex) = 0; + size_t local_restores TSA_GUARDED_BY(mutex) = 0; + std::unordered_map on_cluster_backups TSA_GUARDED_BY(mutex); + std::unordered_map on_cluster_restores TSA_GUARDED_BY(mutex); + std::mutex mutex; +}; + +} diff --git a/src/Backups/BackupCoordinationCleaner.cpp b/src/Backups/BackupCoordinationCleaner.cpp new file mode 100644 index 00000000000..1f5068a94de --- /dev/null +++ b/src/Backups/BackupCoordinationCleaner.cpp @@ -0,0 +1,64 @@ +#include + + +namespace DB +{ + +BackupCoordinationCleaner::BackupCoordinationCleaner(const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_) + : zookeeper_path(zookeeper_path_), with_retries(with_retries_), log(log_) +{ +} + +void BackupCoordinationCleaner::cleanup() +{ + tryRemoveAllNodes(/* throw_if_error = */ true, /* retries_kind = */ WithRetries::kNormal); +} + +bool BackupCoordinationCleaner::tryCleanupAfterError() noexcept +{ + return tryRemoveAllNodes(/* throw_if_error = */ false, /* retries_kind = */ WithRetries::kNormal); +} + +bool BackupCoordinationCleaner::tryRemoveAllNodes(bool throw_if_error, WithRetries::Kind retries_kind) +{ + { + std::lock_guard lock{mutex}; + if (cleanup_result.succeeded) + return true; + if (cleanup_result.exception) + { + if (throw_if_error) + std::rethrow_exception(cleanup_result.exception); + return false; + } + } + + try + { + LOG_TRACE(log, "Removing nodes from ZooKeeper"); + auto holder = with_retries.createRetriesControlHolder("removeAllNodes", retries_kind); + holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zookeeper); + zookeeper->removeRecursive(zookeeper_path); + }); + + std::lock_guard lock{mutex}; + cleanup_result.succeeded = true; + return true; + } + catch (...) + { + LOG_TRACE(log, "Caught exception while removing nodes from ZooKeeper for this restore: {}", + getCurrentExceptionMessage(/* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true)); + + std::lock_guard lock{mutex}; + cleanup_result.exception = std::current_exception(); + + if (throw_if_error) + throw; + return false; + } +} + +} diff --git a/src/Backups/BackupCoordinationCleaner.h b/src/Backups/BackupCoordinationCleaner.h new file mode 100644 index 00000000000..43e095d9f33 --- /dev/null +++ b/src/Backups/BackupCoordinationCleaner.h @@ -0,0 +1,40 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Removes all the nodes from ZooKeeper used to coordinate a BACKUP ON CLUSTER operation or +/// a RESTORE ON CLUSTER operation (successful or not). +/// This class is used by BackupCoordinationOnCluster and RestoreCoordinationOnCluster to cleanup. +class BackupCoordinationCleaner +{ +public: + BackupCoordinationCleaner(const String & zookeeper_path_, const WithRetries & with_retries_, LoggerPtr log_); + + void cleanup(); + bool tryCleanupAfterError() noexcept; + +private: + bool tryRemoveAllNodes(bool throw_if_error, WithRetries::Kind retries_kind); + + const String zookeeper_path; + + /// A reference to a field of the parent object which is either BackupCoordinationOnCluster or RestoreCoordinationOnCluster. + const WithRetries & with_retries; + + const LoggerPtr log; + + struct CleanupResult + { + bool succeeded = false; + std::exception_ptr exception; + }; + CleanupResult cleanup_result TSA_GUARDED_BY(mutex); + + std::mutex mutex; +}; + +} diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index efdc18cc29c..8bd6b4d327d 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -1,5 +1,7 @@ #include + #include +#include #include #include #include @@ -8,27 +10,20 @@ namespace DB { -BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_) - : log(getLogger("BackupCoordinationLocal")), file_infos(plain_backup_) +BackupCoordinationLocal::BackupCoordinationLocal( + const UUID & backup_uuid_, + bool is_plain_backup_, + bool allow_concurrent_backup_, + BackupConcurrencyCounters & concurrency_counters_) + : log(getLogger("BackupCoordinationLocal")) + , concurrency_check(backup_uuid_, /* is_restore = */ false, /* on_cluster = */ false, allow_concurrent_backup_, concurrency_counters_) + , file_infos(is_plain_backup_) { } BackupCoordinationLocal::~BackupCoordinationLocal() = default; -void BackupCoordinationLocal::setStage(const String &, const String &) -{ -} - -void BackupCoordinationLocal::setError(const Exception &) -{ -} - -Strings BackupCoordinationLocal::waitForStage(const String &) -{ - return {}; -} - -Strings BackupCoordinationLocal::waitForStage(const String &, std::chrono::milliseconds) +ZooKeeperRetriesInfo BackupCoordinationLocal::getOnClusterInitializationKeeperRetriesInfo() const { return {}; } @@ -135,15 +130,4 @@ bool BackupCoordinationLocal::startWritingFile(size_t data_file_index) return writing_files.emplace(data_file_index).second; } - -bool BackupCoordinationLocal::hasConcurrentBackups(const std::atomic & num_active_backups) const -{ - if (num_active_backups > 1) - { - LOG_WARNING(log, "Found concurrent backups: num_active_backups={}", num_active_backups); - return true; - } - return false; -} - } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index a7f15c79649..09991c0d301 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -21,13 +22,21 @@ namespace DB class BackupCoordinationLocal : public IBackupCoordination { public: - explicit BackupCoordinationLocal(bool plain_backup_); + explicit BackupCoordinationLocal( + const UUID & backup_uuid_, + bool is_plain_backup_, + bool allow_concurrent_backup_, + BackupConcurrencyCounters & concurrency_counters_); + ~BackupCoordinationLocal() override; - void setStage(const String & new_stage, const String & message) override; - void setError(const Exception & exception) override; - Strings waitForStage(const String & stage_to_wait) override; - Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; + Strings setStage(const String &, const String &, bool) override { return {}; } + void setBackupQueryWasSentToOtherHosts() override {} + bool trySetError(std::exception_ptr) override { return true; } + void finish() override {} + bool tryFinishAfterError() noexcept override { return true; } + void waitForOtherHostsToFinish() override {} + bool tryWaitForOtherHostsToFinishAfterError() noexcept override { return true; } void addReplicatedPartNames(const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, const std::vector & part_names_and_checksums) override; @@ -54,17 +63,18 @@ public: BackupFileInfos getFileInfosForAllHosts() const override; bool startWritingFile(size_t data_file_index) override; - bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; + ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const override; private: LoggerPtr const log; + BackupConcurrencyCheck concurrency_check; - BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables; - BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access; - BackupCoordinationReplicatedSQLObjects TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects; - BackupCoordinationFileInfos TSA_GUARDED_BY(file_infos_mutex) file_infos; + BackupCoordinationReplicatedTables replicated_tables TSA_GUARDED_BY(replicated_tables_mutex); + BackupCoordinationReplicatedAccess replicated_access TSA_GUARDED_BY(replicated_access_mutex); + BackupCoordinationReplicatedSQLObjects replicated_sql_objects TSA_GUARDED_BY(replicated_sql_objects_mutex); + BackupCoordinationFileInfos file_infos TSA_GUARDED_BY(file_infos_mutex); BackupCoordinationKeeperMapTables keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); - std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; + std::unordered_set writing_files TSA_GUARDED_BY(writing_files_mutex); mutable std::mutex replicated_tables_mutex; mutable std::mutex replicated_access_mutex; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationOnCluster.cpp similarity index 73% rename from src/Backups/BackupCoordinationRemote.cpp rename to src/Backups/BackupCoordinationOnCluster.cpp index a60ac0c636f..dc34939f805 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationOnCluster.cpp @@ -1,7 +1,4 @@ -#include - -#include -#include +#include #include #include @@ -26,8 +23,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace Stage = BackupCoordinationStage; - namespace { using PartNameAndChecksum = IBackupCoordination::PartNameAndChecksum; @@ -149,144 +144,152 @@ namespace }; } -size_t BackupCoordinationRemote::findCurrentHostIndex(const Strings & all_hosts, const String & current_host) +Strings BackupCoordinationOnCluster::excludeInitiator(const Strings & all_hosts) +{ + Strings all_hosts_without_initiator = all_hosts; + bool has_initiator = (std::erase(all_hosts_without_initiator, kInitiator) > 0); + chassert(has_initiator); + return all_hosts_without_initiator; +} + +size_t BackupCoordinationOnCluster::findCurrentHostIndex(const String & current_host, const Strings & all_hosts) { auto it = std::find(all_hosts.begin(), all_hosts.end(), current_host); if (it == all_hosts.end()) - return 0; + return all_hosts.size(); return it - all_hosts.begin(); } -BackupCoordinationRemote::BackupCoordinationRemote( - zkutil::GetZooKeeper get_zookeeper_, + +BackupCoordinationOnCluster::BackupCoordinationOnCluster( + const UUID & backup_uuid_, + bool is_plain_backup_, const String & root_zookeeper_path_, + zkutil::GetZooKeeper get_zookeeper_, const BackupKeeperSettings & keeper_settings_, - const String & backup_uuid_, - const Strings & all_hosts_, const String & current_host_, - bool plain_backup_, - bool is_internal_, + const Strings & all_hosts_, + bool allow_concurrent_backup_, + BackupConcurrencyCounters & concurrency_counters_, + ThreadPoolCallbackRunnerUnsafe schedule_, QueryStatusPtr process_list_element_) : root_zookeeper_path(root_zookeeper_path_) - , zookeeper_path(root_zookeeper_path_ + "/backup-" + backup_uuid_) + , zookeeper_path(root_zookeeper_path_ + "/backup-" + toString(backup_uuid_)) , keeper_settings(keeper_settings_) , backup_uuid(backup_uuid_) , all_hosts(all_hosts_) + , all_hosts_without_initiator(excludeInitiator(all_hosts)) , current_host(current_host_) - , current_host_index(findCurrentHostIndex(all_hosts, current_host)) - , plain_backup(plain_backup_) - , is_internal(is_internal_) - , log(getLogger("BackupCoordinationRemote")) - , with_retries( - log, - get_zookeeper_, - keeper_settings, - process_list_element_, - [my_zookeeper_path = zookeeper_path, my_current_host = current_host, my_is_internal = is_internal] - (WithRetries::FaultyKeeper & zk) - { - /// Recreate this ephemeral node to signal that we are alive. - if (my_is_internal) - { - String alive_node_path = my_zookeeper_path + "/stage/alive|" + my_current_host; - - /// Delete the ephemeral node from the previous connection so we don't have to wait for keeper to do it automatically. - zk->tryRemove(alive_node_path); - - zk->createAncestors(alive_node_path); - zk->create(alive_node_path, "", zkutil::CreateMode::Ephemeral); - } - }) + , current_host_index(findCurrentHostIndex(current_host, all_hosts)) + , plain_backup(is_plain_backup_) + , log(getLogger("BackupCoordinationOnCluster")) + , with_retries(log, get_zookeeper_, keeper_settings, process_list_element_, [root_zookeeper_path_](Coordination::ZooKeeperWithFaultInjection::Ptr zk) { zk->sync(root_zookeeper_path_); }) + , concurrency_check(backup_uuid_, /* is_restore = */ false, /* on_cluster = */ true, allow_concurrent_backup_, concurrency_counters_) + , stage_sync(/* is_restore = */ false, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_backup_, with_retries, schedule_, process_list_element_, log) + , cleaner(zookeeper_path, with_retries, log) { createRootNodes(); - - stage_sync.emplace( - zookeeper_path, - with_retries, - log); } -BackupCoordinationRemote::~BackupCoordinationRemote() +BackupCoordinationOnCluster::~BackupCoordinationOnCluster() { - try - { - if (!is_internal) - removeAllNodes(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + tryFinishImpl(); } -void BackupCoordinationRemote::createRootNodes() +void BackupCoordinationOnCluster::createRootNodes() { - auto holder = with_retries.createRetriesControlHolder("createRootNodes"); + auto holder = with_retries.createRetriesControlHolder("createRootNodes", WithRetries::kInitialization); holder.retries_ctl.retryLoop( [&, &zk = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zk); zk->createAncestors(zookeeper_path); - - Coordination::Requests ops; - Coordination::Responses responses; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_part_names", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_mutations", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_data_paths", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_access", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_sql_objects", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/keeper_map_tables", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/file_infos", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/writing_files", "", zkutil::CreateMode::Persistent)); - zk->tryMulti(ops, responses); + zk->createIfNotExists(zookeeper_path, ""); + zk->createIfNotExists(zookeeper_path + "/repl_part_names", ""); + zk->createIfNotExists(zookeeper_path + "/repl_mutations", ""); + zk->createIfNotExists(zookeeper_path + "/repl_data_paths", ""); + zk->createIfNotExists(zookeeper_path + "/repl_access", ""); + zk->createIfNotExists(zookeeper_path + "/repl_sql_objects", ""); + zk->createIfNotExists(zookeeper_path + "/keeper_map_tables", ""); + zk->createIfNotExists(zookeeper_path + "/file_infos", ""); + zk->createIfNotExists(zookeeper_path + "/writing_files", ""); }); } -void BackupCoordinationRemote::removeAllNodes() +Strings BackupCoordinationOnCluster::setStage(const String & new_stage, const String & message, bool sync) { - auto holder = with_retries.createRetriesControlHolder("removeAllNodes"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() + stage_sync.setStage(new_stage, message); + + if (!sync) + return {}; + + return stage_sync.waitForHostsToReachStage(new_stage, all_hosts_without_initiator); +} + +void BackupCoordinationOnCluster::setBackupQueryWasSentToOtherHosts() +{ + backup_query_was_sent_to_other_hosts = true; +} + +bool BackupCoordinationOnCluster::trySetError(std::exception_ptr exception) +{ + return stage_sync.trySetError(exception); +} + +void BackupCoordinationOnCluster::finish() +{ + bool other_hosts_also_finished = false; + stage_sync.finish(other_hosts_also_finished); + + if ((current_host == kInitiator) && (other_hosts_also_finished || !backup_query_was_sent_to_other_hosts)) + cleaner.cleanup(); +} + +bool BackupCoordinationOnCluster::tryFinishAfterError() noexcept +{ + return tryFinishImpl(); +} + +bool BackupCoordinationOnCluster::tryFinishImpl() noexcept +{ + bool other_hosts_also_finished = false; + if (!stage_sync.tryFinishAfterError(other_hosts_also_finished)) + return false; + + if ((current_host == kInitiator) && (other_hosts_also_finished || !backup_query_was_sent_to_other_hosts)) { - /// Usually this function is called by the initiator when a backup is complete so we don't need the coordination anymore. - /// - /// However there can be a rare situation when this function is called after an error occurs on the initiator of a query - /// while some hosts are still making the backup. Removing all the nodes will remove the parent node of the backup coordination - /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some useless part - /// of their backup work before that. Anyway in this case backup won't be finalized (because only an initiator can do that). - with_retries.renewZooKeeper(zk); - zk->removeRecursive(zookeeper_path); - }); + if (!cleaner.tryCleanupAfterError()) + return false; + } + + return true; } - -void BackupCoordinationRemote::setStage(const String & new_stage, const String & message) +void BackupCoordinationOnCluster::waitForOtherHostsToFinish() { - if (is_internal) - stage_sync->set(current_host, new_stage, message); - else - stage_sync->set(current_host, new_stage, /* message */ "", /* all_hosts */ true); + if ((current_host != kInitiator) || !backup_query_was_sent_to_other_hosts) + return; + stage_sync.waitForOtherHostsToFinish(); } -void BackupCoordinationRemote::setError(const Exception & exception) +bool BackupCoordinationOnCluster::tryWaitForOtherHostsToFinishAfterError() noexcept { - stage_sync->setError(current_host, exception); + if (current_host != kInitiator) + return false; + if (!backup_query_was_sent_to_other_hosts) + return true; + return stage_sync.tryWaitForOtherHostsToFinishAfterError(); } -Strings BackupCoordinationRemote::waitForStage(const String & stage_to_wait) +ZooKeeperRetriesInfo BackupCoordinationOnCluster::getOnClusterInitializationKeeperRetriesInfo() const { - return stage_sync->wait(all_hosts, stage_to_wait); + return ZooKeeperRetriesInfo{keeper_settings.max_retries_while_initializing, + static_cast(keeper_settings.retry_initial_backoff_ms.count()), + static_cast(keeper_settings.retry_max_backoff_ms.count())}; } -Strings BackupCoordinationRemote::waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) -{ - return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); -} - - -void BackupCoordinationRemote::serializeToMultipleZooKeeperNodes(const String & path, const String & value, const String & logging_name) +void BackupCoordinationOnCluster::serializeToMultipleZooKeeperNodes(const String & path, const String & value, const String & logging_name) { { auto holder = with_retries.createRetriesControlHolder(logging_name + "::create"); @@ -301,7 +304,7 @@ void BackupCoordinationRemote::serializeToMultipleZooKeeperNodes(const String & if (value.empty()) return; - size_t max_part_size = keeper_settings.keeper_value_max_size; + size_t max_part_size = keeper_settings.value_max_size; if (!max_part_size) max_part_size = value.size(); @@ -324,7 +327,7 @@ void BackupCoordinationRemote::serializeToMultipleZooKeeperNodes(const String & } } -String BackupCoordinationRemote::deserializeFromMultipleZooKeeperNodes(const String & path, const String & logging_name) const +String BackupCoordinationOnCluster::deserializeFromMultipleZooKeeperNodes(const String & path, const String & logging_name) const { Strings part_names; @@ -357,7 +360,7 @@ String BackupCoordinationRemote::deserializeFromMultipleZooKeeperNodes(const Str } -void BackupCoordinationRemote::addReplicatedPartNames( +void BackupCoordinationOnCluster::addReplicatedPartNames( const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, @@ -381,14 +384,14 @@ void BackupCoordinationRemote::addReplicatedPartNames( }); } -Strings BackupCoordinationRemote::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const +Strings BackupCoordinationOnCluster::getReplicatedPartNames(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); return replicated_tables->getPartNames(table_zk_path, replica_name); } -void BackupCoordinationRemote::addReplicatedMutations( +void BackupCoordinationOnCluster::addReplicatedMutations( const String & table_zk_path, const String & table_name_for_logs, const String & replica_name, @@ -412,7 +415,7 @@ void BackupCoordinationRemote::addReplicatedMutations( }); } -std::vector BackupCoordinationRemote::getReplicatedMutations(const String & table_zk_path, const String & replica_name) const +std::vector BackupCoordinationOnCluster::getReplicatedMutations(const String & table_zk_path, const String & replica_name) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); @@ -420,7 +423,7 @@ std::vector BackupCoordinationRemote::getRepl } -void BackupCoordinationRemote::addReplicatedDataPath( +void BackupCoordinationOnCluster::addReplicatedDataPath( const String & table_zk_path, const String & data_path) { { @@ -441,7 +444,7 @@ void BackupCoordinationRemote::addReplicatedDataPath( }); } -Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_zk_path) const +Strings BackupCoordinationOnCluster::getReplicatedDataPaths(const String & table_zk_path) const { std::lock_guard lock{replicated_tables_mutex}; prepareReplicatedTables(); @@ -449,7 +452,7 @@ Strings BackupCoordinationRemote::getReplicatedDataPaths(const String & table_zk } -void BackupCoordinationRemote::prepareReplicatedTables() const +void BackupCoordinationOnCluster::prepareReplicatedTables() const { if (replicated_tables) return; @@ -536,7 +539,7 @@ void BackupCoordinationRemote::prepareReplicatedTables() const replicated_tables->addDataPath(std::move(data_paths)); } -void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) +void BackupCoordinationOnCluster::addReplicatedAccessFilePath(const String & access_zk_path, AccessEntityType access_entity_type, const String & file_path) { { std::lock_guard lock{replicated_access_mutex}; @@ -558,14 +561,14 @@ void BackupCoordinationRemote::addReplicatedAccessFilePath(const String & access }); } -Strings BackupCoordinationRemote::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const +Strings BackupCoordinationOnCluster::getReplicatedAccessFilePaths(const String & access_zk_path, AccessEntityType access_entity_type) const { std::lock_guard lock{replicated_access_mutex}; prepareReplicatedAccess(); return replicated_access->getFilePaths(access_zk_path, access_entity_type, current_host); } -void BackupCoordinationRemote::prepareReplicatedAccess() const +void BackupCoordinationOnCluster::prepareReplicatedAccess() const { if (replicated_access) return; @@ -601,7 +604,7 @@ void BackupCoordinationRemote::prepareReplicatedAccess() const replicated_access->addFilePath(std::move(file_path)); } -void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) +void BackupCoordinationOnCluster::addReplicatedSQLObjectsDir(const String & loader_zk_path, UserDefinedSQLObjectType object_type, const String & dir_path) { { std::lock_guard lock{replicated_sql_objects_mutex}; @@ -631,14 +634,14 @@ void BackupCoordinationRemote::addReplicatedSQLObjectsDir(const String & loader_ }); } -Strings BackupCoordinationRemote::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const +Strings BackupCoordinationOnCluster::getReplicatedSQLObjectsDirs(const String & loader_zk_path, UserDefinedSQLObjectType object_type) const { std::lock_guard lock{replicated_sql_objects_mutex}; prepareReplicatedSQLObjects(); return replicated_sql_objects->getDirectories(loader_zk_path, object_type, current_host); } -void BackupCoordinationRemote::prepareReplicatedSQLObjects() const +void BackupCoordinationOnCluster::prepareReplicatedSQLObjects() const { if (replicated_sql_objects) return; @@ -674,7 +677,7 @@ void BackupCoordinationRemote::prepareReplicatedSQLObjects() const replicated_sql_objects->addDirectory(std::move(directory)); } -void BackupCoordinationRemote::addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) +void BackupCoordinationOnCluster::addKeeperMapTable(const String & table_zookeeper_root_path, const String & table_id, const String & data_path_in_backup) { { std::lock_guard lock{keeper_map_tables_mutex}; @@ -695,7 +698,7 @@ void BackupCoordinationRemote::addKeeperMapTable(const String & table_zookeeper_ }); } -void BackupCoordinationRemote::prepareKeeperMapTables() const +void BackupCoordinationOnCluster::prepareKeeperMapTables() const { if (keeper_map_tables) return; @@ -740,7 +743,7 @@ void BackupCoordinationRemote::prepareKeeperMapTables() const } -String BackupCoordinationRemote::getKeeperMapDataPath(const String & table_zookeeper_root_path) const +String BackupCoordinationOnCluster::getKeeperMapDataPath(const String & table_zookeeper_root_path) const { std::lock_guard lock(keeper_map_tables_mutex); prepareKeeperMapTables(); @@ -748,7 +751,7 @@ String BackupCoordinationRemote::getKeeperMapDataPath(const String & table_zooke } -void BackupCoordinationRemote::addFileInfos(BackupFileInfos && file_infos_) +void BackupCoordinationOnCluster::addFileInfos(BackupFileInfos && file_infos_) { { std::lock_guard lock{file_infos_mutex}; @@ -761,21 +764,21 @@ void BackupCoordinationRemote::addFileInfos(BackupFileInfos && file_infos_) serializeToMultipleZooKeeperNodes(zookeeper_path + "/file_infos/" + current_host, file_infos_str, "addFileInfos"); } -BackupFileInfos BackupCoordinationRemote::getFileInfos() const +BackupFileInfos BackupCoordinationOnCluster::getFileInfos() const { std::lock_guard lock{file_infos_mutex}; prepareFileInfos(); return file_infos->getFileInfos(current_host); } -BackupFileInfos BackupCoordinationRemote::getFileInfosForAllHosts() const +BackupFileInfos BackupCoordinationOnCluster::getFileInfosForAllHosts() const { std::lock_guard lock{file_infos_mutex}; prepareFileInfos(); return file_infos->getFileInfosForAllHosts(); } -void BackupCoordinationRemote::prepareFileInfos() const +void BackupCoordinationOnCluster::prepareFileInfos() const { if (file_infos) return; @@ -801,7 +804,7 @@ void BackupCoordinationRemote::prepareFileInfos() const } } -bool BackupCoordinationRemote::startWritingFile(size_t data_file_index) +bool BackupCoordinationOnCluster::startWritingFile(size_t data_file_index) { { /// Check if this host is already writing this file. @@ -842,66 +845,4 @@ bool BackupCoordinationRemote::startWritingFile(size_t data_file_index) } } -bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) const -{ - /// If its internal concurrency will be checked for the base backup - if (is_internal) - return false; - - std::string backup_stage_path = zookeeper_path + "/stage"; - - bool result = false; - - auto holder = with_retries.createRetriesControlHolder("getAllArchiveSuffixes"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - if (!zk->exists(root_zookeeper_path)) - zk->createAncestors(root_zookeeper_path); - - for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) - { - Coordination::Stat stat; - zk->get(root_zookeeper_path, &stat); - Strings existing_backup_paths = zk->getChildren(root_zookeeper_path); - - for (const auto & existing_backup_path : existing_backup_paths) - { - if (startsWith(existing_backup_path, "restore-")) - continue; - - String existing_backup_uuid = existing_backup_path; - existing_backup_uuid.erase(0, String("backup-").size()); - - if (existing_backup_uuid == toString(backup_uuid)) - continue; - - String status; - if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status)) - { - /// Check if some other backup is in progress - if (status == Stage::SCHEDULED_TO_START) - { - LOG_WARNING(log, "Found a concurrent backup: {}, current backup: {}", existing_backup_uuid, toString(backup_uuid)); - result = true; - return; - } - } - } - - zk->createIfNotExists(backup_stage_path, ""); - auto code = zk->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version); - if (code == Coordination::Error::ZOK) - break; - bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); - if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException::fromPath(code, backup_stage_path); - } - }); - - return result; -} - } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationOnCluster.h similarity index 67% rename from src/Backups/BackupCoordinationRemote.h rename to src/Backups/BackupCoordinationOnCluster.h index 7a56b1a4eb8..7369c2cc746 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationOnCluster.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include #include #include @@ -13,32 +15,35 @@ namespace DB { -/// We try to store data to zookeeper several times due to possible version conflicts. -constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; - /// Implementation of the IBackupCoordination interface performing coordination via ZooKeeper. It's necessary for "BACKUP ON CLUSTER". -class BackupCoordinationRemote : public IBackupCoordination +class BackupCoordinationOnCluster : public IBackupCoordination { public: - using BackupKeeperSettings = WithRetries::KeeperSettings; + /// Empty string as the current host is used to mark the initiator of a BACKUP ON CLUSTER query. + static const constexpr std::string_view kInitiator; - BackupCoordinationRemote( - zkutil::GetZooKeeper get_zookeeper_, + BackupCoordinationOnCluster( + const UUID & backup_uuid_, + bool is_plain_backup_, const String & root_zookeeper_path_, + zkutil::GetZooKeeper get_zookeeper_, const BackupKeeperSettings & keeper_settings_, - const String & backup_uuid_, - const Strings & all_hosts_, const String & current_host_, - bool plain_backup_, - bool is_internal_, + const Strings & all_hosts_, + bool allow_concurrent_backup_, + BackupConcurrencyCounters & concurrency_counters_, + ThreadPoolCallbackRunnerUnsafe schedule_, QueryStatusPtr process_list_element_); - ~BackupCoordinationRemote() override; + ~BackupCoordinationOnCluster() override; - void setStage(const String & new_stage, const String & message) override; - void setError(const Exception & exception) override; - Strings waitForStage(const String & stage_to_wait) override; - Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; + Strings setStage(const String & new_stage, const String & message, bool sync) override; + void setBackupQueryWasSentToOtherHosts() override; + bool trySetError(std::exception_ptr exception) override; + void finish() override; + bool tryFinishAfterError() noexcept override; + void waitForOtherHostsToFinish() override; + bool tryWaitForOtherHostsToFinishAfterError() noexcept override; void addReplicatedPartNames( const String & table_zk_path, @@ -73,13 +78,14 @@ public: BackupFileInfos getFileInfosForAllHosts() const override; bool startWritingFile(size_t data_file_index) override; - bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; + ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const override; - static size_t findCurrentHostIndex(const Strings & all_hosts, const String & current_host); + static Strings excludeInitiator(const Strings & all_hosts); + static size_t findCurrentHostIndex(const String & current_host, const Strings & all_hosts); private: void createRootNodes(); - void removeAllNodes(); + bool tryFinishImpl() noexcept; void serializeToMultipleZooKeeperNodes(const String & path, const String & value, const String & logging_name); String deserializeFromMultipleZooKeeperNodes(const String & path, const String & logging_name) const; @@ -96,26 +102,27 @@ private: const String root_zookeeper_path; const String zookeeper_path; const BackupKeeperSettings keeper_settings; - const String backup_uuid; + const UUID backup_uuid; const Strings all_hosts; + const Strings all_hosts_without_initiator; const String current_host; const size_t current_host_index; const bool plain_backup; - const bool is_internal; LoggerPtr const log; - /// The order of these two fields matters, because stage_sync holds a reference to with_retries object - mutable WithRetries with_retries; - std::optional stage_sync; + const WithRetries with_retries; + BackupConcurrencyCheck concurrency_check; + BackupCoordinationStageSync stage_sync; + BackupCoordinationCleaner cleaner; + std::atomic backup_query_was_sent_to_other_hosts = false; - mutable std::optional TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables; - mutable std::optional TSA_GUARDED_BY(replicated_access_mutex) replicated_access; - mutable std::optional TSA_GUARDED_BY(replicated_sql_objects_mutex) replicated_sql_objects; - mutable std::optional TSA_GUARDED_BY(file_infos_mutex) file_infos; + mutable std::optional replicated_tables TSA_GUARDED_BY(replicated_tables_mutex); + mutable std::optional replicated_access TSA_GUARDED_BY(replicated_access_mutex); + mutable std::optional replicated_sql_objects TSA_GUARDED_BY(replicated_sql_objects_mutex); + mutable std::optional file_infos TSA_GUARDED_BY(file_infos_mutex); mutable std::optional keeper_map_tables TSA_GUARDED_BY(keeper_map_tables_mutex); - std::unordered_set TSA_GUARDED_BY(writing_files_mutex) writing_files; + std::unordered_set writing_files TSA_GUARDED_BY(writing_files_mutex); - mutable std::mutex zookeeper_mutex; mutable std::mutex replicated_tables_mutex; mutable std::mutex replicated_access_mutex; mutable std::mutex replicated_sql_objects_mutex; diff --git a/src/Backups/BackupCoordinationStage.h b/src/Backups/BackupCoordinationStage.h index 9abdc019784..2cd1efb5404 100644 --- a/src/Backups/BackupCoordinationStage.h +++ b/src/Backups/BackupCoordinationStage.h @@ -8,10 +8,6 @@ namespace DB namespace BackupCoordinationStage { - /// This stage is set after concurrency check so ensure we dont start other backup/restores - /// when concurrent backup/restores are not allowed - constexpr const char * SCHEDULED_TO_START = "scheduled to start"; - /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. constexpr const char * GATHERING_METADATA = "gathering metadata"; @@ -46,10 +42,6 @@ namespace BackupCoordinationStage /// Coordination stage meaning that a host finished its work. constexpr const char * COMPLETED = "completed"; - - /// Coordination stage meaning that backup/restore has failed due to an error - /// Check '/error' for the error message - constexpr const char * ERROR = "error"; } } diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 17ef163ce35..1642cab70c7 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -9,267 +9,1084 @@ #include #include #include +#include +#include +#include + namespace DB { -namespace Stage = BackupCoordinationStage; - namespace ErrorCodes { extern const int FAILED_TO_SYNC_BACKUP_OR_RESTORE; + extern const int LOGICAL_ERROR; } +namespace +{ + /// The coordination version is stored in the 'start' node for each host + /// by each host when it starts working on this backup or restore. + /// The initial version didn't use nodes 'finish*' and 'num_hosts'. + constexpr const int kInitialVersion = 1; + constexpr const int kCurrentVersion = 2; +} + +bool BackupCoordinationStageSync::HostInfo::operator ==(const HostInfo & other) const +{ + /// We don't compare `last_connection_time` here. + return (host == other.host) && (started == other.started) && (connected == other.connected) && (finished == other.finished) + && (stages == other.stages) && (!!exception == !!other.exception); +} + +bool BackupCoordinationStageSync::HostInfo::operator !=(const HostInfo & other) const +{ + return !(*this == other); +} + +bool BackupCoordinationStageSync::State::operator ==(const State & other) const = default; +bool BackupCoordinationStageSync::State::operator !=(const State & other) const = default; + BackupCoordinationStageSync::BackupCoordinationStageSync( - const String & root_zookeeper_path_, - WithRetries & with_retries_, - LoggerPtr log_) - : zookeeper_path(root_zookeeper_path_ + "/stage") + bool is_restore_, + const String & zookeeper_path_, + const String & current_host_, + const Strings & all_hosts_, + bool allow_concurrency_, + const WithRetries & with_retries_, + ThreadPoolCallbackRunnerUnsafe schedule_, + QueryStatusPtr process_list_element_, + LoggerPtr log_) + : is_restore(is_restore_) + , operation_name(is_restore ? "restore" : "backup") + , current_host(current_host_) + , current_host_desc(getHostDesc(current_host)) + , all_hosts(all_hosts_) + , allow_concurrency(allow_concurrency_) , with_retries(with_retries_) + , schedule(schedule_) + , process_list_element(process_list_element_) , log(log_) + , failure_after_host_disconnected_for_seconds(with_retries.getKeeperSettings().failure_after_host_disconnected_for_seconds) + , finish_timeout_after_error(with_retries.getKeeperSettings().finish_timeout_after_error) + , sync_period_ms(with_retries.getKeeperSettings().sync_period_ms) + , max_attempts_after_bad_version(with_retries.getKeeperSettings().max_attempts_after_bad_version) + , zookeeper_path(zookeeper_path_) + , root_zookeeper_path(zookeeper_path.parent_path().parent_path()) + , operation_node_path(zookeeper_path.parent_path()) + , operation_node_name(zookeeper_path.parent_path().filename()) + , stage_node_path(zookeeper_path) + , start_node_path(zookeeper_path / ("started|" + current_host)) + , finish_node_path(zookeeper_path / ("finished|" + current_host)) + , num_hosts_node_path(zookeeper_path / "num_hosts") + , alive_node_path(zookeeper_path / ("alive|" + current_host)) + , alive_tracker_node_path(fs::path{root_zookeeper_path} / "alive_tracker") + , error_node_path(zookeeper_path / "error") + , zk_nodes_changed(std::make_shared()) { + if ((zookeeper_path.filename() != "stage") || !operation_node_name.starts_with(is_restore ? "restore-" : "backup-") + || (root_zookeeper_path == operation_node_path)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected path in ZooKeeper specified: {}", zookeeper_path); + } + + initializeState(); createRootNodes(); + + try + { + createStartAndAliveNodes(); + startWatchingThread(); + } + catch (...) + { + trySetError(std::current_exception()); + tryFinishImpl(); + throw; + } } + +BackupCoordinationStageSync::~BackupCoordinationStageSync() +{ + tryFinishImpl(); +} + + +void BackupCoordinationStageSync::initializeState() +{ + std::lock_guard lock{mutex}; + auto now = std::chrono::system_clock::now(); + auto monotonic_now = std::chrono::steady_clock::now(); + + for (const String & host : all_hosts) + state.hosts.emplace(host, HostInfo{.host = host, .last_connection_time = now, .last_connection_time_monotonic = monotonic_now}); +} + + +String BackupCoordinationStageSync::getHostDesc(const String & host) +{ + String res; + if (host.empty()) + { + res = "the initiator"; + } + else + { + try + { + res = "host "; + Poco::URI::decode(host, res); /// Append the decoded host name to `res`. + } + catch (const Poco::URISyntaxException &) + { + res = "host " + host; + } + } + return res; +} + + +String BackupCoordinationStageSync::getHostsDesc(const Strings & hosts) +{ + String res = "["; + for (const String & host : hosts) + { + if (res != "[") + res += ", "; + res += getHostDesc(host); + } + res += "]"; + return res; +} + + void BackupCoordinationStageSync::createRootNodes() { - auto holder = with_retries.createRetriesControlHolder("createRootNodes"); + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::createRootNodes", WithRetries::kInitialization); holder.retries_ctl.retryLoop( [&, &zookeeper = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zookeeper); + zookeeper->createAncestors(root_zookeeper_path); + zookeeper->createIfNotExists(root_zookeeper_path, ""); + }); +} + + +void BackupCoordinationStageSync::createStartAndAliveNodes() +{ + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::createStartAndAliveNodes", WithRetries::kInitialization); + holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zookeeper); - zookeeper->createAncestors(zookeeper_path); - zookeeper->createIfNotExists(zookeeper_path, ""); + createStartAndAliveNodes(zookeeper); }); } -void BackupCoordinationStageSync::set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts) -{ - auto holder = with_retries.createRetriesControlHolder("set"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zookeeper); - if (all_hosts) +void BackupCoordinationStageSync::createStartAndAliveNodes(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper) +{ + /// The "num_hosts" node keeps the number of hosts which started (created the "started" node) + /// but not yet finished (not created the "finished" node). + /// The number of alive hosts can be less than that. + + /// The "alive_tracker" node always keeps an empty string, we track its version only. + /// The "alive_tracker" node increases its version each time when any "alive" nodes are created + /// so we use it to check concurrent backups/restores. + zookeeper->createIfNotExists(alive_tracker_node_path, ""); + + std::optional num_hosts; + int num_hosts_version = -1; + + bool check_concurrency = !allow_concurrency; + int alive_tracker_version = -1; + + for (size_t attempt_no = 1; attempt_no <= max_attempts_after_bad_version; ++attempt_no) + { + if (!num_hosts) { - auto code = zookeeper->trySet(zookeeper_path, new_stage); - if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException::fromPath(code, zookeeper_path); + String num_hosts_str; + Coordination::Stat stat; + if (zookeeper->tryGet(num_hosts_node_path, num_hosts_str, &stat)) + { + num_hosts = parseFromString(num_hosts_str); + num_hosts_version = stat.version; + } + } + + String serialized_error; + if (zookeeper->tryGet(error_node_path, serialized_error)) + { + auto [exception, host] = parseErrorNode(serialized_error); + if (exception) + std::rethrow_exception(exception); + } + + if (check_concurrency) + { + Coordination::Stat stat; + zookeeper->exists(alive_tracker_node_path, &stat); + alive_tracker_version = stat.version; + + checkConcurrency(zookeeper); + check_concurrency = false; + } + + Coordination::Requests requests; + requests.reserve(6); + + size_t operation_node_path_pos = static_cast(-1); + if (!zookeeper->exists(operation_node_path)) + { + operation_node_path_pos = requests.size(); + requests.emplace_back(zkutil::makeCreateRequest(operation_node_path, "", zkutil::CreateMode::Persistent)); + } + + size_t stage_node_path_pos = static_cast(-1); + if (!zookeeper->exists(stage_node_path)) + { + stage_node_path_pos = requests.size(); + requests.emplace_back(zkutil::makeCreateRequest(stage_node_path, "", zkutil::CreateMode::Persistent)); + } + + size_t num_hosts_node_path_pos = requests.size(); + if (num_hosts) + requests.emplace_back(zkutil::makeSetRequest(num_hosts_node_path, toString(*num_hosts + 1), num_hosts_version)); + else + requests.emplace_back(zkutil::makeCreateRequest(num_hosts_node_path, "1", zkutil::CreateMode::Persistent)); + + size_t alive_tracker_node_path_pos = requests.size(); + requests.emplace_back(zkutil::makeSetRequest(alive_tracker_node_path, "", alive_tracker_version)); + + requests.emplace_back(zkutil::makeCreateRequest(start_node_path, std::to_string(kCurrentVersion), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(alive_node_path, "", zkutil::CreateMode::Ephemeral)); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(requests, responses); + + if (code == Coordination::Error::ZOK) + { + LOG_INFO(log, "Created start node #{} in ZooKeeper for {} (coordination version: {})", + num_hosts.value_or(0) + 1, current_host_desc, kCurrentVersion); + return; + } + + auto show_error_before_next_attempt = [&](const String & message) + { + bool will_try_again = (attempt_no < max_attempts_after_bad_version); + LOG_TRACE(log, "{} (attempt #{}){}", message, attempt_no, will_try_again ? ", will try again" : ""); + }; + + if ((responses.size() > operation_node_path_pos) && + (responses[operation_node_path_pos]->error == Coordination::Error::ZNODEEXISTS)) + { + show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper already exists", operation_node_path)); + /// needs another attempt + } + else if ((responses.size() > stage_node_path_pos) && + (responses[stage_node_path_pos]->error == Coordination::Error::ZNODEEXISTS)) + { + show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper already exists", stage_node_path)); + /// needs another attempt + } + else if ((responses.size() > num_hosts_node_path_pos) && num_hosts && + (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZBADVERSION)) + { + show_error_before_next_attempt("Other host changed the 'num_hosts' node in ZooKeeper"); + num_hosts.reset(); /// needs to reread 'num_hosts' again + } + else if ((responses.size() > num_hosts_node_path_pos) && num_hosts && + (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZNONODE)) + { + show_error_before_next_attempt("Other host removed the 'num_hosts' node in ZooKeeper"); + num_hosts.reset(); /// needs to reread 'num_hosts' again + } + else if ((responses.size() > num_hosts_node_path_pos) && !num_hosts && + (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZNODEEXISTS)) + { + show_error_before_next_attempt("Other host created the 'num_hosts' node in ZooKeeper"); + /// needs another attempt + } + else if ((responses.size() > alive_tracker_node_path_pos) && + (responses[alive_tracker_node_path_pos]->error == Coordination::Error::ZBADVERSION)) + { + show_error_before_next_attempt("Concurrent backup or restore changed some 'alive' nodes in ZooKeeper"); + check_concurrency = true; /// needs to recheck for concurrency again } else { - zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); - zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); + zkutil::KeeperMultiException::check(code, requests, responses); } + } + + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Couldn't create the 'start' node in ZooKeeper for {} after {} attempts", + current_host_desc, max_attempts_after_bad_version); +} + + +void BackupCoordinationStageSync::checkConcurrency(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper) +{ + if (allow_concurrency) + return; + + Strings found_operations; + auto code = zookeeper->tryGetChildren(root_zookeeper_path, found_operations); + + if (!((code == Coordination::Error::ZOK) || (code == Coordination::Error::ZNONODE))) + throw zkutil::KeeperException::fromPath(code, root_zookeeper_path); + + if (code == Coordination::Error::ZNONODE) + return; + + for (const String & found_operation : found_operations) + { + if (found_operation.starts_with(is_restore ? "restore-" : "backup-") && (found_operation != operation_node_name)) + { + Strings stages; + code = zookeeper->tryGetChildren(fs::path{root_zookeeper_path} / found_operation / "stage", stages); + + if (!((code == Coordination::Error::ZOK) || (code == Coordination::Error::ZNONODE))) + throw zkutil::KeeperException::fromPath(code, fs::path{root_zookeeper_path} / found_operation / "stage"); + + if (code == Coordination::Error::ZOK) + { + for (const String & stage : stages) + { + if (stage.starts_with("alive")) + BackupConcurrencyCheck::throwConcurrentOperationNotAllowed(is_restore); + } + } + } + } +} + + +void BackupCoordinationStageSync::startWatchingThread() +{ + watching_thread_future = schedule([this]() { watchingThread(); }, Priority{}); +} + + +void BackupCoordinationStageSync::stopWatchingThread() +{ + should_stop_watching_thread = true; + + /// Wake up waiting threads. + if (zk_nodes_changed) + zk_nodes_changed->set(); + state_changed.notify_all(); + + if (watching_thread_future.valid()) + watching_thread_future.wait(); +} + + +void BackupCoordinationStageSync::watchingThread() +{ + while (!should_stop_watching_thread) + { + try + { + /// Check if the current BACKUP or RESTORE command is already cancelled. + checkIfQueryCancelled(); + + /// Reset the `connected` flag for each host, we'll set them to true again after we find the 'alive' nodes. + resetConnectedFlag(); + + /// Recreate the 'alive' node if necessary and read a new state from ZooKeeper. + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::watchingThread"); + auto & zookeeper = holder.faulty_zookeeper; + with_retries.renewZooKeeper(zookeeper); + + if (should_stop_watching_thread) + return; + + /// Recreate the 'alive' node if it was removed. + createAliveNode(zookeeper); + + /// Reads the current state from nodes in ZooKeeper. + readCurrentState(zookeeper); + } + catch (...) + { + tryLogCurrentException(log, "Caugth exception while watching"); + } + + try + { + /// Cancel the query if there is an error on another host or if some host was disconnected too long. + cancelQueryIfError(); + cancelQueryIfDisconnectedTooLong(); + } + catch (...) + { + tryLogCurrentException(log, "Caugth exception while checking if the query should be cancelled"); + } + + zk_nodes_changed->tryWait(sync_period_ms.count()); + } +} + + +void BackupCoordinationStageSync::createAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper) +{ + if (zookeeper->exists(alive_node_path)) + return; + + Coordination::Requests requests; + requests.emplace_back(zkutil::makeCreateRequest(alive_node_path, "", zkutil::CreateMode::Ephemeral)); + requests.emplace_back(zkutil::makeSetRequest(alive_tracker_node_path, "", -1)); + zookeeper->multi(requests); + + LOG_INFO(log, "The alive node was recreated for {}", current_host_desc); +} + + +void BackupCoordinationStageSync::resetConnectedFlag() +{ + std::lock_guard lock{mutex}; + for (auto & [_, host_info] : state.hosts) + host_info.connected = false; +} + + +void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper) +{ + zk_nodes_changed->reset(); + + /// Get zk nodes and subscribe on their changes. + Strings new_zk_nodes = zookeeper->getChildren(stage_node_path, nullptr, zk_nodes_changed); + std::sort(new_zk_nodes.begin(), new_zk_nodes.end()); /// Sorting is necessary because we compare the list of zk nodes with its previous versions. + + State new_state; + + { + std::lock_guard lock{mutex}; + + /// Log all changes in zookeeper nodes in the "stage" folder to make debugging easier. + Strings added_zk_nodes, removed_zk_nodes; + std::set_difference(new_zk_nodes.begin(), new_zk_nodes.end(), zk_nodes.begin(), zk_nodes.end(), back_inserter(added_zk_nodes)); + std::set_difference(zk_nodes.begin(), zk_nodes.end(), new_zk_nodes.begin(), new_zk_nodes.end(), back_inserter(removed_zk_nodes)); + if (!added_zk_nodes.empty()) + LOG_TRACE(log, "Detected new zookeeper nodes appeared in the stage folder: {}", boost::algorithm::join(added_zk_nodes, ", ")); + if (!removed_zk_nodes.empty()) + LOG_TRACE(log, "Detected that some zookeeper nodes disappeared from the stage folder: {}", boost::algorithm::join(removed_zk_nodes, ", ")); + + zk_nodes = new_zk_nodes; + new_state = state; + } + + auto get_host_info = [&](const String & host) -> HostInfo * + { + auto it = new_state.hosts.find(host); + if (it == new_state.hosts.end()) + return nullptr; + return &it->second; + }; + + auto now = std::chrono::system_clock::now(); + auto monotonic_now = std::chrono::steady_clock::now(); + + /// Read the current state from zookeeper nodes. + for (const auto & zk_node : new_zk_nodes) + { + if (zk_node == "error") + { + if (!new_state.host_with_error) + { + String serialized_error = zookeeper->get(error_node_path); + auto [exception, host] = parseErrorNode(serialized_error); + if (auto * host_info = get_host_info(host)) + { + host_info->exception = exception; + new_state.host_with_error = host; + } + } + } + else if (zk_node.starts_with("started|")) + { + String host = zk_node.substr(strlen("started|")); + if (auto * host_info = get_host_info(host)) + { + if (!host_info->started) + { + host_info->version = parseStartNode(zookeeper->get(zookeeper_path / zk_node), host); + host_info->started = true; + } + } + } + else if (zk_node.starts_with("finished|")) + { + String host = zk_node.substr(strlen("finished|")); + if (auto * host_info = get_host_info(host)) + host_info->finished = true; + } + else if (zk_node.starts_with("alive|")) + { + String host = zk_node.substr(strlen("alive|")); + if (auto * host_info = get_host_info(host)) + { + host_info->connected = true; + host_info->last_connection_time = now; + host_info->last_connection_time_monotonic = monotonic_now; + } + } + else if (zk_node.starts_with("current|")) + { + String host_and_stage = zk_node.substr(strlen("current|")); + size_t separator_pos = host_and_stage.find('|'); + if (separator_pos != String::npos) + { + String host = host_and_stage.substr(0, separator_pos); + String stage = host_and_stage.substr(separator_pos + 1); + if (auto * host_info = get_host_info(host)) + { + String result = zookeeper->get(fs::path{zookeeper_path} / zk_node); + host_info->stages[stage] = std::move(result); + + /// The initial version didn't create the 'finish' ZooKeeper nodes so + /// we consider that if the "completed" stage is reached by a host then the host has finished its work. + /// This assumption is not correct if an error happens, but the initial version can't handle errors quite + /// correctly anyway. + if ((host_info->version == kInitialVersion) && (stage == BackupCoordinationStage::COMPLETED)) + host_info->finished = true; + } + } + } + } + + /// Check if the state has been just changed, and if so then wake up waiting threads (see waitHostsReachStage()). + bool was_state_changed = false; + + { + std::lock_guard lock{mutex}; + was_state_changed = (new_state != state); + state = std::move(new_state); + } + + if (was_state_changed) + state_changed.notify_all(); +} + + +int BackupCoordinationStageSync::parseStartNode(const String & start_node_contents, const String & host) const +{ + int version; + if (start_node_contents.empty()) + { + version = kInitialVersion; + } + else if (!tryParse(version, start_node_contents) || (version < kInitialVersion)) + { + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Coordination version {} used by {} is not supported", start_node_contents, getHostDesc(host)); + } + + if (version < kCurrentVersion) + LOG_WARNING(log, "Coordination version {} used by {} is outdated", version, getHostDesc(host)); + return version; +} + + +std::pair BackupCoordinationStageSync::parseErrorNode(const String & error_node_contents) +{ + ReadBufferFromOwnString buf{error_node_contents}; + String host; + readStringBinary(host, buf); + auto exception = std::make_exception_ptr(readException(buf, fmt::format("Got error from {}", getHostDesc(host)))); + return {exception, host}; +} + + +void BackupCoordinationStageSync::checkIfQueryCancelled() +{ + if (process_list_element->checkTimeLimitSoft()) + return; /// Not cancelled. + + std::lock_guard lock{mutex}; + if (state.cancelled) + return; /// Already marked as cancelled. + + state.cancelled = true; + state_changed.notify_all(); +} + + +void BackupCoordinationStageSync::cancelQueryIfError() +{ + std::exception_ptr exception; + + { + std::lock_guard lock{mutex}; + if (state.cancelled || !state.host_with_error) + return; + + state.cancelled = true; + exception = state.hosts.at(*state.host_with_error).exception; + } + + process_list_element->cancelQuery(false, exception); + state_changed.notify_all(); +} + + +void BackupCoordinationStageSync::cancelQueryIfDisconnectedTooLong() +{ + std::exception_ptr exception; + + { + std::lock_guard lock{mutex}; + if (state.cancelled || state.host_with_error || ((failure_after_host_disconnected_for_seconds.count() == 0))) + return; + + auto monotonic_now = std::chrono::steady_clock::now(); + bool info_shown = false; + + for (auto & [host, host_info] : state.hosts) + { + if (!host_info.connected && !host_info.finished && (host != current_host)) + { + auto disconnected_duration = std::chrono::duration_cast(monotonic_now - host_info.last_connection_time_monotonic); + if (disconnected_duration > failure_after_host_disconnected_for_seconds) + { + /// Host `host` was disconnected too long. + /// We can't just throw an exception here because readCurrentState() is called from a background thread. + /// So here we're writingh the error to the `process_list_element` and let it to be thrown later + /// from `process_list_element->checkTimeLimit()`. + String message = fmt::format("The 'alive' node hasn't been updated in ZooKeeper for {} for {} " + "which is more than the specified timeout {}. Last time the 'alive' node was detected at {}", + getHostDesc(host), disconnected_duration, failure_after_host_disconnected_for_seconds, + host_info.last_connection_time); + LOG_WARNING(log, "Lost connection to {}: {}", getHostDesc(host), message); + exception = std::make_exception_ptr(Exception{ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Lost connection to {}: {}", getHostDesc(host), message}); + break; + } + + if ((disconnected_duration >= std::chrono::seconds{1}) && !info_shown) + { + LOG_TRACE(log, "The 'alive' node hasn't been updated in ZooKeeper for {} for {}", getHostDesc(host), disconnected_duration); + info_shown = true; + } + } + } + + if (!exception) + return; + + state.cancelled = true; + } + + process_list_element->cancelQuery(false, exception); + state_changed.notify_all(); +} + + +void BackupCoordinationStageSync::setStage(const String & stage, const String & stage_result) +{ + LOG_INFO(log, "{} reached stage {}", current_host_desc, stage); + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::setStage"); + holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zookeeper); + zookeeper->createIfNotExists(getStageNodePath(stage), stage_result); }); } -void BackupCoordinationStageSync::setError(const String & current_host, const Exception & exception) + +String BackupCoordinationStageSync::getStageNodePath(const String & stage) const { - auto holder = with_retries.createRetriesControlHolder("setError"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() + return fs::path{zookeeper_path} / ("current|" + current_host + "|" + stage); +} + + +bool BackupCoordinationStageSync::trySetError(std::exception_ptr exception) noexcept +{ + try + { + std::rethrow_exception(exception); + } + catch (const Exception & e) + { + return trySetError(e); + } + catch (...) + { + return trySetError(Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode())); + } +} + + +bool BackupCoordinationStageSync::trySetError(const Exception & exception) +{ + try + { + setError(exception); + return true; + } + catch (...) + { + return false; + } +} + + +void BackupCoordinationStageSync::setError(const Exception & exception) +{ + /// Most likely this exception has been already logged so here we're logging it without stacktrace. + String exception_message = getExceptionMessage(exception, /* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true); + LOG_INFO(log, "Sending exception from {} to other hosts: {}", current_host_desc, exception_message); + + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::setError", WithRetries::kErrorHandling); + + holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]() { with_retries.renewZooKeeper(zookeeper); WriteBufferFromOwnString buf; writeStringBinary(current_host, buf); writeException(exception, buf, true); - zookeeper->createIfNotExists(zookeeper_path + "/error", buf.str()); + auto code = zookeeper->tryCreate(error_node_path, buf.str(), zkutil::CreateMode::Persistent); - /// When backup/restore fails, it removes the nodes from Zookeeper. - /// Sometimes it fails to remove all nodes. It's possible that it removes /error node, but fails to remove /stage node, - /// so the following line tries to preserve the error status. - auto code = zookeeper->trySet(zookeeper_path, Stage::ERROR); - if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException::fromPath(code, zookeeper_path); + if (code == Coordination::Error::ZOK) + { + LOG_TRACE(log, "Sent exception from {} to other hosts", current_host_desc); + } + else if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "An error has been already assigned for this {}", operation_name); + } + else + { + throw zkutil::KeeperException::fromPath(code, error_node_path); + } }); } -Strings BackupCoordinationStageSync::wait(const Strings & all_hosts, const String & stage_to_wait) + +Strings BackupCoordinationStageSync::waitForHostsToReachStage(const String & stage_to_wait, const Strings & hosts, std::optional timeout) const { - return waitImpl(all_hosts, stage_to_wait, {}); -} - -Strings BackupCoordinationStageSync::waitFor(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout) -{ - return waitImpl(all_hosts, stage_to_wait, timeout); -} - -namespace -{ - struct UnreadyHost - { - String host; - bool started = false; - }; -} - -struct BackupCoordinationStageSync::State -{ - std::optional results; - std::optional> error; - std::optional disconnected_host; - std::optional unready_host; -}; - -BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState( - WithRetries::RetriesControlHolder & retries_control_holder, - const Strings & zk_nodes, - const Strings & all_hosts, - const String & stage_to_wait) const -{ - auto zookeeper = retries_control_holder.faulty_zookeeper; - auto & retries_ctl = retries_control_holder.retries_ctl; - - std::unordered_set zk_nodes_set{zk_nodes.begin(), zk_nodes.end()}; - - State state; - if (zk_nodes_set.contains("error")) - { - String errors = zookeeper->get(zookeeper_path + "/error"); - ReadBufferFromOwnString buf{errors}; - String host; - readStringBinary(host, buf); - state.error = std::make_pair(host, readException(buf, fmt::format("Got error from {}", host))); - return state; - } - - std::optional unready_host; - - for (const auto & host : all_hosts) - { - if (!zk_nodes_set.contains("current|" + host + "|" + stage_to_wait)) - { - const String started_node_name = "started|" + host; - const String alive_node_name = "alive|" + host; - - bool started = zk_nodes_set.contains(started_node_name); - bool alive = zk_nodes_set.contains(alive_node_name); - - if (!alive) - { - /// If the "alive" node doesn't exist then we don't have connection to the corresponding host. - /// This node is ephemeral so probably it will be recreated soon. We use zookeeper retries to wait. - /// In worst case when we won't manage to see the alive node for a long time we will just abort the backup. - const auto * const suffix = retries_ctl.isLastRetry() ? "" : ", will retry"; - if (started) - retries_ctl.setUserError(Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, - "Lost connection to host {}{}", host, suffix)); - else - retries_ctl.setUserError(Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, - "No connection to host {} yet{}", host, suffix)); - - state.disconnected_host = host; - return state; - } - - if (!unready_host) - unready_host.emplace(UnreadyHost{.host = host, .started = started}); - } - } - - if (unready_host) - { - state.unready_host = std::move(unready_host); - return state; - } - Strings results; - for (const auto & host : all_hosts) - results.emplace_back(zookeeper->get(zookeeper_path + "/current|" + host + "|" + stage_to_wait)); - state.results = std::move(results); + results.resize(hosts.size()); - return state; + std::unique_lock lock{mutex}; + + /// TSA_NO_THREAD_SAFETY_ANALYSIS is here because Clang Thread Safety Analysis doesn't understand std::unique_lock. + auto check_if_hosts_ready = [&](bool time_is_out) TSA_NO_THREAD_SAFETY_ANALYSIS + { + return checkIfHostsReachStage(hosts, stage_to_wait, time_is_out, timeout, results); + }; + + if (timeout) + { + if (!state_changed.wait_for(lock, *timeout, [&] { return check_if_hosts_ready(/* time_is_out = */ false); })) + check_if_hosts_ready(/* time_is_out = */ true); + } + else + { + state_changed.wait(lock, [&] { return check_if_hosts_ready(/* time_is_out = */ false); }); + } + + return results; } -Strings BackupCoordinationStageSync::waitImpl( - const Strings & all_hosts, const String & stage_to_wait, std::optional timeout) const + +bool BackupCoordinationStageSync::checkIfHostsReachStage( + const Strings & hosts, + const String & stage_to_wait, + bool time_is_out, + std::optional timeout, + Strings & results) const { - if (all_hosts.empty()) - return {}; + if (should_stop_watching_thread) + throw Exception(ErrorCodes::LOGICAL_ERROR, "finish() was called while waiting for a stage"); - /// Wait until all hosts are ready or an error happens or time is out. + process_list_element->checkTimeLimit(); - bool use_timeout = timeout.has_value(); - std::chrono::steady_clock::time_point end_of_timeout; - if (use_timeout) - end_of_timeout = std::chrono::steady_clock::now() + std::chrono::duration_cast(*timeout); - - State state; - for (;;) + for (size_t i = 0; i != hosts.size(); ++i) { - LOG_INFO(log, "Waiting for the stage {}", stage_to_wait); - /// Set by ZooKepper when list of zk nodes have changed. - auto watch = std::make_shared(); - Strings zk_nodes; - { - auto holder = with_retries.createRetriesControlHolder("waitImpl"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zookeeper); - watch->reset(); - /// Get zk nodes and subscribe on their changes. - zk_nodes = zookeeper->getChildren(zookeeper_path, nullptr, watch); + const String & host = hosts[i]; + auto it = state.hosts.find(host); - /// Read the current state of zk nodes. - state = readCurrentState(holder, zk_nodes, all_hosts, stage_to_wait); - }); + if (it == state.hosts.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "waitForHostsToReachStage() was called for unexpected {}, all hosts are {}", getHostDesc(host), getHostsDesc(all_hosts)); + + const HostInfo & host_info = it->second; + auto stage_it = host_info.stages.find(stage_to_wait); + if (stage_it != host_info.stages.end()) + { + results[i] = stage_it->second; + continue; } - /// Analyze the current state of zk nodes. - chassert(state.results || state.error || state.disconnected_host || state.unready_host); - - if (state.results || state.error || state.disconnected_host) - break; /// Everything is ready or error happened. - - /// Log what we will wait. - const auto & unready_host = *state.unready_host; - LOG_INFO(log, "Waiting on ZooKeeper watch for any node to be changed (currently waiting for host {}{})", - unready_host.host, - (!unready_host.started ? " which didn't start the operation yet" : "")); - - /// Wait until `watch_callback` is called by ZooKeeper meaning that zk nodes have changed. + if (host_info.finished) { - if (use_timeout) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "{} finished without coming to stage {}", getHostDesc(host), stage_to_wait); + } + + String host_status; + if (!host_info.started) + host_status = fmt::format(": the host hasn't started working on this {} yet", operation_name); + else if (!host_info.connected) + host_status = fmt::format(": the host is currently disconnected, last connection was at {}", host_info.last_connection_time); + + if (!time_is_out) + { + LOG_TRACE(log, "Waiting for {} to reach stage {}{}", getHostDesc(host), stage_to_wait, host_status); + return false; + } + else + { + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Waited longer than timeout {} for {} to reach stage {}{}", + *timeout, getHostDesc(host), stage_to_wait, host_status); + } + } + + LOG_INFO(log, "Hosts {} reached stage {}", getHostsDesc(hosts), stage_to_wait); + return true; +} + + +void BackupCoordinationStageSync::finish(bool & other_hosts_also_finished) +{ + tryFinishImpl(other_hosts_also_finished, /* throw_if_error = */ true, /* retries_kind = */ WithRetries::kNormal); +} + + +bool BackupCoordinationStageSync::tryFinishAfterError(bool & other_hosts_also_finished) noexcept +{ + return tryFinishImpl(other_hosts_also_finished, /* throw_if_error = */ false, /* retries_kind = */ WithRetries::kErrorHandling); +} + + +bool BackupCoordinationStageSync::tryFinishImpl() +{ + bool other_hosts_also_finished; + return tryFinishAfterError(other_hosts_also_finished); +} + + +bool BackupCoordinationStageSync::tryFinishImpl(bool & other_hosts_also_finished, bool throw_if_error, WithRetries::Kind retries_kind) +{ + auto get_value_other_hosts_also_finished = [&] TSA_REQUIRES(mutex) + { + other_hosts_also_finished = true; + for (const auto & [host, host_info] : state.hosts) + { + if ((host != current_host) && !host_info.finished) + other_hosts_also_finished = false; + } + }; + + { + std::lock_guard lock{mutex}; + if (finish_result.succeeded) + { + get_value_other_hosts_also_finished(); + return true; + } + if (finish_result.exception) + { + if (throw_if_error) + std::rethrow_exception(finish_result.exception); + return false; + } + } + + try + { + stopWatchingThread(); + + auto holder = with_retries.createRetriesControlHolder("BackupStageSync::finish", retries_kind); + holder.retries_ctl.retryLoop([&, &zookeeper = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zookeeper); + createFinishNodeAndRemoveAliveNode(zookeeper); + }); + + std::lock_guard lock{mutex}; + finish_result.succeeded = true; + get_value_other_hosts_also_finished(); + return true; + } + catch (...) + { + LOG_TRACE(log, "Caught exception while creating the 'finish' node for {}: {}", + current_host_desc, + getCurrentExceptionMessage(/* with_stacktrace= */ false, /* check_embedded_stacktrace= */ true)); + + std::lock_guard lock{mutex}; + finish_result.exception = std::current_exception(); + if (throw_if_error) + throw; + return false; + } +} + + +void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper) +{ + if (zookeeper->exists(finish_node_path)) + return; + + std::optional num_hosts; + int num_hosts_version = -1; + + for (size_t attempt_no = 1; attempt_no <= max_attempts_after_bad_version; ++attempt_no) + { + if (!num_hosts) + { + Coordination::Stat stat; + num_hosts = parseFromString(zookeeper->get(num_hosts_node_path, &stat)); + num_hosts_version = stat.version; + } + + Coordination::Requests requests; + requests.reserve(3); + + requests.emplace_back(zkutil::makeCreateRequest(finish_node_path, "", zkutil::CreateMode::Persistent)); + + size_t num_hosts_node_path_pos = requests.size(); + requests.emplace_back(zkutil::makeSetRequest(num_hosts_node_path, toString(*num_hosts - 1), num_hosts_version)); + + size_t alive_node_path_pos = static_cast(-1); + if (zookeeper->exists(alive_node_path)) + { + alive_node_path_pos = requests.size(); + requests.emplace_back(zkutil::makeRemoveRequest(alive_node_path, -1)); + } + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(requests, responses); + + if (code == Coordination::Error::ZOK) + { + --*num_hosts; + String hosts_left_desc = ((*num_hosts == 0) ? "no hosts left" : fmt::format("{} hosts left", *num_hosts)); + LOG_INFO(log, "Created the 'finish' node in ZooKeeper for {}, {}", current_host_desc, hosts_left_desc); + return; + } + + auto show_error_before_next_attempt = [&](const String & message) + { + bool will_try_again = (attempt_no < max_attempts_after_bad_version); + LOG_TRACE(log, "{} (attempt #{}){}", message, attempt_no, will_try_again ? ", will try again" : ""); + }; + + if ((responses.size() > num_hosts_node_path_pos) && + (responses[num_hosts_node_path_pos]->error == Coordination::Error::ZBADVERSION)) + { + show_error_before_next_attempt("Other host changed the 'num_hosts' node in ZooKeeper"); + num_hosts.reset(); /// needs to reread 'num_hosts' again + } + else if ((responses.size() > alive_node_path_pos) && + (responses[alive_node_path_pos]->error == Coordination::Error::ZNONODE)) + { + show_error_before_next_attempt(fmt::format("Node {} in ZooKeeper doesn't exist", alive_node_path_pos)); + /// needs another attempt + } + else + { + zkutil::KeeperMultiException::check(code, requests, responses); + } + } + + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Couldn't create the 'finish' node for {} after {} attempts", + current_host_desc, max_attempts_after_bad_version); +} + + +void BackupCoordinationStageSync::waitForOtherHostsToFinish() const +{ + tryWaitForOtherHostsToFinishImpl(/* reason = */ "", /* throw_if_error = */ true, /* timeout = */ {}); +} + + +bool BackupCoordinationStageSync::tryWaitForOtherHostsToFinishAfterError() const noexcept +{ + std::optional timeout; + if (finish_timeout_after_error.count() != 0) + timeout = finish_timeout_after_error; + + String reason = fmt::format("{} needs other hosts to finish before cleanup", current_host_desc); + return tryWaitForOtherHostsToFinishImpl(reason, /* throw_if_error = */ false, timeout); +} + + +bool BackupCoordinationStageSync::tryWaitForOtherHostsToFinishImpl(const String & reason, bool throw_if_error, std::optional timeout) const +{ + std::unique_lock lock{mutex}; + + /// TSA_NO_THREAD_SAFETY_ANALYSIS is here because Clang Thread Safety Analysis doesn't understand std::unique_lock. + auto check_if_other_hosts_finish = [&](bool time_is_out) TSA_NO_THREAD_SAFETY_ANALYSIS + { + return checkIfOtherHostsFinish(reason, throw_if_error, time_is_out, timeout); + }; + + if (timeout) + { + if (state_changed.wait_for(lock, *timeout, [&] { return check_if_other_hosts_finish(/* time_is_out = */ false); })) + return true; + return check_if_other_hosts_finish(/* time_is_out = */ true); + } + else + { + state_changed.wait(lock, [&] { return check_if_other_hosts_finish(/* time_is_out = */ false); }); + return true; + } +} + + +bool BackupCoordinationStageSync::checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional timeout) const +{ + if (should_stop_watching_thread) + throw Exception(ErrorCodes::LOGICAL_ERROR, "finish() was called while waiting for other hosts to finish"); + + if (throw_if_error) + process_list_element->checkTimeLimit(); + + for (const auto & [host, host_info] : state.hosts) + { + if ((host == current_host) || host_info.finished) + continue; + + String host_status; + if (!host_info.started) + host_status = fmt::format(": the host hasn't started working on this {} yet", operation_name); + else if (!host_info.connected) + host_status = fmt::format(": the host is currently disconnected, last connection was at {}", host_info.last_connection_time); + + if (!time_is_out) + { + String reason_text = reason.empty() ? "" : (" because " + reason); + LOG_TRACE(log, "Waiting for {} to finish{}{}", getHostDesc(host), reason_text, host_status); + return false; + } + else + { + String reason_text = reason.empty() ? "" : fmt::format(" (reason of waiting: {})", reason); + if (!throw_if_error) { - auto current_time = std::chrono::steady_clock::now(); - if ((current_time > end_of_timeout) - || !watch->tryWait(std::chrono::duration_cast(end_of_timeout - current_time).count())) - break; + LOG_INFO(log, "Waited longer than timeout {} for {} to finish{}{}", + *timeout, getHostDesc(host), host_status, reason_text); + return false; } else { - watch->wait(); + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, + "Waited longer than timeout {} for {} to finish{}{}", + *timeout, getHostDesc(host), host_status, reason_text); } } } - /// Rethrow an error raised originally on another host. - if (state.error) - state.error->second.rethrow(); - - /// Another host terminated without errors. - if (state.disconnected_host) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "No connection to host {}", *state.disconnected_host); - - /// Something's unready, timeout is probably not enough. - if (state.unready_host) - { - const auto & unready_host = *state.unready_host; - throw Exception( - ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, - "Waited for host {} too long (> {}){}", - unready_host.host, - to_string(*timeout), - unready_host.started ? "" : ": Operation didn't start"); - } - - LOG_TRACE(log, "Everything is Ok. All hosts achieved stage {}", stage_to_wait); - return std::move(*state.results); + LOG_TRACE(log, "Other hosts finished working on this {}", operation_name); + return true; } } diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index a06c5c61041..32f660af997 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -10,33 +10,190 @@ class BackupCoordinationStageSync { public: BackupCoordinationStageSync( - const String & root_zookeeper_path_, - WithRetries & with_retries_, + bool is_restore_, /// true if this is a RESTORE ON CLUSTER command, false if this is a BACKUP ON CLUSTER command + const String & zookeeper_path_, /// path to the "stage" folder in ZooKeeper + const String & current_host_, /// the current host, or an empty string if it's the initiator of the BACKUP/RESTORE ON CLUSTER command + const Strings & all_hosts_, /// all the hosts (including the initiator and the current host) performing the BACKUP/RESTORE ON CLUSTER command + bool allow_concurrency_, /// whether it's allowed to have concurrent backups or restores. + const WithRetries & with_retries_, + ThreadPoolCallbackRunnerUnsafe schedule_, + QueryStatusPtr process_list_element_, LoggerPtr log_); + ~BackupCoordinationStageSync(); + /// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that. - void set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts = false); - void setError(const String & current_host, const Exception & exception); + void setStage(const String & stage, const String & stage_result = {}); - /// Sets the stage of the current host and waits until all hosts come to the same stage. - /// The function returns the messages all hosts set when they come to the required stage. - Strings wait(const Strings & all_hosts, const String & stage_to_wait); + /// Waits until all the specified hosts come to the specified stage. + /// The function returns the results which specified hosts set when they came to the required stage. + /// If it doesn't happen before the timeout then the function will stop waiting and throw an exception. + Strings waitForHostsToReachStage(const String & stage_to_wait, const Strings & hosts, std::optional timeout = {}) const; - /// Almost the same as setAndWait() but this one stops waiting and throws an exception after a specific amount of time. - Strings waitFor(const Strings & all_hosts, const String & stage_to_wait, std::chrono::milliseconds timeout); + /// Waits until all the other hosts finish their work. + /// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled. + void waitForOtherHostsToFinish() const; + + /// Lets other host know that the current host has finished its work. + void finish(bool & other_hosts_also_finished); + + /// Lets other hosts know that the current host has encountered an error. + bool trySetError(std::exception_ptr exception) noexcept; + + /// Waits until all the other hosts finish their work (as a part of error-handling process). + /// Doesn't stops waiting if some host encounters an error or gets cancelled. + bool tryWaitForOtherHostsToFinishAfterError() const noexcept; + + /// Lets other host know that the current host has finished its work (as a part of error-handling process). + bool tryFinishAfterError(bool & other_hosts_also_finished) noexcept; + + /// Returns a printable name of a specific host. For empty host the function returns "initiator". + static String getHostDesc(const String & host); + static String getHostsDesc(const Strings & hosts); private: + /// Initializes the original state. It will be updated then with readCurrentState(). + void initializeState(); + + /// Creates the root node in ZooKeeper. void createRootNodes(); - struct State; - State readCurrentState(WithRetries::RetriesControlHolder & retries_control_holder, const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const; + /// Atomically creates both 'start' and 'alive' nodes and also checks that there is no concurrent backup or restore if `allow_concurrency` is false. + void createStartAndAliveNodes(); + void createStartAndAliveNodes(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); - Strings waitImpl(const Strings & all_hosts, const String & stage_to_wait, std::optional timeout) const; + /// Deserialize the version of a node stored in the 'start' node. + int parseStartNode(const String & start_node_contents, const String & host) const; - String zookeeper_path; - /// A reference to the field of parent object - BackupCoordinationRemote or RestoreCoordinationRemote - WithRetries & with_retries; - LoggerPtr log; + /// Recreates the 'alive' node if it doesn't exist. It's an ephemeral node so it's removed automatically after disconnections. + void createAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); + + /// Checks that there is no concurrent backup or restore if `allow_concurrency` is false. + void checkConcurrency(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); + + /// Watching thread periodically reads the current state from ZooKeeper and recreates the 'alive' node. + void startWatchingThread(); + void stopWatchingThread(); + void watchingThread(); + + /// Reads the current state from ZooKeeper without throwing exceptions. + void readCurrentState(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); + String getStageNodePath(const String & stage) const; + + /// Lets other hosts know that the current host has encountered an error. + bool trySetError(const Exception & exception); + void setError(const Exception & exception); + + /// Deserializes an error stored in the error node. + static std::pair parseErrorNode(const String & error_node_contents); + + /// Reset the `connected` flag for each host. + void resetConnectedFlag(); + + /// Checks if the current query is cancelled, and if so then the function sets the `cancelled` flag in the current state. + void checkIfQueryCancelled(); + + /// Checks if the current state contains an error, and if so then the function passes this error to the query status + /// to cancel the current BACKUP or RESTORE command. + void cancelQueryIfError(); + + /// Checks if some host was disconnected for too long, and if so then the function generates an error and pass it to the query status + /// to cancel the current BACKUP or RESTORE command. + void cancelQueryIfDisconnectedTooLong(); + + /// Used by waitForHostsToReachStage() to check if everything is ready to return. + bool checkIfHostsReachStage(const Strings & hosts, const String & stage_to_wait, bool time_is_out, std::optional timeout, Strings & results) const TSA_REQUIRES(mutex); + + /// Creates the 'finish' node. + bool tryFinishImpl(); + bool tryFinishImpl(bool & other_hosts_also_finished, bool throw_if_error, WithRetries::Kind retries_kind); + void createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); + + /// Waits until all the other hosts finish their work. + bool tryWaitForOtherHostsToFinishImpl(const String & reason, bool throw_if_error, std::optional timeout) const; + bool checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional timeout) const TSA_REQUIRES(mutex); + + const bool is_restore; + const String operation_name; + const String current_host; + const String current_host_desc; + const Strings all_hosts; + const bool allow_concurrency; + + /// A reference to a field of the parent object which is either BackupCoordinationOnCluster or RestoreCoordinationOnCluster. + const WithRetries & with_retries; + + const ThreadPoolCallbackRunnerUnsafe schedule; + const QueryStatusPtr process_list_element; + const LoggerPtr log; + + const std::chrono::seconds failure_after_host_disconnected_for_seconds; + const std::chrono::seconds finish_timeout_after_error; + const std::chrono::milliseconds sync_period_ms; + const size_t max_attempts_after_bad_version; + + /// Paths in ZooKeeper. + const std::filesystem::path zookeeper_path; + const String root_zookeeper_path; + const String operation_node_path; + const String operation_node_name; + const String stage_node_path; + const String start_node_path; + const String finish_node_path; + const String num_hosts_node_path; + const String alive_node_path; + const String alive_tracker_node_path; + const String error_node_path; + + std::shared_ptr zk_nodes_changed; + + /// We store list of previously found ZooKeeper nodes to show better logging messages. + Strings zk_nodes; + + /// Information about one host read from ZooKeeper. + struct HostInfo + { + String host; + bool started = false; + bool connected = false; + bool finished = false; + int version = 0; + std::map stages = {}; /// std::map because we need to compare states + std::exception_ptr exception = nullptr; + + std::chrono::time_point last_connection_time = {}; + std::chrono::time_point last_connection_time_monotonic = {}; + + bool operator ==(const HostInfo & other) const; + bool operator !=(const HostInfo & other) const; + }; + + /// Information about all the host participating in the current BACKUP or RESTORE operation. + struct State + { + std::map hosts; /// std::map because we need to compare states + std::optional host_with_error; + bool cancelled = false; + + bool operator ==(const State & other) const; + bool operator !=(const State & other) const; + }; + + State state TSA_GUARDED_BY(mutex); + mutable std::condition_variable state_changed; + + std::future watching_thread_future; + std::atomic should_stop_watching_thread = false; + + struct FinishResult + { + bool succeeded = false; + std::exception_ptr exception; + bool other_hosts_also_finished = false; + }; + FinishResult finish_result TSA_GUARDED_BY(mutex); + + mutable std::mutex mutex; }; } diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index ae73630d41c..00a4471d994 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -102,7 +102,6 @@ BackupEntriesCollector::BackupEntriesCollector( , read_settings(read_settings_) , context(context_) , process_list_element(context->getProcessListElement()) - , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , collect_metadata_timeout(context->getConfigRef().getUInt64( "backups.collect_metadata_timeout", context->getConfigRef().getUInt64("backups.consistent_metadata_snapshot_timeout", 600000))) , attempts_to_collect_metadata_before_sleep(context->getConfigRef().getUInt("backups.attempts_to_collect_metadata_before_sleep", 2)) @@ -176,21 +175,7 @@ Strings BackupEntriesCollector::setStage(const String & new_stage, const String checkIsQueryCancelled(); current_stage = new_stage; - backup_coordination->setStage(new_stage, message); - - if (new_stage == Stage::formatGatheringMetadata(0)) - { - return backup_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); - } - if (new_stage.starts_with(Stage::GATHERING_METADATA)) - { - auto current_time = std::chrono::steady_clock::now(); - auto end_of_timeout = std::max(current_time, collect_metadata_end_time); - return backup_coordination->waitForStage( - new_stage, std::chrono::duration_cast(end_of_timeout - current_time)); - } - - return backup_coordination->waitForStage(new_stage); + return backup_coordination->setStage(new_stage, message, /* sync = */ true); } void BackupEntriesCollector::checkIsQueryCancelled() const diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index ae076a84c8b..504489cce6b 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -111,10 +111,6 @@ private: ContextPtr context; QueryStatusPtr process_list_element; - /// The time a BACKUP ON CLUSTER or RESTORE ON CLUSTER command will wait until all the nodes receive the BACKUP (or RESTORE) query and start working. - /// This setting is similar to `distributed_ddl_task_timeout`. - const std::chrono::milliseconds on_cluster_first_sync_timeout; - /// The time a BACKUP command will try to collect the metadata of tables & databases. const std::chrono::milliseconds collect_metadata_timeout; diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index ee2f38c785b..c9e0f25f9a0 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -5,6 +5,7 @@ namespace DB { + class IDisk; using DiskPtr = std::shared_ptr; class SeekableReadBuffer; @@ -63,9 +64,13 @@ public: virtual void copyFile(const String & destination, const String & source, size_t size) = 0; + /// Removes a file written to the backup, if it still exists. virtual void removeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; + /// Removes the backup folder if it's empty or contains empty subfolders. + virtual void removeEmptyDirectories() = 0; + virtual const ReadSettings & getReadSettings() const = 0; virtual const WriteSettings & getWriteSettings() const = 0; virtual size_t getWriteBufferSize() const = 0; diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index c3b88f245ab..c90a030a1e7 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -81,6 +81,7 @@ public: void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + void removeEmptyDirectories() override {} private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index aeb07b154f5..794fb5be936 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -91,16 +91,36 @@ std::unique_ptr BackupWriterDisk::writeFile(const String & file_nam void BackupWriterDisk::removeFile(const String & file_name) { disk->removeFileIfExists(root_path / file_name); - if (disk->existsDirectory(root_path) && disk->isDirectoryEmpty(root_path)) - disk->removeDirectory(root_path); } void BackupWriterDisk::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) disk->removeFileIfExists(root_path / file_name); - if (disk->existsDirectory(root_path) && disk->isDirectoryEmpty(root_path)) - disk->removeDirectory(root_path); +} + +void BackupWriterDisk::removeEmptyDirectories() +{ + removeEmptyDirectoriesImpl(root_path); +} + +void BackupWriterDisk::removeEmptyDirectoriesImpl(const fs::path & current_dir) +{ + if (!disk->existsDirectory(current_dir)) + return; + + if (disk->isDirectoryEmpty(current_dir)) + { + disk->removeDirectory(current_dir); + return; + } + + /// Backups are not too deep, so recursion is good enough here. + for (auto it = disk->iterateDirectory(current_dir); it->isValid(); it->next()) + removeEmptyDirectoriesImpl(current_dir / it->name()); + + if (disk->isDirectoryEmpty(current_dir)) + disk->removeDirectory(current_dir); } void BackupWriterDisk::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 3d3253877bd..c77513935a9 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -50,9 +50,11 @@ public: void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + void removeEmptyDirectories() override; private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + void removeEmptyDirectoriesImpl(const std::filesystem::path & current_dir); const DiskPtr disk; const std::filesystem::path root_path; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 681513bf7ce..80f084d241c 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -106,16 +106,36 @@ std::unique_ptr BackupWriterFile::writeFile(const String & file_nam void BackupWriterFile::removeFile(const String & file_name) { (void)fs::remove(root_path / file_name); - if (fs::is_directory(root_path) && fs::is_empty(root_path)) - (void)fs::remove(root_path); } void BackupWriterFile::removeFiles(const Strings & file_names) { for (const auto & file_name : file_names) (void)fs::remove(root_path / file_name); - if (fs::is_directory(root_path) && fs::is_empty(root_path)) - (void)fs::remove(root_path); +} + +void BackupWriterFile::removeEmptyDirectories() +{ + removeEmptyDirectoriesImpl(root_path); +} + +void BackupWriterFile::removeEmptyDirectoriesImpl(const fs::path & current_dir) +{ + if (!fs::is_directory(current_dir)) + return; + + if (fs::is_empty(current_dir)) + { + (void)fs::remove(current_dir); + return; + } + + /// Backups are not too deep, so recursion is good enough here. + for (const auto & it : std::filesystem::directory_iterator{current_dir}) + removeEmptyDirectoriesImpl(it.path()); + + if (fs::is_empty(current_dir)) + (void)fs::remove(current_dir); } void BackupWriterFile::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index ebe9a0f02cb..a2169ac7b4b 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -42,9 +42,11 @@ public: void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + void removeEmptyDirectories() override; private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + void removeEmptyDirectoriesImpl(const std::filesystem::path & current_dir); const std::filesystem::path root_path; const DataSourceDescription data_source_description; diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index a04f1c915b9..4ccf477b369 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -74,6 +74,7 @@ public: void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + void removeEmptyDirectories() override {} private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index b95a2e10b4d..af3fa5531b8 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -147,11 +147,11 @@ BackupImpl::BackupImpl( BackupImpl::~BackupImpl() { - if ((open_mode == OpenMode::WRITE) && !is_internal_backup && !writing_finalized && !std::uncaught_exceptions() && !std::current_exception()) + if ((open_mode == OpenMode::WRITE) && !writing_finalized && !corrupted) { /// It is suspicious to destroy BackupImpl without finalization while writing a backup when there is no exception. - LOG_ERROR(log, "BackupImpl is not finalized when destructor is called. Stack trace: {}", StackTrace().toString()); - chassert(false && "BackupImpl is not finalized when destructor is called."); + LOG_ERROR(log, "BackupImpl is not finalized or marked as corrupted when destructor is called. Stack trace: {}", StackTrace().toString()); + chassert(false, "BackupImpl is not finalized or marked as corrupted when destructor is called."); } try @@ -196,9 +196,6 @@ void BackupImpl::open() if (open_mode == OpenMode::READ) readBackupMetadata(); - - if ((open_mode == OpenMode::WRITE) && base_backup_info) - base_backup_uuid = getBaseBackupUnlocked()->getUUID(); } void BackupImpl::close() @@ -280,6 +277,8 @@ std::shared_ptr BackupImpl::getBaseBackupUnlocked() const toString(base_backup->getUUID()), (base_backup_uuid ? toString(*base_backup_uuid) : "")); } + + base_backup_uuid = base_backup->getUUID(); } return base_backup; } @@ -369,7 +368,7 @@ void BackupImpl::writeBackupMetadata() if (base_backup_in_use) { *out << "" << xml << base_backup_info->toString() << ""; - *out << "" << toString(*base_backup_uuid) << ""; + *out << "" << getBaseBackupUnlocked()->getUUID() << ""; } } @@ -594,9 +593,6 @@ bool BackupImpl::checkLockFile(bool throw_if_failed) const void BackupImpl::removeLockFile() { - if (is_internal_backup) - return; /// Internal backup must not remove the lock file (it's still used by the initiator). - if (checkLockFile(false)) writer->removeFile(lock_file_name); } @@ -989,8 +985,11 @@ void BackupImpl::finalizeWriting() if (open_mode != OpenMode::WRITE) throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is not opened for writing"); + if (corrupted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup can't be finalized after an error happened"); + if (writing_finalized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is already finalized"); + return; if (!is_internal_backup) { @@ -1015,20 +1014,58 @@ void BackupImpl::setCompressedSize() } -void BackupImpl::tryRemoveAllFiles() +bool BackupImpl::setIsCorrupted() noexcept { - if (open_mode != OpenMode::WRITE) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Backup is not opened for writing"); - - if (is_internal_backup) - return; - try { - LOG_INFO(log, "Removing all files of backup {}", backup_name_for_logging); + std::lock_guard lock{mutex}; + if (open_mode != OpenMode::WRITE) + { + LOG_ERROR(log, "Backup is not opened for writing. Stack trace: {}", StackTrace().toString()); + chassert(false, "Backup is not opened for writing when setIsCorrupted() is called"); + return false; + } + + if (writing_finalized) + { + LOG_WARNING(log, "An error happened after the backup was completed successfully, the backup must be correct!"); + return false; + } + + if (corrupted) + return true; + + LOG_WARNING(log, "An error happened, the backup won't be completed"); + closeArchive(/* finalize= */ false); + corrupted = true; + return true; + } + catch (...) + { + DB::tryLogCurrentException(log, "Caught exception while setting that the backup was corrupted"); + return false; + } +} + + +bool BackupImpl::tryRemoveAllFiles() noexcept +{ + try + { + std::lock_guard lock{mutex}; + if (!corrupted) + { + LOG_ERROR(log, "Backup is not set as corrupted. Stack trace: {}", StackTrace().toString()); + chassert(false, "Backup is not set as corrupted when tryRemoveAllFiles() is called"); + return false; + } + + LOG_INFO(log, "Removing all files of backup {}", backup_name_for_logging); + Strings files_to_remove; + if (use_archive) { files_to_remove.push_back(archive_params.archive_name); @@ -1041,14 +1078,17 @@ void BackupImpl::tryRemoveAllFiles() } if (!checkLockFile(false)) - return; + return false; writer->removeFiles(files_to_remove); removeLockFile(); + writer->removeEmptyDirectories(); + return true; } catch (...) { - DB::tryLogCurrentException(__PRETTY_FUNCTION__); + DB::tryLogCurrentException(log, "Caught exception while removing files of a corrupted backup"); + return false; } } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index d7846104c4c..4b0f9f879ec 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -86,7 +86,8 @@ public: void writeFile(const BackupFileInfo & info, BackupEntryPtr entry) override; bool supportsWritingInMultipleThreads() const override { return !use_archive; } void finalizeWriting() override; - void tryRemoveAllFiles() override; + bool setIsCorrupted() noexcept override; + bool tryRemoveAllFiles() noexcept override; private: void open(); @@ -146,13 +147,14 @@ private: int version; mutable std::optional base_backup_info; mutable std::shared_ptr base_backup; - std::optional base_backup_uuid; + mutable std::optional base_backup_uuid; std::shared_ptr archive_reader; std::shared_ptr archive_writer; String lock_file_name; std::atomic lock_file_before_first_file_checked = false; bool writing_finalized = false; + bool corrupted = false; bool deduplicate_files = true; bool use_same_s3_credentials_for_base_backup = false; bool use_same_password_for_base_backup = false; diff --git a/src/Backups/BackupKeeperSettings.cpp b/src/Backups/BackupKeeperSettings.cpp new file mode 100644 index 00000000000..180633cea1f --- /dev/null +++ b/src/Backups/BackupKeeperSettings.cpp @@ -0,0 +1,58 @@ +#include + +#include +#include +#include + + +namespace DB +{ + +namespace Setting +{ + extern const SettingsUInt64 backup_restore_keeper_max_retries; + extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms; + extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms; + extern const SettingsUInt64 backup_restore_failure_after_host_disconnected_for_seconds; + extern const SettingsUInt64 backup_restore_keeper_max_retries_while_initializing; + extern const SettingsUInt64 backup_restore_keeper_max_retries_while_handling_error; + extern const SettingsUInt64 backup_restore_finish_timeout_after_error_sec; + extern const SettingsUInt64 backup_restore_keeper_value_max_size; + extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multi; + extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multiread; + extern const SettingsFloat backup_restore_keeper_fault_injection_probability; + extern const SettingsUInt64 backup_restore_keeper_fault_injection_seed; +} + +BackupKeeperSettings BackupKeeperSettings::fromContext(const ContextPtr & context) +{ + BackupKeeperSettings keeper_settings; + + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + + keeper_settings.max_retries = settings[Setting::backup_restore_keeper_max_retries]; + keeper_settings.retry_initial_backoff_ms = std::chrono::milliseconds{settings[Setting::backup_restore_keeper_retry_initial_backoff_ms]}; + keeper_settings.retry_max_backoff_ms = std::chrono::milliseconds{settings[Setting::backup_restore_keeper_retry_max_backoff_ms]}; + + keeper_settings.failure_after_host_disconnected_for_seconds = std::chrono::seconds{settings[Setting::backup_restore_failure_after_host_disconnected_for_seconds]}; + keeper_settings.max_retries_while_initializing = settings[Setting::backup_restore_keeper_max_retries_while_initializing]; + keeper_settings.max_retries_while_handling_error = settings[Setting::backup_restore_keeper_max_retries_while_handling_error]; + keeper_settings.finish_timeout_after_error = std::chrono::seconds(settings[Setting::backup_restore_finish_timeout_after_error_sec]); + + if (config.has("backups.sync_period_ms")) + keeper_settings.sync_period_ms = std::chrono::milliseconds{config.getUInt64("backups.sync_period_ms")}; + + if (config.has("backups.max_attempts_after_bad_version")) + keeper_settings.max_attempts_after_bad_version = config.getUInt64("backups.max_attempts_after_bad_version"); + + keeper_settings.value_max_size = settings[Setting::backup_restore_keeper_value_max_size]; + keeper_settings.batch_size_for_multi = settings[Setting::backup_restore_batch_size_for_keeper_multi]; + keeper_settings.batch_size_for_multiread = settings[Setting::backup_restore_batch_size_for_keeper_multiread]; + keeper_settings.fault_injection_probability = settings[Setting::backup_restore_keeper_fault_injection_probability]; + keeper_settings.fault_injection_seed = settings[Setting::backup_restore_keeper_fault_injection_seed]; + + return keeper_settings; +} + +} diff --git a/src/Backups/BackupKeeperSettings.h b/src/Backups/BackupKeeperSettings.h new file mode 100644 index 00000000000..6c4b2187094 --- /dev/null +++ b/src/Backups/BackupKeeperSettings.h @@ -0,0 +1,64 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Settings for [Zoo]Keeper-related works during BACKUP or RESTORE. +struct BackupKeeperSettings +{ + /// Maximum number of retries in the middle of a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. + /// Should be big enough so the whole operation won't be cancelled in the middle of it because of a temporary ZooKeeper failure. + UInt64 max_retries{1000}; + + /// Initial backoff timeout for ZooKeeper operations during backup or restore. + std::chrono::milliseconds retry_initial_backoff_ms{100}; + + /// Max backoff timeout for ZooKeeper operations during backup or restore. + std::chrono::milliseconds retry_max_backoff_ms{5000}; + + /// If a host during BACKUP ON CLUSTER or RESTORE ON CLUSTER doesn't recreate its 'alive' node in ZooKeeper + /// for this amount of time then the whole backup or restore is considered as failed. + /// Should be bigger than any reasonable time for a host to reconnect to ZooKeeper after a failure. + /// Set to zero to disable (if it's zero and some host crashed then BACKUP ON CLUSTER or RESTORE ON CLUSTER will be waiting + /// for the crashed host forever until the operation is explicitly cancelled with KILL QUERY). + std::chrono::seconds failure_after_host_disconnected_for_seconds{3600}; + + /// Maximum number of retries during the initialization of a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. + /// Shouldn't be too big because if the operation is going to fail then it's better if it fails faster. + UInt64 max_retries_while_initializing{20}; + + /// Maximum number of retries while handling an error of a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. + /// Shouldn't be too big because those retries are just for cleanup after the operation has failed already. + UInt64 max_retries_while_handling_error{20}; + + /// How long the initiator should wait for other host to handle the 'error' node and finish their work. + std::chrono::seconds finish_timeout_after_error{180}; + + /// How often the "stage" folder in ZooKeeper must be scanned in a background thread to track changes done by other hosts. + std::chrono::milliseconds sync_period_ms{5000}; + + /// Number of attempts after getting error ZBADVERSION from ZooKeeper. + size_t max_attempts_after_bad_version{10}; + + /// Maximum size of data of a ZooKeeper's node during backup. + UInt64 value_max_size{1048576}; + + /// Maximum size of a batch for a multi request. + UInt64 batch_size_for_multi{1000}; + + /// Maximum size of a batch for a multiread request. + UInt64 batch_size_for_multiread{10000}; + + /// Approximate probability of failure for a keeper request during backup or restore. Valid value is in interval [0.0f, 1.0f]. + Float64 fault_injection_probability{0}; + + /// Seed for `fault_injection_probability`: 0 - random seed, otherwise the setting value. + UInt64 fault_injection_seed{0}; + + static BackupKeeperSettings fromContext(const ContextPtr & context); +}; + +} diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 9b8117c6587..915989735c3 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -74,6 +74,17 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) return res; } +bool BackupSettings::isAsync(const ASTBackupQuery & query) +{ + if (query.settings) + { + const auto * field = query.settings->as().changes.tryGet("async"); + if (field) + return field->safeGet(); + } + return false; /// `async` is false by default. +} + void BackupSettings::copySettingsToQuery(ASTBackupQuery & query) const { auto query_settings = std::make_shared(); diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 8c2ea21df01..fa1e5025935 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -101,6 +101,8 @@ struct BackupSettings static BackupSettings fromBackupQuery(const ASTBackupQuery & query); void copySettingsToQuery(ASTBackupQuery & query) const; + static bool isAsync(const ASTBackupQuery & query); + struct Util { static std::vector clusterHostIDsFromAST(const IAST & ast); diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index d3889295598..8480dc5d64d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -1,4 +1,6 @@ #include + +#include #include #include #include @@ -6,9 +8,9 @@ #include #include #include -#include +#include #include -#include +#include #include #include #include @@ -43,21 +45,11 @@ namespace CurrentMetrics namespace DB { -namespace Setting -{ - extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multiread; - extern const SettingsUInt64 backup_restore_keeper_max_retries; - extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms; - extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms; - extern const SettingsUInt64 backup_restore_keeper_fault_injection_seed; - extern const SettingsFloat backup_restore_keeper_fault_injection_probability; -} namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int CONCURRENT_ACCESS_NOT_SUPPORTED; extern const int QUERY_WAS_CANCELLED; } @@ -66,102 +58,6 @@ namespace Stage = BackupCoordinationStage; namespace { - std::shared_ptr makeBackupCoordination(const ContextPtr & context, const BackupSettings & backup_settings, bool remote) - { - if (remote) - { - String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - - auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - - BackupCoordinationRemote::BackupKeeperSettings keeper_settings = WithRetries::KeeperSettings::fromContext(context); - - auto all_hosts = BackupSettings::Util::filterHostIDs( - backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); - - return std::make_shared( - get_zookeeper, - root_zk_path, - keeper_settings, - toString(*backup_settings.backup_uuid), - all_hosts, - backup_settings.host_id, - !backup_settings.deduplicate_files, - backup_settings.internal, - context->getProcessListElement()); - } - - return std::make_shared(!backup_settings.deduplicate_files); - } - - std::shared_ptr - makeRestoreCoordination(const ContextPtr & context, const RestoreSettings & restore_settings, bool remote) - { - if (remote) - { - String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - - auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - - RestoreCoordinationRemote::RestoreKeeperSettings keeper_settings - { - .keeper_max_retries = context->getSettingsRef()[Setting::backup_restore_keeper_max_retries], - .keeper_retry_initial_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_initial_backoff_ms], - .keeper_retry_max_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_max_backoff_ms], - .batch_size_for_keeper_multiread = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multiread], - .keeper_fault_injection_probability = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_probability], - .keeper_fault_injection_seed = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_seed] - }; - - auto all_hosts = BackupSettings::Util::filterHostIDs( - restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); - - return std::make_shared( - get_zookeeper, - root_zk_path, - keeper_settings, - toString(*restore_settings.restore_uuid), - all_hosts, - restore_settings.host_id, - restore_settings.internal, - context->getProcessListElement()); - } - - return std::make_shared(); - } - - /// Sends information about an exception to IBackupCoordination or IRestoreCoordination. - template - void sendExceptionToCoordination(std::shared_ptr coordination, const Exception & exception) - { - try - { - if (coordination) - coordination->setError(exception); - } - catch (...) // NOLINT(bugprone-empty-catch) - { - } - } - - /// Sends information about the current exception to IBackupCoordination or IRestoreCoordination. - template - void sendCurrentExceptionToCoordination(std::shared_ptr coordination) - { - try - { - throw; - } - catch (const Exception & e) - { - sendExceptionToCoordination(coordination, e); - } - catch (...) - { - sendExceptionToCoordination(coordination, Exception(getCurrentExceptionMessageAndPattern(true, true), getCurrentExceptionCode())); - } - } - bool isFinishedSuccessfully(BackupStatus status) { return (status == BackupStatus::BACKUP_CREATED) || (status == BackupStatus::RESTORED); @@ -262,24 +158,27 @@ namespace /// while the thread pool is still occupied with the waiting task then a scheduled task can be never executed). enum class BackupsWorker::ThreadPoolId : uint8_t { - /// "BACKUP ON CLUSTER ASYNC" waits in background while "BACKUP ASYNC" is finished on the nodes of the cluster, then finalizes the backup. - BACKUP_ASYNC_ON_CLUSTER = 0, + /// Making a list of files to copy or copying those files. + BACKUP, - /// "BACKUP ASYNC" waits in background while all file infos are built and then it copies the backup's files. - BACKUP_ASYNC = 1, + /// Creating of tables and databases during RESTORE and filling them with data. + RESTORE, - /// Making a list of files to copy and copying of those files is always sequential, so those operations can share one thread pool. - BACKUP_MAKE_FILES_LIST = 2, - BACKUP_COPY_FILES = BACKUP_MAKE_FILES_LIST, + /// We need background threads for ASYNC backups and restores. + ASYNC_BACKGROUND_BACKUP, + ASYNC_BACKGROUND_RESTORE, - /// "RESTORE ON CLUSTER ASYNC" waits in background while "BACKUP ASYNC" is finished on the nodes of the cluster, then finalizes the backup. - RESTORE_ASYNC_ON_CLUSTER = 3, + /// We need background threads for coordination workers (see BackgroundCoordinationStageSync). + ON_CLUSTER_COORDINATION_BACKUP, + ON_CLUSTER_COORDINATION_RESTORE, - /// "RESTORE ASYNC" waits in background while the data of all tables are restored. - RESTORE_ASYNC = 4, - - /// Restores from backups. - RESTORE = 5, + /// We need separate threads for internal backups and restores. + /// An internal backup is a helper backup invoked on some shard and replica by a BACKUP ON CLUSTER command, + /// (see BackupSettings.internal); and the same for restores. + ASYNC_BACKGROUND_INTERNAL_BACKUP, + ASYNC_BACKGROUND_INTERNAL_RESTORE, + ON_CLUSTER_COORDINATION_INTERNAL_BACKUP, + ON_CLUSTER_COORDINATION_INTERNAL_RESTORE, }; @@ -312,22 +211,26 @@ public: switch (thread_pool_id) { - case ThreadPoolId::BACKUP_ASYNC: - case ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER: - case ThreadPoolId::BACKUP_COPY_FILES: + case ThreadPoolId::BACKUP: + case ThreadPoolId::ASYNC_BACKGROUND_BACKUP: + case ThreadPoolId::ON_CLUSTER_COORDINATION_BACKUP: + case ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_BACKUP: + case ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_BACKUP: { metric_threads = CurrentMetrics::BackupsThreads; metric_active_threads = CurrentMetrics::BackupsThreadsActive; metric_active_threads = CurrentMetrics::BackupsThreadsScheduled; max_threads = num_backup_threads; /// We don't use thread pool queues for thread pools with a lot of tasks otherwise that queue could be memory-wasting. - use_queue = (thread_pool_id != ThreadPoolId::BACKUP_COPY_FILES); + use_queue = (thread_pool_id != ThreadPoolId::BACKUP); break; } - case ThreadPoolId::RESTORE_ASYNC: - case ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER: case ThreadPoolId::RESTORE: + case ThreadPoolId::ASYNC_BACKGROUND_RESTORE: + case ThreadPoolId::ON_CLUSTER_COORDINATION_RESTORE: + case ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_RESTORE: + case ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_RESTORE: { metric_threads = CurrentMetrics::RestoreThreads; metric_active_threads = CurrentMetrics::RestoreThreadsActive; @@ -352,12 +255,20 @@ public: void wait() { auto wait_sequence = { - ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER, - ThreadPoolId::RESTORE_ASYNC, + /// ASYNC_BACKGROUND_BACKUP must be before ASYNC_BACKGROUND_INTERNAL_BACKUP, + /// ASYNC_BACKGROUND_RESTORE must be before ASYNC_BACKGROUND_INTERNAL_RESTORE, + /// and everything else is after those ones. + ThreadPoolId::ASYNC_BACKGROUND_BACKUP, + ThreadPoolId::ASYNC_BACKGROUND_RESTORE, + ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_BACKUP, + ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_RESTORE, + /// Others: + ThreadPoolId::BACKUP, ThreadPoolId::RESTORE, - ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER, - ThreadPoolId::BACKUP_ASYNC, - ThreadPoolId::BACKUP_COPY_FILES, + ThreadPoolId::ON_CLUSTER_COORDINATION_BACKUP, + ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_BACKUP, + ThreadPoolId::ON_CLUSTER_COORDINATION_RESTORE, + ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_RESTORE, }; for (auto thread_pool_id : wait_sequence) @@ -392,6 +303,7 @@ BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup , log(getLogger("BackupsWorker")) , backup_log(global_context->getBackupLog()) , process_list(global_context->getProcessList()) + , concurrency_counters(std::make_unique()) { } @@ -405,7 +317,7 @@ ThreadPool & BackupsWorker::getThreadPool(ThreadPoolId thread_pool_id) } -OperationID BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context) +std::pair BackupsWorker::start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context) { const ASTBackupQuery & backup_query = typeid_cast(*backup_or_restore_query); if (backup_query.kind == ASTBackupQuery::Kind::BACKUP) @@ -414,180 +326,147 @@ OperationID BackupsWorker::start(const ASTPtr & backup_or_restore_query, Context } -OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context) +struct BackupsWorker::BackupStarter { - auto backup_query = std::static_pointer_cast(query->clone()); - auto backup_settings = BackupSettings::fromBackupQuery(*backup_query); - - auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); - String backup_name_for_logging = backup_info.toStringForLogging(); - - if (!backup_settings.backup_uuid) - backup_settings.backup_uuid = UUIDHelpers::generateV4(); - - /// `backup_id` will be used as a key to the `infos` map, so it should be unique. - OperationID backup_id; - if (backup_settings.internal) - backup_id = "internal-" + toString(UUIDHelpers::generateV4()); /// Always generate `backup_id` for internal backup to avoid collision if both internal and non-internal backups are on the same host - else if (!backup_settings.id.empty()) - backup_id = backup_settings.id; - else - backup_id = toString(*backup_settings.backup_uuid); - + BackupsWorker & backups_worker; + std::shared_ptr backup_query; + ContextPtr query_context; /// We have to keep `query_context` until the end of the operation because a pointer to it is stored inside the ThreadGroup we're using. + ContextMutablePtr backup_context; + BackupSettings backup_settings; + BackupInfo backup_info; + String backup_id; + String backup_name_for_logging; + bool on_cluster; + bool is_internal_backup; std::shared_ptr backup_coordination; + ClusterPtr cluster; BackupMutablePtr backup; + std::shared_ptr process_list_element_holder; - /// Called in exception handlers below. This lambda function can be called on a separate thread, so it can't capture local variables by reference. - auto on_exception = [this](BackupMutablePtr & backup_, const OperationID & backup_id_, const String & backup_name_for_logging_, - const BackupSettings & backup_settings_, const std::shared_ptr & backup_coordination_) + BackupStarter(BackupsWorker & backups_worker_, const ASTPtr & query_, const ContextPtr & context_) + : backups_worker(backups_worker_) + , backup_query(std::static_pointer_cast(query_->clone())) + , query_context(context_) + , backup_context(Context::createCopy(query_context)) { - /// Something bad happened, the backup has not built. - tryLogCurrentException(log, fmt::format("Failed to make {} {}", (backup_settings_.internal ? "internal backup" : "backup"), backup_name_for_logging_)); - setStatusSafe(backup_id_, getBackupStatusFromCurrentException()); - sendCurrentExceptionToCoordination(backup_coordination_); + backup_context->makeQueryContext(); + backup_settings = BackupSettings::fromBackupQuery(*backup_query); + backup_info = BackupInfo::fromAST(*backup_query->backup_name); + backup_name_for_logging = backup_info.toStringForLogging(); + is_internal_backup = backup_settings.internal; + on_cluster = !backup_query->cluster.empty() || is_internal_backup; - if (backup_ && remove_backup_files_after_failure) - backup_->tryRemoveAllFiles(); - backup_.reset(); - }; + if (!backup_settings.backup_uuid) + backup_settings.backup_uuid = UUIDHelpers::generateV4(); + + /// `backup_id` will be used as a key to the `infos` map, so it should be unique. + if (is_internal_backup) + backup_id = "internal-" + toString(UUIDHelpers::generateV4()); /// Always generate `backup_id` for internal backup to avoid collision if both internal and non-internal backups are on the same host + else if (!backup_settings.id.empty()) + backup_id = backup_settings.id; + else + backup_id = toString(*backup_settings.backup_uuid); - try - { String base_backup_name; if (backup_settings.base_backup_info) base_backup_name = backup_settings.base_backup_info->toStringForLogging(); - addInfo(backup_id, + /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. + auto process_list_element = backup_context->getProcessListElement(); + if (process_list_element) + process_list_element_holder = process_list_element->getProcessListEntry(); + + backups_worker.addInfo(backup_id, backup_name_for_logging, base_backup_name, - context->getCurrentQueryId(), - backup_settings.internal, - context->getProcessListElement(), + backup_context->getCurrentQueryId(), + is_internal_backup, + process_list_element, BackupStatus::CREATING_BACKUP); + } - if (backup_settings.internal) + void doBackup() + { + chassert(!backup_coordination); + if (on_cluster && !is_internal_backup) { - /// The following call of makeBackupCoordination() is not essential because doBackup() will later create a backup coordination - /// if it's not created here. However to handle errors better it's better to make a coordination here because this way - /// if an exception will be thrown in startMakingBackup() other hosts will know about that. - backup_coordination = makeBackupCoordination(context, backup_settings, /* remote= */ true); + backup_query->cluster = backup_context->getMacros()->expand(backup_query->cluster); + cluster = backup_context->getCluster(backup_query->cluster); + backup_settings.cluster_host_ids = cluster->getHostIDs(); + } + backup_coordination = backups_worker.makeBackupCoordination(on_cluster, backup_settings, backup_context); + + chassert(!backup); + backup = backups_worker.openBackupForWriting(backup_info, backup_settings, backup_coordination, backup_context); + + backups_worker.doBackup( + backup, backup_query, backup_id, backup_name_for_logging, backup_settings, backup_coordination, backup_context, + on_cluster, cluster); + } + + void onException() + { + /// Something bad happened, the backup has not built. + tryLogCurrentException(backups_worker.log, fmt::format("Failed to make {} {}", + (is_internal_backup ? "internal backup" : "backup"), + backup_name_for_logging)); + + bool should_remove_files_in_backup = backup && !is_internal_backup && backups_worker.remove_backup_files_after_failure; + + if (backup && !backup->setIsCorrupted()) + should_remove_files_in_backup = false; + + if (backup_coordination && backup_coordination->trySetError(std::current_exception())) + { + bool other_hosts_finished = backup_coordination->tryWaitForOtherHostsToFinishAfterError(); + + if (should_remove_files_in_backup && other_hosts_finished) + backup->tryRemoveAllFiles(); + + backup_coordination->tryFinishAfterError(); } - /// Prepare context to use. - ContextPtr context_in_use = context; - ContextMutablePtr mutable_context; - bool on_cluster = !backup_query->cluster.empty(); - if (on_cluster || backup_settings.async) - { - /// We have to clone the query context here because: - /// if this is an "ON CLUSTER" query we need to change some settings, and - /// if this is an "ASYNC" query it's going to be executed in another thread. - context_in_use = mutable_context = Context::createCopy(context); - mutable_context->makeQueryContext(); - } + backups_worker.setStatusSafe(backup_id, getBackupStatusFromCurrentException()); + } +}; - if (backup_settings.async) - { - auto & thread_pool = getThreadPool(on_cluster ? ThreadPoolId::BACKUP_ASYNC_ON_CLUSTER : ThreadPoolId::BACKUP_ASYNC); - /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. - auto process_list_element = context_in_use->getProcessListElement(); +std::pair BackupsWorker::startMakingBackup(const ASTPtr & query, const ContextPtr & context) +{ + auto starter = std::make_shared(*this, query, context); - thread_pool.scheduleOrThrowOnError( - [this, - backup_query, - backup_id, - backup_name_for_logging, - backup_info, - backup_settings, - backup_coordination, - context_in_use, - mutable_context, - on_exception, - process_list_element_holder = process_list_element ? process_list_element->getProcessListEntry() : nullptr] + try + { + auto thread_pool_id = starter->is_internal_backup ? ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_BACKUP: ThreadPoolId::ASYNC_BACKGROUND_BACKUP; + String thread_name = starter->is_internal_backup ? "BackupAsyncInt" : "BackupAsync"; + auto schedule = threadPoolCallbackRunnerUnsafe(thread_pools->getThreadPool(thread_pool_id), thread_name); + + schedule([starter] + { + try { - BackupMutablePtr backup_async; - try - { - setThreadName("BackupWorker"); - CurrentThread::QueryScope query_scope(context_in_use); - doBackup( - backup_async, - backup_query, - backup_id, - backup_name_for_logging, - backup_info, - backup_settings, - backup_coordination, - context_in_use, - mutable_context); - } - catch (...) - { - on_exception(backup_async, backup_id, backup_name_for_logging, backup_settings, backup_coordination); - } - }); - } - else - { - doBackup( - backup, - backup_query, - backup_id, - backup_name_for_logging, - backup_info, - backup_settings, - backup_coordination, - context_in_use, - mutable_context); - } + starter->doBackup(); + } + catch (...) + { + starter->onException(); + } + }, + Priority{}); - return backup_id; + return {starter->backup_id, BackupStatus::CREATING_BACKUP}; } catch (...) { - on_exception(backup, backup_id, backup_name_for_logging, backup_settings, backup_coordination); + starter->onException(); throw; } } -void BackupsWorker::doBackup( - BackupMutablePtr & backup, - const std::shared_ptr & backup_query, - const OperationID & backup_id, - const String & backup_name_for_logging, - const BackupInfo & backup_info, - BackupSettings backup_settings, - std::shared_ptr backup_coordination, - const ContextPtr & context, - ContextMutablePtr mutable_context) +BackupMutablePtr BackupsWorker::openBackupForWriting(const BackupInfo & backup_info, const BackupSettings & backup_settings, std::shared_ptr backup_coordination, const ContextPtr & context) const { - bool on_cluster = !backup_query->cluster.empty(); - assert(!on_cluster || mutable_context); - - /// Checks access rights if this is not ON CLUSTER query. - /// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.) - auto required_access = BackupUtils::getRequiredAccessToBackup(backup_query->elements); - if (!on_cluster) - context->checkAccess(required_access); - - ClusterPtr cluster; - if (on_cluster) - { - backup_query->cluster = context->getMacros()->expand(backup_query->cluster); - cluster = context->getCluster(backup_query->cluster); - backup_settings.cluster_host_ids = cluster->getHostIDs(); - } - - /// Make a backup coordination. - if (!backup_coordination) - backup_coordination = makeBackupCoordination(context, backup_settings, /* remote= */ on_cluster); - - if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(std::ref(num_active_backups))) - throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); - - /// Opens a backup for writing. + LOG_TRACE(log, "Opening backup for writing"); BackupFactory::CreateParams backup_create_params; backup_create_params.open_mode = IBackup::OpenMode::WRITE; backup_create_params.context = context; @@ -608,37 +487,57 @@ void BackupsWorker::doBackup( backup_create_params.azure_attempt_to_create_container = backup_settings.azure_attempt_to_create_container; backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings); backup_create_params.write_settings = getWriteSettingsForBackup(context); - backup = BackupFactory::instance().createBackup(backup_create_params); + auto backup = BackupFactory::instance().createBackup(backup_create_params); + LOG_INFO(log, "Opened backup for writing"); + return backup; +} + + +void BackupsWorker::doBackup( + BackupMutablePtr backup, + const std::shared_ptr & backup_query, + const OperationID & backup_id, + const String & backup_name_for_logging, + const BackupSettings & backup_settings, + std::shared_ptr backup_coordination, + ContextMutablePtr context, + bool on_cluster, + const ClusterPtr & cluster) +{ + bool is_internal_backup = backup_settings.internal; + + /// Checks access rights if this is not ON CLUSTER query. + /// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.) + auto required_access = BackupUtils::getRequiredAccessToBackup(backup_query->elements); + if (!on_cluster) + context->checkAccess(required_access); + + maybeSleepForTesting(); /// Write the backup. - if (on_cluster) + if (on_cluster && !is_internal_backup) { - DDLQueryOnClusterParams params; - params.cluster = cluster; - params.only_shard_num = backup_settings.shard_num; - params.only_replica_num = backup_settings.replica_num; - params.access_to_check = required_access; + /// Send the BACKUP query to other hosts. backup_settings.copySettingsToQuery(*backup_query); - - // executeDDLQueryOnCluster() will return without waiting for completion - mutable_context->setSetting("distributed_ddl_task_timeout", Field{0}); - mutable_context->setSetting("distributed_ddl_output_mode", Field{"none"}); - executeDDLQueryOnCluster(backup_query, mutable_context, params); + sendQueryToOtherHosts(*backup_query, cluster, backup_settings.shard_num, backup_settings.replica_num, + context, required_access, backup_coordination->getOnClusterInitializationKeeperRetriesInfo()); + backup_coordination->setBackupQueryWasSentToOtherHosts(); /// Wait until all the hosts have written their backup entries. - backup_coordination->waitForStage(Stage::COMPLETED); - backup_coordination->setStage(Stage::COMPLETED,""); + backup_coordination->waitForOtherHostsToFinish(); } else { backup_query->setCurrentDatabase(context->getCurrentDatabase()); + auto read_settings = getReadSettingsForBackup(context, backup_settings); + /// Prepare backup entries. BackupEntries backup_entries; { BackupEntriesCollector backup_entries_collector( backup_query->elements, backup_settings, backup_coordination, - backup_create_params.read_settings, context, getThreadPool(ThreadPoolId::BACKUP_MAKE_FILES_LIST)); + read_settings, context, getThreadPool(ThreadPoolId::BACKUP)); backup_entries = backup_entries_collector.run(); } @@ -646,11 +545,11 @@ void BackupsWorker::doBackup( chassert(backup); chassert(backup_coordination); chassert(context); - buildFileInfosForBackupEntries(backup, backup_entries, backup_create_params.read_settings, backup_coordination, context->getProcessListElement()); - writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, backup_settings.internal, context->getProcessListElement()); + buildFileInfosForBackupEntries(backup, backup_entries, read_settings, backup_coordination, context->getProcessListElement()); + writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, is_internal_backup, context->getProcessListElement()); - /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). - backup_coordination->setStage(Stage::COMPLETED,""); + /// We have written our backup entries (there is no need to sync it with other hosts because it's the last stage). + backup_coordination->setStage(Stage::COMPLETED, "", /* sync = */ false); } size_t num_files = 0; @@ -660,9 +559,9 @@ void BackupsWorker::doBackup( UInt64 compressed_size = 0; /// Finalize backup (write its metadata). - if (!backup_settings.internal) + backup->finalizeWriting(); + if (!is_internal_backup) { - backup->finalizeWriting(); num_files = backup->getNumFiles(); total_size = backup->getTotalSize(); num_entries = backup->getNumEntries(); @@ -673,19 +572,22 @@ void BackupsWorker::doBackup( /// Close the backup. backup.reset(); - LOG_INFO(log, "{} {} was created successfully", (backup_settings.internal ? "Internal backup" : "Backup"), backup_name_for_logging); + /// The backup coordination is not needed anymore. + backup_coordination->finish(); + /// NOTE: we need to update metadata again after backup->finalizeWriting(), because backup metadata is written there. setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0); + /// NOTE: setStatus is called after setNumFilesAndSize in order to have actual information in a backup log record + LOG_INFO(log, "{} {} was created successfully", (is_internal_backup ? "Internal backup" : "Backup"), backup_name_for_logging); setStatus(backup_id, BackupStatus::BACKUP_CREATED); } void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr backup_coordination, QueryStatusPtr process_list_element) { - backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, ""); - backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS); - backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, getThreadPool(ThreadPoolId::BACKUP_MAKE_FILES_LIST), process_list_element)); + backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, "", /* sync = */ true); + backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, getThreadPool(ThreadPoolId::BACKUP), process_list_element)); } @@ -694,12 +596,11 @@ void BackupsWorker::writeBackupEntries( BackupEntries && backup_entries, const OperationID & backup_id, std::shared_ptr backup_coordination, - bool internal, + bool is_internal_backup, QueryStatusPtr process_list_element) { LOG_TRACE(log, "{}, num backup entries={}", Stage::WRITING_BACKUP, backup_entries.size()); - backup_coordination->setStage(Stage::WRITING_BACKUP, ""); - backup_coordination->waitForStage(Stage::WRITING_BACKUP); + backup_coordination->setStage(Stage::WRITING_BACKUP, "", /* sync = */ true); auto file_infos = backup_coordination->getFileInfos(); if (file_infos.size() != backup_entries.size()) @@ -715,7 +616,7 @@ void BackupsWorker::writeBackupEntries( std::atomic_bool failed = false; bool always_single_threaded = !backup->supportsWritingInMultipleThreads(); - auto & thread_pool = getThreadPool(ThreadPoolId::BACKUP_COPY_FILES); + auto & thread_pool = getThreadPool(ThreadPoolId::BACKUP); std::vector writing_order; if (test_randomize_order) @@ -751,7 +652,7 @@ void BackupsWorker::writeBackupEntries( maybeSleepForTesting(); // Update metadata - if (!internal) + if (!is_internal_backup) { setNumFilesAndSize( backup_id, @@ -783,142 +684,139 @@ void BackupsWorker::writeBackupEntries( } -OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) +struct BackupsWorker::RestoreStarter { - auto restore_query = std::static_pointer_cast(query->clone()); - auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); - - auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); - String backup_name_for_logging = backup_info.toStringForLogging(); - - if (!restore_settings.restore_uuid) - restore_settings.restore_uuid = UUIDHelpers::generateV4(); - - /// `restore_id` will be used as a key to the `infos` map, so it should be unique. - OperationID restore_id; - if (restore_settings.internal) - restore_id = "internal-" + toString(UUIDHelpers::generateV4()); /// Always generate `restore_id` for internal restore to avoid collision if both internal and non-internal restores are on the same host - else if (!restore_settings.id.empty()) - restore_id = restore_settings.id; - else - restore_id = toString(*restore_settings.restore_uuid); - + BackupsWorker & backups_worker; + std::shared_ptr restore_query; + ContextPtr query_context; /// We have to keep `query_context` until the end of the operation because a pointer to it is stored inside the ThreadGroup we're using. + ContextMutablePtr restore_context; + RestoreSettings restore_settings; + BackupInfo backup_info; + String restore_id; + String backup_name_for_logging; + bool on_cluster; + bool is_internal_restore; std::shared_ptr restore_coordination; + ClusterPtr cluster; + std::shared_ptr process_list_element_holder; - /// Called in exception handlers below. This lambda function can be called on a separate thread, so it can't capture local variables by reference. - auto on_exception = [this](const OperationID & restore_id_, const String & backup_name_for_logging_, - const RestoreSettings & restore_settings_, const std::shared_ptr & restore_coordination_) + RestoreStarter(BackupsWorker & backups_worker_, const ASTPtr & query_, const ContextPtr & context_) + : backups_worker(backups_worker_) + , restore_query(std::static_pointer_cast(query_->clone())) + , query_context(context_) + , restore_context(Context::createCopy(query_context)) { - /// Something bad happened, some data were not restored. - tryLogCurrentException(log, fmt::format("Failed to restore from {} {}", (restore_settings_.internal ? "internal backup" : "backup"), backup_name_for_logging_)); - setStatusSafe(restore_id_, getRestoreStatusFromCurrentException()); - sendCurrentExceptionToCoordination(restore_coordination_); - }; + restore_context->makeQueryContext(); + restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); + backup_info = BackupInfo::fromAST(*restore_query->backup_name); + backup_name_for_logging = backup_info.toStringForLogging(); + is_internal_restore = restore_settings.internal; + on_cluster = !restore_query->cluster.empty() || is_internal_restore; + + if (!restore_settings.restore_uuid) + restore_settings.restore_uuid = UUIDHelpers::generateV4(); + + /// `restore_id` will be used as a key to the `infos` map, so it should be unique. + if (is_internal_restore) + restore_id = "internal-" + toString(UUIDHelpers::generateV4()); /// Always generate `restore_id` for internal restore to avoid collision if both internal and non-internal restores are on the same host + else if (!restore_settings.id.empty()) + restore_id = restore_settings.id; + else + restore_id = toString(*restore_settings.restore_uuid); - try - { String base_backup_name; if (restore_settings.base_backup_info) base_backup_name = restore_settings.base_backup_info->toStringForLogging(); - addInfo(restore_id, + /// process_list_element_holder is used to make an element in ProcessList live while BACKUP is working asynchronously. + auto process_list_element = restore_context->getProcessListElement(); + if (process_list_element) + process_list_element_holder = process_list_element->getProcessListEntry(); + + backups_worker.addInfo(restore_id, backup_name_for_logging, base_backup_name, - context->getCurrentQueryId(), - restore_settings.internal, - context->getProcessListElement(), + restore_context->getCurrentQueryId(), + is_internal_restore, + process_list_element, BackupStatus::RESTORING); + } - if (restore_settings.internal) + void doRestore() + { + chassert(!restore_coordination); + if (on_cluster && !is_internal_restore) { - /// The following call of makeRestoreCoordination() is not essential because doRestore() will later create a restore coordination - /// if it's not created here. However to handle errors better it's better to make a coordination here because this way - /// if an exception will be thrown in startRestoring() other hosts will know about that. - restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ true); + restore_query->cluster = restore_context->getMacros()->expand(restore_query->cluster); + cluster = restore_context->getCluster(restore_query->cluster); + restore_settings.cluster_host_ids = cluster->getHostIDs(); + } + restore_coordination = backups_worker.makeRestoreCoordination(on_cluster, restore_settings, restore_context); + + backups_worker.doRestore( + restore_query, + restore_id, + backup_name_for_logging, + backup_info, + restore_settings, + restore_coordination, + restore_context, + on_cluster, + cluster); + } + + void onException() + { + /// Something bad happened, some data were not restored. + tryLogCurrentException(backups_worker.log, fmt::format("Failed to restore from {} {}", (is_internal_restore ? "internal backup" : "backup"), backup_name_for_logging)); + + if (restore_coordination && restore_coordination->trySetError(std::current_exception())) + { + restore_coordination->tryWaitForOtherHostsToFinishAfterError(); + restore_coordination->tryFinishAfterError(); } - /// Prepare context to use. - ContextMutablePtr context_in_use = context; - bool on_cluster = !restore_query->cluster.empty(); - if (restore_settings.async || on_cluster) - { - /// We have to clone the query context here because: - /// if this is an "ON CLUSTER" query we need to change some settings, and - /// if this is an "ASYNC" query it's going to be executed in another thread. - context_in_use = Context::createCopy(context); - context_in_use->makeQueryContext(); - } + backups_worker.setStatusSafe(restore_id, getRestoreStatusFromCurrentException()); + } +}; - if (restore_settings.async) - { - auto & thread_pool = getThreadPool(on_cluster ? ThreadPoolId::RESTORE_ASYNC_ON_CLUSTER : ThreadPoolId::RESTORE_ASYNC); - /// process_list_element_holder is used to make an element in ProcessList live while RESTORE is working asynchronously. - auto process_list_element = context_in_use->getProcessListElement(); +std::pair BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePtr context) +{ + auto starter = std::make_shared(*this, query, context); - thread_pool.scheduleOrThrowOnError( - [this, - restore_query, - restore_id, - backup_name_for_logging, - backup_info, - restore_settings, - restore_coordination, - context_in_use, - on_exception, - process_list_element_holder = process_list_element ? process_list_element->getProcessListEntry() : nullptr] + try + { + auto thread_pool_id = starter->is_internal_restore ? ThreadPoolId::ASYNC_BACKGROUND_INTERNAL_RESTORE : ThreadPoolId::ASYNC_BACKGROUND_RESTORE; + String thread_name = starter->is_internal_restore ? "RestoreAsyncInt" : "RestoreAsync"; + auto schedule = threadPoolCallbackRunnerUnsafe(thread_pools->getThreadPool(thread_pool_id), thread_name); + + schedule([starter] + { + try { - try - { - setThreadName("RestorerWorker"); - CurrentThread::QueryScope query_scope(context_in_use); - doRestore( - restore_query, - restore_id, - backup_name_for_logging, - backup_info, - restore_settings, - restore_coordination, - context_in_use); - } - catch (...) - { - on_exception(restore_id, backup_name_for_logging, restore_settings, restore_coordination); - } - }); - } - else - { - doRestore( - restore_query, - restore_id, - backup_name_for_logging, - backup_info, - restore_settings, - restore_coordination, - context_in_use); - } + starter->doRestore(); + } + catch (...) + { + starter->onException(); + } + }, + Priority{}); - return restore_id; + return {starter->restore_id, BackupStatus::RESTORING}; } catch (...) { - on_exception(restore_id, backup_name_for_logging, restore_settings, restore_coordination); + starter->onException(); throw; } } -void BackupsWorker::doRestore( - const std::shared_ptr & restore_query, - const OperationID & restore_id, - const String & backup_name_for_logging, - const BackupInfo & backup_info, - RestoreSettings restore_settings, - std::shared_ptr restore_coordination, - ContextMutablePtr context) +BackupPtr BackupsWorker::openBackupForReading(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context) const { - /// Open the backup for reading. + LOG_TRACE(log, "Opening backup for reading"); BackupFactory::CreateParams backup_open_params; backup_open_params.open_mode = IBackup::OpenMode::READ; backup_open_params.context = context; @@ -931,32 +829,35 @@ void BackupsWorker::doRestore( backup_open_params.read_settings = getReadSettingsForRestore(context); backup_open_params.write_settings = getWriteSettingsForRestore(context); backup_open_params.is_internal_backup = restore_settings.internal; - BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); + auto backup = BackupFactory::instance().createBackup(backup_open_params); + LOG_TRACE(log, "Opened backup for reading"); + return backup; +} + + +void BackupsWorker::doRestore( + const std::shared_ptr & restore_query, + const OperationID & restore_id, + const String & backup_name_for_logging, + const BackupInfo & backup_info, + RestoreSettings restore_settings, + std::shared_ptr restore_coordination, + ContextMutablePtr context, + bool on_cluster, + const ClusterPtr & cluster) +{ + bool is_internal_restore = restore_settings.internal; + + maybeSleepForTesting(); + + /// Open the backup for reading. + BackupPtr backup = openBackupForReading(backup_info, restore_settings, context); String current_database = context->getCurrentDatabase(); + /// Checks access rights if this is ON CLUSTER query. /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) - ClusterPtr cluster; - bool on_cluster = !restore_query->cluster.empty(); - - if (on_cluster) - { - restore_query->cluster = context->getMacros()->expand(restore_query->cluster); - cluster = context->getCluster(restore_query->cluster); - restore_settings.cluster_host_ids = cluster->getHostIDs(); - } - - /// Make a restore coordination. - if (!restore_coordination) - restore_coordination = makeRestoreCoordination(context, restore_settings, /* remote= */ on_cluster); - - if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores))) - throw Exception( - ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, - "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); - - - if (on_cluster) + if (on_cluster && !is_internal_restore) { /// We cannot just use access checking provided by the function executeDDLQueryOnCluster(): it would be incorrect /// because different replicas can contain different set of tables and so the required access rights can differ too. @@ -975,27 +876,21 @@ void BackupsWorker::doRestore( } /// Do RESTORE. - if (on_cluster) + if (on_cluster && !is_internal_restore) { - - DDLQueryOnClusterParams params; - params.cluster = cluster; - params.only_shard_num = restore_settings.shard_num; - params.only_replica_num = restore_settings.replica_num; + /// Send the RESTORE query to other hosts. restore_settings.copySettingsToQuery(*restore_query); + sendQueryToOtherHosts(*restore_query, cluster, restore_settings.shard_num, restore_settings.replica_num, + context, {}, restore_coordination->getOnClusterInitializationKeeperRetriesInfo()); + restore_coordination->setRestoreQueryWasSentToOtherHosts(); - // executeDDLQueryOnCluster() will return without waiting for completion - context->setSetting("distributed_ddl_task_timeout", Field{0}); - context->setSetting("distributed_ddl_output_mode", Field{"none"}); - - executeDDLQueryOnCluster(restore_query, context, params); - - /// Wait until all the hosts have written their backup entries. - restore_coordination->waitForStage(Stage::COMPLETED); - restore_coordination->setStage(Stage::COMPLETED,""); + /// Wait until all the hosts have done with their restoring work. + restore_coordination->waitForOtherHostsToFinish(); } else { + maybeSleepForTesting(); + restore_query->setCurrentDatabase(current_database); auto after_task_callback = [&] @@ -1011,11 +906,115 @@ void BackupsWorker::doRestore( restorer.run(RestorerFromBackup::RESTORE); } - LOG_INFO(log, "Restored from {} {} successfully", (restore_settings.internal ? "internal backup" : "backup"), backup_name_for_logging); + /// The restore coordination is not needed anymore. + restore_coordination->finish(); + + LOG_INFO(log, "Restored from {} {} successfully", (is_internal_restore ? "internal backup" : "backup"), backup_name_for_logging); setStatus(restore_id, BackupStatus::RESTORED); } +void BackupsWorker::sendQueryToOtherHosts(const ASTBackupQuery & backup_or_restore_query, const ClusterPtr & cluster, + size_t only_shard_num, size_t only_replica_num, ContextMutablePtr context, const AccessRightsElements & access_to_check, + const ZooKeeperRetriesInfo & retries_info) const +{ + chassert(cluster); + + DDLQueryOnClusterParams params; + params.cluster = cluster; + params.only_shard_num = only_shard_num; + params.only_replica_num = only_replica_num; + params.access_to_check = access_to_check; + params.retries_info = retries_info; + + context->setSetting("distributed_ddl_task_timeout", Field{0}); + context->setSetting("distributed_ddl_output_mode", Field{"never_throw"}); + + // executeDDLQueryOnCluster() will return without waiting for completion + executeDDLQueryOnCluster(backup_or_restore_query.clone(), context, params); + + maybeSleepForTesting(); +} + + +std::shared_ptr +BackupsWorker::makeBackupCoordination(bool on_cluster, const BackupSettings & backup_settings, const ContextPtr & context) const +{ + if (!on_cluster) + { + return std::make_shared( + *backup_settings.backup_uuid, !backup_settings.deduplicate_files, allow_concurrent_backups, *concurrency_counters); + } + + bool is_internal_backup = backup_settings.internal; + + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; + auto keeper_settings = BackupKeeperSettings::fromContext(context); + + auto all_hosts = BackupSettings::Util::filterHostIDs( + backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num); + all_hosts.emplace_back(BackupCoordinationOnCluster::kInitiator); + + String current_host = is_internal_backup ? backup_settings.host_id : String{BackupCoordinationOnCluster::kInitiator}; + + auto thread_pool_id = is_internal_backup ? ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_BACKUP : ThreadPoolId::ON_CLUSTER_COORDINATION_BACKUP; + String thread_name = is_internal_backup ? "BackupCoordInt" : "BackupCoord"; + auto schedule = threadPoolCallbackRunnerUnsafe(thread_pools->getThreadPool(thread_pool_id), thread_name); + + return std::make_shared( + *backup_settings.backup_uuid, + !backup_settings.deduplicate_files, + root_zk_path, + get_zookeeper, + keeper_settings, + current_host, + all_hosts, + allow_concurrent_backups, + *concurrency_counters, + schedule, + context->getProcessListElement()); +} + +std::shared_ptr +BackupsWorker::makeRestoreCoordination(bool on_cluster, const RestoreSettings & restore_settings, const ContextPtr & context) const +{ + if (!on_cluster) + { + return std::make_shared( + *restore_settings.restore_uuid, allow_concurrent_restores, *concurrency_counters); + } + + bool is_internal_restore = restore_settings.internal; + + String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; + auto keeper_settings = BackupKeeperSettings::fromContext(context); + + auto all_hosts = BackupSettings::Util::filterHostIDs( + restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); + all_hosts.emplace_back(BackupCoordinationOnCluster::kInitiator); + + String current_host = is_internal_restore ? restore_settings.host_id : String{RestoreCoordinationOnCluster::kInitiator}; + + auto thread_pool_id = is_internal_restore ? ThreadPoolId::ON_CLUSTER_COORDINATION_INTERNAL_RESTORE : ThreadPoolId::ON_CLUSTER_COORDINATION_RESTORE; + String thread_name = is_internal_restore ? "RestoreCoordInt" : "RestoreCoord"; + auto schedule = threadPoolCallbackRunnerUnsafe(thread_pools->getThreadPool(thread_pool_id), thread_name); + + return std::make_shared( + *restore_settings.restore_uuid, + root_zk_path, + get_zookeeper, + keeper_settings, + current_host, + all_hosts, + allow_concurrent_restores, + *concurrency_counters, + schedule, + context->getProcessListElement()); +} + + void BackupsWorker::addInfo(const OperationID & id, const String & name, const String & base_backup_name, const String & query_id, bool internal, QueryStatusPtr process_list_element, BackupStatus status) { @@ -1135,23 +1134,25 @@ void BackupsWorker::maybeSleepForTesting() const } -void BackupsWorker::wait(const OperationID & backup_or_restore_id, bool rethrow_exception) +BackupStatus BackupsWorker::wait(const OperationID & backup_or_restore_id, bool rethrow_exception) { std::unique_lock lock{infos_mutex}; + BackupStatus current_status; status_changed.wait(lock, [&] { auto it = infos.find(backup_or_restore_id); if (it == infos.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown backup ID {}", backup_or_restore_id); const auto & info = it->second.info; - auto current_status = info.status; + current_status = info.status; if (rethrow_exception && isFailedOrCancelled(current_status)) std::rethrow_exception(info.exception); if (isFinalStatus(current_status)) return true; - LOG_INFO(log, "Waiting {} {}", isBackupStatus(info.status) ? "backup" : "restore", info.name); + LOG_INFO(log, "Waiting {} {} to complete", isBackupStatus(current_status) ? "backup" : "restore", info.name); return false; }); + return current_status; } void BackupsWorker::waitAll() @@ -1175,9 +1176,11 @@ void BackupsWorker::waitAll() LOG_INFO(log, "Backups and restores finished"); } -void BackupsWorker::cancel(const BackupOperationID & backup_or_restore_id, bool wait_) +BackupStatus BackupsWorker::cancel(const BackupOperationID & backup_or_restore_id, bool wait_) { QueryStatusPtr process_list_element; + BackupStatus current_status; + { std::unique_lock lock{infos_mutex}; auto it = infos.find(backup_or_restore_id); @@ -1186,17 +1189,20 @@ void BackupsWorker::cancel(const BackupOperationID & backup_or_restore_id, bool const auto & extended_info = it->second; const auto & info = extended_info.info; - if (isFinalStatus(info.status) || !extended_info.process_list_element) - return; + current_status = info.status; + if (isFinalStatus(current_status) || !extended_info.process_list_element) + return current_status; - LOG_INFO(log, "Cancelling {} {}", isBackupStatus(info.status) ? "backup" : "restore", info.name); + LOG_INFO(log, "Cancelling {} {}", isBackupStatus(current_status) ? "backup" : "restore", info.name); process_list_element = extended_info.process_list_element; } process_list.sendCancelToQuery(process_list_element); - if (wait_) - wait(backup_or_restore_id, /* rethrow_exception= */ false); + if (!wait_) + return current_status; + + return wait(backup_or_restore_id, /* rethrow_exception= */ false); } diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 946562b575f..37f91e269a9 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -23,6 +23,7 @@ using BackupMutablePtr = std::shared_ptr; using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; +class BackupConcurrencyCounters; using DataRestoreTasks = std::vector>; struct ReadSettings; class BackupLog; @@ -31,6 +32,10 @@ using ThreadGroupPtr = std::shared_ptr; class QueryStatus; using QueryStatusPtr = std::shared_ptr; class ProcessList; +class Cluster; +using ClusterPtr = std::shared_ptr; +class AccessRightsElements; +struct ZooKeeperRetriesInfo; /// Manager of backups and restores: executes backups and restores' threads in the background. @@ -47,18 +52,18 @@ public: /// Starts executing a BACKUP or RESTORE query. Returns ID of the operation. /// For asynchronous operations the function throws no exceptions on failure usually, /// call getInfo() on a returned operation id to check for errors. - BackupOperationID start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context); + std::pair start(const ASTPtr & backup_or_restore_query, ContextMutablePtr context); /// Waits until the specified backup or restore operation finishes or stops. /// The function returns immediately if the operation is already finished. - void wait(const BackupOperationID & backup_or_restore_id, bool rethrow_exception = true); + BackupStatus wait(const BackupOperationID & backup_or_restore_id, bool rethrow_exception = true); /// Waits until all running backup and restore operations finish or stop. void waitAll(); /// Cancels the specified backup or restore operation. /// The function does nothing if this operation has already finished. - void cancel(const BackupOperationID & backup_or_restore_id, bool wait_ = true); + BackupStatus cancel(const BackupOperationID & backup_or_restore_id, bool wait_ = true); /// Cancels all running backup and restore operations. void cancelAll(bool wait_ = true); @@ -67,26 +72,32 @@ public: std::vector getAllInfos() const; private: - BackupOperationID startMakingBackup(const ASTPtr & query, const ContextPtr & context); + std::pair startMakingBackup(const ASTPtr & query, const ContextPtr & context); + struct BackupStarter; + + BackupMutablePtr openBackupForWriting(const BackupInfo & backup_info, const BackupSettings & backup_settings, std::shared_ptr backup_coordination, const ContextPtr & context) const; void doBackup( - BackupMutablePtr & backup, + BackupMutablePtr backup, const std::shared_ptr & backup_query, const BackupOperationID & backup_id, const String & backup_name_for_logging, - const BackupInfo & backup_info, - BackupSettings backup_settings, + const BackupSettings & backup_settings, std::shared_ptr backup_coordination, - const ContextPtr & context, - ContextMutablePtr mutable_context); + ContextMutablePtr context, + bool on_cluster, + const ClusterPtr & cluster); /// Builds file infos for specified backup entries. void buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr backup_coordination, QueryStatusPtr process_list_element); /// Write backup entries to an opened backup. - void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, const BackupOperationID & backup_id, std::shared_ptr backup_coordination, bool internal, QueryStatusPtr process_list_element); + void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, const BackupOperationID & backup_id, std::shared_ptr backup_coordination, bool is_internal_backup, QueryStatusPtr process_list_element); - BackupOperationID startRestoring(const ASTPtr & query, ContextMutablePtr context); + std::pair startRestoring(const ASTPtr & query, ContextMutablePtr context); + struct RestoreStarter; + + BackupPtr openBackupForReading(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context) const; void doRestore( const std::shared_ptr & restore_query, @@ -95,7 +106,17 @@ private: const BackupInfo & backup_info, RestoreSettings restore_settings, std::shared_ptr restore_coordination, - ContextMutablePtr context); + ContextMutablePtr context, + bool on_cluster, + const ClusterPtr & cluster); + + std::shared_ptr makeBackupCoordination(bool on_cluster, const BackupSettings & backup_settings, const ContextPtr & context) const; + std::shared_ptr makeRestoreCoordination(bool on_cluster, const RestoreSettings & restore_settings, const ContextPtr & context) const; + + /// Sends a BACKUP or RESTORE query to other hosts. + void sendQueryToOtherHosts(const ASTBackupQuery & backup_or_restore_query, const ClusterPtr & cluster, + size_t only_shard_num, size_t only_replica_num, ContextMutablePtr context, const AccessRightsElements & access_to_check, + const ZooKeeperRetriesInfo & retries_info) const; /// Run data restoring tasks which insert data to tables. void restoreTablesData(const BackupOperationID & restore_id, BackupPtr backup, DataRestoreTasks && tasks, ThreadPool & thread_pool, QueryStatusPtr process_list_element); @@ -139,6 +160,8 @@ private: std::shared_ptr backup_log; ProcessList & process_list; + + std::unique_ptr concurrency_counters; }; } diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index 0aa2d34657f..126b4d764da 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -121,8 +121,13 @@ public: /// Finalizes writing the backup, should be called after all entries have been successfully written. virtual void finalizeWriting() = 0; - /// Try to remove all files copied to the backup. Used after an exception or it the backup was cancelled. - virtual void tryRemoveAllFiles() = 0; + /// Sets that a non-retriable error happened while the backup was being written which means that + /// the backup is most likely corrupted and it can't be finalized. + /// This function is called while handling an exception or if the backup was cancelled. + virtual bool setIsCorrupted() noexcept = 0; + + /// Try to remove all files copied to the backup. Could be used after setIsCorrupted(). + virtual bool tryRemoveAllFiles() noexcept = 0; }; using BackupPtr = std::shared_ptr; diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 166a2c5bbbc..c0eb90de89b 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -5,26 +5,44 @@ namespace DB { -class Exception; struct BackupFileInfo; using BackupFileInfos = std::vector; enum class AccessEntityType : uint8_t; enum class UserDefinedSQLObjectType : uint8_t; +struct ZooKeeperRetriesInfo; /// Replicas use this class to coordinate what they're writing to a backup while executing BACKUP ON CLUSTER. -/// There are two implementation of this interface: BackupCoordinationLocal and BackupCoordinationRemote. +/// There are two implementation of this interface: BackupCoordinationLocal and BackupCoordinationOnCluster. /// BackupCoordinationLocal is used while executing BACKUP without ON CLUSTER and performs coordination in memory. -/// BackupCoordinationRemote is used while executing BACKUP with ON CLUSTER and performs coordination via ZooKeeper. +/// BackupCoordinationOnCluster is used while executing BACKUP with ON CLUSTER and performs coordination via ZooKeeper. class IBackupCoordination { public: virtual ~IBackupCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; - virtual void setError(const Exception & exception) = 0; - virtual Strings waitForStage(const String & stage_to_wait) = 0; - virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; + virtual Strings setStage(const String & new_stage, const String & message, bool sync) = 0; + + /// Sets that the backup query was sent to other hosts. + /// Function waitForOtherHostsToFinish() will check that to find out if it should really wait or not. + virtual void setBackupQueryWasSentToOtherHosts() = 0; + + /// Lets other hosts know that the current host has encountered an error. + virtual bool trySetError(std::exception_ptr exception) = 0; + + /// Lets other hosts know that the current host has finished its work. + virtual void finish() = 0; + + /// Lets other hosts know that the current host has finished its work (as a part of error-handling process). + virtual bool tryFinishAfterError() noexcept = 0; + + /// Waits until all the other hosts finish their work. + /// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled. + virtual void waitForOtherHostsToFinish() = 0; + + /// Waits until all the other hosts finish their work (as a part of error-handling process). + /// Doesn't stops waiting if some host encounters an error or gets cancelled. + virtual bool tryWaitForOtherHostsToFinishAfterError() noexcept = 0; struct PartNameAndChecksum { @@ -87,9 +105,7 @@ public: /// Starts writing a specified file, the function returns false if that file is already being written concurrently. virtual bool startWritingFile(size_t data_file_index) = 0; - /// This function is used to check if concurrent backups are running - /// other than the backup passed to the function - virtual bool hasConcurrentBackups(const std::atomic & num_active_backups) const = 0; + virtual ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const = 0; }; } diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 37229534286..daabf1745f3 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -5,26 +5,42 @@ namespace DB { -class Exception; enum class UserDefinedSQLObjectType : uint8_t; class ASTCreateQuery; +struct ZooKeeperRetriesInfo; /// Replicas use this class to coordinate what they're reading from a backup while executing RESTORE ON CLUSTER. -/// There are two implementation of this interface: RestoreCoordinationLocal and RestoreCoordinationRemote. +/// There are two implementation of this interface: RestoreCoordinationLocal and RestoreCoordinationOnCluster. /// RestoreCoordinationLocal is used while executing RESTORE without ON CLUSTER and performs coordination in memory. -/// RestoreCoordinationRemote is used while executing RESTORE with ON CLUSTER and performs coordination via ZooKeeper. +/// RestoreCoordinationOnCluster is used while executing RESTORE with ON CLUSTER and performs coordination via ZooKeeper. class IRestoreCoordination { public: virtual ~IRestoreCoordination() = default; /// Sets the current stage and waits for other hosts to come to this stage too. - virtual void setStage(const String & new_stage, const String & message) = 0; - virtual void setError(const Exception & exception) = 0; - virtual Strings waitForStage(const String & stage_to_wait) = 0; - virtual Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) = 0; + virtual Strings setStage(const String & new_stage, const String & message, bool sync) = 0; - static constexpr const char * kErrorStatus = "error"; + /// Sets that the restore query was sent to other hosts. + /// Function waitForOtherHostsToFinish() will check that to find out if it should really wait or not. + virtual void setRestoreQueryWasSentToOtherHosts() = 0; + + /// Lets other hosts know that the current host has encountered an error. + virtual bool trySetError(std::exception_ptr exception) = 0; + + /// Lets other hosts know that the current host has finished its work. + virtual void finish() = 0; + + /// Lets other hosts know that the current host has finished its work (as a part of error-handling process). + virtual bool tryFinishAfterError() noexcept = 0; + + /// Waits until all the other hosts finish their work. + /// Stops waiting and throws an exception if another host encounters an error or if some host gets cancelled. + virtual void waitForOtherHostsToFinish() = 0; + + /// Waits until all the other hosts finish their work (as a part of error-handling process). + /// Doesn't stops waiting if some host encounters an error or gets cancelled. + virtual bool tryWaitForOtherHostsToFinishAfterError() noexcept = 0; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. virtual bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) = 0; @@ -49,9 +65,7 @@ public: /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). virtual void generateUUIDForTable(ASTCreateQuery & create_query) = 0; - /// This function is used to check if concurrent restores are running - /// other than the restore passed to the function - virtual bool hasConcurrentRestores(const std::atomic & num_active_restores) const = 0; + virtual ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const = 0; }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 9fe22f874b4..569f58f1909 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -1,32 +1,24 @@ #include + #include #include +#include #include namespace DB { -RestoreCoordinationLocal::RestoreCoordinationLocal() : log(getLogger("RestoreCoordinationLocal")) +RestoreCoordinationLocal::RestoreCoordinationLocal( + const UUID & restore_uuid, bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_) + : log(getLogger("RestoreCoordinationLocal")) + , concurrency_check(restore_uuid, /* is_restore = */ true, /* on_cluster = */ false, allow_concurrent_restore_, concurrency_counters_) { } RestoreCoordinationLocal::~RestoreCoordinationLocal() = default; -void RestoreCoordinationLocal::setStage(const String &, const String &) -{ -} - -void RestoreCoordinationLocal::setError(const Exception &) -{ -} - -Strings RestoreCoordinationLocal::waitForStage(const String &) -{ - return {}; -} - -Strings RestoreCoordinationLocal::waitForStage(const String &, std::chrono::milliseconds) +ZooKeeperRetriesInfo RestoreCoordinationLocal::getOnClusterInitializationKeeperRetriesInfo() const { return {}; } @@ -63,7 +55,7 @@ void RestoreCoordinationLocal::generateUUIDForTable(ASTCreateQuery & create_quer { String query_str = serializeAST(create_query); - auto find_in_map = [&] + auto find_in_map = [&]() TSA_REQUIRES(mutex) { auto it = create_query_uuids.find(query_str); if (it != create_query_uuids.end()) @@ -91,14 +83,4 @@ void RestoreCoordinationLocal::generateUUIDForTable(ASTCreateQuery & create_quer } } -bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic & num_active_restores) const -{ - if (num_active_restores > 1) - { - LOG_WARNING(log, "Found concurrent backups: num_active_restores={}", num_active_restores); - return true; - } - return false; -} - } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 35f93574b68..6be357c4b7e 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -12,19 +13,20 @@ namespace DB { class ASTCreateQuery; - /// Implementation of the IRestoreCoordination interface performing coordination in memory. class RestoreCoordinationLocal : public IRestoreCoordination { public: - RestoreCoordinationLocal(); + RestoreCoordinationLocal(const UUID & restore_uuid_, bool allow_concurrent_restore_, BackupConcurrencyCounters & concurrency_counters_); ~RestoreCoordinationLocal() override; - /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; - void setError(const Exception & exception) override; - Strings waitForStage(const String & stage_to_wait) override; - Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; + Strings setStage(const String &, const String &, bool) override { return {}; } + void setRestoreQueryWasSentToOtherHosts() override {} + bool trySetError(std::exception_ptr) override { return true; } + void finish() override {} + bool tryFinishAfterError() noexcept override { return true; } + void waitForOtherHostsToFinish() override {} + bool tryWaitForOtherHostsToFinishAfterError() noexcept override { return true; } /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -49,15 +51,16 @@ public: /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). void generateUUIDForTable(ASTCreateQuery & create_query) override; - bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; + ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const override; private: LoggerPtr const log; + BackupConcurrencyCheck concurrency_check; - std::set> acquired_tables_in_replicated_databases; - std::unordered_set acquired_data_in_replicated_tables; - std::unordered_map create_query_uuids; - std::unordered_set acquired_data_in_keeper_map_tables; + std::set> acquired_tables_in_replicated_databases TSA_GUARDED_BY(mutex); + std::unordered_set acquired_data_in_replicated_tables TSA_GUARDED_BY(mutex); + std::unordered_map create_query_uuids TSA_GUARDED_BY(mutex); + std::unordered_set acquired_data_in_keeper_map_tables TSA_GUARDED_BY(mutex); mutable std::mutex mutex; }; diff --git a/src/Backups/RestoreCoordinationOnCluster.cpp b/src/Backups/RestoreCoordinationOnCluster.cpp new file mode 100644 index 00000000000..2029ad8b072 --- /dev/null +++ b/src/Backups/RestoreCoordinationOnCluster.cpp @@ -0,0 +1,318 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +RestoreCoordinationOnCluster::RestoreCoordinationOnCluster( + const UUID & restore_uuid_, + const String & root_zookeeper_path_, + zkutil::GetZooKeeper get_zookeeper_, + const BackupKeeperSettings & keeper_settings_, + const String & current_host_, + const Strings & all_hosts_, + bool allow_concurrent_restore_, + BackupConcurrencyCounters & concurrency_counters_, + ThreadPoolCallbackRunnerUnsafe schedule_, + QueryStatusPtr process_list_element_) + : root_zookeeper_path(root_zookeeper_path_) + , keeper_settings(keeper_settings_) + , restore_uuid(restore_uuid_) + , zookeeper_path(root_zookeeper_path_ + "/restore-" + toString(restore_uuid_)) + , all_hosts(all_hosts_) + , all_hosts_without_initiator(BackupCoordinationOnCluster::excludeInitiator(all_hosts)) + , current_host(current_host_) + , current_host_index(BackupCoordinationOnCluster::findCurrentHostIndex(current_host, all_hosts)) + , log(getLogger("RestoreCoordinationOnCluster")) + , with_retries(log, get_zookeeper_, keeper_settings, process_list_element_, [root_zookeeper_path_](Coordination::ZooKeeperWithFaultInjection::Ptr zk) { zk->sync(root_zookeeper_path_); }) + , concurrency_check(restore_uuid_, /* is_restore = */ true, /* on_cluster = */ true, allow_concurrent_restore_, concurrency_counters_) + , stage_sync(/* is_restore = */ true, fs::path{zookeeper_path} / "stage", current_host, all_hosts, allow_concurrent_restore_, with_retries, schedule_, process_list_element_, log) + , cleaner(zookeeper_path, with_retries, log) +{ + createRootNodes(); +} + +RestoreCoordinationOnCluster::~RestoreCoordinationOnCluster() +{ + tryFinishImpl(); +} + +void RestoreCoordinationOnCluster::createRootNodes() +{ + auto holder = with_retries.createRetriesControlHolder("createRootNodes", WithRetries::kInitialization); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + zk->createAncestors(zookeeper_path); + zk->createIfNotExists(zookeeper_path, ""); + zk->createIfNotExists(zookeeper_path + "/repl_databases_tables_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_tables_data_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_access_storages_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/repl_sql_objects_acquired", ""); + zk->createIfNotExists(zookeeper_path + "/keeper_map_tables", ""); + zk->createIfNotExists(zookeeper_path + "/table_uuids", ""); + }); +} + +Strings RestoreCoordinationOnCluster::setStage(const String & new_stage, const String & message, bool sync) +{ + stage_sync.setStage(new_stage, message); + + if (!sync) + return {}; + + return stage_sync.waitForHostsToReachStage(new_stage, all_hosts_without_initiator); +} + +void RestoreCoordinationOnCluster::setRestoreQueryWasSentToOtherHosts() +{ + restore_query_was_sent_to_other_hosts = true; +} + +bool RestoreCoordinationOnCluster::trySetError(std::exception_ptr exception) +{ + return stage_sync.trySetError(exception); +} + +void RestoreCoordinationOnCluster::finish() +{ + bool other_hosts_also_finished = false; + stage_sync.finish(other_hosts_also_finished); + + if ((current_host == kInitiator) && (other_hosts_also_finished || !restore_query_was_sent_to_other_hosts)) + cleaner.cleanup(); +} + +bool RestoreCoordinationOnCluster::tryFinishAfterError() noexcept +{ + return tryFinishImpl(); +} + +bool RestoreCoordinationOnCluster::tryFinishImpl() noexcept +{ + bool other_hosts_also_finished = false; + if (!stage_sync.tryFinishAfterError(other_hosts_also_finished)) + return false; + + if ((current_host == kInitiator) && (other_hosts_also_finished || !restore_query_was_sent_to_other_hosts)) + { + if (!cleaner.tryCleanupAfterError()) + return false; + } + + return true; +} + +void RestoreCoordinationOnCluster::waitForOtherHostsToFinish() +{ + if ((current_host != kInitiator) || !restore_query_was_sent_to_other_hosts) + return; + stage_sync.waitForOtherHostsToFinish(); +} + +bool RestoreCoordinationOnCluster::tryWaitForOtherHostsToFinishAfterError() noexcept +{ + if (current_host != kInitiator) + return false; + if (!restore_query_was_sent_to_other_hosts) + return true; + return stage_sync.tryWaitForOtherHostsToFinishAfterError(); +} + +ZooKeeperRetriesInfo RestoreCoordinationOnCluster::getOnClusterInitializationKeeperRetriesInfo() const +{ + return ZooKeeperRetriesInfo{keeper_settings.max_retries_while_initializing, + static_cast(keeper_settings.retry_initial_backoff_ms.count()), + static_cast(keeper_settings.retry_max_backoff_ms.count())}; +} + +bool RestoreCoordinationOnCluster::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) +{ + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireCreatingTableInReplicatedDatabase"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = zookeeper_path + "/repl_databases_tables_acquired/" + escapeForFileName(database_zk_path); + zk->createIfNotExists(path, ""); + + path += "/" + escapeForFileName(table_name); + auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException::fromPath(code, path); + + if (code == Coordination::Error::ZOK) + { + result = true; + return; + } + + /// We need to check who created that node + result = zk->get(path) == toString(current_host_index); + }); + return result; +} + +bool RestoreCoordinationOnCluster::acquireInsertingDataIntoReplicatedTable(const String & table_zk_path) +{ + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireInsertingDataIntoReplicatedTable"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = zookeeper_path + "/repl_tables_data_acquired/" + escapeForFileName(table_zk_path); + auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException::fromPath(code, path); + + if (code == Coordination::Error::ZOK) + { + result = true; + return; + } + + /// We need to check who created that node + result = zk->get(path) == toString(current_host_index); + }); + return result; +} + +bool RestoreCoordinationOnCluster::acquireReplicatedAccessStorage(const String & access_storage_zk_path) +{ + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireReplicatedAccessStorage"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = zookeeper_path + "/repl_access_storages_acquired/" + escapeForFileName(access_storage_zk_path); + auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException::fromPath(code, path); + + if (code == Coordination::Error::ZOK) + { + result = true; + return; + } + + /// We need to check who created that node + result = zk->get(path) == toString(current_host_index); + }); + return result; +} + +bool RestoreCoordinationOnCluster::acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) +{ + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireReplicatedSQLObjects"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = zookeeper_path + "/repl_sql_objects_acquired/" + escapeForFileName(loader_zk_path); + zk->createIfNotExists(path, ""); + + path += "/"; + switch (object_type) + { + case UserDefinedSQLObjectType::Function: + path += "functions"; + break; + } + + auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException::fromPath(code, path); + + if (code == Coordination::Error::ZOK) + { + result = true; + return; + } + + /// We need to check who created that node + result = zk->get(path) == toString(current_host_index); + }); + return result; +} + +bool RestoreCoordinationOnCluster::acquireInsertingDataForKeeperMap(const String & root_zk_path, const String & table_unique_id) +{ + bool lock_acquired = false; + auto holder = with_retries.createRetriesControlHolder("acquireInsertingDataForKeeperMap"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + /// we need to remove leading '/' from root_zk_path + auto normalized_root_zk_path = root_zk_path.substr(1); + std::string restore_lock_path = fs::path(zookeeper_path) / "keeper_map_tables" / escapeForFileName(normalized_root_zk_path); + zk->createAncestors(restore_lock_path); + auto code = zk->tryCreate(restore_lock_path, table_unique_id, zkutil::CreateMode::Persistent); + + if (code == Coordination::Error::ZOK) + { + lock_acquired = true; + return; + } + + if (code == Coordination::Error::ZNODEEXISTS) + lock_acquired = table_unique_id == zk->get(restore_lock_path); + else + zkutil::KeeperException::fromPath(code, restore_lock_path); + }); + return lock_acquired; +} + +void RestoreCoordinationOnCluster::generateUUIDForTable(ASTCreateQuery & create_query) +{ + String query_str = serializeAST(create_query); + CreateQueryUUIDs new_uuids{create_query, /* generate_random= */ true, /* force_random= */ true}; + String new_uuids_str = new_uuids.toString(); + + auto holder = with_retries.createRetriesControlHolder("generateUUIDForTable"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = zookeeper_path + "/table_uuids/" + escapeForFileName(query_str); + Coordination::Error res = zk->tryCreate(path, new_uuids_str, zkutil::CreateMode::Persistent); + + if (res == Coordination::Error::ZOK) + { + new_uuids.copyToQuery(create_query); + return; + } + + if (res == Coordination::Error::ZNODEEXISTS) + { + CreateQueryUUIDs::fromString(zk->get(path)).copyToQuery(create_query); + return; + } + + zkutil::KeeperException::fromPath(res, path); + }); +} + +} diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationOnCluster.h similarity index 62% rename from src/Backups/RestoreCoordinationRemote.h rename to src/Backups/RestoreCoordinationOnCluster.h index a3d57e9a4d0..87a8dd3ce83 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationOnCluster.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include #include @@ -9,28 +11,33 @@ namespace DB { /// Implementation of the IRestoreCoordination interface performing coordination via ZooKeeper. It's necessary for "RESTORE ON CLUSTER". -class RestoreCoordinationRemote : public IRestoreCoordination +class RestoreCoordinationOnCluster : public IRestoreCoordination { public: - using RestoreKeeperSettings = WithRetries::KeeperSettings; + /// Empty string as the current host is used to mark the initiator of a RESTORE ON CLUSTER query. + static const constexpr std::string_view kInitiator; - RestoreCoordinationRemote( - zkutil::GetZooKeeper get_zookeeper_, + RestoreCoordinationOnCluster( + const UUID & restore_uuid_, const String & root_zookeeper_path_, - const RestoreKeeperSettings & keeper_settings_, - const String & restore_uuid_, - const Strings & all_hosts_, + zkutil::GetZooKeeper get_zookeeper_, + const BackupKeeperSettings & keeper_settings_, const String & current_host_, - bool is_internal_, + const Strings & all_hosts_, + bool allow_concurrent_restore_, + BackupConcurrencyCounters & concurrency_counters_, + ThreadPoolCallbackRunnerUnsafe schedule_, QueryStatusPtr process_list_element_); - ~RestoreCoordinationRemote() override; + ~RestoreCoordinationOnCluster() override; - /// Sets the current stage and waits for other hosts to come to this stage too. - void setStage(const String & new_stage, const String & message) override; - void setError(const Exception & exception) override; - Strings waitForStage(const String & stage_to_wait) override; - Strings waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) override; + Strings setStage(const String & new_stage, const String & message, bool sync) override; + void setRestoreQueryWasSentToOtherHosts() override; + bool trySetError(std::exception_ptr exception) override; + void finish() override; + bool tryFinishAfterError() noexcept override; + void waitForOtherHostsToFinish() override; + bool tryWaitForOtherHostsToFinishAfterError() noexcept override; /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -55,27 +62,27 @@ public: /// (because otherwise the macro "{uuid}" in the ZooKeeper path will not work correctly). void generateUUIDForTable(ASTCreateQuery & create_query) override; - bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; + ZooKeeperRetriesInfo getOnClusterInitializationKeeperRetriesInfo() const override; private: void createRootNodes(); - void removeAllNodes(); + bool tryFinishImpl() noexcept; - /// get_zookeeper will provide a zookeeper client without any fault injection - const zkutil::GetZooKeeper get_zookeeper; const String root_zookeeper_path; - const RestoreKeeperSettings keeper_settings; - const String restore_uuid; + const BackupKeeperSettings keeper_settings; + const UUID restore_uuid; const String zookeeper_path; const Strings all_hosts; + const Strings all_hosts_without_initiator; const String current_host; const size_t current_host_index; - const bool is_internal; LoggerPtr const log; - mutable WithRetries with_retries; - std::optional stage_sync; - mutable std::mutex mutex; + const WithRetries with_retries; + BackupConcurrencyCheck concurrency_check; + BackupCoordinationStageSync stage_sync; + BackupCoordinationCleaner cleaner; + std::atomic restore_query_was_sent_to_other_hosts = false; }; } diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp deleted file mode 100644 index 0a69bc0eafb..00000000000 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ /dev/null @@ -1,379 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace Stage = BackupCoordinationStage; - -RestoreCoordinationRemote::RestoreCoordinationRemote( - zkutil::GetZooKeeper get_zookeeper_, - const String & root_zookeeper_path_, - const RestoreKeeperSettings & keeper_settings_, - const String & restore_uuid_, - const Strings & all_hosts_, - const String & current_host_, - bool is_internal_, - QueryStatusPtr process_list_element_) - : get_zookeeper(get_zookeeper_) - , root_zookeeper_path(root_zookeeper_path_) - , keeper_settings(keeper_settings_) - , restore_uuid(restore_uuid_) - , zookeeper_path(root_zookeeper_path_ + "/restore-" + restore_uuid_) - , all_hosts(all_hosts_) - , current_host(current_host_) - , current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host)) - , is_internal(is_internal_) - , log(getLogger("RestoreCoordinationRemote")) - , with_retries( - log, - get_zookeeper_, - keeper_settings, - process_list_element_, - [my_zookeeper_path = zookeeper_path, my_current_host = current_host, my_is_internal = is_internal] - (WithRetries::FaultyKeeper & zk) - { - /// Recreate this ephemeral node to signal that we are alive. - if (my_is_internal) - { - String alive_node_path = my_zookeeper_path + "/stage/alive|" + my_current_host; - - /// Delete the ephemeral node from the previous connection so we don't have to wait for keeper to do it automatically. - zk->tryRemove(alive_node_path); - - zk->createAncestors(alive_node_path); - zk->create(alive_node_path, "", zkutil::CreateMode::Ephemeral); - } - }) -{ - createRootNodes(); - - stage_sync.emplace( - zookeeper_path, - with_retries, - log); -} - -RestoreCoordinationRemote::~RestoreCoordinationRemote() -{ - try - { - if (!is_internal) - removeAllNodes(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - -void RestoreCoordinationRemote::createRootNodes() -{ - auto holder = with_retries.createRetriesControlHolder("createRootNodes"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - zk->createAncestors(zookeeper_path); - - Coordination::Requests ops; - Coordination::Responses responses; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_databases_tables_acquired", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_tables_data_acquired", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_access_storages_acquired", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/repl_sql_objects_acquired", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/keeper_map_tables", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/table_uuids", "", zkutil::CreateMode::Persistent)); - zk->tryMulti(ops, responses); - }); -} - -void RestoreCoordinationRemote::setStage(const String & new_stage, const String & message) -{ - if (is_internal) - stage_sync->set(current_host, new_stage, message); - else - stage_sync->set(current_host, new_stage, /* message */ "", /* all_hosts */ true); -} - -void RestoreCoordinationRemote::setError(const Exception & exception) -{ - stage_sync->setError(current_host, exception); -} - -Strings RestoreCoordinationRemote::waitForStage(const String & stage_to_wait) -{ - return stage_sync->wait(all_hosts, stage_to_wait); -} - -Strings RestoreCoordinationRemote::waitForStage(const String & stage_to_wait, std::chrono::milliseconds timeout) -{ - return stage_sync->waitFor(all_hosts, stage_to_wait, timeout); -} - -bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) -{ - bool result = false; - auto holder = with_retries.createRetriesControlHolder("acquireCreatingTableInReplicatedDatabase"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - String path = zookeeper_path + "/repl_databases_tables_acquired/" + escapeForFileName(database_zk_path); - zk->createIfNotExists(path, ""); - - path += "/" + escapeForFileName(table_name); - auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException::fromPath(code, path); - - if (code == Coordination::Error::ZOK) - { - result = true; - return; - } - - /// We need to check who created that node - result = zk->get(path) == toString(current_host_index); - }); - return result; -} - -bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const String & table_zk_path) -{ - bool result = false; - auto holder = with_retries.createRetriesControlHolder("acquireInsertingDataIntoReplicatedTable"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - String path = zookeeper_path + "/repl_tables_data_acquired/" + escapeForFileName(table_zk_path); - auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException::fromPath(code, path); - - if (code == Coordination::Error::ZOK) - { - result = true; - return; - } - - /// We need to check who created that node - result = zk->get(path) == toString(current_host_index); - }); - return result; -} - -bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & access_storage_zk_path) -{ - bool result = false; - auto holder = with_retries.createRetriesControlHolder("acquireReplicatedAccessStorage"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - String path = zookeeper_path + "/repl_access_storages_acquired/" + escapeForFileName(access_storage_zk_path); - auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException::fromPath(code, path); - - if (code == Coordination::Error::ZOK) - { - result = true; - return; - } - - /// We need to check who created that node - result = zk->get(path) == toString(current_host_index); - }); - return result; -} - -bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loader_zk_path, UserDefinedSQLObjectType object_type) -{ - bool result = false; - auto holder = with_retries.createRetriesControlHolder("acquireReplicatedSQLObjects"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - String path = zookeeper_path + "/repl_sql_objects_acquired/" + escapeForFileName(loader_zk_path); - zk->createIfNotExists(path, ""); - - path += "/"; - switch (object_type) - { - case UserDefinedSQLObjectType::Function: - path += "functions"; - break; - } - - auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException::fromPath(code, path); - - if (code == Coordination::Error::ZOK) - { - result = true; - return; - } - - /// We need to check who created that node - result = zk->get(path) == toString(current_host_index); - }); - return result; -} - -bool RestoreCoordinationRemote::acquireInsertingDataForKeeperMap(const String & root_zk_path, const String & table_unique_id) -{ - bool lock_acquired = false; - auto holder = with_retries.createRetriesControlHolder("acquireInsertingDataForKeeperMap"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - /// we need to remove leading '/' from root_zk_path - auto normalized_root_zk_path = root_zk_path.substr(1); - std::string restore_lock_path = fs::path(zookeeper_path) / "keeper_map_tables" / escapeForFileName(normalized_root_zk_path); - zk->createAncestors(restore_lock_path); - auto code = zk->tryCreate(restore_lock_path, table_unique_id, zkutil::CreateMode::Persistent); - - if (code == Coordination::Error::ZOK) - { - lock_acquired = true; - return; - } - - if (code == Coordination::Error::ZNODEEXISTS) - lock_acquired = table_unique_id == zk->get(restore_lock_path); - else - zkutil::KeeperException::fromPath(code, restore_lock_path); - }); - return lock_acquired; -} - -void RestoreCoordinationRemote::generateUUIDForTable(ASTCreateQuery & create_query) -{ - String query_str = serializeAST(create_query); - CreateQueryUUIDs new_uuids{create_query, /* generate_random= */ true, /* force_random= */ true}; - String new_uuids_str = new_uuids.toString(); - - auto holder = with_retries.createRetriesControlHolder("generateUUIDForTable"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - String path = zookeeper_path + "/table_uuids/" + escapeForFileName(query_str); - Coordination::Error res = zk->tryCreate(path, new_uuids_str, zkutil::CreateMode::Persistent); - - if (res == Coordination::Error::ZOK) - { - new_uuids.copyToQuery(create_query); - return; - } - - if (res == Coordination::Error::ZNODEEXISTS) - { - CreateQueryUUIDs::fromString(zk->get(path)).copyToQuery(create_query); - return; - } - - zkutil::KeeperException::fromPath(res, path); - }); -} - -void RestoreCoordinationRemote::removeAllNodes() -{ - /// Usually this function is called by the initiator when a restore operation is complete so we don't need the coordination anymore. - /// - /// However there can be a rare situation when this function is called after an error occurs on the initiator of a query - /// while some hosts are still restoring something. Removing all the nodes will remove the parent node of the restore coordination - /// at `zookeeper_path` which might cause such hosts to stop with exception "ZNONODE". Or such hosts might still do some part - /// of their restore work before that. - - auto holder = with_retries.createRetriesControlHolder("removeAllNodes"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - zk->removeRecursive(zookeeper_path); - }); -} - -bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic &) const -{ - /// If its internal concurrency will be checked for the base restore - if (is_internal) - return false; - - bool result = false; - std::string path = zookeeper_path + "/stage"; - - auto holder = with_retries.createRetriesControlHolder("createRootNodes"); - holder.retries_ctl.retryLoop( - [&, &zk = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zk); - - if (! zk->exists(root_zookeeper_path)) - zk->createAncestors(root_zookeeper_path); - - for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) - { - Coordination::Stat stat; - zk->get(root_zookeeper_path, &stat); - Strings existing_restore_paths = zk->getChildren(root_zookeeper_path); - for (const auto & existing_restore_path : existing_restore_paths) - { - if (startsWith(existing_restore_path, "backup-")) - continue; - - String existing_restore_uuid = existing_restore_path; - existing_restore_uuid.erase(0, String("restore-").size()); - - if (existing_restore_uuid == toString(restore_uuid)) - continue; - - String status; - if (zk->tryGet(root_zookeeper_path + "/" + existing_restore_path + "/stage", status)) - { - /// Check if some other restore is in progress - if (status == Stage::SCHEDULED_TO_START) - { - LOG_WARNING(log, "Found a concurrent restore: {}, current restore: {}", existing_restore_uuid, toString(restore_uuid)); - result = true; - return; - } - } - } - - zk->createIfNotExists(path, ""); - auto code = zk->trySet(path, Stage::SCHEDULED_TO_START, stat.version); - if (code == Coordination::Error::ZOK) - break; - bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); - if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException::fromPath(code, path); - } - }); - - return result; -} - -} diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index eb4ba9424ff..29579aa7348 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -100,7 +100,6 @@ RestorerFromBackup::RestorerFromBackup( , context(context_) , process_list_element(context->getProcessListElement()) , after_task_callback(after_task_callback_) - , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) , log(getLogger("RestorerFromBackup")) , tables_dependencies("RestorerFromBackup") @@ -119,12 +118,14 @@ RestorerFromBackup::~RestorerFromBackup() } } -void RestorerFromBackup::run(Mode mode) +void RestorerFromBackup::run(Mode mode_) { /// run() can be called onle once. if (!current_stage.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Already restoring"); + mode = mode_; + /// Find other hosts working along with us to execute this ON CLUSTER query. all_hosts = BackupSettings::Util::filterHostIDs( restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num); @@ -139,6 +140,7 @@ void RestorerFromBackup::run(Mode mode) setStage(Stage::FINDING_TABLES_IN_BACKUP); findDatabasesAndTablesInBackup(); waitFutures(); + logNumberOfDatabasesAndTablesToRestore(); /// Check access rights. setStage(Stage::CHECKING_ACCESS_RIGHTS); @@ -228,20 +230,8 @@ void RestorerFromBackup::setStage(const String & new_stage, const String & messa if (restore_coordination) { - restore_coordination->setStage(new_stage, message); - - /// The initiator of a RESTORE ON CLUSTER query waits for other hosts to complete their work (see waitForStage(Stage::COMPLETED) in BackupsWorker::doRestore), - /// but other hosts shouldn't wait for each others' completion. (That's simply unnecessary and also - /// the initiator may start cleaning up (e.g. removing restore-coordination ZooKeeper nodes) once all other hosts are in Stage::COMPLETED.) - bool need_wait = (new_stage != Stage::COMPLETED); - - if (need_wait) - { - if (new_stage == Stage::FINDING_TABLES_IN_BACKUP) - restore_coordination->waitForStage(new_stage, on_cluster_first_sync_timeout); - else - restore_coordination->waitForStage(new_stage); - } + /// There is no need to sync Stage::COMPLETED with other hosts because it's the last stage. + restore_coordination->setStage(new_stage, message, /* sync = */ (new_stage != Stage::COMPLETED)); } } @@ -384,8 +374,12 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup() } } } +} - LOG_INFO(log, "Will restore {} databases and {} tables", getNumDatabases(), getNumTables()); +void RestorerFromBackup::logNumberOfDatabasesAndTablesToRestore() const +{ + std::string_view action = (mode == CHECK_ACCESS_ONLY) ? "check access rights for restoring" : "restore"; + LOG_INFO(log, "Will {} {} databases and {} tables", action, getNumDatabases(), getNumTables()); } void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional & partitions) diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index e0130ccfcb4..87290618487 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -53,7 +53,7 @@ public: using DataRestoreTasks = std::vector; /// Restores the metadata of databases and tables and returns tasks to restore the data of tables. - void run(Mode mode); + void run(Mode mode_); BackupPtr getBackup() const { return backup; } const RestoreSettings & getRestoreSettings() const { return restore_settings; } @@ -80,10 +80,10 @@ private: ContextMutablePtr context; QueryStatusPtr process_list_element; std::function after_task_callback; - std::chrono::milliseconds on_cluster_first_sync_timeout; std::chrono::milliseconds create_table_timeout; LoggerPtr log; + Mode mode = Mode::RESTORE; Strings all_hosts; DDLRenamingMap renaming_map; std::vector root_paths_in_backup; @@ -97,6 +97,7 @@ private: void findDatabaseInBackupImpl(const String & database_name_in_backup, const std::set & except_table_names); void findEverythingInBackup(const std::set & except_database_names, const std::set & except_table_names); + void logNumberOfDatabasesAndTablesToRestore() const; size_t getNumDatabases() const; size_t getNumTables() const; diff --git a/src/Backups/WithRetries.cpp b/src/Backups/WithRetries.cpp index 772f746e40a..9c18be3ca9e 100644 --- a/src/Backups/WithRetries.cpp +++ b/src/Backups/WithRetries.cpp @@ -1,57 +1,34 @@ #include -#include #include + namespace DB { -namespace Setting -{ - extern const SettingsUInt64 backup_restore_keeper_max_retries; - extern const SettingsUInt64 backup_restore_keeper_retry_initial_backoff_ms; - extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms; - extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multiread; - extern const SettingsFloat backup_restore_keeper_fault_injection_probability; - extern const SettingsUInt64 backup_restore_keeper_fault_injection_seed; - extern const SettingsUInt64 backup_restore_keeper_value_max_size; - extern const SettingsUInt64 backup_restore_batch_size_for_keeper_multi; -} - -WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr context) -{ - return - { - .keeper_max_retries = context->getSettingsRef()[Setting::backup_restore_keeper_max_retries], - .keeper_retry_initial_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_initial_backoff_ms], - .keeper_retry_max_backoff_ms = context->getSettingsRef()[Setting::backup_restore_keeper_retry_max_backoff_ms], - .batch_size_for_keeper_multiread = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multiread], - .keeper_fault_injection_probability = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_probability], - .keeper_fault_injection_seed = context->getSettingsRef()[Setting::backup_restore_keeper_fault_injection_seed], - .keeper_value_max_size = context->getSettingsRef()[Setting::backup_restore_keeper_value_max_size], - .batch_size_for_keeper_multi = context->getSettingsRef()[Setting::backup_restore_batch_size_for_keeper_multi], - }; -} WithRetries::WithRetries( - LoggerPtr log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_) + LoggerPtr log_, zkutil::GetZooKeeper get_zookeeper_, const BackupKeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_) : log(log_) , get_zookeeper(get_zookeeper_) , settings(settings_) , process_list_element(process_list_element_) , callback(callback_) - , global_zookeeper_retries_info( - settings.keeper_max_retries, settings.keeper_retry_initial_backoff_ms, settings.keeper_retry_max_backoff_ms) {} -WithRetries::RetriesControlHolder::RetriesControlHolder(const WithRetries * parent, const String & name) - : info(parent->global_zookeeper_retries_info) - , retries_ctl(name, parent->log, info, parent->process_list_element) +WithRetries::RetriesControlHolder::RetriesControlHolder(const WithRetries * parent, const String & name, Kind kind) + : info( (kind == kInitialization) ? parent->settings.max_retries_while_initializing + : (kind == kErrorHandling) ? parent->settings.max_retries_while_handling_error + : parent->settings.max_retries, + parent->settings.retry_initial_backoff_ms.count(), + parent->settings.retry_max_backoff_ms.count()) + /// We don't use process_list_element while handling an error because the error handling can't be cancellable. + , retries_ctl(name, parent->log, info, (kind == kErrorHandling) ? nullptr : parent->process_list_element) , faulty_zookeeper(parent->getFaultyZooKeeper()) {} -WithRetries::RetriesControlHolder WithRetries::createRetriesControlHolder(const String & name) +WithRetries::RetriesControlHolder WithRetries::createRetriesControlHolder(const String & name, Kind kind) const { - return RetriesControlHolder(this, name); + return RetriesControlHolder(this, name, kind); } void WithRetries::renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const @@ -62,8 +39,8 @@ void WithRetries::renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const { zookeeper = get_zookeeper(); my_faulty_zookeeper->setKeeper(zookeeper); - - callback(my_faulty_zookeeper); + if (callback) + callback(my_faulty_zookeeper); } else { @@ -71,7 +48,7 @@ void WithRetries::renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const } } -const WithRetries::KeeperSettings & WithRetries::getKeeperSettings() const +const BackupKeeperSettings & WithRetries::getKeeperSettings() const { return settings; } @@ -88,8 +65,8 @@ WithRetries::FaultyKeeper WithRetries::getFaultyZooKeeper() const /// The reason is that ZooKeeperWithFaultInjection may reset the underlying pointer and there could be a race condition /// when the same object is used from multiple threads. auto faulty_zookeeper = ZooKeeperWithFaultInjection::createInstance( - settings.keeper_fault_injection_probability, - settings.keeper_fault_injection_seed, + settings.fault_injection_probability, + settings.fault_injection_seed, current_zookeeper, log->name(), log); diff --git a/src/Backups/WithRetries.h b/src/Backups/WithRetries.h index f795a963911..e465fbb1e50 100644 --- a/src/Backups/WithRetries.h +++ b/src/Backups/WithRetries.h @@ -1,9 +1,11 @@ #pragma once -#include +#include #include +#include #include + namespace DB { @@ -15,20 +17,13 @@ class WithRetries { public: using FaultyKeeper = Coordination::ZooKeeperWithFaultInjection::Ptr; - using RenewerCallback = std::function; + using RenewerCallback = std::function; - struct KeeperSettings + enum Kind { - UInt64 keeper_max_retries{0}; - UInt64 keeper_retry_initial_backoff_ms{0}; - UInt64 keeper_retry_max_backoff_ms{0}; - UInt64 batch_size_for_keeper_multiread{10000}; - Float64 keeper_fault_injection_probability{0}; - UInt64 keeper_fault_injection_seed{42}; - UInt64 keeper_value_max_size{1048576}; - UInt64 batch_size_for_keeper_multi{1000}; - - static KeeperSettings fromContext(ContextPtr context); + kNormal, + kInitialization, + kErrorHandling, }; /// For simplicity a separate ZooKeeperRetriesInfo and a faulty [Zoo]Keeper client @@ -48,23 +43,23 @@ public: private: friend class WithRetries; - RetriesControlHolder(const WithRetries * parent, const String & name); + RetriesControlHolder(const WithRetries * parent, const String & name, Kind kind); }; - RetriesControlHolder createRetriesControlHolder(const String & name); - WithRetries(LoggerPtr log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback); + RetriesControlHolder createRetriesControlHolder(const String & name, Kind kind = Kind::kNormal) const; + WithRetries(LoggerPtr log, zkutil::GetZooKeeper get_zookeeper_, const BackupKeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback = {}); /// Used to re-establish new connection inside a retry loop. void renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const; - const KeeperSettings & getKeeperSettings() const; + const BackupKeeperSettings & getKeeperSettings() const; private: /// This will provide a special wrapper which is useful for testing FaultyKeeper getFaultyZooKeeper() const; LoggerPtr log; zkutil::GetZooKeeper get_zookeeper; - KeeperSettings settings; + BackupKeeperSettings settings; QueryStatusPtr process_list_element; /// This callback is called each time when a new [Zoo]Keeper session is created. @@ -76,7 +71,6 @@ private: /// it could lead just to a failed backup which could possibly be successful /// if there were a little bit more retries. RenewerCallback callback; - ZooKeeperRetriesInfo global_zookeeper_retries_info; /// This is needed only to protect zookeeper object mutable std::mutex zookeeper_mutex; diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index d68537513da..320fc06cb2f 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -627,7 +627,7 @@ PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with return PreformattedMessage{stream.str(), e.tryGetMessageFormatString(), e.getMessageFormatStringArgs()}; } -std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) +std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace, bool check_embedded_stacktrace) { try { @@ -635,7 +635,7 @@ std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) } catch (...) { - return getCurrentExceptionMessage(with_stacktrace); + return getCurrentExceptionMessage(with_stacktrace, check_embedded_stacktrace); } } diff --git a/src/Common/Exception.h b/src/Common/Exception.h index a4f55f41caa..8ec640ff642 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -329,7 +329,7 @@ void tryLogException(std::exception_ptr e, const AtomicLogger & logger, const st std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); PreformattedMessage getExceptionMessageAndPattern(const Exception & e, bool with_stacktrace, bool check_embedded_stacktrace = false); -std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace); +std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace, bool check_embedded_stacktrace = false); template diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 6c269e22c35..cdbade04a59 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2660,29 +2660,44 @@ The maximum amount of data consumed by temporary files on disk in bytes for all The maximum amount of data consumed by temporary files on disk in bytes for all concurrently running queries. Zero means unlimited. )", 0)\ \ - DECLARE(UInt64, backup_restore_keeper_max_retries, 20, R"( -Max retries for keeper operations during backup or restore + DECLARE(UInt64, backup_restore_keeper_max_retries, 1000, R"( +Max retries for [Zoo]Keeper operations in the middle of a BACKUP or RESTORE operation. +Should be big enough so the whole operation won't fail because of a temporary [Zoo]Keeper failure. )", 0) \ DECLARE(UInt64, backup_restore_keeper_retry_initial_backoff_ms, 100, R"( Initial backoff timeout for [Zoo]Keeper operations during backup or restore )", 0) \ DECLARE(UInt64, backup_restore_keeper_retry_max_backoff_ms, 5000, R"( Max backoff timeout for [Zoo]Keeper operations during backup or restore +)", 0) \ + DECLARE(UInt64, backup_restore_failure_after_host_disconnected_for_seconds, 3600, R"( +If a host during a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation doesn't recreate its ephemeral 'alive' node in ZooKeeper for this amount of time then the whole backup or restore is considered as failed. +This value should be bigger than any reasonable time for a host to reconnect to ZooKeeper after a failure. +Zero means unlimited. +)", 0) \ + DECLARE(UInt64, backup_restore_keeper_max_retries_while_initializing, 20, R"( +Max retries for [Zoo]Keeper operations during the initialization of a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. +)", 0) \ + DECLARE(UInt64, backup_restore_keeper_max_retries_while_handling_error, 20, R"( +Max retries for [Zoo]Keeper operations while handling an error of a BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. +)", 0) \ + DECLARE(UInt64, backup_restore_finish_timeout_after_error_sec, 180, R"( +How long the initiator should wait for other host to react to the 'error' node and stop their work on the current BACKUP ON CLUSTER or RESTORE ON CLUSTER operation. +)", 0) \ + DECLARE(UInt64, backup_restore_keeper_value_max_size, 1048576, R"( +Maximum size of data of a [Zoo]Keeper's node during backup +)", 0) \ + DECLARE(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, R"( +Maximum size of batch for multi request to [Zoo]Keeper during backup or restore +)", 0) \ + DECLARE(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, R"( +Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore )", 0) \ DECLARE(Float, backup_restore_keeper_fault_injection_probability, 0.0f, R"( Approximate probability of failure for a keeper request during backup or restore. Valid value is in interval [0.0f, 1.0f] )", 0) \ DECLARE(UInt64, backup_restore_keeper_fault_injection_seed, 0, R"( 0 - random seed, otherwise the setting value -)", 0) \ - DECLARE(UInt64, backup_restore_keeper_value_max_size, 1048576, R"( -Maximum size of data of a [Zoo]Keeper's node during backup -)", 0) \ - DECLARE(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, R"( -Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore -)", 0) \ - DECLARE(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, R"( -Maximum size of batch for multi request to [Zoo]Keeper during backup or restore )", 0) \ DECLARE(UInt64, backup_restore_s3_retry_attempts, 1000, R"( Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3fe3e960dc6..b6dd68e1571 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -113,6 +113,11 @@ static std::initializer_list #include +#include +#include #include #include #include @@ -18,13 +20,13 @@ namespace DB namespace { - Block getResultRow(const BackupOperationInfo & info) + Block getResultRow(const String & id, BackupStatus status) { auto column_id = ColumnString::create(); auto column_status = ColumnInt8::create(); - column_id->insert(info.id); - column_status->insert(static_cast(info.status)); + column_id->insert(id); + column_status->insert(static_cast(status)); Block res_columns; res_columns.insert(0, {std::move(column_id), std::make_shared(), "id"}); @@ -36,15 +38,18 @@ namespace BlockIO InterpreterBackupQuery::execute() { + const ASTBackupQuery & backup_query = query_ptr->as(); auto & backups_worker = context->getBackupsWorker(); - auto id = backups_worker.start(query_ptr, context); - auto info = backups_worker.getInfo(id); - if (info.exception) - std::rethrow_exception(info.exception); + auto [id, status] = backups_worker.start(query_ptr, context); + + /// Wait if it's a synchronous operation. + bool async = BackupSettings::isAsync(backup_query); + if (!async) + status = backups_worker.wait(id); BlockIO res_io; - res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(info))); + res_io.pipeline = QueryPipeline(std::make_shared(getResultRow(id, status))); return res_io; } diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 316eced1ed6..2a4a5f3370f 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -889,7 +889,7 @@ private: } }; - auto max_multiread_size = with_retries->getKeeperSettings().batch_size_for_keeper_multiread; + auto max_multiread_size = with_retries->getKeeperSettings().batch_size_for_multiread; auto keys_it = data_children.begin(); while (keys_it != data_children.end()) @@ -941,9 +941,8 @@ void StorageKeeperMap::backupData(BackupEntriesCollector & backup_entries_collec ( getLogger(fmt::format("StorageKeeperMapBackup ({})", getStorageID().getNameForLogs())), [&] { return getClient(); }, - WithRetries::KeeperSettings::fromContext(backup_entries_collector.getContext()), - backup_entries_collector.getContext()->getProcessListElement(), - [](WithRetries::FaultyKeeper &) {} + BackupKeeperSettings::fromContext(backup_entries_collector.getContext()), + backup_entries_collector.getContext()->getProcessListElement() ); backup_entries_collector.addBackupEntries( @@ -972,9 +971,8 @@ void StorageKeeperMap::restoreDataFromBackup(RestorerFromBackup & restorer, cons ( getLogger(fmt::format("StorageKeeperMapRestore ({})", getStorageID().getNameForLogs())), [&] { return getClient(); }, - WithRetries::KeeperSettings::fromContext(restorer.getContext()), - restorer.getContext()->getProcessListElement(), - [](WithRetries::FaultyKeeper &) {} + BackupKeeperSettings::fromContext(restorer.getContext()), + restorer.getContext()->getProcessListElement() ); bool allow_non_empty_tables = restorer.isNonEmptyTableAllowed(); @@ -1037,7 +1035,7 @@ void StorageKeeperMap::restoreDataImpl( CompressedReadBufferFromFile compressed_in{std::move(in_from_file)}; fs::path data_path_fs(zk_data_path); - auto max_multi_size = with_retries->getKeeperSettings().batch_size_for_keeper_multi; + auto max_multi_size = with_retries->getKeeperSettings().batch_size_for_multi; Coordination::Requests create_requests; const auto flush_create_requests = [&] diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index bac783501e1..2ec04e74075 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2125,6 +2125,16 @@ class ClickHouseCluster: ], ) + def remove_file_from_container(self, container_id, path): + self.exec_in_container( + container_id, + [ + "bash", + "-c", + "rm {}".format(path), + ], + ) + def wait_for_url( self, url="http://localhost:8123/ping", conn_timeout=2, interval=2, timeout=60 ): @@ -4128,6 +4138,9 @@ class ClickHouseInstance: self.docker_id, local_path, dest_path ) + def remove_file_from_container(self, path): + return self.cluster.remove_file_from_container(self.docker_id, path) + def get_process_pid(self, process_name): output = self.exec_in_container( [ diff --git a/tests/integration/helpers/config_manager.py b/tests/integration/helpers/config_manager.py new file mode 100644 index 00000000000..0a080a33477 --- /dev/null +++ b/tests/integration/helpers/config_manager.py @@ -0,0 +1,65 @@ +import os + + +class ConfigManager: + """Allows to temporarily add configuration files to the "config.d" or "users.d" directories. + + Can act as a context manager: + + with ConfigManager() as cm: + cm.add_main_config("configs/test_specific_config.xml") # copy "configs/test_specific_config.xml" to "/etc/clickhouse-server/config.d" + ... + # "/etc/clickhouse-server/config.d/test_specific_config.xml" is removed automatically + + """ + + def __init__(self): + self.__added_configs = [] + + def add_main_config(self, node_or_nodes, local_path, reload_config=True): + """Temporarily adds a configuration file to the "config.d" directory.""" + self.__add_config( + node_or_nodes, local_path, dest_dir="config.d", reload_config=reload_config + ) + + def add_user_config(self, node_or_nodes, local_path, reload_config=True): + """Temporarily adds a configuration file to the "users.d" directory.""" + self.__add_config( + node_or_nodes, local_path, dest_dir="users.d", reload_config=reload_config + ) + + def reset(self, reload_config=True): + """Removes all configuration files added by this ConfigManager.""" + if not self.__added_configs: + return + for node, dest_path in self.__added_configs: + node.remove_file_from_container(dest_path) + if reload_config: + for node, _ in self.__added_configs: + node.query("SYSTEM RELOAD CONFIG") + self.__added_configs = [] + + def __add_config(self, node_or_nodes, local_path, dest_dir, reload_config): + nodes_to_add_config = ( + node_or_nodes if (type(node_or_nodes) is list) else [node_or_nodes] + ) + for node in nodes_to_add_config: + src_path = os.path.join(node.cluster.base_dir, local_path) + dest_path = os.path.join( + "/etc/clickhouse-server", dest_dir, os.path.basename(local_path) + ) + node.copy_file_to_container(src_path, dest_path) + if reload_config: + for node in nodes_to_add_config: + node.query("SYSTEM RELOAD CONFIG") + for node in nodes_to_add_config: + dest_path = os.path.join( + "/etc/clickhouse-server", dest_dir, os.path.basename(local_path) + ) + self.__added_configs.append((node, dest_path)) + + def __enter__(self): + return self + + def __exit__(self, type, value, traceback): + self.reset() diff --git a/tests/integration/test_backup_restore_on_cluster/configs/faster_zk_disconnect_detect.xml b/tests/integration/test_backup_restore_on_cluster/configs/faster_zk_disconnect_detect.xml new file mode 100644 index 00000000000..cfc6672ede4 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/faster_zk_disconnect_detect.xml @@ -0,0 +1,12 @@ + + + + zoo1 + 2181 + + 500 + 0 + 1000 + 5000 + + diff --git a/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml index 0886f4bc722..38947be6a5d 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/lesser_timeouts.xml @@ -1,6 +1,6 @@ - 1000 + 1000 10000 3000 diff --git a/tests/integration/test_backup_restore_on_cluster/configs/shutdown_cancel_backups.xml b/tests/integration/test_backup_restore_on_cluster/configs/shutdown_cancel_backups.xml new file mode 100644 index 00000000000..e0c0e0b32cd --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/shutdown_cancel_backups.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_backup_restore_on_cluster/configs/slow_backups.xml b/tests/integration/test_backup_restore_on_cluster/configs/slow_backups.xml new file mode 100644 index 00000000000..933c3250054 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/slow_backups.xml @@ -0,0 +1,7 @@ + + + true + + 12 + 2 + diff --git a/tests/integration/test_backup_restore_on_cluster/configs/zookeeper_retries.xml b/tests/integration/test_backup_restore_on_cluster/configs/zookeeper_retries.xml index 1283f28a8cb..7af54d2dd95 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/zookeeper_retries.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/zookeeper_retries.xml @@ -1,9 +1,12 @@ - 1000 - 1 - 1 + 50 + 100 + 1000 + 10 + 2 + 3 42 0.002 diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index a1082c563d1..257938a75c5 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1153,7 +1153,7 @@ def test_get_error_from_other_host(): node1.query("INSERT INTO tbl VALUES (3)") backup_name = new_backup_name() - expected_error = "Got error from node2.*Table default.tbl was not found" + expected_error = "Got error from host node2.*Table default.tbl was not found" assert re.search( expected_error, node1.query_and_get_error( diff --git a/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py b/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py new file mode 100644 index 00000000000..f63dc2aef3d --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/test_cancel_backup.py @@ -0,0 +1,780 @@ +import os +import random +import time +import uuid + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.config_manager import ConfigManager +from helpers.network import PartitionManager +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +main_configs = [ + "configs/backups_disk.xml", + "configs/cluster.xml", + "configs/lesser_timeouts.xml", # Default timeouts are quite big (a few minutes), the tests don't need them to be that big. + "configs/slow_backups.xml", + "configs/shutdown_cancel_backups.xml", +] + +user_configs = [ + "configs/zookeeper_retries.xml", +] + +node1 = cluster.add_instance( + "node1", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "node1", "shard": "shard1"}, + with_zookeeper=True, + stay_alive=True, # Necessary for "test_shutdown_cancel_backup" +) + +node2 = cluster.add_instance( + "node2", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "node2", "shard": "shard1"}, + with_zookeeper=True, + stay_alive=True, # Necessary for "test_shutdown_cancel_backup" +) + +nodes = [node1, node2] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + node1.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' SYNC") + + +# Utilities + + +# Gets a printable version the name of a node. +def get_node_name(node): + return "node1" if (node == node1) else "node2" + + +# Choose a random instance. +def random_node(): + return random.choice(nodes) + + +# Makes table "tbl" and fill it with data. +def create_and_fill_table(node, num_parts=10, on_cluster=True): + # We use partitioning to make sure there will be more files in a backup. + partition_by_clause = " PARTITION BY x%" + str(num_parts) if num_parts > 1 else "" + node.query( + "CREATE TABLE tbl " + + ("ON CLUSTER 'cluster' " if on_cluster else "") + + "(x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') " + + "ORDER BY tuple()" + + partition_by_clause + ) + if num_parts > 0: + node.query(f"INSERT INTO tbl SELECT number FROM numbers({num_parts})") + + +# Generates an ID suitable both as backup id or restore id. +def random_id(): + return uuid.uuid4().hex + + +# Generates a backup name prepared for using in BACKUP and RESTORE queries. +def get_backup_name(backup_id): + return f"Disk('backups', '{backup_id}')" + + +# Reads the status of a backup or a restore from system.backups. +def get_status(initiator, backup_id=None, restore_id=None): + id = backup_id if backup_id is not None else restore_id + return initiator.query(f"SELECT status FROM system.backups WHERE id='{id}'").rstrip( + "\n" + ) + + +# Reads the error message of a failed backup or a failed restore from system.backups. +def get_error(initiator, backup_id=None, restore_id=None): + id = backup_id if backup_id is not None else restore_id + return initiator.query(f"SELECT error FROM system.backups WHERE id='{id}'").rstrip( + "\n" + ) + + +# Waits until the status of a backup or a restore becomes a desired one. +# Returns how many seconds the function was waiting. +def wait_status( + initiator, + status="BACKUP_CREATED", + backup_id=None, + restore_id=None, + timeout=None, +): + print(f"Waiting for status {status}") + id = backup_id if backup_id is not None else restore_id + operation_name = "backup" if backup_id is not None else "restore" + current_status = get_status(initiator, backup_id=backup_id, restore_id=restore_id) + waited = 0 + while ( + (current_status != status) + and (current_status in ["CREATING_BACKUP", "RESTORING"]) + and ((timeout is None) or (waited < timeout)) + ): + sleep_time = 1 if (timeout is None) else min(1, timeout - waited) + time.sleep(sleep_time) + waited += sleep_time + current_status = get_status( + initiator, backup_id=backup_id, restore_id=restore_id + ) + start_time, end_time = ( + initiator.query( + f"SELECT start_time, end_time FROM system.backups WHERE id='{id}'" + ) + .splitlines()[0] + .split("\t") + ) + print( + f"{get_node_name(initiator)} : Got status {current_status} for {operation_name} {id} after waiting {waited} seconds " + f"(start_time = {start_time}, end_time = {end_time})" + ) + assert current_status == status + + +# Returns how many entries are in system.processes corresponding to a specified backup or restore. +def get_num_system_processes( + node_or_nodes, backup_id=None, restore_id=None, is_initial_query=None +): + id = backup_id if backup_id is not None else restore_id + query_kind = "Backup" if backup_id is not None else "Restore" + total = 0 + filter_for_is_initial_query = ( + f" AND (is_initial_query = {is_initial_query})" + if is_initial_query is not None + else "" + ) + nodes_to_consider = ( + node_or_nodes if (type(node_or_nodes) is list) else [node_or_nodes] + ) + for node in nodes_to_consider: + count = int( + node.query( + f"SELECT count() FROM system.processes WHERE (query_kind='{query_kind}') AND (query LIKE '%{id}%'){filter_for_is_initial_query}" + ) + ) + total += count + return total + + +# Waits until the number of entries in system.processes corresponding to a specified backup or restore becomes a desired one. +# Returns how many seconds the function was waiting. +def wait_num_system_processes( + node_or_nodes, + num_system_processes=0, + backup_id=None, + restore_id=None, + is_initial_query=None, + timeout=None, +): + print(f"Waiting for number of system processes = {num_system_processes}") + id = backup_id if backup_id is not None else restore_id + operation_name = "backup" if backup_id is not None else "restore" + current_count = get_num_system_processes( + node_or_nodes, + backup_id=backup_id, + restore_id=restore_id, + is_initial_query=is_initial_query, + ) + + def is_current_count_ok(): + return (current_count == num_system_processes) or ( + num_system_processes == "1+" and current_count >= 1 + ) + + waited = 0 + while not is_current_count_ok() and ((timeout is None) or (waited < timeout)): + sleep_time = 1 if (timeout is None) else min(1, timeout - waited) + time.sleep(sleep_time) + waited += sleep_time + current_count = get_num_system_processes( + node_or_nodes, + backup_id=backup_id, + restore_id=restore_id, + is_initial_query=is_initial_query, + ) + if is_current_count_ok(): + print( + f"Got {current_count} system processes for {operation_name} {id} after waiting {waited} seconds" + ) + else: + nodes_to_consider = ( + node_or_nodes if (type(node_or_nodes) is list) else [node_or_nodes] + ) + for node in nodes_to_consider: + count = get_num_system_processes( + node, backup_id=backup_id, restore_id=restore_id + ) + print( + f"{get_node_name(node)}: Got {count} system processes for {operation_name} {id} after waiting {waited} seconds" + ) + assert False + return waited + + +# Kills a BACKUP or RESTORE query. +# Returns how many seconds the KILL QUERY was executing. +def kill_query( + node, backup_id=None, restore_id=None, is_initial_query=None, timeout=None +): + id = backup_id if backup_id is not None else restore_id + query_kind = "Backup" if backup_id is not None else "Restore" + operation_name = "backup" if backup_id is not None else "restore" + print(f"{get_node_name(node)}: Cancelling {operation_name} {id}") + filter_for_is_initial_query = ( + f" AND (is_initial_query = {is_initial_query})" + if is_initial_query is not None + else "" + ) + node.query( + f"KILL QUERY WHERE (query_kind='{query_kind}') AND (query LIKE '%{id}%'){filter_for_is_initial_query} SYNC" + ) + node.query("SYSTEM FLUSH LOGS") + duration = ( + int( + node.query( + f"SELECT query_duration_ms FROM system.query_log WHERE query_kind='KillQuery' AND query LIKE '%{id}%' AND type='QueryFinish'" + ) + ) + / 1000 + ) + print( + f"{get_node_name(node)}: Cancelled {operation_name} {id} after {duration} seconds" + ) + if timeout is not None: + assert duration < timeout + + +# Stops all ZooKeeper servers. +def stop_zookeeper_servers(zoo_nodes): + print(f"Stopping ZooKeeper servers {zoo_nodes}") + old_time = time.monotonic() + cluster.stop_zookeeper_nodes(zoo_nodes) + print( + f"Stopped ZooKeeper servers {zoo_nodes} in {time.monotonic() - old_time} seconds" + ) + + +# Starts all ZooKeeper servers back. +def start_zookeeper_servers(zoo_nodes): + print(f"Starting ZooKeeper servers {zoo_nodes}") + old_time = time.monotonic() + cluster.start_zookeeper_nodes(zoo_nodes) + print( + f"Started ZooKeeper servers {zoo_nodes} in {time.monotonic() - old_time} seconds" + ) + + +# Sleeps for random amount of time. +def random_sleep(max_seconds): + if random.randint(0, 5) > 0: + sleep(random.uniform(0, max_seconds)) + + +def sleep(seconds): + print(f"Sleeping {seconds} seconds") + time.sleep(seconds) + + +# Checks that BACKUP and RESTORE cleaned up properly with no trash left in ZooKeeper, backups folder, and logs. +class NoTrashChecker: + def __init__(self): + self.expect_backups = [] + self.expect_unfinished_backups = [] + self.expect_errors = [] + self.allow_errors = [] + self.check_zookeeper = True + + # Sleep 1 second to ensure this NoTrashChecker won't collect errors from a possible previous NoTrashChecker. + time.sleep(1) + + self.__start_time_for_collecting_errors = time.gmtime() + self.__previous_list_of_backups = set( + os.listdir(os.path.join(node1.cluster.instances_dir, "backups")) + ) + + self.__previous_list_of_znodes = set( + node1.query( + "SELECT name FROM system.zookeeper WHERE path = '/clickhouse/backups' " + + "AND NOT (name == 'alive_tracker')" + ).splitlines() + ) + + def __enter__(self): + return self + + def __exit__(self, type, value, traceback): + list_of_znodes = set( + node1.query( + "SELECT name FROM system.zookeeper WHERE path = '/clickhouse/backups' " + + "AND NOT (name == 'alive_tracker')" + ).splitlines() + ) + new_znodes = list_of_znodes.difference(self.__previous_list_of_znodes) + if new_znodes: + print(f"Found nodes in ZooKeeper: {new_znodes}") + for node in new_znodes: + print( + f"Nodes in '/clickhouse/backups/{node}':\n" + + node1.query( + f"SELECT name FROM system.zookeeper WHERE path = '/clickhouse/backups/{node}'" + ) + ) + print( + f"Nodes in '/clickhouse/backups/{node}/stage':\n" + + node1.query( + f"SELECT name FROM system.zookeeper WHERE path = '/clickhouse/backups/{node}/stage'" + ) + ) + if self.check_zookeeper: + assert new_znodes == set() + + list_of_backups = set( + os.listdir(os.path.join(node1.cluster.instances_dir, "backups")) + ) + new_backups = list_of_backups.difference(self.__previous_list_of_backups) + unfinished_backups = set( + backup + for backup in new_backups + if not os.path.exists( + os.path.join(node1.cluster.instances_dir, "backups", backup, ".backup") + ) + ) + new_backups = set( + backup for backup in new_backups if backup not in unfinished_backups + ) + if new_backups: + print(f"Found new backups: {new_backups}") + if unfinished_backups: + print(f"Found unfinished backups: {unfinished_backups}") + assert new_backups == set(self.expect_backups) + assert unfinished_backups == set(self.expect_unfinished_backups) + + all_errors = set() + start_time = time.strftime( + "%Y-%m-%d %H:%M:%S", self.__start_time_for_collecting_errors + ) + for node in nodes: + errors_query_result = node.query( + "SELECT name FROM system.errors WHERE last_error_time >= toDateTime('" + + start_time + + "') " + + "AND NOT ((name == 'KEEPER_EXCEPTION') AND (last_error_message LIKE '%Fault injection%')) " + + "AND NOT (name == 'NO_ELEMENTS_IN_CONFIG')" + ) + errors = errors_query_result.splitlines() + if errors: + print(f"{get_node_name(node)}: Found errors: {errors}") + print( + node.query( + "SELECT name, last_error_message FROM system.errors WHERE last_error_time >= toDateTime('" + + start_time + + "')" + ) + ) + for error in errors: + assert (error in self.expect_errors) or (error in self.allow_errors) + all_errors.update(errors) + + not_found_expected_errors = set(self.expect_errors).difference(all_errors) + if not_found_expected_errors: + print(f"Not found expected errors: {not_found_expected_errors}") + assert False + + +__backup_id_of_successful_backup = None + + +# Generates a backup which will be used to test RESTORE. +def get_backup_id_of_successful_backup(): + global __backup_id_of_successful_backup + if __backup_id_of_successful_backup is None: + __backup_id_of_successful_backup = random_id() + with NoTrashChecker() as no_trash_checker: + print("Will make backup successfully") + backup_id = __backup_id_of_successful_backup + create_and_fill_table(random_node()) + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC" + ) + wait_status(initiator, "BACKUP_CREATED", backup_id=backup_id) + assert get_num_system_processes(nodes, backup_id=backup_id) == 0 + no_trash_checker.expect_backups = [backup_id] + + # Dropping the table before restoring. + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") + + return __backup_id_of_successful_backup + + +# Actual tests + + +# Test that a BACKUP operation can be cancelled with KILL QUERY. +def test_cancel_backup(): + with NoTrashChecker() as no_trash_checker: + create_and_fill_table(random_node()) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + backup_id = random_id() + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC" + ) + + assert get_status(initiator, backup_id=backup_id) == "CREATING_BACKUP" + assert get_num_system_processes(initiator, backup_id=backup_id) >= 1 + + # We shouldn't wait too long here, because otherwise the backup might be completed before we cancel it. + random_sleep(3) + + node_to_cancel, cancel_as_initiator = random.choice( + [(node1, False), (node2, False), (initiator, True)] + ) + + wait_num_system_processes( + node_to_cancel, + "1+", + backup_id=backup_id, + is_initial_query=cancel_as_initiator, + ) + + print( + f"Cancelling on {'initiator' if cancel_as_initiator else 'node'} {get_node_name(node_to_cancel)}" + ) + + # The timeout is 2 seconds here because a backup must be cancelled quickly. + kill_query( + node_to_cancel, + backup_id=backup_id, + is_initial_query=cancel_as_initiator, + timeout=3, + ) + + if cancel_as_initiator: + assert get_status(initiator, backup_id=backup_id) == "BACKUP_CANCELLED" + wait_status(initiator, "BACKUP_CANCELLED", backup_id=backup_id, timeout=3) + + assert "QUERY_WAS_CANCELLED" in get_error(initiator, backup_id=backup_id) + assert get_num_system_processes(nodes, backup_id=backup_id) == 0 + no_trash_checker.expect_errors = ["QUERY_WAS_CANCELLED"] + + +# Test that a RESTORE operation can be cancelled with KILL QUERY. +def test_cancel_restore(): + # Make backup. + backup_id = get_backup_id_of_successful_backup() + + # Cancel restoring. + with NoTrashChecker() as no_trash_checker: + print("Will cancel restoring") + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + restore_id = random_id() + initiator.query( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {get_backup_name(backup_id)} SETTINGS id='{restore_id}' ASYNC" + ) + + assert get_status(initiator, restore_id=restore_id) == "RESTORING" + assert get_num_system_processes(initiator, restore_id=restore_id) >= 1 + + # We shouldn't wait too long here, because otherwise the restore might be completed before we cancel it. + random_sleep(3) + + node_to_cancel, cancel_as_initiator = random.choice( + [(node1, False), (node2, False), (initiator, True)] + ) + + wait_num_system_processes( + node_to_cancel, + "1+", + restore_id=restore_id, + is_initial_query=cancel_as_initiator, + ) + + print( + f"Cancelling on {'initiator' if cancel_as_initiator else 'node'} {get_node_name(node_to_cancel)}" + ) + + # The timeout is 2 seconds here because a restore must be cancelled quickly. + kill_query( + node_to_cancel, + restore_id=restore_id, + is_initial_query=cancel_as_initiator, + timeout=3, + ) + + if cancel_as_initiator: + assert get_status(initiator, restore_id=restore_id) == "RESTORE_CANCELLED" + wait_status(initiator, "RESTORE_CANCELLED", restore_id=restore_id, timeout=3) + + assert "QUERY_WAS_CANCELLED" in get_error(initiator, restore_id=restore_id) + assert get_num_system_processes(nodes, restore_id=restore_id) == 0 + no_trash_checker.expect_errors = ["QUERY_WAS_CANCELLED"] + + # Restore successfully. + with NoTrashChecker() as no_trash_checker: + print("Will restore from backup successfully") + restore_id = random_id() + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + initiator.query( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {get_backup_name(backup_id)} SETTINGS id='{restore_id}' ASYNC" + ) + + wait_status(initiator, "RESTORED", restore_id=restore_id) + assert get_num_system_processes(nodes, restore_id=restore_id) == 0 + + +# Test that shutdown cancels a running backup and doesn't wait until it finishes. +def test_shutdown_cancels_backup(): + with NoTrashChecker() as no_trash_checker: + create_and_fill_table(random_node()) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + backup_id = random_id() + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC" + ) + + assert get_status(initiator, backup_id=backup_id) == "CREATING_BACKUP" + assert get_num_system_processes(initiator, backup_id=backup_id) >= 1 + + # We shouldn't wait too long here, because otherwise the backup might be completed before we cancel it. + random_sleep(3) + + node_to_restart = random.choice([node1, node2]) + wait_num_system_processes(node_to_restart, "1+", backup_id=backup_id) + + print(f"{get_node_name(node_to_restart)}: Restarting...") + node_to_restart.restart_clickhouse() # Must cancel the backup. + print(f"{get_node_name(node_to_restart)}: Restarted") + + wait_num_system_processes(nodes, 0, backup_id=backup_id) + + if initiator != node_to_restart: + assert get_status(initiator, backup_id=backup_id) == "BACKUP_CANCELLED" + assert "QUERY_WAS_CANCELLED" in get_error(initiator, backup_id=backup_id) + + # The information about this cancelled backup must be stored in system.backup_log + initiator.query("SYSTEM FLUSH LOGS") + assert initiator.query( + f"SELECT status FROM system.backup_log WHERE id='{backup_id}' ORDER BY status" + ) == TSV(["CREATING_BACKUP", "BACKUP_CANCELLED"]) + + no_trash_checker.expect_errors = ["QUERY_WAS_CANCELLED"] + + +# After an error backup should clean the destination folder and used nodes in ZooKeeper. +# No unexpected errors must be generated. +def test_error_leaves_no_trash(): + with NoTrashChecker() as no_trash_checker: + # We create table "tbl" on one node only in order to make "BACKUP TABLE tbl ON CLUSTER" fail + # (because of the non-existing table on another node). + create_and_fill_table(random_node(), on_cluster=False) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + backup_id = random_id() + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC" + ) + + wait_status(initiator, "BACKUP_FAILED", backup_id=backup_id) + assert "UNKNOWN_TABLE" in get_error(initiator, backup_id=backup_id) + + assert get_num_system_processes(nodes, backup_id=backup_id) == 0 + no_trash_checker.expect_errors = ["UNKNOWN_TABLE"] + + +# A backup must be stopped if Zookeeper is disconnected longer than `failure_after_host_disconnected_for_seconds`. +def test_long_disconnection_stops_backup(): + with NoTrashChecker() as no_trash_checker, ConfigManager() as config_manager: + # Config "faster_zk_disconnect_detect.xml" is used in this test to decrease number of retries when reconnecting to ZooKeeper. + # Without this config this test can take several minutes (instead of seconds) to run. + config_manager.add_main_config(nodes, "configs/faster_zk_disconnect_detect.xml") + + create_and_fill_table(random_node(), num_parts=100) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + backup_id = random_id() + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC", + settings={"backup_restore_failure_after_host_disconnected_for_seconds": 3}, + ) + + assert get_status(initiator, backup_id=backup_id) == "CREATING_BACKUP" + assert get_num_system_processes(initiator, backup_id=backup_id) >= 1 + + no_trash_checker.expect_unfinished_backups = [backup_id] + no_trash_checker.allow_errors = [ + "FAILED_TO_SYNC_BACKUP_OR_RESTORE", + "KEEPER_EXCEPTION", + "SOCKET_TIMEOUT", + "CANNOT_READ_ALL_DATA", + "NETWORK_ERROR", + "TABLE_IS_READ_ONLY", + ] + no_trash_checker.check_zookeeper = False + + with PartitionManager() as pm: + random_sleep(3) + + time_before_disconnection = time.monotonic() + + node_to_drop_zk_connection = random_node() + print( + f"Dropping connection between {get_node_name(node_to_drop_zk_connection)} and ZooKeeper" + ) + pm.drop_instance_zk_connections(node_to_drop_zk_connection) + + # Being disconnected from ZooKeeper a backup is expected to fail. + wait_status(initiator, "BACKUP_FAILED", backup_id=backup_id) + + time_to_fail = time.monotonic() - time_before_disconnection + error = get_error(initiator, backup_id=backup_id) + print(f"error={error}") + assert "Lost connection" in error + + # A backup is expected to fail, but it isn't expected to fail too soon. + print(f"Backup failed after {time_to_fail} seconds disconnection") + assert time_to_fail > 3 + assert time_to_fail < 30 + + +# A backup must NOT be stopped if Zookeeper is disconnected shorter than `failure_after_host_disconnected_for_seconds`. +def test_short_disconnection_doesnt_stop_backup(): + with NoTrashChecker() as no_trash_checker, ConfigManager() as config_manager: + use_faster_zk_disconnect_detect = random.choice([True, False]) + if use_faster_zk_disconnect_detect: + print("Using faster_zk_disconnect_detect.xml") + config_manager.add_main_config( + nodes, "configs/faster_zk_disconnect_detect.xml" + ) + + create_and_fill_table(random_node()) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + backup_id = random_id() + initiator.query( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {get_backup_name(backup_id)} SETTINGS id='{backup_id}' ASYNC", + settings={"backup_restore_failure_after_host_disconnected_for_seconds": 6}, + ) + + assert get_status(initiator, backup_id=backup_id) == "CREATING_BACKUP" + assert get_num_system_processes(initiator, backup_id=backup_id) >= 1 + + # Dropping connection for less than `failure_after_host_disconnected_for_seconds` + with PartitionManager() as pm: + random_sleep(3) + node_to_drop_zk_connection = random_node() + print( + f"Dropping connection between {get_node_name(node_to_drop_zk_connection)} and ZooKeeper" + ) + pm.drop_instance_zk_connections(node_to_drop_zk_connection) + random_sleep(3) + print( + f"Restoring connection between {get_node_name(node_to_drop_zk_connection)} and ZooKeeper" + ) + + # Backup must be successful. + wait_status(initiator, "BACKUP_CREATED", backup_id=backup_id) + assert get_num_system_processes(nodes, backup_id=backup_id) == 0 + + no_trash_checker.expect_backups = [backup_id] + no_trash_checker.allow_errors = [ + "KEEPER_EXCEPTION", + "SOCKET_TIMEOUT", + "CANNOT_READ_ALL_DATA", + "NETWORK_ERROR", + "TABLE_IS_READ_ONLY", + ] + + +# A restore must NOT be stopped if Zookeeper is disconnected shorter than `failure_after_host_disconnected_for_seconds`. +def test_short_disconnection_doesnt_stop_restore(): + # Make a backup. + backup_id = get_backup_id_of_successful_backup() + + # Restore from the backup. + with NoTrashChecker() as no_trash_checker, ConfigManager() as config_manager: + use_faster_zk_disconnect_detect = random.choice([True, False]) + if use_faster_zk_disconnect_detect: + print("Using faster_zk_disconnect_detect.xml") + config_manager.add_main_config( + nodes, "configs/faster_zk_disconnect_detect.xml" + ) + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator") + + restore_id = random_id() + initiator.query( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {get_backup_name(backup_id)} SETTINGS id='{restore_id}' ASYNC", + settings={"backup_restore_failure_after_host_disconnected_for_seconds": 6}, + ) + + assert get_status(initiator, restore_id=restore_id) == "RESTORING" + assert get_num_system_processes(initiator, restore_id=restore_id) >= 1 + + # Dropping connection for less than `failure_after_host_disconnected_for_seconds` + with PartitionManager() as pm: + random_sleep(3) + node_to_drop_zk_connection = random_node() + print( + f"Dropping connection between {get_node_name(node_to_drop_zk_connection)} and ZooKeeper" + ) + pm.drop_instance_zk_connections(node_to_drop_zk_connection) + random_sleep(3) + print( + f"Restoring connection between {get_node_name(node_to_drop_zk_connection)} and ZooKeeper" + ) + + # Restore must be successful. + wait_status(initiator, "RESTORED", restore_id=restore_id) + assert get_num_system_processes(nodes, restore_id=restore_id) == 0 + + no_trash_checker.allow_errors = [ + "KEEPER_EXCEPTION", + "SOCKET_TIMEOUT", + "CANNOT_READ_ALL_DATA", + "NETWORK_ERROR", + "TABLE_IS_READ_ONLY", + ] diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index 846c41592f7..3dea986e3d9 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -145,7 +145,7 @@ def wait_for_restore(node, restore_id): def check_backup_error(error): expected_errors = [ - "Concurrent backups not supported", + "Concurrent backups are not allowed", "BACKUP_ALREADY_EXISTS", ] assert any([expected_error in error for expected_error in expected_errors]) @@ -153,7 +153,7 @@ def check_backup_error(error): def check_restore_error(error): expected_errors = [ - "Concurrent restores not supported", + "Concurrent restores are not allowed", "Cannot restore the table default.tbl because it already contains some data", ] assert any([expected_error in error for expected_error in expected_errors]) From 7c3ba9324a76ab05ebd132b80bc358f48b135f43 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 30 Oct 2024 22:09:14 +0100 Subject: [PATCH 672/816] Correct test "test_stop_other_host_during_backup[False]" and remove test "test_stop_other_host_during_backup[True]" because it was replaced by new test "test_long_disconnection_stops_backup". --- .../test_backup_restore_on_cluster/test.py | 26 +++++-------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 257938a75c5..4d4fe0e665a 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -1162,8 +1162,7 @@ def test_get_error_from_other_host(): ) -@pytest.mark.parametrize("kill", [False, True]) -def test_stop_other_host_during_backup(kill): +def test_shutdown_waits_for_backup(): node1.query( "CREATE TABLE tbl ON CLUSTER 'cluster' (" "x UInt8" @@ -1182,7 +1181,7 @@ def test_stop_other_host_during_backup(kill): # If kill=False the pending backup must be completed # If kill=True the pending backup might be completed or failed - node2.stop_clickhouse(kill=kill) + node2.stop_clickhouse(kill=False) assert_eq_with_retry( node1, @@ -1192,22 +1191,11 @@ def test_stop_other_host_during_backup(kill): ) status = node1.query(f"SELECT status FROM system.backups WHERE id='{id}'").strip() - - if kill: - expected_statuses = ["BACKUP_CREATED", "BACKUP_FAILED"] - else: - expected_statuses = ["BACKUP_CREATED", "BACKUP_CANCELLED"] - - assert status in expected_statuses + assert status == "BACKUP_CREATED" node2.start_clickhouse() - if status == "BACKUP_CREATED": - node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") - node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") - node1.query("SYSTEM SYNC REPLICA tbl") - assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) - elif status == "BACKUP_FAILED": - assert not os.path.exists( - os.path.join(get_path_to_backup(backup_name), ".backup") - ) + node1.query("DROP TABLE tbl ON CLUSTER 'cluster' SYNC") + node1.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}") + node1.query("SYSTEM SYNC REPLICA tbl") + assert node1.query("SELECT * FROM tbl ORDER BY x") == TSV([3, 5]) From 8d622000b05c7bb54d7e4587a0568bdba327d059 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:26:13 -0300 Subject: [PATCH 673/816] remove unrelated file --- .../Impl/Parquet/ParquetFilterCondition.h | 49 ------------------- 1 file changed, 49 deletions(-) delete mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h deleted file mode 100644 index a09eaa9ced0..00000000000 --- a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.h +++ /dev/null @@ -1,49 +0,0 @@ -#pragma once - -#include - -#if USE_PARQUET - -#include - -namespace DB -{ - -class ParquetFilterCondition -{ - struct ConditionElement - { - enum Function - { - /// Atoms of a Boolean expression. - FUNCTION_EQUALS, - FUNCTION_NOT_EQUALS, - FUNCTION_IN, - FUNCTION_NOT_IN, - /// Can take any value. - FUNCTION_UNKNOWN, - /// Operators of the logical expression. - FUNCTION_NOT, - FUNCTION_AND, - FUNCTION_OR, - /// Constants - ALWAYS_FALSE, - ALWAYS_TRUE, - }; - - using ColumnPtr = IColumn::Ptr; - using HashesForColumns = std::vector>; - using KeyColumns = std::vector; - - Function function; - // each entry represents a list of hashes per column - // suppose there are three columns with 2 rows each - // hashes_per_column.size() == 3 and hashes_per_column[0].size() == 2 - HashesForColumns hashes_per_column; - KeyColumns key_columns; - }; -}; - -} - -#endif From 33fdddf9d9327ccc62ac9e0eae3bc022c25f5975 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:26:37 -0300 Subject: [PATCH 674/816] remove unrelated file --- .../Formats/Impl/Parquet/ParquetFilterCondition.cpp | 5 ----- 1 file changed, 5 deletions(-) delete mode 100644 src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp diff --git a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp b/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp deleted file mode 100644 index 27be594d3c2..00000000000 --- a/src/Processors/Formats/Impl/Parquet/ParquetFilterCondition.cpp +++ /dev/null @@ -1,5 +0,0 @@ -// -// Created by laptop on 10/29/24. -// - -#include "ParquetFilterCondition.h" From fc1fd46686722c5bb13c95edf7051c4e21be7b68 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 30 Oct 2024 23:36:15 +0100 Subject: [PATCH 675/816] fix test --- ...eplicas_join_algo_and_analyzer_4.reference | 29 ++++++++++++++++++ ...allel_replicas_join_algo_and_analyzer_4.sh | 30 +++++++++++-------- 2 files changed, 46 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference index 9fc156b5fb0..8464317f7e6 100644 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference @@ -27,3 +27,32 @@ SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP 500030000 500040000 SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` ALL LEFT JOIN (SELECT `__table4`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table4`) AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` GROUP BY `__table1`.`item_id` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP BY `__table1`.`item_id` +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` ALL LEFT JOIN (SELECT `__table4`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table4`) AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh index a588fa47c2d..0e1f07b6ac5 100755 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: long, no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -74,20 +75,23 @@ query3=" ORDER BY price_sold " -for query in "${query1}" "${query2}" "${query3}"; do - for enable_parallel_replicas in {0..1}; do - ${CLICKHOUSE_CLIENT} --query=" - set enable_analyzer=1; - set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; +for prefer_local_plan in {0..1}; do + for query in "${query1}" "${query2}" "${query3}"; do + for enable_parallel_replicas in {0..1}; do + ${CLICKHOUSE_CLIENT} --query=" + set enable_analyzer=1; + set parallel_replicas_local_plan=${prefer_local_plan}; + set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; - ${query}; + ${query}; - SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') - FROM - ( - EXPLAIN actions=1 ${query} - ) - WHERE explain LIKE '%ParallelReplicas%'; - " + SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') + FROM + ( + EXPLAIN actions=1 ${query} + ) + WHERE explain LIKE '%ParallelReplicas%'; + " + done done done From 26da759cf20459c38bc504b2d9a790c38c66c5a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Oct 2024 07:51:20 +0100 Subject: [PATCH 676/816] Add changelog for 24.10 --- CHANGELOG.md | 169 +++++++++++++++++++++++++-------------------------- 1 file changed, 83 insertions(+), 86 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index adb3fbe22ba..c9d44a49b7d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,123 +21,119 @@ * Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. Note that new inegration is disabled by default, to use it, please set `` to `false` in server config. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). -* A new `--progress-table` option in clickhouse-client prints a table with metrics changing during query execution; a new `--enable-progress-table-toggle` is associated with the `--progress-table` option, and toggles the rendering of the progress table by pressing the control key (Space). [#63689](https://github.com/ClickHouse/ClickHouse/pull/63689) ([Maria Khristenko](https://github.com/mariaKhr)). -* This allows to grant access to the wildcard prefixes. `GRANT SELECT ON db.table_pefix_* TO user`. [#65311](https://github.com/ClickHouse/ClickHouse/pull/65311) ([pufit](https://github.com/pufit)). -* Add system.query_metric_log which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). -* A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Support --copy mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). -* Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). -* Support aggregate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. This is for spark compatibility. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). -* Support function arrayElementOrNull. It returns null if array index is out of range or map key not found. [#69646](https://github.com/ClickHouse/ClickHouse/pull/69646) ([李扬](https://github.com/taiyang-li)). -* Allows users to specify regular expressions through new `message_regexp` and `message_regexp_negative` fields in the `config.xml` file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. [#69657](https://github.com/ClickHouse/ClickHouse/pull/69657) ([Peter Nguyen](https://github.com/petern48)). -* Re-added `RIPEMD160` function, which computes the RIPEMD-160 cryptographic hash of a string. Example: `SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))` returns `37F332F68DB77BD9D7EDD4969571AD671CF9DD3B`. [#70087](https://github.com/ClickHouse/ClickHouse/pull/70087) ([Dergousov Maxim](https://github.com/m7kss1)). +* Allow to grant access to the wildcard prefixes. `GRANT SELECT ON db.table_pefix_* TO user`. [#65311](https://github.com/ClickHouse/ClickHouse/pull/65311) ([pufit](https://github.com/pufit)). +* If you press space bar during query runtime, the client will display a real-time table with detailed metrics. You can enable it globally with the new `--progress-table` option in clickhouse-client; a new `--enable-progress-table-toggle` is associated with the `--progress-table` option, and toggles the rendering of the progress table by pressing the control key (Space). [#63689](https://github.com/ClickHouse/ClickHouse/pull/63689) ([Maria Khristenko](https://github.com/mariaKhr)), [#70423](https://github.com/ClickHouse/ClickHouse/pull/70423) ([Julia Kartseva](https://github.com/jkartseva)). * Allow to cache read files for object storage table engines and data lakes using hash from ETag + file path as cache key. [#70135](https://github.com/ClickHouse/ClickHouse/pull/70135) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Support reading Iceberg tables on HDFS. [#70268](https://github.com/ClickHouse/ClickHouse/pull/70268) ([flynn](https://github.com/ucasfl)). -* Supports standard CTE, `with insert`, as previously only supports `insert ... with ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). +* Support creating a table with a query: `CREATE TABLE ... CLONE AS ...`. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of the `MergeTree` family Closes [#65015](https://github.com/ClickHouse/ClickHouse/issues/65015). [#69091](https://github.com/ClickHouse/ClickHouse/pull/69091) ([tuanpach](https://github.com/tuanpach)). +* Add a new system table, `system.query_metric_log` which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). +* A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support the `--copy` mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). +* Add a builin HTML page for visualizing merges which is available at the `/merges` path. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). +* Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). +* A new aggregate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. This is for spark compatibility. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). +* A new function `arrayElementOrNull`. It returns `NULL` if the array index is out of range or a Map key not found. [#69646](https://github.com/ClickHouse/ClickHouse/pull/69646) ([李扬](https://github.com/taiyang-li)). +* Allows users to specify regular expressions through new `message_regexp` and `message_regexp_negative` fields in the `config.xml` file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. [#69657](https://github.com/ClickHouse/ClickHouse/pull/69657) ([Peter Nguyen](https://github.com/petern48)). +* Added `RIPEMD160` function, which computes the RIPEMD-160 cryptographic hash of a string. Example: `SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))` returns `37F332F68DB77BD9D7EDD4969571AD671CF9DD3B`. [#70087](https://github.com/ClickHouse/ClickHouse/pull/70087) ([Dergousov Maxim](https://github.com/m7kss1)). +* Support reading `Iceberg` tables on `HDFS`. [#70268](https://github.com/ClickHouse/ClickHouse/pull/70268) ([flynn](https://github.com/ucasfl)). +* Support for CTE in the form of `WITH ... INSERT`, as previously we only supported `INSERT ... WITH ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). +* MongoDB integration: support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expressions unsupported by MongoDB. Note that the new inegration is disabled by default, to use it, please set `` to `false` in server config. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). +* A new function `getSettingOrDefault` added to return the default value and avoid exception if a custom setting is not found in the current profile. [#69917](https://github.com/ClickHouse/ClickHouse/pull/69917) ([Shankar](https://github.com/shiyer7474)). #### Experimental feature -* Refreshable materialized views are not experimental anymore. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). -* Support Dynamic type in most functions by executing them on internal types inside Dynamic. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). -* Allow to read/write JSON type as binary string in RowBinary format under settings `input_format_binary_read_json_as_string/output_format_binary_write_json_as_string`. [#70288](https://github.com/ClickHouse/ClickHouse/pull/70288) ([Pavel Kruglov](https://github.com/Avogar)). -* Allow to serialize/deserialize JSON column as single String column in Native format. For output use setting `output_format_native_write_json_as_string`. For input, use serialization version `1` before the column data. [#70312](https://github.com/ClickHouse/ClickHouse/pull/70312) ([Pavel Kruglov](https://github.com/Avogar)). -* Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refreshable materialized views are production ready. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). +* Parallel replicas are moved from experimental to beta. Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)), ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support for the `Dynamic` type in most functions by executing them on internal types inside `Dynamic`. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow to read/write the `JSON` type as a binary string in `RowBinary` format under settings `input_format_binary_read_json_as_string/output_format_binary_write_json_as_string`. [#70288](https://github.com/ClickHouse/ClickHouse/pull/70288) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow to serialize/deserialize `JSON` column as single String column in the Native format. For output use setting `output_format_native_write_json_as_string`. For input, use serialization version `1` before the column data. [#70312](https://github.com/ClickHouse/ClickHouse/pull/70312) ([Pavel Kruglov](https://github.com/Avogar)). * Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the `merge_selector_use_blurry_base` MergeTree-level setting. [#70645](https://github.com/ClickHouse/ClickHouse/pull/70645) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Implement generic ser/de between Avro's `Union` and ClickHouse's `Variant` types. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). #### Performance Improvement +* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov), [Julia Kartseva](https://github.com/jkartseva)). Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). +* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). +* Improved performance of parsing formats with high number of missed values (e.g. `JSONEachRow`). [#69875](https://github.com/ClickHouse/ClickHouse/pull/69875) ([Anton Popov](https://github.com/CurtizJ)). +* Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. [#69862](https://github.com/ClickHouse/ClickHouse/pull/69862) ([LiuNeng](https://github.com/liuneng1994)). * Support minmax index for `pointInPolygon`. [#62085](https://github.com/ClickHouse/ClickHouse/pull/62085) ([JackyWoo](https://github.com/JackyWoo)). -* Add support for parquet bloom filters. [#62966](https://github.com/ClickHouse/ClickHouse/pull/62966) ([Arthur Passos](https://github.com/arthurpassos)). +* Use bloom filters when reading Parquet files. [#62966](https://github.com/ClickHouse/ClickHouse/pull/62966) ([Arthur Passos](https://github.com/arthurpassos)). * Lock-free parts rename to avoid INSERT affect SELECT (due to parts lock) (under normal circumstances with `fsync_part_directory`, QPS of SELECT with INSERT in parallel, increased 2x, under heavy load the effect is even bigger). Note, this only includes `ReplicatedMergeTree` for now. [#64955](https://github.com/ClickHouse/ClickHouse/pull/64955) ([Azat Khuzhin](https://github.com/azat)). * Respect `ttl_only_drop_parts` on `materialize ttl`; only read necessary columns to recalculate TTL and drop parts by replacing them with an empty one. [#65488](https://github.com/ClickHouse/ClickHouse/pull/65488) ([Andrey Zvonov](https://github.com/zvonand)). -* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Optimized thread creation in the ThreadPool to minimize lock contention. Thread creation is now performed outside of the critical section to avoid delays in job scheduling and thread management under high load conditions. This leads to a much more responsive ClickHouse under heavy concurrent load. [#68694](https://github.com/ClickHouse/ClickHouse/pull/68694) ([filimonov](https://github.com/filimonov)). -* Enable reading LowCardinality string columns from ORC. [#69481](https://github.com/ClickHouse/ClickHouse/pull/69481) ([李扬](https://github.com/taiyang-li)). -* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). -* Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. [#69862](https://github.com/ClickHouse/ClickHouse/pull/69862) ([LiuNeng](https://github.com/liuneng1994)). -* Improved performance of parsing formats with high number of missed values (e.g. `JSONEachRow`). [#69875](https://github.com/ClickHouse/ClickHouse/pull/69875) ([Anton Popov](https://github.com/CurtizJ)). +* Enable reading `LowCardinality` string columns from `ORC`. [#69481](https://github.com/ClickHouse/ClickHouse/pull/69481) ([李扬](https://github.com/taiyang-li)). * Use `LowCardinality` for `ProfileEvents` in system logs such as `part_log`, `query_views_log`, `filesystem_cache_log`. [#70152](https://github.com/ClickHouse/ClickHouse/pull/70152) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Improve performance of FromUnixTimestamp/ToUnixTimestamp functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). +* Improve performance of `fromUnixTimestamp`/`toUnixTimestamp` functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). +* Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. This was leading to a poorer performance when a single filesystem (e.g., tmpfs) didn't support the `preadv2` syscall while others do. [#70299](https://github.com/ClickHouse/ClickHouse/pull/70299) ([Antonio Andelic](https://github.com/antonio2368)). +* `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). #### Improvement -* Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). -* Allow empty needle in functions replaceRegexp*. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). -* Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). -* `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). -* Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). -* Symbolic links for tables in the `data/database_name/` directory are created for the actual paths to the table's data, depending on the storage policy, instead of the `store/...` directory on the default disk. [#61777](https://github.com/ClickHouse/ClickHouse/pull/61777) ([Kirill](https://github.com/kirillgarbar)). -* While parsing an Enum field from JSON, a string containing an integer will be interpreted as the corresponding Enum element. This closes [#65119](https://github.com/ClickHouse/ClickHouse/issues/65119). [#66801](https://github.com/ClickHouse/ClickHouse/pull/66801) ([scanhex12](https://github.com/scanhex12)). -* Allow `TRIM` -ing `LEADING` or `TRAILING` empty string as a no-op. Closes [#67792](https://github.com/ClickHouse/ClickHouse/issues/67792). [#68455](https://github.com/ClickHouse/ClickHouse/pull/68455) ([Peter Nguyen](https://github.com/petern48)). -* Support creating a table with a query: `CREATE TABLE ... CLONE AS ...`. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of the `MergeTree` family Closes [#65015](https://github.com/ClickHouse/ClickHouse/issues/65015). [#69091](https://github.com/ClickHouse/ClickHouse/pull/69091) ([tuanpach](https://github.com/tuanpach)). -* Improve compatibility of cast(timestamp as string) with spark. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). -* Always use the new analyzer to calculate constant expressions when `enable_analyzer` is set to `true`. Support calculation of `executable()` table function arguments without using `SELECT` query for constant expression. [#69292](https://github.com/ClickHouse/ClickHouse/pull/69292) ([Dmitry Novik](https://github.com/novikd)). -* Add `enable_secure_identifiers` to disallow insecure identifiers. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). -* Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior of the show create query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}`, or it can cause ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). -* Improve restoring of access entities' dependencies [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). -* Implement generic SerDe between Avro Union and ClickHouse Variant type. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). -* CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses (MergeTree tables). [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). -* Added user-level settings `min_free_disk_bytes_to_throw_insert` and `min_free_disk_ratio_to_throw_insert` to prevent insertions on disks that are almost full. [#69755](https://github.com/ClickHouse/ClickHouse/pull/69755) ([Marco Vilas Boas](https://github.com/marco-vb)). -* If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add new column readonly_duration to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). -* Change the join to sort settings type to unsigned int. [#69886](https://github.com/ClickHouse/ClickHouse/pull/69886) ([kevinyhzou](https://github.com/KevinyhZou)). -* Support 64-bit XID in Keeper. It can be enabled with `use_xid_64` config. [#69908](https://github.com/ClickHouse/ClickHouse/pull/69908) ([Antonio Andelic](https://github.com/antonio2368)). -* New function getSettingOrDefault() added to return the default value and avoid exception if a custom setting is not found in the current profile. [#69917](https://github.com/ClickHouse/ClickHouse/pull/69917) ([Shankar](https://github.com/shiyer7474)). -* Enhance OpenTelemetry span logging to include query settings. [#70011](https://github.com/ClickHouse/ClickHouse/pull/70011) ([sharathks118](https://github.com/sharathks118)). -* Add info to higher-order array functions if lambda result type is unexpected. [#70093](https://github.com/ClickHouse/ClickHouse/pull/70093) ([ttanay](https://github.com/ttanay)). -* Keeper improvement: less blocking during cluster changes. [#70275](https://github.com/ClickHouse/ClickHouse/pull/70275) ([Antonio Andelic](https://github.com/antonio2368)). -* Embedded documentation for settings will be strictly more detailed and complete than the documentation on the website. This is the first step before making the website documentation always auto-generated from the source code. This has long-standing implications: - it will be guaranteed to have every setting; - there is no chance of having default values obsolete; - we can generate this documentation for each ClickHouse version; - the documentation can be displayed by the server itself even without Internet access. Generate the docs on the website from the source code. [#70289](https://github.com/ClickHouse/ClickHouse/pull/70289) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add `WITH IMPLICIT` and `FINAL` keywords to the `SHOW GRANTS` command. Fix a minor bug with implicit grants: [#70094](https://github.com/ClickHouse/ClickHouse/issues/70094). [#70293](https://github.com/ClickHouse/ClickHouse/pull/70293) ([pufit](https://github.com/pufit)). -* Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. [#70299](https://github.com/ClickHouse/ClickHouse/pull/70299) ([Antonio Andelic](https://github.com/antonio2368)). -* Respect `compatibility` for MergeTree settings. The `compatibility` value is taken from the `default` profile on server startup, and default MergeTree settings are changed accordingly. Further changes of the `compatibility` setting do not affect MergeTree settings. [#70322](https://github.com/ClickHouse/ClickHouse/pull/70322) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Clickhouse-client realtime metrics follow-up: restore cursor when ctrl-c cancels query; immediately stop intercepting keystrokes when the query is canceled; display the metrics table if `--progress-table` is on, and toggling is disabled. [#70423](https://github.com/ClickHouse/ClickHouse/pull/70423) ([Julia Kartseva](https://github.com/jkartseva)). +* `CREATE TABLE AS` will copy `PRIMARY KEY`, `ORDER BY`, and similar clauses (of `MergeTree` tables). [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). +* Support 64-bit XID in Keeper. It can be enabled with the `use_xid_64` configuration value. [#69908](https://github.com/ClickHouse/ClickHouse/pull/69908) ([Antonio Andelic](https://github.com/antonio2368)). * Command-line arguments for Bool settings are set to true when no value is provided for the argument (e.g. `clickhouse-client --optimize_aggregation_in_order --query "SELECT 1"`). [#70459](https://github.com/ClickHouse/ClickHouse/pull/70459) ([davidtsuk](https://github.com/davidtsuk)). +* Added user-level settings `min_free_disk_bytes_to_throw_insert` and `min_free_disk_ratio_to_throw_insert` to prevent insertions on disks that are almost full. [#69755](https://github.com/ClickHouse/ClickHouse/pull/69755) ([Marco Vilas Boas](https://github.com/marco-vb)). +* Embedded documentation for settings will be strictly more detailed and complete than the documentation on the website. This is the first step before making the website documentation always auto-generated from the source code. This has long-standing implications: - it will be guaranteed to have every setting; - there is no chance of having default values obsolete; - we can generate this documentation for each ClickHouse version; - the documentation can be displayed by the server itself even without Internet access. Generate the docs on the website from the source code. [#70289](https://github.com/ClickHouse/ClickHouse/pull/70289) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow empty needle in the function `replace`, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). +* Allow empty needle in functions `replaceRegexp*`. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). +* Symbolic links for tables in the `data/database_name/` directory are created for the actual paths to the table's data, depending on the storage policy, instead of the `store/...` directory on the default disk. [#61777](https://github.com/ClickHouse/ClickHouse/pull/61777) ([Kirill](https://github.com/kirillgarbar)). +* While parsing an `Enum` field from `JSON`, a string containing an integer will be interpreted as the corresponding `Enum` element. This closes [#65119](https://github.com/ClickHouse/ClickHouse/issues/65119). [#66801](https://github.com/ClickHouse/ClickHouse/pull/66801) ([scanhex12](https://github.com/scanhex12)). +* Allow `TRIM` -ing `LEADING` or `TRAILING` empty string as a no-op. Closes [#67792](https://github.com/ClickHouse/ClickHouse/issues/67792). [#68455](https://github.com/ClickHouse/ClickHouse/pull/68455) ([Peter Nguyen](https://github.com/petern48)). +* Improve compatibility of `cast(timestamp as String)` with Spark. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). +* Always use the new analyzer to calculate constant expressions when `enable_analyzer` is set to `true`. Support calculation of `executable` table function arguments without using `SELECT` query for constant expressions. [#69292](https://github.com/ClickHouse/ClickHouse/pull/69292) ([Dmitry Novik](https://github.com/novikd)). +* Add a setting `enable_secure_identifiers` to disallow identifiers with special characters. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). +* Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior in the `SHOW CREATE TABLE` query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}` and when it could lead to ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). +* Improve restoring of access entities' dependencies [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). +* If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add new column `readonly_duration` to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Change the type of `join_output_by_rowlist_perkey_rows_threshold` setting type to unsigned integer. [#69886](https://github.com/ClickHouse/ClickHouse/pull/69886) ([kevinyhzou](https://github.com/KevinyhZou)). +* Enhance OpenTelemetry span logging to include query settings. [#70011](https://github.com/ClickHouse/ClickHouse/pull/70011) ([sharathks118](https://github.com/sharathks118)). +* Add diagnostic info about higher-order array functions if lambda result type is unexpected. [#70093](https://github.com/ClickHouse/ClickHouse/pull/70093) ([ttanay](https://github.com/ttanay)). +* Keeper improvement: less locking during cluster changes. [#70275](https://github.com/ClickHouse/ClickHouse/pull/70275) ([Antonio Andelic](https://github.com/antonio2368)). +* Add `WITH IMPLICIT` and `FINAL` keywords to the `SHOW GRANTS` command. Fix a minor bug with implicit grants: [#70094](https://github.com/ClickHouse/ClickHouse/issues/70094). [#70293](https://github.com/ClickHouse/ClickHouse/pull/70293) ([pufit](https://github.com/pufit)). +* Respect `compatibility` for MergeTree settings. The `compatibility` value is taken from the `default` profile on server startup, and default MergeTree settings are changed accordingly. Further changes of the `compatibility` setting do not affect MergeTree settings. [#70322](https://github.com/ClickHouse/ClickHouse/pull/70322) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Avoid spamming the logs with large HTTP response bodies in case of errors during inter-server communication. [#70487](https://github.com/ClickHouse/ClickHouse/pull/70487) ([Vladimir Cherkasov](https://github.com/vdimir)). * Added a new setting `max_parts_to_move` to control the maximum number of parts that can be moved at once. [#70520](https://github.com/ClickHouse/ClickHouse/pull/70520) ([Vladimir Cherkasov](https://github.com/vdimir)). * Limit the frequency of certain log messages. [#70601](https://github.com/ClickHouse/ClickHouse/pull/70601) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Don't do validation when synchronizing user_directories from keeper. [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). +* Don't do validation when synchronizing ACL from Keeper. It's validating during creation. It shouldn't matter that much, but there are installations with tens of thousands or even more user created, and the unnecessary hash validation can take a long time to finish during server startup (it synchronizes everything from keeper). [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). * `CHECK TABLE` with `PART` qualifier was incorrectly formatted in the client. [#70660](https://github.com/ClickHouse/ClickHouse/pull/70660) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Support write column index and offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). -* Support parse `DateTime64` for microsecond and timezone in joda syntax. [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support writing the column index and the offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). +* Support parsing `DateTime64` for microsecond and timezone in joda syntax ("joda" is a popular Java library for date and time, and the "joda syntax" is that library's style). [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). * Changed an approach to figure out if a cloud storage supports [batch delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) or not. [#70786](https://github.com/ClickHouse/ClickHouse/pull/70786) ([Vitaly Baranov](https://github.com/vitlibar)). -* Support for Parquet page V2 on native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). -* Add an HTML page for visualizing merges. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* A check if table has both `storage_policy` and `disk` set after alter query is added. A check if a new storage policy is compatible with an old one when using `disk` setting is added. [#70839](https://github.com/ClickHouse/ClickHouse/pull/70839) ([Kirill](https://github.com/kirillgarbar)). -* Add system.s3_queue_settings and system.azure_queue_settings. [#70841](https://github.com/ClickHouse/ClickHouse/pull/70841) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support for Parquet page V2 in the native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). +* A check if table has both `storage_policy` and `disk` set. A check if a new storage policy is compatible with an old one when using `disk` setting is added. [#70839](https://github.com/ClickHouse/ClickHouse/pull/70839) ([Kirill](https://github.com/kirillgarbar)). +* Add `system.s3_queue_settings` and `system.azure_queue_settings`. [#70841](https://github.com/ClickHouse/ClickHouse/pull/70841) ([Kseniia Sumarokova](https://github.com/kssenii)). * Functions `base58Encode` and `base58Decode` now accept arguments of type `FixedString`. Example: `SELECT base58Encode(toFixedString('plaintext', 9));`. [#70846](https://github.com/ClickHouse/ClickHouse/pull/70846) ([Faizan Patel](https://github.com/faizan2786)). * Add the `partition` column to every entry type of the part log. Previously, it was set only for some entries. This closes [#70819](https://github.com/ClickHouse/ClickHouse/issues/70819). [#70848](https://github.com/ClickHouse/ClickHouse/pull/70848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add merge start and mutate start events into `system.part_log` which helps with merges analysis and visualization. [#70850](https://github.com/ClickHouse/ClickHouse/pull/70850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). +* Add `MergeStart` and `MutateStart` events into `system.part_log` which helps with merges analysis and visualization. [#70850](https://github.com/ClickHouse/ClickHouse/pull/70850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add a profile event about the number of merged source parts. It allows the monitoring of the fanout of the merge tree in production. [#70908](https://github.com/ClickHouse/ClickHouse/pull/70908) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). -* Background downloads to filesystem cache was enabled back. [#70929](https://github.com/ClickHouse/ClickHouse/pull/70929) ([Nikita Taranov](https://github.com/nickitat)). +* Background downloads to the filesystem cache were enabled back. [#70929](https://github.com/ClickHouse/ClickHouse/pull/70929) ([Nikita Taranov](https://github.com/nickitat)). * Add a new merge selector algorithm, named `Trivial`, for professional usage only. It is worse than the `Simple` merge selector. [#70969](https://github.com/ClickHouse/ClickHouse/pull/70969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Support CREATE OR REPLACE VIEW atomically. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([tuanpach](https://github.com/tuanpach)) +* Support for atomic `CREATE OR REPLACE VIEW`. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([tuanpach](https://github.com/tuanpach)) +* Added `strict_once` mode to aggregate function `windowFunnel` to avoid counting one event several times in case it matches multiple conditions, close [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835). [#69738](https://github.com/ClickHouse/ClickHouse/pull/69738) ([Vladimir Cherkasov](https://github.com/vdimir)). #### Bug Fix (user-visible misbehavior in an official stable release) * Apply configuration updates in global context object. It fixes issues like [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308). [#62944](https://github.com/ClickHouse/ClickHouse/pull/62944) ([Amos Bird](https://github.com/amosbird)). * Fix `ReadSettings` not using user set values, because defaults were only used. [#65625](https://github.com/ClickHouse/ClickHouse/pull/65625) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix type mismatch issue in sumMapFiltered when using signed arguments. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). +* Fix type mismatch issue in `sumMapFiltered` when using signed arguments. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). * Fix toHour-like conversion functions' monotonicity when optional time zone argument is passed. [#60264](https://github.com/ClickHouse/ClickHouse/pull/60264) ([Amos Bird](https://github.com/amosbird)). -* Relax `supportsPrewhere` check for StorageMerge. This fixes [#61064](https://github.com/ClickHouse/ClickHouse/issues/61064). It was hardened unnecessarily in [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#61091](https://github.com/ClickHouse/ClickHouse/pull/61091) ([Amos Bird](https://github.com/amosbird)). +* Relax `supportsPrewhere` check for `Merge` tables. This fixes [#61064](https://github.com/ClickHouse/ClickHouse/issues/61064). It was hardened unnecessarily in [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#61091](https://github.com/ClickHouse/ClickHouse/pull/61091) ([Amos Bird](https://github.com/amosbird)). * Fix `use_concurrency_control` setting handling for proper `concurrent_threads_soft_limit_num` limit enforcing. This enables concurrency control by default because previously it was broken. [#61473](https://github.com/ClickHouse/ClickHouse/pull/61473) ([Sergei Trifonov](https://github.com/serxa)). -* Fix incorrect JOIN ON section optimization in case of `IS NULL` check under any other function (like `NOT`) that may lead to wrong results. Closes [#67915](https://github.com/ClickHouse/ClickHouse/issues/67915). [#68049](https://github.com/ClickHouse/ClickHouse/pull/68049) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix incorrect `JOIN ON` section optimization in case of `IS NULL` check under any other function (like `NOT`) that may lead to wrong results. Closes [#67915](https://github.com/ClickHouse/ClickHouse/issues/67915). [#68049](https://github.com/ClickHouse/ClickHouse/pull/68049) ([Vladimir Cherkasov](https://github.com/vdimir)). * Prevent `ALTER` queries that would make the `CREATE` query of tables invalid. [#68574](https://github.com/ClickHouse/ClickHouse/pull/68574) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Fix inconsistent AST formatting for `negate` (`-`) and `NOT` functions with tuples and arrays. [#68600](https://github.com/ClickHouse/ClickHouse/pull/68600) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix insertion of incomplete type into Dynamic during deserialization. It could lead to `Parameter out of bound` errors. [#69291](https://github.com/ClickHouse/ClickHouse/pull/69291) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix inf loop after `restore replica` in the replicated merge tree with zero copy. [#69293](https://github.com/ClickHouse/ClickHouse/pull/69293) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix insertion of incomplete type into `Dynamic` during deserialization. It could lead to `Parameter out of bound` errors. [#69291](https://github.com/ClickHouse/ClickHouse/pull/69291) ([Pavel Kruglov](https://github.com/Avogar)). +* Zero-copy replication, which is experimental and should not be used in production: fix inf loop after `restore replica` in the replicated merge tree with zero copy. [#69293](https://github.com/CljmnickHouse/ClickHouse/pull/69293) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Return back default value of `processing_threads_num` as number of cpu cores in storage `S3Queue`. [#69384](https://github.com/ClickHouse/ClickHouse/pull/69384) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns ( fixes [#41971](https://github.com/ClickHouse/ClickHouse/issues/41971) ). [#69556](https://github.com/ClickHouse/ClickHouse/pull/69556) ([Eliot Hautefeuille](https://github.com/hileef)). +* Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns (fixes [#41971](https://github.com/ClickHouse/ClickHouse/issues/41971)). [#69556](https://github.com/ClickHouse/ClickHouse/pull/69556) ([Eliot Hautefeuille](https://github.com/hileef)). * Fix crash during insertion into FixedString column in PostgreSQL engine. [#69584](https://github.com/ClickHouse/ClickHouse/pull/69584) ([Pavel Kruglov](https://github.com/Avogar)). * Fix crash when executing `create view t as (with recursive 42 as ttt select ttt);`. [#69676](https://github.com/ClickHouse/ClickHouse/pull/69676) ([Han Fei](https://github.com/hanfei1991)). -* Added `strict_once` mode to aggregate function `windowFunnel` to avoid counting one event several times in case it matches multiple conditions, close [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835). [#69738](https://github.com/ClickHouse/ClickHouse/pull/69738) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fixed `maxMapState` throwing 'Bad get' if value type is DateTime64. [#69787](https://github.com/ClickHouse/ClickHouse/pull/69787) ([Michael Kolupaev](https://github.com/al13n321)). * Fix `getSubcolumn` with `LowCardinality` columns by overriding `useDefaultImplementationForLowCardinalityColumns` to return `true`. [#69831](https://github.com/ClickHouse/ClickHouse/pull/69831) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). -* Fix permanent blocked distributed sends if DROP of distributed table fails. [#69843](https://github.com/ClickHouse/ClickHouse/pull/69843) ([Azat Khuzhin](https://github.com/azat)). +* Fix permanent blocked distributed sends if a DROP of distributed table failed. [#69843](https://github.com/ClickHouse/ClickHouse/pull/69843) ([Azat Khuzhin](https://github.com/azat)). * Fix non-cancellable queries containing WITH FILL with NaN keys. This closes [#69261](https://github.com/ClickHouse/ClickHouse/issues/69261). [#69845](https://github.com/ClickHouse/ClickHouse/pull/69845) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix analyzer default with old compatibility value. [#69895](https://github.com/ClickHouse/ClickHouse/pull/69895) ([Raúl Marín](https://github.com/Algunenano)). * Don't check dependencies during CREATE OR REPLACE VIEW during DROP of old table. Previously CREATE OR REPLACE query failed when there are dependent tables of the recreated view. [#69907](https://github.com/ClickHouse/ClickHouse/pull/69907) ([Pavel Kruglov](https://github.com/Avogar)). -* Implement missing decimal cases for `zeroField`. Fixes [#69730](https://github.com/ClickHouse/ClickHouse/issues/69730). [#69978](https://github.com/ClickHouse/ClickHouse/pull/69978) ([Arthur Passos](https://github.com/arthurpassos)). -* Now SQL security will work with parameterized views correctly. [#69984](https://github.com/ClickHouse/ClickHouse/pull/69984) ([pufit](https://github.com/pufit)). -* Fix parsing for definers. [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). +* Something for Decimal. Fixes [#69730](https://github.com/ClickHouse/ClickHouse/issues/69730). [#69978](https://github.com/ClickHouse/ClickHouse/pull/69978) ([Arthur Passos](https://github.com/arthurpassos)). +* Now DEFINER/INVOKER will work with parameterized views. [#69984](https://github.com/ClickHouse/ClickHouse/pull/69984) ([pufit](https://github.com/pufit)). +* Fix parsing for view's definers. [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). * Fixed a bug when the timezone could change the result of the query with a `Date` or `Date32` arguments. [#70036](https://github.com/ClickHouse/ClickHouse/pull/70036) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Fixes `Block structure mismatch` for queries with nested views and `WHERE` condition. Fixes [#66209](https://github.com/ClickHouse/ClickHouse/issues/66209). [#70054](https://github.com/ClickHouse/ClickHouse/pull/70054) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Avoid reusing columns among different named tuples when evaluating `tuple` functions. This fixes [#70022](https://github.com/ClickHouse/ClickHouse/issues/70022). [#70103](https://github.com/ClickHouse/ClickHouse/pull/70103) ([Amos Bird](https://github.com/amosbird)). @@ -150,31 +146,32 @@ * Now ClickHouse will consider more errors as retriable and will not mark data parts as broken in case of such errors. [#70145](https://github.com/ClickHouse/ClickHouse/pull/70145) ([alesapin](https://github.com/alesapin)). * Use correct `max_types` parameter during Dynamic type creation for JSON subcolumn. [#70147](https://github.com/ClickHouse/ClickHouse/pull/70147) ([Pavel Kruglov](https://github.com/Avogar)). * Fix the password being displayed in `system.query_log` for users with bcrypt password authentication method. [#70148](https://github.com/ClickHouse/ClickHouse/pull/70148) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix event counter for native interface (InterfaceNativeSendBytes). [#70153](https://github.com/ClickHouse/ClickHouse/pull/70153) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix possible crash in JSON column. [#70172](https://github.com/ClickHouse/ClickHouse/pull/70172) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix event counter for the native interface (InterfaceNativeSendBytes). [#70153](https://github.com/ClickHouse/ClickHouse/pull/70153) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible crash related to JSON columns. [#70172](https://github.com/ClickHouse/ClickHouse/pull/70172) ([Pavel Kruglov](https://github.com/Avogar)). * Fix multiple issues with arrayMin and arrayMax. [#70207](https://github.com/ClickHouse/ClickHouse/pull/70207) ([Raúl Marín](https://github.com/Algunenano)). -* Respect setting allow_simdjson in JSON type parser. [#70218](https://github.com/ClickHouse/ClickHouse/pull/70218) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix server segfault on creating a materialized view with two selects and an `INTERSECT`, e.g. `CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);`. [#70264](https://github.com/ClickHouse/ClickHouse/pull/70264) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Respect setting allow_simdjson in the JSON type parser. [#70218](https://github.com/ClickHouse/ClickHouse/pull/70218) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix a null pointer dereference on creating a materialized view with two selects and an `INTERSECT`, e.g. `CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);`. [#70264](https://github.com/ClickHouse/ClickHouse/pull/70264) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Don't modify global settings with startup scripts. Previously, changing a setting in a startup script would change it globally. [#70310](https://github.com/ClickHouse/ClickHouse/pull/70310) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix ALTER of Dynamic type with reducing max_types parameter that could lead to server crash. [#70328](https://github.com/ClickHouse/ClickHouse/pull/70328) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix ALTER of `Dynamic` type with reducing max_types parameter that could lead to server crash. [#70328](https://github.com/ClickHouse/ClickHouse/pull/70328) ([Pavel Kruglov](https://github.com/Avogar)). * Fix crash when using WITH FILL incorrectly. [#70338](https://github.com/ClickHouse/ClickHouse/pull/70338) ([Raúl Marín](https://github.com/Algunenano)). * Fix possible use-after-free in `SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf`. [#70358](https://github.com/ClickHouse/ClickHouse/pull/70358) ([Azat Khuzhin](https://github.com/azat)). * Fix crash during GROUP BY JSON sub-object subcolumn. [#70374](https://github.com/ClickHouse/ClickHouse/pull/70374) ([Pavel Kruglov](https://github.com/Avogar)). * Don't prefetch parts for vertical merges if part has no rows. [#70452](https://github.com/ClickHouse/ClickHouse/pull/70452) ([Antonio Andelic](https://github.com/antonio2368)). * Fix crash in WHERE with lambda functions. [#70464](https://github.com/ClickHouse/ClickHouse/pull/70464) ([Raúl Marín](https://github.com/Algunenano)). -* Fix table creation with `CREATE ... AS table_function()` with database `Replicated` and unavailable table function source on secondary replica. [#70511](https://github.com/ClickHouse/ClickHouse/pull/70511) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix table creation with `CREATE ... AS table_function(...)` with database `Replicated` and unavailable table function source on secondary replica. [#70511](https://github.com/ClickHouse/ClickHouse/pull/70511) ([Kseniia Sumarokova](https://github.com/kssenii)). * Ignore all output on async insert with `wait_for_async_insert=1`. Closes [#62644](https://github.com/ClickHouse/ClickHouse/issues/62644). [#70530](https://github.com/ClickHouse/ClickHouse/pull/70530) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Ignore frozen_metadata.txt while traversing shadow directory from system.remote_data_paths. [#70590](https://github.com/ClickHouse/ClickHouse/pull/70590) ([Aleksei Filatov](https://github.com/aalexfvk)). * Fix creation of stateful window functions on misaligned memory. [#70631](https://github.com/ClickHouse/ClickHouse/pull/70631) ([Raúl Marín](https://github.com/Algunenano)). * Fixed rare crashes in `SELECT`-s and merges after adding a column of `Array` type with non-empty default expression. [#70695](https://github.com/ClickHouse/ClickHouse/pull/70695) ([Anton Popov](https://github.com/CurtizJ)). -* Insert into table function s3 respect query settings. [#70696](https://github.com/ClickHouse/ClickHouse/pull/70696) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix infinite recursion when infering a proto schema with skip unsupported fields enabled. [#70697](https://github.com/ClickHouse/ClickHouse/pull/70697) ([Raúl Marín](https://github.com/Algunenano)). +* Insert into table function s3 will respect query settings. [#70696](https://github.com/ClickHouse/ClickHouse/pull/70696) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix infinite recursion when inferring a protobuf schema when skipping unsupported fields is enabled. [#70697](https://github.com/ClickHouse/ClickHouse/pull/70697) ([Raúl Marín](https://github.com/Algunenano)). * Disable enable_named_columns_in_function_tuple by default. [#70833](https://github.com/ClickHouse/ClickHouse/pull/70833) ([Raúl Marín](https://github.com/Algunenano)). * Fix S3Queue table engine setting processing_threads_num not being effective in case it was deduced from the number of cpu cores on the server. [#70837](https://github.com/ClickHouse/ClickHouse/pull/70837) ([Kseniia Sumarokova](https://github.com/kssenii)). * Normalize named tuple arguments in aggregation states. This fixes [#69732](https://github.com/ClickHouse/ClickHouse/issues/69732) . [#70853](https://github.com/ClickHouse/ClickHouse/pull/70853) ([Amos Bird](https://github.com/amosbird)). * Fix a logical error due to negative zeros in the two-level hash table. This closes [#70973](https://github.com/ClickHouse/ClickHouse/issues/70973). [#70979](https://github.com/ClickHouse/ClickHouse/pull/70979) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix `limit by`, `limit with ties` for distributed and parallel replicas. [#70880](https://github.com/ClickHouse/ClickHouse/pull/70880) ([Nikita Taranov](https://github.com/nickitat)). + ### ClickHouse release 24.9, 2024-09-26 #### Backward Incompatible Change From 0604ff1871341683220d10ef3932894b8a95cfcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Oct 2024 07:53:15 +0100 Subject: [PATCH 677/816] Add changelog for 24.10 --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c9d44a49b7d..3cb0212d359 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -28,7 +28,7 @@ * Add a new system table, `system.query_metric_log` which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). * A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support the `--copy` mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). -* Add a builin HTML page for visualizing merges which is available at the `/merges` path. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a builtin HTML page for visualizing merges which is available at the `/merges` path. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). * Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). * A new aggregate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. This is for spark compatibility. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). From 6f1d690779a44354513e0dec00c1c7d4dec0ad85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Oct 2024 08:44:59 +0100 Subject: [PATCH 678/816] Changelog for 24.10 --- CHANGELOG.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3cb0212d359..ee14151ad06 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -64,6 +64,7 @@ * Improve performance of `fromUnixTimestamp`/`toUnixTimestamp` functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). * Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. This was leading to a poorer performance when a single filesystem (e.g., tmpfs) didn't support the `preadv2` syscall while others do. [#70299](https://github.com/ClickHouse/ClickHouse/pull/70299) ([Antonio Andelic](https://github.com/antonio2368)). * `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). +* Don't do validation when synchronizing ACL from Keeper. It's validating during creation. It shouldn't matter that much, but there are installations with tens of thousands or even more user created, and the unnecessary hash validation can take a long time to finish during server startup (it synchronizes everything from keeper). [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). #### Improvement * `CREATE TABLE AS` will copy `PRIMARY KEY`, `ORDER BY`, and similar clauses (of `MergeTree` tables). [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). @@ -81,8 +82,8 @@ * Add a setting `enable_secure_identifiers` to disallow identifiers with special characters. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). * Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior in the `SHOW CREATE TABLE` query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}` and when it could lead to ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). * Improve restoring of access entities' dependencies [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). -* If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add new column `readonly_duration` to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* If you run `clickhouse-client` or other CLI application, and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add new column `readonly_duration` to the `system.replicas` table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). * Change the type of `join_output_by_rowlist_perkey_rows_threshold` setting type to unsigned integer. [#69886](https://github.com/ClickHouse/ClickHouse/pull/69886) ([kevinyhzou](https://github.com/KevinyhZou)). * Enhance OpenTelemetry span logging to include query settings. [#70011](https://github.com/ClickHouse/ClickHouse/pull/70011) ([sharathks118](https://github.com/sharathks118)). * Add diagnostic info about higher-order array functions if lambda result type is unexpected. [#70093](https://github.com/ClickHouse/ClickHouse/pull/70093) ([ttanay](https://github.com/ttanay)). @@ -92,12 +93,11 @@ * Avoid spamming the logs with large HTTP response bodies in case of errors during inter-server communication. [#70487](https://github.com/ClickHouse/ClickHouse/pull/70487) ([Vladimir Cherkasov](https://github.com/vdimir)). * Added a new setting `max_parts_to_move` to control the maximum number of parts that can be moved at once. [#70520](https://github.com/ClickHouse/ClickHouse/pull/70520) ([Vladimir Cherkasov](https://github.com/vdimir)). * Limit the frequency of certain log messages. [#70601](https://github.com/ClickHouse/ClickHouse/pull/70601) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Don't do validation when synchronizing ACL from Keeper. It's validating during creation. It shouldn't matter that much, but there are installations with tens of thousands or even more user created, and the unnecessary hash validation can take a long time to finish during server startup (it synchronizes everything from keeper). [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). * `CHECK TABLE` with `PART` qualifier was incorrectly formatted in the client. [#70660](https://github.com/ClickHouse/ClickHouse/pull/70660) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support writing the column index and the offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). * Support parsing `DateTime64` for microsecond and timezone in joda syntax ("joda" is a popular Java library for date and time, and the "joda syntax" is that library's style). [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). * Changed an approach to figure out if a cloud storage supports [batch delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) or not. [#70786](https://github.com/ClickHouse/ClickHouse/pull/70786) ([Vitaly Baranov](https://github.com/vitlibar)). -* Support for Parquet page V2 in the native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). +* Support for Parquet page v2 in the native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). * A check if table has both `storage_policy` and `disk` set. A check if a new storage policy is compatible with an old one when using `disk` setting is added. [#70839](https://github.com/ClickHouse/ClickHouse/pull/70839) ([Kirill](https://github.com/kirillgarbar)). * Add `system.s3_queue_settings` and `system.azure_queue_settings`. [#70841](https://github.com/ClickHouse/ClickHouse/pull/70841) ([Kseniia Sumarokova](https://github.com/kssenii)). * Functions `base58Encode` and `base58Decode` now accept arguments of type `FixedString`. Example: `SELECT base58Encode(toFixedString('plaintext', 9));`. [#70846](https://github.com/ClickHouse/ClickHouse/pull/70846) ([Faizan Patel](https://github.com/faizan2786)). From e126092c1f4123f26caf7c7f29ef2ebded6434d3 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 31 Oct 2024 08:51:52 +0000 Subject: [PATCH 679/816] Improve system.query_metric_log to remove flakiness - Use an interval of 400ms instead of 1234ms for the 2500ms sleep to avoid having a last collection that may clash with the finish one. - Move the check for number of events to a separate check. This way we don't have to remove the first and last event, which makes the check less good the fewer events we have. - Add explicit comments of what each check does for readability. --- .../03203_system_query_metric_log.reference | 36 ++++++--- .../03203_system_query_metric_log.sh | 75 ++++++++++++------- 2 files changed, 74 insertions(+), 37 deletions(-) diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.reference b/tests/queries/0_stateless/03203_system_query_metric_log.reference index d761659fce2..940b0c4e178 100644 --- a/tests/queries/0_stateless/03203_system_query_metric_log.reference +++ b/tests/queries/0_stateless/03203_system_query_metric_log.reference @@ -1,12 +1,30 @@ -number_of_metrics_1000_ok timestamp_diff_in_metrics_1000_ok -initial_data_1000_ok -data_1000_ok -number_of_metrics_1234_ok timestamp_diff_in_metrics_1234_ok -initial_data_1234_ok -data_1234_ok -number_of_metrics_123_ok timestamp_diff_in_metrics_123_ok -initial_data_123_ok -data_123_ok +--Interval 1000: check that amount of events is correct +1 +--Interval 1000: check that the delta/diff between the events is correct +1 +--Interval 1000: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 1000: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Interval 400: check that amount of events is correct +1 +--Interval 400: check that the delta/diff between the events is correct +1 +--Interval 400: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 400: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Interval 123: check that amount of events is correct +1 +--Interval 123: check that the delta/diff between the events is correct +1 +--Interval 123: check that the Query, SelectQuery and InitialQuery values are correct for the first event +1 +--Interval 123: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct +1 +--Check that a query_metric_log_interval=0 disables the collection 0 +-Check that a query which execution time is less than query_metric_log_interval is never collected 0 +--Check that there is a final event when queries finish 3 diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.sh b/tests/queries/0_stateless/03203_system_query_metric_log.sh index 1c189c6ce41..b66e274df78 100755 --- a/tests/queries/0_stateless/03203_system_query_metric_log.sh +++ b/tests/queries/0_stateless/03203_system_query_metric_log.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) readonly query_prefix=$CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT --query-id="${query_prefix}_1000" -q "SELECT sleep(2.5) FORMAT Null" & -$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1234" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=1234 FORMAT Null" & +$CLICKHOUSE_CLIENT --query-id="${query_prefix}_400" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=400 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_123" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=123 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_0" -q "SELECT sleep(2.5) SETTINGS query_metric_log_interval=0 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_fast" -q "SELECT sleep(0.1) FORMAT Null" & @@ -20,32 +20,42 @@ function check_log() { interval=$1 + # Check that the amount of events collected is correct, leaving a 20% of margin. + $CLICKHOUSE_CLIENT -m -q """ + SELECT '--Interval $interval: check that amount of events is correct'; + SELECT + count() BETWEEN (ceil(2500 / $interval) * 0.8) AND (ceil(2500 / $interval) * 1.2) + FROM system.query_metric_log + WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' + """ + # We calculate the diff of each row with its previous row to check whether the intervals at # which data is collected is right. The first row is always skipped because the diff with the # preceding one (itself) is 0. The last row is also skipped, because it doesn't contain a full # interval. $CLICKHOUSE_CLIENT --max_threads=1 -m -q """ - WITH diff AS ( - SELECT - row_number() OVER () AS row, - count() OVER () as total_rows, - event_time_microseconds, - first_value(event_time_microseconds) OVER (ORDER BY event_time_microseconds ROWS BETWEEN 1 PRECEDING AND 0 FOLLOWING) as prev, - dateDiff('ms', prev, event_time_microseconds) AS diff - FROM system.query_metric_log - WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' - ORDER BY event_time_microseconds - OFFSET 1 - ) - SELECT if(count() BETWEEN ((ceil(2500 / $interval) - 2) * 0.8) AND ((ceil(2500 / $interval) - 2) * 1.2), 'number_of_metrics_${interval}_ok', 'number_of_metrics_${interval}_error'), - if(avg(diff) BETWEEN $interval * 0.8 AND $interval * 1.2, 'timestamp_diff_in_metrics_${interval}_ok', 'timestamp_diff_in_metrics_${interval}_error') - FROM diff WHERE row < total_rows + SELECT '--Interval $interval: check that the delta/diff between the events is correct'; + WITH diff AS ( + SELECT + row_number() OVER () AS row, + count() OVER () as total_rows, + event_time_microseconds, + first_value(event_time_microseconds) OVER (ORDER BY event_time_microseconds ROWS BETWEEN 1 PRECEDING AND 0 FOLLOWING) as prev, + dateDiff('ms', prev, event_time_microseconds) AS diff + FROM system.query_metric_log + WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' + ORDER BY event_time_microseconds + OFFSET 1 + ) + SELECT avg(diff) BETWEEN $interval * 0.8 AND $interval * 1.2 + FROM diff WHERE row < total_rows """ # Check that the first event contains information from the beginning of the query. # Notice the rest of the events won't contain these because the diff will be 0. $CLICKHOUSE_CLIENT -m -q """ - SELECT if(ProfileEvent_Query = 1 AND ProfileEvent_SelectQuery = 1 AND ProfileEvent_InitialQuery = 1, 'initial_data_${interval}_ok', 'initial_data_${interval}_error') + SELECT '--Interval $interval: check that the Query, SelectQuery and InitialQuery values are correct for the first event'; + SELECT ProfileEvent_Query = 1 AND ProfileEvent_SelectQuery = 1 AND ProfileEvent_InitialQuery = 1 FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' ORDER BY event_time_microseconds @@ -55,27 +65,36 @@ function check_log() # Also check that it contains some data that we know it's going to be there. # Notice the Sleep events can be in any of the rows, not only in the first one. $CLICKHOUSE_CLIENT -m -q """ - SELECT if(sum(ProfileEvent_SleepFunctionCalls) = 1 AND - sum(ProfileEvent_SleepFunctionMicroseconds) = 2500000 AND - sum(ProfileEvent_SleepFunctionElapsedMicroseconds) = 2500000 AND - sum(ProfileEvent_Query) = 1 AND - sum(ProfileEvent_SelectQuery) = 1 AND - sum(ProfileEvent_InitialQuery) = 1, - 'data_${interval}_ok', 'data_${interval}_error') + SELECT '--Interval $interval: check that the SleepFunctionCalls, SleepFunctionMilliseconds and ProfileEvent_SleepFunctionElapsedMicroseconds are correct'; + SELECT sum(ProfileEvent_SleepFunctionCalls) = 1 AND + sum(ProfileEvent_SleepFunctionMicroseconds) = 2500000 AND + sum(ProfileEvent_SleepFunctionElapsedMicroseconds) = 2500000 AND + sum(ProfileEvent_Query) = 1 AND + sum(ProfileEvent_SelectQuery) = 1 AND + sum(ProfileEvent_InitialQuery) = 1 FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' """ } check_log 1000 -check_log 1234 +check_log 400 check_log 123 # query_metric_log_interval=0 disables the collection altogether -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_0'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '--Check that a query_metric_log_interval=0 disables the collection'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_0' +""" # a quick query that takes less than query_metric_log_interval is never collected -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_fast'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '-Check that a query which execution time is less than query_metric_log_interval is never collected'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_fast' +""" # a query that takes more than query_metric_log_interval is collected including the final row -$CLICKHOUSE_CLIENT -m -q """SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_1000'""" +$CLICKHOUSE_CLIENT -m -q """ + SELECT '--Check that there is a final event when queries finish'; + SELECT count() FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_1000' +""" From 4aa06a8ed5cf66317fb124e01dbae94c29832b4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Oct 2024 11:21:22 +0100 Subject: [PATCH 680/816] Update Changelog --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ee14151ad06..90285582b4e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -50,8 +50,8 @@ * Implement generic ser/de between Avro's `Union` and ClickHouse's `Variant` types. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). #### Performance Improvement -* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov), [Julia Kartseva](https://github.com/jkartseva)). Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). -* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). +* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov), [Julia Kartseva](https://github.com/jkartseva)). Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). +* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). * Improved performance of parsing formats with high number of missed values (e.g. `JSONEachRow`). [#69875](https://github.com/ClickHouse/ClickHouse/pull/69875) ([Anton Popov](https://github.com/CurtizJ)). * Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. [#69862](https://github.com/ClickHouse/ClickHouse/pull/69862) ([LiuNeng](https://github.com/liuneng1994)). * Support minmax index for `pointInPolygon`. [#62085](https://github.com/ClickHouse/ClickHouse/pull/62085) ([JackyWoo](https://github.com/JackyWoo)). From 41e4076c5c0b7207327e7a9eff143a8346a936cd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Oct 2024 13:18:30 +0100 Subject: [PATCH 681/816] Fix test --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 90871b8c0ad..a1737c55c26 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -521,6 +521,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBu size_t buffer_size = prefer_bigger_buffer_size ? std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) : read_settings.remote_fs_buffer_size; + if (object_size) + buffer_size = std::min(object_size, buffer_size); auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); impl = std::make_unique( From 3184b1ef11afa500782118c9f663517ab4ebf20b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Oct 2024 12:24:03 +0000 Subject: [PATCH 682/816] Fix reading of LowCardinality dictionary in Dynamic column --- .../Serializations/ISerialization.cpp | 8 ++++++++ src/DataTypes/Serializations/ISerialization.h | 2 ++ .../SerializationLowCardinality.cpp | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 2 +- ...dynamic_low_cardinality_dict_bug.reference | 20 +++++++++++++++++++ ...03260_dynamic_low_cardinality_dict_bug.sql | 12 +++++++++++ 6 files changed, 44 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.reference create mode 100644 tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.sql diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index fdcdf9e0cda..42f1505118b 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -434,6 +434,14 @@ bool ISerialization::isDynamicSubcolumn(const DB::ISerialization::SubstreamPath return false; } +bool ISerialization::isLowCardinalityDictionarySubcolumn(const DB::ISerialization::SubstreamPath & path, size_t prefix_len) +{ + if (prefix_len == 0 || prefix_len > path.size()) + return false; + + return path[prefix_len - 1].type == SubstreamType::DictionaryKeys; +} + ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) { assert(prefix_len <= path.size()); diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 7bd58a8a981..e8056ea9665 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -463,6 +463,8 @@ public: /// Returns true if stream with specified path corresponds to dynamic subcolumn. static bool isDynamicSubcolumn(const SubstreamPath & path, size_t prefix_len); + static bool isLowCardinalityDictionarySubcolumn(const SubstreamPath & path, size_t prefix_len); + protected: template State * checkAndGetState(const StatePtr & state) const; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index baaab6ba3c3..248fe2681b0 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -54,7 +54,7 @@ void SerializationLowCardinality::enumerateStreams( .withSerializationInfo(data.serialization_info); settings.path.back().data = dict_data; - dict_inner_serialization->enumerateStreams(settings, callback, dict_data); + callback(settings.path); settings.path.back() = Substream::DictionaryIndexes; settings.path.back().data = data; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 898bf5a2933..9b93762a797 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -262,7 +262,7 @@ MergeTreeReaderWide::FileStreams::iterator MergeTreeReaderWide::addStream(const /*num_columns_in_mark=*/ 1); auto stream_settings = settings; - stream_settings.is_low_cardinality_dictionary = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; + stream_settings.is_low_cardinality_dictionary = ISerialization::isLowCardinalityDictionarySubcolumn(substream_path, substream_path.size()); auto create_stream = [&]() { diff --git a/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.reference b/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.reference new file mode 100644 index 00000000000..8ae0f8e9f14 --- /dev/null +++ b/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.reference @@ -0,0 +1,20 @@ +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 +12345678 diff --git a/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.sql b/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.sql new file mode 100644 index 00000000000..c5b981d5965 --- /dev/null +++ b/tests/queries/0_stateless/03260_dynamic_low_cardinality_dict_bug.sql @@ -0,0 +1,12 @@ +set allow_experimental_dynamic_type = 1; +set min_bytes_to_use_direct_io = 0; + +drop table if exists test; +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity=1, use_adaptive_write_buffer_for_dynamic_subcolumns=0, max_compress_block_size=8, min_compress_block_size=8, use_compact_variant_discriminators_serialization=0; + +insert into test select number, '12345678'::LowCardinality(String) from numbers(20); + +select d.`LowCardinality(String)` from test settings max_threads=1; + +drop table test; + From 1563689c034992866c2de6ede7776c41888395ac Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Oct 2024 13:31:54 +0100 Subject: [PATCH 683/816] Transfer changes from sync --- src/Core/Settings.cpp | 6 +++++ src/Core/SettingsChangesHistory.cpp | 4 +++- .../IO/CachedOnDiskReadBufferFromFile.cpp | 6 +++++ src/IO/ReadSettings.h | 2 ++ src/Interpreters/Cache/FileSegment.cpp | 9 +++++++- src/Interpreters/Context.cpp | 5 +++++ src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 22 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataPartBuilder.cpp | 18 +++++++++------ .../MergeTree/MergeTreeDataPartBuilder.h | 12 ++++++---- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../MergeTree/MergeTreePartsMover.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 17 files changed, 69 insertions(+), 33 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index cdaa305e804..6b16cc132bc 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4842,6 +4842,12 @@ Limit on size of a single batch of file segments that a read buffer can request )", 0) \ M(UInt64, filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, 1000, R"( Wait time to lock cache for space reservation in filesystem cache +)", 0) \ + M(Bool, filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage, true, R"( +Wait time to lock cache for space reservation in filesystem cache +)", 0) \ + M(Bool, filesystem_cache_enable_background_download_during_fetch, true, R"( +Wait time to lock cache for space reservation in filesystem cache )", 0) \ M(UInt64, temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds, (10 * 60 * 1000), R"( Wait time to lock cache for space reservation for temporary data in filesystem cache diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ad9499c6d86..c36add485bb 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -103,7 +103,9 @@ static std::initializer_listfront(), read_type); } + + if (file_segments && !file_segments->empty() && !file_segments->front().isCompleted()) + { + file_segments->completeAndPopFront(settings.filesystem_cache_allow_background_download); + file_segments = {}; + } } void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index ac3d7fc9faf..24392891e72 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -107,6 +107,8 @@ struct ReadSettings size_t filesystem_cache_segments_batch_size = 20; size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; bool filesystem_cache_allow_background_download = true; + bool filesystem_cache_allow_background_download_for_metadata_files_in_packed_storage = true; + bool filesystem_cache_allow_background_download_during_fetch = true; bool use_page_cache_for_disks_without_file_cache = false; bool read_from_page_cache_if_exists_otherwise_bypass_cache = false; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 7081ac81ae4..5e42bf0113a 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1003,7 +1003,14 @@ void FileSegmentsHolder::reset() ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size()); for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();) - file_segment_it = completeAndPopFrontImpl(false); + { + /// One might think it would have been more correct to do `false` here, + /// not to allow background download for file segments that we actually did not start reading. + /// But actually we would only do that, if those file segments were already read partially by some other thread/query + /// but they were not put to the download queue, because current thread was holding them in Holder. + /// So as a culprit, we need to allow to happen what would have happened if we did not exist. + file_segment_it = completeAndPopFrontImpl(true); + } file_segments.clear(); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8962be59f86..9b775b9eb61 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -193,6 +193,8 @@ namespace Setting extern const SettingsUInt64 filesystem_cache_max_download_size; extern const SettingsUInt64 filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; extern const SettingsUInt64 filesystem_cache_segments_batch_size; + extern const SettingsBool filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage; + extern const SettingsBool filesystem_cache_enable_background_download_during_fetch; extern const SettingsBool http_make_head_request; extern const SettingsUInt64 http_max_fields; extern const SettingsUInt64 http_max_field_name_size; @@ -5687,6 +5689,9 @@ ReadSettings Context::getReadSettings() const res.filesystem_cache_segments_batch_size = settings_ref[Setting::filesystem_cache_segments_batch_size]; res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings_ref[Setting::filesystem_cache_reserve_space_wait_lock_timeout_milliseconds]; + res.filesystem_cache_allow_background_download_for_metadata_files_in_packed_storage + = settings_ref[Setting::filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage]; + res.filesystem_cache_allow_background_download_during_fetch = settings_ref[Setting::filesystem_cache_enable_background_download_during_fetch]; res.filesystem_cache_max_download_size = settings_ref[Setting::filesystem_cache_max_download_size]; res.skip_download_if_exceeds_query_cache = settings_ref[Setting::skip_download_if_exceeds_query_cache]; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e13ec5a7515..1d79ae5aacb 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -908,7 +908,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( { part_storage_for_loading->commitTransaction(); - MergeTreeDataPartBuilder builder(data, part_name, volume, part_relative_path, part_dir); + MergeTreeDataPartBuilder builder(data, part_name, volume, part_relative_path, part_dir, getReadSettings()); new_data_part = builder.withPartFormatFromDisk().build(); new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 20d7528d38a..41783ffddb0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -833,7 +833,7 @@ MergeTreeDataPartBuilder IMergeTreeDataPart::getProjectionPartBuilder(const Stri { const char * projection_extension = is_temp_projection ? ".tmp_proj" : ".proj"; auto projection_storage = getDataPartStorage().getProjection(projection_name + projection_extension, !is_temp_projection); - MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage); + MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage, getReadSettings()); return builder.withPartInfo(MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION).withParentPart(this); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 74d6d60ba1b..06471bbe2ba 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -342,13 +342,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const if (global_ctx->parent_part) { auto data_part_storage = global_ctx->parent_part->getDataPartStorage().getProjection(local_tmp_part_basename, /* use parent transaction */ false); - builder.emplace(*global_ctx->data, global_ctx->future_part->name, data_part_storage); + builder.emplace(*global_ctx->data, global_ctx->future_part->name, data_part_storage, getReadSettings()); builder->withParentPart(global_ctx->parent_part); } else { auto local_single_disk_volume = std::make_shared("volume_" + global_ctx->future_part->name, global_ctx->disk, 0); - builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename)); + builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename, getReadSettings())); builder->withPartStorageType(global_ctx->future_part->part_format.storage_type); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ebb082f399..1ed70f7dd4e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1423,7 +1423,7 @@ void MergeTreeData::loadUnexpectedDataPart(UnexpectedPartLoadState & state) try { - state.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + state.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartInfo(part_info) .withPartFormatFromDisk() .build(); @@ -1438,7 +1438,7 @@ void MergeTreeData::loadUnexpectedDataPart(UnexpectedPartLoadState & state) /// Build a fake part and mark it as broken in case of filesystem error. /// If the error impacts part directory instead of single files, /// an exception will be thrown during detach and silently ignored. - state.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + state.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -1472,7 +1472,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( /// Build a fake part and mark it as broken in case of filesystem error. /// If the error impacts part directory instead of single files, /// an exception will be thrown during detach and silently ignored. - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -1493,7 +1493,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( try { - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartInfo(part_info) .withPartFormatFromDisk() .build(); @@ -3732,9 +3732,9 @@ MergeTreeDataPartFormat MergeTreeData::choosePartFormatOnDisk(size_t bytes_uncom } MergeTreeDataPartBuilder MergeTreeData::getDataPartBuilder( - const String & name, const VolumePtr & volume, const String & part_dir) const + const String & name, const VolumePtr & volume, const String & part_dir, const ReadSettings & read_settings_) const { - return MergeTreeDataPartBuilder(*this, name, volume, relative_data_path, part_dir); + return MergeTreeDataPartBuilder(*this, name, volume, relative_data_path, part_dir, read_settings_); } void MergeTreeData::changeSettings( @@ -5812,7 +5812,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons /// Load this part from the directory `temp_part_dir`. auto load_part = [&] { - MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, parent_part_dir, part_dir_name); + MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, parent_part_dir, part_dir_name, getReadSettings()); builder.withPartFormatFromDisk(); part = std::move(builder).build(); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); @@ -5827,7 +5827,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons if (!part) { /// Make a fake data part only to copy its files to /detached/. - part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, parent_part_dir, part_dir_name} + part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, parent_part_dir, part_dir_name, getReadSettings()} .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -6473,7 +6473,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const LOG_DEBUG(log, "Checking part {}", new_name); auto single_disk_volume = std::make_shared("volume_" + old_name, disk); - auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir / new_name) + auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir / new_name, getReadSettings()) .withPartFormatFromDisk() .build(); @@ -7528,7 +7528,7 @@ std::pair MergeTreeData::cloneAn std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), with_copy); - auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) + auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage, getReadSettings()) .withPartFormatFromDisk() .build(); @@ -8786,7 +8786,7 @@ std::pair MergeTreeData::createE VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); auto tmp_dir_holder = getTemporaryPartDirectoryHolder(EMPTY_PART_TMP_PREFIX + new_part_name); - auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name) + auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name, getReadSettings()) .withBytesAndRowsOnDisk(0, 0) .withPartInfo(new_part_info) .build(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7a9730e8627..8438ac412c9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -241,7 +241,7 @@ public: MergeTreeDataPartFormat choosePartFormat(size_t bytes_uncompressed, size_t rows_count) const; MergeTreeDataPartFormat choosePartFormatOnDisk(size_t bytes_uncompressed, size_t rows_count) const; - MergeTreeDataPartBuilder getDataPartBuilder(const String & name, const VolumePtr & volume, const String & part_dir) const; + MergeTreeDataPartBuilder getDataPartBuilder(const String & name, const VolumePtr & volume, const String & part_dir, const ReadSettings & read_settings_) const; /// Auxiliary object to add a set of parts into the working set in two steps: /// * First, as PreActive parts (the parts are ready, but not yet in the active set). diff --git a/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp b/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp index 37f578b0c25..6ec4bc31d90 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp @@ -14,20 +14,22 @@ namespace ErrorCodes } MergeTreeDataPartBuilder::MergeTreeDataPartBuilder( - const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_) + const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_, const ReadSettings & read_settings_) : data(data_) , name(std::move(name_)) , volume(std::move(volume_)) , root_path(std::move(root_path_)) , part_dir(std::move(part_dir_)) + , read_settings(read_settings_) { } MergeTreeDataPartBuilder::MergeTreeDataPartBuilder( - const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_) + const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_, const ReadSettings & read_settings_) : data(data_) , name(std::move(name_)) , part_storage(std::move(part_storage_)) + , read_settings(read_settings_) { } @@ -73,7 +75,8 @@ MutableDataPartStoragePtr MergeTreeDataPartBuilder::getPartStorageByType( MergeTreeDataPartStorageType storage_type_, const VolumePtr & volume_, const String & root_path_, - const String & part_dir_) + const String & part_dir_, + const ReadSettings &) /// Unused here, but used in private repo. { if (!volume_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create part storage, because volume is not specified"); @@ -112,7 +115,7 @@ MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartType(MergeTreeDataP MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartStorageType(MergeTreeDataPartStorageType storage_type_) { - part_storage = getPartStorageByType(storage_type_, volume, root_path, part_dir); + part_storage = getPartStorageByType(storage_type_, volume, root_path, part_dir, read_settings); return *this; } @@ -126,7 +129,8 @@ MergeTreeDataPartBuilder::PartStorageAndMarkType MergeTreeDataPartBuilder::getPartStorageAndMarkType( const VolumePtr & volume_, const String & root_path_, - const String & part_dir_) + const String & part_dir_, + const ReadSettings & read_settings_) { auto disk = volume_->getDisk(); auto part_relative_path = fs::path(root_path_) / part_dir_; @@ -138,7 +142,7 @@ MergeTreeDataPartBuilder::getPartStorageAndMarkType( if (MarkType::isMarkFileExtension(ext)) { - auto storage = getPartStorageByType(MergeTreeDataPartStorageType::Full, volume_, root_path_, part_dir_); + auto storage = getPartStorageByType(MergeTreeDataPartStorageType::Full, volume_, root_path_, part_dir_, read_settings_); return {std::move(storage), MarkType(ext)}; } } @@ -156,7 +160,7 @@ MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartFormatFromDisk() MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartFormatFromVolume() { assert(volume); - auto [storage, mark_type] = getPartStorageAndMarkType(volume, root_path, part_dir); + auto [storage, mark_type] = getPartStorageAndMarkType(volume, root_path, part_dir, read_settings); if (!storage || !mark_type) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartBuilder.h b/src/Storages/MergeTree/MergeTreeDataPartBuilder.h index 0f54ff0a631..bce881a1970 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartBuilder.h +++ b/src/Storages/MergeTree/MergeTreeDataPartBuilder.h @@ -21,8 +21,8 @@ using VolumePtr = std::shared_ptr; class MergeTreeDataPartBuilder { public: - MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_); - MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_); + MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_, const ReadSettings & read_settings_); + MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_, const ReadSettings & read_settings_); std::shared_ptr build(); @@ -42,7 +42,8 @@ public: static PartStorageAndMarkType getPartStorageAndMarkType( const VolumePtr & volume_, const String & root_path_, - const String & part_dir_); + const String & part_dir_, + const ReadSettings & read_settings); private: Self & withPartFormatFromVolume(); @@ -52,7 +53,8 @@ private: MergeTreeDataPartStorageType storage_type_, const VolumePtr & volume_, const String & root_path_, - const String & part_dir_); + const String & part_dir_, + const ReadSettings & read_settings); const MergeTreeData & data; const String name; @@ -64,6 +66,8 @@ private: std::optional part_type; MutableDataPartStoragePtr part_storage; const IMergeTreeDataPart * parent_part = nullptr; + + const ReadSettings read_settings; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 67fef759ed4..12dbd529f70 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -609,7 +609,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( } } - auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir, getReadSettings()) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) .build(); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 48a4a37f444..e9c9f2b4b06 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -280,7 +280,7 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, read_settings, write_settings, cancellation_hook); } - MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); + MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage, getReadSettings()); cloned_part.part = std::move(builder).withPartFormatFromDisk().build(); LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part.part->getDataPartStorage().getFullPath()); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2e7847fc99f..92e0193fff9 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2286,7 +2286,7 @@ bool MutateTask::prepare() String tmp_part_dir_name = prefix + ctx->future_part->name; ctx->temporary_directory_lock = ctx->data->getTemporaryPartDirectoryHolder(tmp_part_dir_name); - auto builder = ctx->data->getDataPartBuilder(ctx->future_part->name, single_disk_volume, tmp_part_dir_name); + auto builder = ctx->data->getDataPartBuilder(ctx->future_part->name, single_disk_volume, tmp_part_dir_name, getReadSettings()); builder.withPartFormat(ctx->future_part->part_format); builder.withPartInfo(ctx->future_part->part_info); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b5b07a129bd..e5b40c07f69 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2092,7 +2092,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const auto part_old_name = part_info->getPartNameV1(); const auto volume = std::make_shared("volume_" + part_old_name, disk); - auto part = getDataPartBuilder(entry.new_part_name, volume, fs::path(DETACHED_DIR_NAME) / part_old_name) + auto part = getDataPartBuilder(entry.new_part_name, volume, fs::path(DETACHED_DIR_NAME) / part_old_name, getReadSettings()) .withPartFormatFromDisk() .build(); From 9ea9e9422e478b84e8c750ba69e005a16d8ff30f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 31 Oct 2024 14:45:16 +0100 Subject: [PATCH 684/816] Fix bad cleanup of output format in client when an exception happens --- src/Client/ClientBase.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 73885ba522d..b6bf637ab44 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1454,8 +1454,22 @@ void ClientBase::resetOutput() /// Order is important: format, compression, file - if (output_format) - output_format->finalize(); + try + { + if (output_format) + output_format->finalize(); + } + catch (...) + { + /// We need to make sure we continue resetting output_format (will stop threads on parallel output) + /// as well as cleaning other output related setup + if (!have_error) + { + client_exception + = std::make_unique(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); + have_error = true; + } + } output_format.reset(); logs_out_stream.reset(); From 33cbc540d523888eea630f467718ac84f723f068 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Thu, 31 Oct 2024 13:49:24 +0000 Subject: [PATCH 685/816] Update kill.md - remove ON CLUSTER for KILL MUTATION ON CLUSTER is not valid for KILL MUTATION, and will result in an exception. Correcting the docs for this syntax --- docs/en/sql-reference/statements/kill.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/kill.md b/docs/en/sql-reference/statements/kill.md index 667a5b51f5c..ff6f64a97fe 100644 --- a/docs/en/sql-reference/statements/kill.md +++ b/docs/en/sql-reference/statements/kill.md @@ -83,7 +83,7 @@ The presence of long-running or incomplete mutations often indicates that a Clic - Or manually kill some of these mutations by sending a `KILL` command. ``` sql -KILL MUTATION [ON CLUSTER cluster] +KILL MUTATION WHERE [TEST] [FORMAT format] @@ -135,7 +135,6 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' -- Cancel the specific mutation: KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' ``` -:::tip If you are killing a mutation in ClickHouse Cloud or in a self-managed cluster, then be sure to use the ```ON CLUSTER [cluster-name]``` option, in order to ensure the mutation is killed on all replicas::: The query is useful when a mutation is stuck and cannot finish (e.g. if some function in the mutation query throws an exception when applied to the data contained in the table). From b7907051b3eb7b2d669d48beda5dfde130d93b12 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Oct 2024 13:52:15 +0000 Subject: [PATCH 686/816] Fix comments, update tests --- src/DataTypes/Serializations/ISerialization.cpp | 6 +++--- src/DataTypes/Serializations/ISerialization.h | 2 +- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 2 +- .../02240_get_type_serialization_streams.reference | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 42f1505118b..90ad822e6f5 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -434,12 +434,12 @@ bool ISerialization::isDynamicSubcolumn(const DB::ISerialization::SubstreamPath return false; } -bool ISerialization::isLowCardinalityDictionarySubcolumn(const DB::ISerialization::SubstreamPath & path, size_t prefix_len) +bool ISerialization::isLowCardinalityDictionarySubcolumn(const DB::ISerialization::SubstreamPath & path) { - if (prefix_len == 0 || prefix_len > path.size()) + if (path.empty()) return false; - return path[prefix_len - 1].type == SubstreamType::DictionaryKeys; + return path[path.size() - 1].type == SubstreamType::DictionaryKeys; } ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index e8056ea9665..400bdbf32d3 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -463,7 +463,7 @@ public: /// Returns true if stream with specified path corresponds to dynamic subcolumn. static bool isDynamicSubcolumn(const SubstreamPath & path, size_t prefix_len); - static bool isLowCardinalityDictionarySubcolumn(const SubstreamPath & path, size_t prefix_len); + static bool isLowCardinalityDictionarySubcolumn(const SubstreamPath & path); protected: template diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 9b93762a797..77231d8d392 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -262,7 +262,7 @@ MergeTreeReaderWide::FileStreams::iterator MergeTreeReaderWide::addStream(const /*num_columns_in_mark=*/ 1); auto stream_settings = settings; - stream_settings.is_low_cardinality_dictionary = ISerialization::isLowCardinalityDictionarySubcolumn(substream_path, substream_path.size()); + stream_settings.is_low_cardinality_dictionary = ISerialization::isLowCardinalityDictionarySubcolumn(substream_path); auto create_stream = [&]() { diff --git a/tests/queries/0_stateless/02240_get_type_serialization_streams.reference b/tests/queries/0_stateless/02240_get_type_serialization_streams.reference index 15e9bf87562..eb16198e877 100644 --- a/tests/queries/0_stateless/02240_get_type_serialization_streams.reference +++ b/tests/queries/0_stateless/02240_get_type_serialization_streams.reference @@ -1,7 +1,7 @@ ['{ArraySizes}','{ArrayElements, Regular}'] ['{ArraySizes}','{ArrayElements, TupleElement(keys), Regular}','{ArrayElements, TupleElement(values), Regular}'] ['{TupleElement(1), Regular}','{TupleElement(2), Regular}','{TupleElement(3), Regular}'] -['{DictionaryKeys, Regular}','{DictionaryIndexes}'] +['{DictionaryKeys}','{DictionaryIndexes}'] ['{NullMap}','{NullableElements, Regular}'] ['{ArraySizes}','{ArrayElements, Regular}'] ['{ArraySizes}','{ArrayElements, TupleElement(keys), Regular}','{ArrayElements, TupleElement(values), Regular}'] From 936d6b22518e7711adc4991663f6474b42805eb8 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 31 Oct 2024 14:05:33 +0000 Subject: [PATCH 687/816] Fix unescaping in named collections --- .../NamedCollectionsMetadataStorage.cpp | 2 +- tests/integration/test_named_collections/test.py | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index b8413bfadd7..8bb411f1437 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -568,7 +568,7 @@ std::vector NamedCollectionsMetadataStorage::listCollections() cons std::vector collections; collections.reserve(paths.size()); for (const auto & path : paths) - collections.push_back(std::filesystem::path(path).stem()); + collections.push_back(unescapeForFileName(std::filesystem::path(path).stem())); return collections; } diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index ed80898ebc7..bd04bb9e3c8 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -794,3 +794,17 @@ def test_keeper_storage_remove_on_cluster(cluster, ignore, expected_raise): node.query( f"DROP NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster`" ) + + +@pytest.mark.parametrize( + "instance_name", + [("node"), ("node_with_keeper")], +) +def test_name_escaping(cluster, instance_name): + node = cluster.instances[instance_name] + + node.query("DROP NAMED COLLECTION IF EXISTS test;") + node.query("CREATE NAMED COLLECTION `test_!strange/symbols!` AS key1=1, key2=2") + node.restart_clickhouse() + + node.query("DROP NAMED COLLECTION `test_!strange/symbols!`") From cdb479d10daeb0edd4bd1ff2c9e400b6cb77c07d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 31 Oct 2024 14:37:37 +0000 Subject: [PATCH 688/816] Fix debug log timestamp Increase the error margin for the test to avoid flakiness in the intervals where the number of events is smaller. --- src/Interpreters/QueryMetricLog.cpp | 6 +++--- tests/queries/0_stateless/03203_system_query_metric_log.sh | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 8a84c95a5a3..5ab3fe590e0 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -100,7 +100,7 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint start_time, U const auto query_info = process_list.getQueryInfo(query_id, false, true, false); if (!query_info) { - LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryInfo", query_id); + LOG_TRACE(logger, "Query {} is not running anymore, so we couldn't get its QueryStatusInfo", query_id); return; } @@ -156,8 +156,8 @@ std::optional QueryMetricLog::createLogMetricElement(cons { /// fmtlib supports subsecond formatting in 10.0.0. We're in 9.1.0, so we need to add the milliseconds ourselves. auto seconds = std::chrono::time_point_cast(query_info_time); - auto milliseconds = std::chrono::duration_cast(query_info_time - seconds).count(); - LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:05}. Schedule next: {}", query_id, seconds, milliseconds, schedule_next); + auto microseconds = std::chrono::duration_cast(query_info_time - seconds).count(); + LOG_DEBUG(logger, "Collecting query_metric_log for query {} with QueryStatusInfo from {:%Y.%m.%d %H:%M:%S}.{:06}. Schedule next: {}", query_id, seconds, microseconds, schedule_next); std::unique_lock lock(queries_mutex); auto query_status_it = queries.find(query_id); diff --git a/tests/queries/0_stateless/03203_system_query_metric_log.sh b/tests/queries/0_stateless/03203_system_query_metric_log.sh index b66e274df78..bf94be79d7c 100755 --- a/tests/queries/0_stateless/03203_system_query_metric_log.sh +++ b/tests/queries/0_stateless/03203_system_query_metric_log.sh @@ -24,7 +24,7 @@ function check_log() $CLICKHOUSE_CLIENT -m -q """ SELECT '--Interval $interval: check that amount of events is correct'; SELECT - count() BETWEEN (ceil(2500 / $interval) * 0.8) AND (ceil(2500 / $interval) * 1.2) + count() BETWEEN ((ceil(2500 / $interval) - 1) * 0.8) AND ((ceil(2500 / $interval) + 1) * 1.2) FROM system.query_metric_log WHERE event_date >= yesterday() AND query_id = '${query_prefix}_${interval}' """ From a57c64e6b01dde6084e40162142bf1325f59f11c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 31 Oct 2024 14:59:04 +0000 Subject: [PATCH 689/816] fix async inserts with empty blocks via native protocol --- src/Interpreters/AsynchronousInsertQueue.cpp | 7 +++++ ..._async_insert_native_empty_block.reference | 9 +++++++ .../03257_async_insert_native_empty_block.sh | 27 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/03257_async_insert_native_empty_block.reference create mode 100755 tests/queries/0_stateless/03257_async_insert_native_empty_block.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5cc97effad6..8b8a6d4e9ef 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1121,6 +1121,13 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); Block block_to_insert = *block; + if (block_to_insert.rows() == 0) + { + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows(), block_to_insert.bytes()); + entry->resetChunk(); + continue; + } + if (!isCompatibleHeader(block_to_insert, header)) convertBlockToHeader(block_to_insert, header); diff --git a/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference b/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference new file mode 100644 index 00000000000..6df2a541bff --- /dev/null +++ b/tests/queries/0_stateless/03257_async_insert_native_empty_block.reference @@ -0,0 +1,9 @@ +1 name1 +2 name2 +3 +4 +5 +Ok Preprocessed 2 +Ok Preprocessed 3 +Ok Preprocessed 0 +Ok Preprocessed 0 diff --git a/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh b/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh new file mode 100755 index 00000000000..43a5472914d --- /dev/null +++ b/tests/queries/0_stateless/03257_async_insert_native_empty_block.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query " + DROP TABLE IF EXISTS json_square_brackets; + CREATE TABLE json_square_brackets (id UInt32, name String) ENGINE = MergeTree ORDER BY tuple() +" + +MY_CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --async_insert 1 --wait_for_async_insert 1" + +echo '[{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '[{"id": 3}, {"id": 4}, {"id": 5}]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '[]' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +echo '' | $MY_CLICKHOUSE_CLIENT -q "INSERT INTO json_square_brackets FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT --query " + SYSTEM FLUSH LOGS; + SELECT * FROM json_square_brackets ORDER BY id; + SELECT status, data_kind, rows FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 'json_square_brackets' ORDER BY event_time_microseconds; + DROP TABLE json_square_brackets; +" From 2f0a8ecdcb0073f4b24a225b5a9608a31353e89e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 31 Oct 2024 16:02:38 +0100 Subject: [PATCH 690/816] Expose one more simple merge selector setting --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 +++ src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8b3c7bdf3fb..a39b8a7a40b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -70,6 +70,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool ttl_only_drop_parts; extern const MergeTreeSettingsUInt64 parts_to_throw_insert; extern const MergeTreeSettingsMergeSelectorAlgorithm merge_selector_algorithm; + extern const MergeTreeSettingsBool merge_selector_enable_heuristic_to_remove_small_parts_at_right; } namespace ErrorCodes @@ -540,6 +541,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( /// Override value from table settings simple_merge_settings.window_size = (*data_settings)[MergeTreeSetting::merge_selector_window_size]; simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once]; + simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSettings::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; + if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only]) simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds]; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 3d2c9c63598..5d7d9cb3c6b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -99,6 +99,7 @@ namespace ErrorCodes DECLARE(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ DECLARE(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", 0) \ + DECLARE(Bool, merge_selector_enable_heuristic_to_remove_small_parts_at_right, true, "Enable heuristic for selecting parts for merge which removes parts from right side of range, if their size is less than specified ratio (0.01) of sum_size. Works for Simple and StochasticSimple merge selectors", 0) \ \ /** Inserts settings. */ \ DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 53d4f2aacf722cbb2fbadabf2b7899fe1f9f6fc0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 31 Oct 2024 16:33:40 +0100 Subject: [PATCH 691/816] Followup --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a39b8a7a40b..62ad9d4a52a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -541,7 +541,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( /// Override value from table settings simple_merge_settings.window_size = (*data_settings)[MergeTreeSetting::merge_selector_window_size]; simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once]; - simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSettings::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; + simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSetting::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only]) simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds]; From 4784c3f0a3e15d908148878270ba7695cadb22c8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 31 Oct 2024 17:12:43 +0100 Subject: [PATCH 692/816] Better style for some sever-level settings --- src/Core/ServerSettings.cpp | 7 +++++++ src/Interpreters/Context.cpp | 26 +++++++++++++++----------- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index ead40061493..637c3196f33 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -192,6 +192,13 @@ namespace DB DECLARE(UInt64, parts_killer_pool_size, 128, "Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \ DECLARE(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \ + \ + DECLARE(UInt64, prefetch_threadpool_pool_size, 100, "Size of background pool for prefetches for remote object storages", 0) \ + DECLARE(UInt64, prefetch_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ + DECLARE(UInt64, load_marks_threadpool_pool_size, 50, "Size of background pool for marks loading", 0) \ + DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ + DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \ + DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fbf0cbd0eb7..4f82ed7b046 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -273,6 +273,13 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_replicated_sends_network_bandwidth_for_server; extern const ServerSettingsUInt64 tables_loader_background_pool_size; extern const ServerSettingsUInt64 tables_loader_foreground_pool_size; + extern const ServerSettingsUInt64 prefetch_threadpool_pool_size; + extern const ServerSettingsUInt64 prefetch_threadpool_queue_size; + extern const ServerSettingsUInt64 load_marks_threadpool_pool_size; + extern const ServerSettingsUInt64 load_marks_threadpool_queue_size; + extern const ServerSettingsUInt64 threadpool_writer_pool_size; + extern const ServerSettingsUInt64 threadpool_writer_queue_size; + } namespace ErrorCodes @@ -3215,9 +3222,8 @@ void Context::clearMarkCache() const ThreadPool & Context::getLoadMarksThreadpool() const { callOnce(shared->load_marks_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); - auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::load_marks_threadpool_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::load_marks_threadpool_queue_size]; shared->load_marks_threadpool = std::make_unique( CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, CurrentMetrics::MarksLoaderThreadsScheduled, pool_size, pool_size, queue_size); }); @@ -3410,9 +3416,9 @@ AsynchronousMetrics * Context::getAsynchronousMetrics() const ThreadPool & Context::getPrefetchThreadpool() const { callOnce(shared->prefetch_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::prefetch_threadpool_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::prefetch_threadpool_queue_size]; + shared->prefetch_threadpool = std::make_unique( CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, CurrentMetrics::IOPrefetchThreadsScheduled, pool_size, pool_size, queue_size); }); @@ -3422,8 +3428,7 @@ ThreadPool & Context::getPrefetchThreadpool() const size_t Context::getPrefetchThreadpoolSize() const { - const auto & config = getConfigRef(); - return config.getUInt(".prefetch_threadpool_pool_size", 100); + return shared->server_settings[ServerSetting::prefetch_threadpool_pool_size]; } ThreadPool & Context::getBuildVectorSimilarityIndexThreadPool() const @@ -5696,9 +5701,8 @@ IOUringReader & Context::getIOUringReader() const ThreadPool & Context::getThreadPoolWriter() const { callOnce(shared->threadpool_writer_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); - auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + auto pool_size = shared->server_settings[ServerSetting::threadpool_writer_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::threadpool_writer_queue_size]; shared->threadpool_writer = std::make_unique( CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); From 542dac1815858e55147a5db80e58690bb8b72df2 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 28 Oct 2024 10:31:50 +0000 Subject: [PATCH 693/816] Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string --- src/DataTypes/DataTypeObject.cpp | 10 +++++ src/DataTypes/DataTypeObject.h | 3 ++ .../Serializations/SerializationObject.cpp | 11 +---- .../Serializations/SerializationObject.h | 3 -- .../SerializationObjectDynamicPath.cpp | 8 ++-- .../Serializations/SerializationSubObject.cpp | 8 ++-- src/Functions/FunctionsConversion.cpp | 42 ++++++++++++++----- ...61_tuple_map_object_to_json_cast.reference | 23 ++++++++++ .../03261_tuple_map_object_to_json_cast.sql | 14 +++++++ 9 files changed, 91 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference create mode 100644 tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 18bfed9c5c3..d744e851ea9 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -1,6 +1,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -522,6 +525,13 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: return std::make_shared(schema_format, std::move(typed_paths), std::move(paths_to_skip), std::move(path_regexps_to_skip), max_dynamic_paths, max_dynamic_types); } +const DataTypePtr & DataTypeObject::getTypeOfSharedData() +{ + /// Array(Tuple(String, String)) + static const DataTypePtr type = std::make_shared(std::make_shared(DataTypes{std::make_shared(), std::make_shared()}, Names{"paths", "values"})); + return type; +} + static DataTypePtr createJSON(const ASTPtr & arguments) { auto context = CurrentThread::getQueryContext(); diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 7eb2e7729de..32ed6a7ee86 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -63,6 +63,9 @@ public: size_t getMaxDynamicTypes() const { return max_dynamic_types; } size_t getMaxDynamicPaths() const { return max_dynamic_paths; } + /// Shared data has type Array(Tuple(String, String)). + static const DataTypePtr & getTypeOfSharedData(); + private: SchemaFormat schema_format; /// Set of paths with types that were specified in type declaration. diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 0fbf8c54a22..3e1badb25ca 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -25,7 +25,7 @@ SerializationObject::SerializationObject( : typed_path_serializations(std::move(typed_path_serializations_)) , paths_to_skip(paths_to_skip_) , dynamic_serialization(std::make_shared()) - , shared_data_serialization(getTypeOfSharedData()->getDefaultSerialization()) + , shared_data_serialization(DataTypeObject::getTypeOfSharedData()->getDefaultSerialization()) { /// We will need sorted order of typed paths to serialize them in order for consistency. sorted_typed_paths.reserve(typed_path_serializations.size()); @@ -38,13 +38,6 @@ SerializationObject::SerializationObject( path_regexps_to_skip.emplace_back(regexp_str); } -const DataTypePtr & SerializationObject::getTypeOfSharedData() -{ - /// Array(Tuple(String, String)) - static const DataTypePtr type = std::make_shared(std::make_shared(DataTypes{std::make_shared(), std::make_shared()}, Names{"paths", "values"})); - return type; -} - bool SerializationObject::shouldSkipPath(const String & path) const { if (paths_to_skip.contains(path)) @@ -168,7 +161,7 @@ void SerializationObject::enumerateStreams(EnumerateStreamsSettings & settings, settings.path.push_back(Substream::ObjectSharedData); auto shared_data_substream_data = SubstreamData(shared_data_serialization) - .withType(getTypeOfSharedData()) + .withType(DataTypeObject::getTypeOfSharedData()) .withColumn(column_object ? column_object->getSharedDataPtr() : nullptr) .withSerializationInfo(data.serialization_info) .withDeserializeState(deserialize_state ? deserialize_state->shared_data_state : nullptr); diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 420293ba428..8bc72312da1 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -111,9 +111,6 @@ private: DeserializeBinaryBulkSettings & settings, SubstreamsDeserializeStatesCache * cache); - /// Shared data has type Array(Tuple(String, String)). - static const DataTypePtr & getTypeOfSharedData(); - struct TypedPathSubcolumnCreator : public ISubcolumnCreator { String path; diff --git a/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp index 5323079c54b..c1f26eca792 100644 --- a/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp +++ b/src/DataTypes/Serializations/SerializationObjectDynamicPath.cpp @@ -18,7 +18,7 @@ SerializationObjectDynamicPath::SerializationObjectDynamicPath( , path(path_) , path_subcolumn(path_subcolumn_) , dynamic_serialization(std::make_shared()) - , shared_data_serialization(SerializationObject::getTypeOfSharedData()->getDefaultSerialization()) + , shared_data_serialization(DataTypeObject::getTypeOfSharedData()->getDefaultSerialization()) , max_dynamic_types(max_dynamic_types_) { } @@ -67,8 +67,8 @@ void SerializationObjectDynamicPath::enumerateStreams( { settings.path.push_back(Substream::ObjectSharedData); auto shared_data_substream_data = SubstreamData(shared_data_serialization) - .withType(data.type ? SerializationObject::getTypeOfSharedData() : nullptr) - .withColumn(data.column ? SerializationObject::getTypeOfSharedData()->createColumn() : nullptr) + .withType(data.type ? DataTypeObject::getTypeOfSharedData() : nullptr) + .withColumn(data.column ? DataTypeObject::getTypeOfSharedData()->createColumn() : nullptr) .withSerializationInfo(data.serialization_info) .withDeserializeState(deserialize_state->nested_state); settings.path.back().data = shared_data_substream_data; @@ -164,7 +164,7 @@ void SerializationObjectDynamicPath::deserializeBinaryBulkWithMultipleStreams( settings.path.push_back(Substream::ObjectSharedData); /// Initialize shared_data column if needed. if (result_column->empty()) - dynamic_path_state->shared_data = SerializationObject::getTypeOfSharedData()->createColumn(); + dynamic_path_state->shared_data = DataTypeObject::getTypeOfSharedData()->createColumn(); size_t prev_size = result_column->size(); shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(dynamic_path_state->shared_data, limit, settings, dynamic_path_state->nested_state, cache); /// If we need to read a subcolumn from Dynamic column, create an empty Dynamic column, fill it and extract subcolumn. diff --git a/src/DataTypes/Serializations/SerializationSubObject.cpp b/src/DataTypes/Serializations/SerializationSubObject.cpp index 9084d46f9b2..ff61cb55572 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.cpp +++ b/src/DataTypes/Serializations/SerializationSubObject.cpp @@ -17,7 +17,7 @@ SerializationSubObject::SerializationSubObject( : path_prefix(path_prefix_) , typed_paths_serializations(typed_paths_serializations_) , dynamic_serialization(std::make_shared()) - , shared_data_serialization(SerializationObject::getTypeOfSharedData()->getDefaultSerialization()) + , shared_data_serialization(DataTypeObject::getTypeOfSharedData()->getDefaultSerialization()) { } @@ -64,8 +64,8 @@ void SerializationSubObject::enumerateStreams( /// We will need to read shared data to find all paths with requested prefix. settings.path.push_back(Substream::ObjectSharedData); auto shared_data_substream_data = SubstreamData(shared_data_serialization) - .withType(data.type ? SerializationObject::getTypeOfSharedData() : nullptr) - .withColumn(data.column ? SerializationObject::getTypeOfSharedData()->createColumn() : nullptr) + .withType(data.type ? DataTypeObject::getTypeOfSharedData() : nullptr) + .withColumn(data.column ? DataTypeObject::getTypeOfSharedData()->createColumn() : nullptr) .withSerializationInfo(data.serialization_info) .withDeserializeState(deserialize_state ? deserialize_state->shared_data_state : nullptr); settings.path.back().data = shared_data_substream_data; @@ -208,7 +208,7 @@ void SerializationSubObject::deserializeBinaryBulkWithMultipleStreams( settings.path.push_back(Substream::ObjectSharedData); /// If it's a new object column, reinitialize column for shared data. if (result_column->empty()) - sub_object_state->shared_data = SerializationObject::getTypeOfSharedData()->createColumn(); + sub_object_state->shared_data = DataTypeObject::getTypeOfSharedData()->createColumn(); size_t prev_size = column_object.size(); shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(sub_object_state->shared_data, limit, settings, sub_object_state->shared_data_state, cache); settings.path.pop_back(); diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 0f6311c9716..ee04916e7b4 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -3921,7 +3921,7 @@ private: } } - WrapperType createTupleToObjectWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const + WrapperType createTupleToObjectDeprecatedWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const { if (!from_tuple.haveExplicitNames()) throw Exception(ErrorCodes::TYPE_MISMATCH, @@ -3968,7 +3968,7 @@ private: }; } - WrapperType createMapToObjectWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const + WrapperType createMapToObjectDeprecatedWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const { auto key_value_types = from_map.getKeyValueTypes(); @@ -4048,11 +4048,11 @@ private: { if (const auto * from_tuple = checkAndGetDataType(from_type.get())) { - return createTupleToObjectWrapper(*from_tuple, to_type->hasNullableSubcolumns()); + return createTupleToObjectDeprecatedWrapper(*from_tuple, to_type->hasNullableSubcolumns()); } else if (const auto * from_map = checkAndGetDataType(from_type.get())) { - return createMapToObjectWrapper(*from_map, to_type->hasNullableSubcolumns()); + return createMapToObjectDeprecatedWrapper(*from_map, to_type->hasNullableSubcolumns()); } else if (checkAndGetDataType(from_type.get())) { @@ -4081,23 +4081,43 @@ private: "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); } + WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_object) const { if (checkAndGetDataType(from_type.get())) { return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) { - auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context)->assumeMutable(); - res->finalize(); - return res; + return ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context); + }; + } + + /// Cast Tuple/Object/Map to JSON type through serializing into JSON string and parsing back into JSON column. + /// Potentially we can do smarter conversion Tuple -> JSON with type preservation, but it's questionable how exactly Tuple should be + /// converted to JSON (for example, should we recursively convert nested Array(Tuple) to Array(JSON) or not, should we infer types from String fields, etc). + if (checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) + { + auto json_string = ColumnString::create(); + ColumnStringHelpers::WriteHelper write_helper(assert_cast(*json_string), input_rows_count); + auto & write_buffer = write_helper.getWriteBuffer(); + FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; + auto serialization = arguments[0].type->getDefaultSerialization(); + for (size_t i = 0; i < input_rows_count; ++i) + { + serialization->serializeTextJSON(*arguments[0].column, i, write_buffer, format_settings); + write_helper.rowWritten(); + } + write_helper.finalize(); + + ColumnsWithTypeAndName args_with_json_string = {ColumnWithTypeAndName(json_string->getPtr(), std::make_shared(), "")}; + return ConvertImplGenericFromString::execute(args_with_json_string, result_type, nullable_source, input_rows_count, context); }; } /// TODO: support CAST between JSON types with different parameters - /// support CAST from Map to JSON - /// support CAST from Tuple to JSON - /// support CAST from Object('json') to JSON - throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String/Map/Object/Tuple. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); } WrapperType createVariantToVariantWrapper(const DataTypeVariant & from_variant, const DataTypeVariant & to_variant) const diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference new file mode 100644 index 00000000000..0ae94e68663 --- /dev/null +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.reference @@ -0,0 +1,23 @@ +Map to JSON +{"a":"0","b":"1970-01-01","c":[],"d":[{"e":"0"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a0":"0","b0":"1970-01-01","c0":[],"d0":[{"e0":"0"}]} {'a0':'Int64','b0':'Date','c0':'Array(Nullable(String))','d0':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a1":"1","b1":"1970-01-02","c1":["0"],"d1":[{"e1":"1"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(String))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a2":"2","b2":"1970-01-03","c2":["0","1"],"d2":[{"e2":"2"}]} {'a2':'Int64','b2':'Date','c2':'Array(Nullable(String))','d2':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a0":"3","b0":"1970-01-04","c0":["0","1","2"],"d0":[{"e0":"3"}]} {'a0':'Int64','b0':'Date','c0':'Array(Nullable(String))','d0':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a1":"4","b1":"1970-01-05","c1":["0","1","2","3"],"d1":[{"e1":"4"}]} {'a1':'Int64','b1':'Date','c1':'Array(Nullable(String))','d1':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +Tuple to JSON +{"a":"0","b":"1970-01-01","c":[],"d":[{"e":"0"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"1","b":"1970-01-02","c":["0"],"d":[{"e":"1"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"2","b":"1970-01-03","c":["0","1"],"d":[{"e":"2"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":[{"e":"3"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":[{"e":"4"}]} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d':'Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))'} +Object to JSON +{"a":"0","b":"1970-01-01","c":[],"d":{"e":["0"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} +{"a":"1","b":"1970-01-02","c":["0"],"d":{"e":["1"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} +{"a":"2","b":"1970-01-03","c":["0","1"],"d":{"e":["2"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} +{"a":"3","b":"1970-01-04","c":["0","1","2"],"d":{"e":["3"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} +{"a":"4","b":"1970-01-05","c":["0","1","2","3"],"d":{"e":["4"]}} {'a':'Int64','b':'Date','c':'Array(Nullable(String))','d.e':'Array(Nullable(Int64))'} diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql new file mode 100644 index 00000000000..fcec7eb3af4 --- /dev/null +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql @@ -0,0 +1,14 @@ +set allow_experimental_json_type = 1; +set allow_experimental_object_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +select 'Map to JSON'; +select map('a', number::UInt32, 'b', toDate(number), 'c', range(number), 'd', [map('e', number::UInt32)])::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); +select map('a' || number % 3, number::UInt32, 'b' || number % 3, toDate(number), 'c' || number % 3, range(number), 'd' || number % 3, [map('e' || number % 3, number::UInt32)])::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); + +select 'Tuple to JSON'; +select tuple(number::UInt32 as a, toDate(number) as b, range(number) as c, [tuple(number::UInt32 as e)] as d)::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); + +select 'Object to JSON'; +select toJSONString(map('a', number::UInt32, 'b', toDate(number), 'c', range(number), 'd', [map('e', number::UInt32)]))::Object('json')::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); From 83f434dffb6bad82abdc791179196b32e1a7f347 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 31 Oct 2024 16:25:17 +0000 Subject: [PATCH 694/816] fix simple path --- src/Processors/Transforms/FillingTransform.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 4a8965dcfaa..dd116a9972a 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -608,9 +608,6 @@ void FillingTransform::transformRange( const auto current_value = (*input_fill_columns[i])[range_begin]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; - logDebug("current value", current_value.dump()); - logDebug("fill from", fill_from.dump()); - if (!fill_from.isNull() && !equals(current_value, fill_from)) { filling_row.initUsingFrom(i); @@ -663,6 +660,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + filling_row_changed = false; } /// Initialize staleness border for current row to generate it's prefix @@ -679,6 +677,7 @@ void FillingTransform::transformRange( interpolate(result_columns, interpolate_block); insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + filling_row_changed = false; } while (filling_row.next(next_row, filling_row_changed)); } From 390429dee53f0c758d823166f9f09024dbed07ae Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 31 Oct 2024 17:34:17 +0100 Subject: [PATCH 695/816] Fix build --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a1737c55c26..563bdc44760 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -522,7 +522,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBu ? std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) : read_settings.remote_fs_buffer_size; if (object_size) - buffer_size = std::min(object_size, buffer_size); + buffer_size = std::min(object_size, buffer_size); auto & reader = context_->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); impl = std::make_unique( From b16a18ed66e8d93e32b2f5956614f1c0a23b40e4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 31 Oct 2024 15:02:18 +0100 Subject: [PATCH 696/816] Add test for mixed version on hosts doing backup or restore. --- src/Backups/BackupCoordinationStageSync.cpp | 49 +++++-- src/Backups/BackupCoordinationStageSync.h | 5 +- .../configs/cluster_different_versions.xml | 16 +++ .../test_different_versions.py | 125 ++++++++++++++++++ 4 files changed, 186 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_backup_restore_on_cluster/configs/cluster_different_versions.xml create mode 100644 tests/integration/test_backup_restore_on_cluster/test_different_versions.py diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 1642cab70c7..9a05f9490c2 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -27,9 +27,24 @@ namespace { /// The coordination version is stored in the 'start' node for each host /// by each host when it starts working on this backup or restore. - /// The initial version didn't use nodes 'finish*' and 'num_hosts'. - constexpr const int kInitialVersion = 1; - constexpr const int kCurrentVersion = 2; + enum Version + { + kInitialVersion = 1, + + /// This old version didn't create the 'finish' node, it uses stage "completed" to tell other hosts that the work is done. + /// If an error happened this old version didn't change any nodes to tell other hosts that the error handling is done. + /// So while using this old version hosts couldn't know when other hosts are done with the error handling, + /// and that situation caused weird errors in the logs somehow. + /// Also this old version didn't create the 'start' node for the initiator. + kVersionWithoutFinishNode = 1, + + /// Now we create the 'finish' node both if the work is done or if the error handling is done. + + kCurrentVersion = 2, + }; + + /// Empty string as the current host is used to mark the initiator of a BACKUP ON CLUSTER or RESTORE ON CLUSTER query. + const constexpr std::string_view kInitiator; } bool BackupCoordinationStageSync::HostInfo::operator ==(const HostInfo & other) const @@ -547,11 +562,9 @@ void BackupCoordinationStageSync::readCurrentState(Coordination::ZooKeeperWithFa String result = zookeeper->get(fs::path{zookeeper_path} / zk_node); host_info->stages[stage] = std::move(result); - /// The initial version didn't create the 'finish' ZooKeeper nodes so - /// we consider that if the "completed" stage is reached by a host then the host has finished its work. - /// This assumption is not correct if an error happens, but the initial version can't handle errors quite - /// correctly anyway. - if ((host_info->version == kInitialVersion) && (stage == BackupCoordinationStage::COMPLETED)) + /// That old version didn't create the 'finish' node so we consider that a host finished its work + /// if it reached the "completed" stage. + if ((host_info->version == kVersionWithoutFinishNode) && (stage == BackupCoordinationStage::COMPLETED)) host_info->finished = true; } } @@ -933,6 +946,15 @@ void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordinatio if (zookeeper->exists(finish_node_path)) return; + /// If the initiator of the query has that old version then it doesn't expect us to create the 'finish' node and moreover + /// the initiator can start removing all the nodes immediately after all hosts report about reaching the "completed" status. + /// So to avoid weird errors in the logs we won't create the 'finish' node if the initiator of the query has that old version. + if ((getInitiatorVersion() == kVersionWithoutFinishNode) && (current_host != kInitiator)) + { + LOG_INFO(log, "Skipped creating the 'finish' node because the initiator uses outdated version {}", getInitiatorVersion()); + return; + } + std::optional num_hosts; int num_hosts_version = -1; @@ -1001,6 +1023,17 @@ void BackupCoordinationStageSync::createFinishNodeAndRemoveAliveNode(Coordinatio } +int BackupCoordinationStageSync::getInitiatorVersion() const +{ + std::lock_guard lock{mutex}; + auto it = state.hosts.find(String{kInitiator}); + if (it == state.hosts.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no initiator of this {} query, it's a bug", operation_name); + const HostInfo & host_info = it->second; + return host_info.version; +} + + void BackupCoordinationStageSync::waitForOtherHostsToFinish() const { tryWaitForOtherHostsToFinishImpl(/* reason = */ "", /* throw_if_error = */ true, /* timeout = */ {}); diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index 32f660af997..dc0d3c3c83d 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -109,6 +109,9 @@ private: bool tryFinishImpl(bool & other_hosts_also_finished, bool throw_if_error, WithRetries::Kind retries_kind); void createFinishNodeAndRemoveAliveNode(Coordination::ZooKeeperWithFaultInjection::Ptr zookeeper); + /// Returns the version used by the initiator. + int getInitiatorVersion() const; + /// Waits until all the other hosts finish their work. bool tryWaitForOtherHostsToFinishImpl(const String & reason, bool throw_if_error, std::optional timeout) const; bool checkIfOtherHostsFinish(const String & reason, bool throw_if_error, bool time_is_out, std::optional timeout) const TSA_REQUIRES(mutex); @@ -157,7 +160,7 @@ private: bool started = false; bool connected = false; bool finished = false; - int version = 0; + int version = 1; std::map stages = {}; /// std::map because we need to compare states std::exception_ptr exception = nullptr; diff --git a/tests/integration/test_backup_restore_on_cluster/configs/cluster_different_versions.xml b/tests/integration/test_backup_restore_on_cluster/configs/cluster_different_versions.xml new file mode 100644 index 00000000000..f70b255da18 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/configs/cluster_different_versions.xml @@ -0,0 +1,16 @@ + + + + + + new_node + 9000 + + + old_node + 9000 + + + + + diff --git a/tests/integration/test_backup_restore_on_cluster/test_different_versions.py b/tests/integration/test_backup_restore_on_cluster/test_different_versions.py new file mode 100644 index 00000000000..b5eea7a1902 --- /dev/null +++ b/tests/integration/test_backup_restore_on_cluster/test_different_versions.py @@ -0,0 +1,125 @@ +import random + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) + +main_configs = [ + "configs/backups_disk.xml", + "configs/cluster_different_versions.xml", +] + +user_configs = [] + +new_node = cluster.add_instance( + "new_node", + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "new_node", "shard": "shard1"}, + with_zookeeper=True, +) + +old_node = cluster.add_instance( + "old_node", + image="clickhouse/clickhouse-server", + tag="24.9.2.42", + with_installed_binary=True, + main_configs=main_configs, + user_configs=user_configs, + external_dirs=["/backups/"], + macros={"replica": "old_node", "shard": "shard1"}, + with_zookeeper=True, +) + +nodes = [new_node, old_node] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + new_node.query("DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster_ver' SYNC") + + +backup_id_counter = 0 + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('backups', '{backup_id_counter}')" + + +# Gets a printable version the name of a node. +def get_node_name(node): + return "new_node" if (node == new_node) else "old_node" + + +# Choose a random instance. +def random_node(): + return random.choice(nodes) + + +def test_different_versions(): + new_node.query( + "CREATE TABLE tbl" + " ON CLUSTER 'cluster_ver'" + " (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}')" + " ORDER BY tuple()" + ) + + new_node.query(f"INSERT INTO tbl VALUES (1)") + old_node.query(f"INSERT INTO tbl VALUES (2)") + + backup_name = new_backup_name() + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator for BACKUP") + initiator.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster_ver' TO {backup_name}") + + new_node.query("DROP TABLE tbl ON CLUSTER 'cluster_ver' SYNC") + + initiator = random_node() + print(f"Using {get_node_name(initiator)} as initiator for RESTORE") + initiator.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster_ver' FROM {backup_name}") + + new_node.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster_ver' tbl") + assert new_node.query("SELECT * FROM tbl ORDER BY x") == TSV([1, 2]) + assert old_node.query("SELECT * FROM tbl ORDER BY x") == TSV([1, 2]) + + # Error NO_ELEMENTS_IN_CONFIG is unrelated. + assert ( + new_node.query( + "SELECT name, last_error_message FROM system.errors WHERE NOT (" + "(name == 'NO_ELEMENTS_IN_CONFIG')" + ")" + ) + == "" + ) + + # Error FAILED_TO_SYNC_BACKUP_OR_RESTORE: "No connection to host new_node:9000 yet, will retry" is generated by the old version + # when it fails to connect to other host because that other host hasn't started yet. + # This is not an error actually, just an exception thrown and caught. The new version doesn't throw this exception. + assert ( + old_node.query( + "SELECT name, last_error_message FROM system.errors WHERE NOT (" + "(name == 'NO_ELEMENTS_IN_CONFIG') OR" + "((name == 'FAILED_TO_SYNC_BACKUP_OR_RESTORE') AND (last_error_message == 'No connection to host new_node:9000 yet, will retry'))" + ")" + ) + == "" + ) From 1000ef0e022516536cbd680fa6a206bf5401295c Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 31 Oct 2024 16:39:31 +0000 Subject: [PATCH 697/816] some improves --- src/Interpreters/FillingRow.cpp | 20 ++++++++----- .../Transforms/FillingTransform.cpp | 30 +++++++++++-------- src/Processors/Transforms/FillingTransform.h | 1 + 3 files changed, 31 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 98c18e9b2ae..384ad669206 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -13,7 +13,7 @@ namespace DB constexpr static bool debug_logging_enabled = false; template -inline static void logDebug(String fmt_str, Args&&... args) +inline static void logDebug(const char * fmt_str, Args&&... args) { if constexpr (debug_logging_enabled) LOG_DEBUG(getLogger("FillingRow"), "{}", fmt::format(fmt::runtime(fmt_str), std::forward(args)...)); @@ -117,7 +117,7 @@ bool FillingRow::isConstraintsSatisfied(size_t pos) const chassert(hasSomeConstraints(pos)); int direction = getDirection(pos); - logDebug("constraint: {}, row: {}, direction: {}", constraints[pos].dump(), row[pos].dump(), direction); + logDebug("constraint: {}, row: {}, direction: {}", constraints[pos], row[pos], direction); return less(row[pos], constraints[pos], direction); } @@ -230,7 +230,7 @@ bool FillingRow::next(const FillingRow & next_original_row, bool& value_changed) bool FillingRow::shift(const FillingRow & next_original_row, bool& value_changed) { - logDebug("next_original_row: {}, current: {}", next_original_row.dump(), dump()); + logDebug("next_original_row: {}, current: {}", next_original_row, *this); for (size_t pos = 0; pos < size(); ++pos) { @@ -318,15 +318,12 @@ void FillingRow::updateConstraintsWithStalenessRow(const Columns& base_row, size for (size_t i = 0; i < size(); ++i) { const auto& descr = getFillDescription(i); - constraints[i] = descr.fill_to; if (!descr.fill_staleness.isNull()) { Field staleness_border = (*base_row[i])[row_ind]; descr.staleness_step_func(staleness_border, 1); - - if (constraints[i].isNull() || less(staleness_border, constraints[i], getDirection(i))) - constraints[i] = std::move(staleness_border); + constraints[i] = findBorder(descr.fill_to, staleness_border, getDirection(i)); } } } @@ -350,3 +347,12 @@ WriteBuffer & operator<<(WriteBuffer & out, const FillingRow & row) } } + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::FillingRow & row, format_context & ctx) const + { + return fmt::format_to(ctx.out(), "{}", row.dump()); + } +}; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index dd116a9972a..ab782f3e521 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -20,7 +20,7 @@ namespace DB constexpr static bool debug_logging_enabled = false; template -inline static void logDebug(String key, const T & value, const char * separator = " : ") +inline static void logDebug(const char * key, const T & value, const char * separator = " : ") { if constexpr (debug_logging_enabled) { @@ -235,6 +235,7 @@ FillingTransform::FillingTransform( fill_column_positions.push_back(block_position); auto & descr = filling_row.getFillDescription(i); + running_with_staleness |= !descr.fill_staleness.isNull(); const Block & output_header = getOutputPort().getHeader(); const DataTypePtr & type = removeNullable(output_header.getByPosition(block_position).type); @@ -663,23 +664,26 @@ void FillingTransform::transformRange( filling_row_changed = false; } - /// Initialize staleness border for current row to generate it's prefix - filling_row.updateConstraintsWithStalenessRow(input_fill_columns, row_ind); - - while (filling_row.shift(next_row, filling_row_changed)) + if (running_with_staleness) { - logDebug("filling_row after shift", filling_row); + /// Initialize staleness border for current row to generate it's prefix + filling_row.updateConstraintsWithStalenessRow(input_fill_columns, row_ind); - do + while (filling_row.shift(next_row, filling_row_changed)) { - logDebug("inserting prefix filling_row", filling_row); + logDebug("filling_row after shift", filling_row); - interpolate(result_columns, interpolate_block); - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); - copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); - filling_row_changed = false; + do + { + logDebug("inserting prefix filling_row", filling_row); - } while (filling_row.next(next_row, filling_row_changed)); + interpolate(result_columns, interpolate_block); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, interpolate_block); + copyRowFromColumns(res_sort_prefix_columns, input_sort_prefix_columns, row_ind); + filling_row_changed = false; + + } while (filling_row.next(next_row, filling_row_changed)); + } } /// new valid filling row was generated but not inserted, will use it during suffix generation diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index a8866a97103..92ca4fe6c9e 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -84,6 +84,7 @@ private: SortDescription sort_prefix; const InterpolateDescriptionPtr interpolate_description; /// Contains INTERPOLATE columns + bool running_with_staleness = false; /// True if STALENESS clause was used. FillingRow filling_row; /// Current row, which is used to fill gaps. FillingRow next_row; /// Row to which we need to generate filling rows. bool filling_row_inserted = false; From 9021aeaaff66f7a0c0daeb37d1cd42157c5a15aa Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 31 Oct 2024 16:57:51 +0000 Subject: [PATCH 698/816] Add docs --- docs/en/sql-reference/data-types/newjson.md | 46 +++++++++++++++++++-- 1 file changed, 43 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 68952590eb9..2f54d45cd64 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -58,10 +58,10 @@ SELECT json FROM test; └───────────────────────────────────┘ ``` -Using CAST from 'String': +Using CAST from `String`: ```sql -SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json; +SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON AS json; ``` ```text @@ -70,7 +70,47 @@ SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::JSON as json └────────────────────────────────────────────────┘ ``` -CAST from `JSON`, named `Tuple`, `Map` and `Object('json')` to `JSON` type will be supported later. +Using CAST from `Tuple`: + +```sql +SELECT (tuple(42 AS b) AS a, [1, 2, 3] AS c, 'Hello, World!' AS d)::JSON AS json; +``` + +```text +┌─json───────────────────────────────────────────┐ +│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ +└────────────────────────────────────────────────┘ +``` + +Using CAST from `Map`: + +```sql +SELECT map('a', map('b', 42), 'c', [1,2,3], 'd', 'Hello, World!')::JSON AS json; +``` + +```text +┌─json───────────────────────────────────────────┐ +│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ +└────────────────────────────────────────────────┘ +``` + +Using CAST from deprecated `Object('json')`: + +```sql + SELECT '{"a" : {"b" : 42},"c" : [1, 2, 3], "d" : "Hello, World!"}'::Object('json')::JSON AS json; + ``` + +```text +┌─json───────────────────────────────────────────┐ +│ {"a":{"b":42},"c":[1,2,3],"d":"Hello, World!"} │ +└────────────────────────────────────────────────┘ +``` + +:::note +CAST from `Tuple`/`Map`/`Object('json')` to `JSON` is implemented via serializing the column into `String` column containing JSON objects and deserializing it back to `JSON` type column. +::: + +CAST between `JSON` types with different arguments will be supported later. ## Reading JSON paths as subcolumns From ca389d0d71c96998f0c9feeca6ffae913a02fa77 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Oct 2024 18:43:56 +0100 Subject: [PATCH 699/816] Move settings to cloud level --- src/Core/Settings.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 404f5a6b090..ee814e72447 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4846,12 +4846,6 @@ Limit on size of a single batch of file segments that a read buffer can request )", 0) \ DECLARE(UInt64, filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, 1000, R"( Wait time to lock cache for space reservation in filesystem cache -)", 0) \ - DECLARE(Bool, filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage, true, R"( -Wait time to lock cache for space reservation in filesystem cache -)", 0) \ - DECLARE(Bool, filesystem_cache_enable_background_download_during_fetch, true, R"( -Wait time to lock cache for space reservation in filesystem cache )", 0) \ DECLARE(UInt64, temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds, (10 * 60 * 1000), R"( Wait time to lock cache for space reservation for temporary data in filesystem cache @@ -5112,6 +5106,12 @@ Only in ClickHouse Cloud. A maximum number of unacknowledged in-flight packets i )", 0) \ DECLARE(UInt64, distributed_cache_data_packet_ack_window, DistributedCache::ACK_DATA_PACKET_WINDOW, R"( Only in ClickHouse Cloud. A window for sending ACK for DataPacket sequence in a single distributed cache read request +)", 0) \ + DECLARE(Bool, filesystem_cache_enable_background_download_for_metadata_files_in_packed_storage, true, R"( +Only in ClickHouse Cloud. Wait time to lock cache for space reservation in filesystem cache +)", 0) \ + DECLARE(Bool, filesystem_cache_enable_background_download_during_fetch, true, R"( +Only in ClickHouse Cloud. Wait time to lock cache for space reservation in filesystem cache )", 0) \ \ DECLARE(Bool, parallelize_output_from_storages, true, R"( @@ -5122,6 +5122,7 @@ The setting allows a user to provide own deduplication semantic in MergeTree/Rep For example, by providing a unique value for the setting in each INSERT statement, user can avoid the same inserted data being deduplicated. + Possible values: - Any string From e5be813de559b197e020d4a474fb0bed5d0a2637 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Oct 2024 18:50:43 +0100 Subject: [PATCH 700/816] Sync --- src/Core/Settings.cpp | 3 +++ src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 3b63d1231af..7ed24bb85fd 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5111,6 +5111,9 @@ Only in ClickHouse Cloud. A maximum number of unacknowledged in-flight packets i )", 0) \ DECLARE(UInt64, distributed_cache_data_packet_ack_window, DistributedCache::ACK_DATA_PACKET_WINDOW, R"( Only in ClickHouse Cloud. A window for sending ACK for DataPacket sequence in a single distributed cache read request +)", 0) \ + DECLARE(Bool, distributed_cache_discard_connection_if_unread_data, true, R"( +Only in ClickHouse Cloud. Discard connection if some data is unread. )", 0) \ \ DECLARE(Bool, parallelize_output_from_storages, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3fe3e960dc6..7ea388f18dd 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,6 +64,7 @@ static std::initializer_list Date: Thu, 31 Oct 2024 19:46:35 +0100 Subject: [PATCH 701/816] add requirements and fix warning --- docker/test/style/Dockerfile | 2 +- docker/test/style/requirements.txt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index fa6b087eb7d..564301f447c 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -28,7 +28,7 @@ COPY requirements.txt / RUN pip3 install --no-cache-dir -r requirements.txt RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 -ENV LC_ALL en_US.UTF-8 +ENV LC_ALL=en_US.UTF-8 # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH diff --git a/docker/test/style/requirements.txt b/docker/test/style/requirements.txt index cc87f6e548d..aab20b5bee0 100644 --- a/docker/test/style/requirements.txt +++ b/docker/test/style/requirements.txt @@ -12,6 +12,7 @@ charset-normalizer==3.3.2 click==8.1.7 codespell==2.2.1 cryptography==43.0.1 +datacompy==0.7.3 Deprecated==1.2.14 dill==0.3.8 flake8==4.0.1 @@ -23,6 +24,7 @@ mccabe==0.6.1 multidict==6.0.5 mypy==1.8.0 mypy-extensions==1.0.0 +pandas==2.2.3 packaging==24.1 pathspec==0.9.0 pip==24.1.1 From d8fd18c38e28c6a21a083610b252411a5d2dba26 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 31 Oct 2024 19:06:51 +0000 Subject: [PATCH 702/816] Fix test: add more retries --- .../Scheduler/Workload/WorkloadEntityStorageBase.cpp | 8 ++++---- tests/integration/test_scheduler/test.py | 6 ++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 1b7a559698c..968dfd90796 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -578,15 +578,15 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vectorsecond)) { changes.emplace_back(entity_name, entity, it->second); // Update entities that are present in both `new_entities` and `entities` - LOG_TRACE(log, "Entity {} was updated", entity_name); + LOG_TRACE(log, "Workload entity {} was updated", entity_name); } else - LOG_TRACE(log, "Entity {} is the same", entity_name); + LOG_TRACE(log, "Workload entity {} is the same", entity_name); } else { changes.emplace_back(entity_name, entity, ASTPtr{}); // Remove entities that are not present in `new_entities` - LOG_TRACE(log, "Entity {} was dropped", entity_name); + LOG_TRACE(log, "Workload entity {} was dropped", entity_name); } } for (const auto & [entity_name, entity] : new_entities) @@ -594,7 +594,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector Date: Fri, 1 Nov 2024 10:09:54 +0800 Subject: [PATCH 703/816] add uts --- .../0_stateless/03258_quantile_exact_weighted_issue.reference | 2 ++ .../queries/0_stateless/03258_quantile_exact_weighted_issue.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03258_quantile_exact_weighted_issue.reference create mode 100644 tests/queries/0_stateless/03258_quantile_exact_weighted_issue.sql diff --git a/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.reference b/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.reference new file mode 100644 index 00000000000..69afec5d545 --- /dev/null +++ b/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.reference @@ -0,0 +1,2 @@ +AggregateFunction(quantilesExactWeighted(0.2, 0.4, 0.6, 0.8), UInt64, UInt8) +AggregateFunction(quantilesExactWeightedInterpolated(0.2, 0.4, 0.6, 0.8), UInt64, UInt8) diff --git a/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.sql b/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.sql new file mode 100644 index 00000000000..3069389f4e2 --- /dev/null +++ b/tests/queries/0_stateless/03258_quantile_exact_weighted_issue.sql @@ -0,0 +1,2 @@ +SELECT toTypeName(quantilesExactWeightedState(0.2, 0.4, 0.6, 0.8)(number + 1, 1) AS x) FROM numbers(49999); +SELECT toTypeName(quantilesExactWeightedInterpolatedState(0.2, 0.4, 0.6, 0.8)(number + 1, 1) AS x) FROM numbers(49999); From 7726866767c31a3aa85e573331c7286fecf4c6e3 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 31 Oct 2024 22:25:04 -0400 Subject: [PATCH 704/816] Fix inconsistent AST formatting when granting wrong wildcard grants --- docs/en/sql-reference/statements/grant.md | 1 + src/Access/Common/AccessRightsElement.cpp | 2 -- src/Parsers/Access/ParserGrantQuery.cpp | 3 +++ tests/queries/0_stateless/03141_wildcard_grants.sql | 2 ++ 4 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 19305675ec8..d00d70ab578 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -117,6 +117,7 @@ GRANT SELECT ON db*.* TO john -- correct GRANT SELECT ON *.my_table TO john -- wrong GRANT SELECT ON foo*bar TO john -- wrong GRANT SELECT ON *suffix TO john -- wrong +GRANT SELECT(foo) ON db.table* TO john -- wrong ``` ## Privileges diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 3a78420f411..3a02047e2b4 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -127,8 +127,6 @@ void AccessRightsElement::formatColumnNames(WriteBuffer & buffer) const if (std::exchange(need_comma, true)) buffer << ", "; buffer << backQuoteIfNeed(column); - if (wildcard) - buffer << "*"; } buffer << ")"; } diff --git a/src/Parsers/Access/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp index e29cf11273b..4a0d24559a3 100644 --- a/src/Parsers/Access/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -155,6 +155,9 @@ namespace for (auto & [access_flags, columns] : access_and_columns) { + if (wildcard && !columns.empty()) + return false; + AccessRightsElement element; element.access_flags = access_flags; element.columns = std::move(columns); diff --git a/tests/queries/0_stateless/03141_wildcard_grants.sql b/tests/queries/0_stateless/03141_wildcard_grants.sql index 45962d9b929..e71fa531134 100644 --- a/tests/queries/0_stateless/03141_wildcard_grants.sql +++ b/tests/queries/0_stateless/03141_wildcard_grants.sql @@ -19,4 +19,6 @@ REVOKE SELECT ON team*.* FROM user_03141; SHOW GRANTS FOR user_03141; SELECT '---'; +GRANT SELECT(bar) ON foo.test* TO user_03141; -- { clientError SYNTAX_ERROR } + DROP USER user_03141; From 06a23c0a792642d04b5fcacb7b3e06d85ddd298e Mon Sep 17 00:00:00 2001 From: Chang Chen Date: Fri, 1 Nov 2024 11:42:22 +0800 Subject: [PATCH 705/816] fix debug build --- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 84923c49c62..d486850a9db 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -494,7 +494,7 @@ public: nodes.push_back(impl.branch.queue); for (auto & [_, branch] : impl.branch.branch.branches) { - for (auto & [_, child] : branch.children) + for (auto & [_1, child] : branch.children) child->addRawPointerNodes(nodes); } } From 7bd984ceea3a0f366dd62d2407a910a17690be09 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 31 Oct 2024 23:52:09 -0400 Subject: [PATCH 706/816] fix tests --- src/Access/Common/AccessRightsElement.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 3a02047e2b4..3a78420f411 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -127,6 +127,8 @@ void AccessRightsElement::formatColumnNames(WriteBuffer & buffer) const if (std::exchange(need_comma, true)) buffer << ", "; buffer << backQuoteIfNeed(column); + if (wildcard) + buffer << "*"; } buffer << ")"; } From e851e8f3e48df739ac270d7b8672b1cd38dbad2e Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Fri, 1 Nov 2024 08:29:12 +0000 Subject: [PATCH 707/816] Restart CI From a50bc3bac15867ce0ee2d90afa480efdc9c98670 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 1 Nov 2024 08:50:54 +0000 Subject: [PATCH 708/816] Update version_date.tsv and changelogs after v24.10.1.2812-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.10.1.2812-stable.md | 412 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 418 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.10.1.2812-stable.md diff --git a/SECURITY.md b/SECURITY.md index db302da8ecd..1b0648dc489 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,9 +14,10 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.10 | ✔️ | | 24.9 | ✔️ | | 24.8 | ✔️ | -| 24.7 | ✔️ | +| 24.7 | ❌ | | 24.6 | ❌ | | 24.5 | ❌ | | 24.4 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index dfe6a420260..bc76bdbb619 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 991c25ad142..93acf1a5773 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -35,7 +35,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5dc88b49e31..506a627b11c 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.10.1.2812-stable.md b/docs/changelogs/v24.10.1.2812-stable.md new file mode 100644 index 00000000000..c26bbf706ff --- /dev/null +++ b/docs/changelogs/v24.10.1.2812-stable.md @@ -0,0 +1,412 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.10.1.2812-stable (9cd0a3738d5) FIXME as compared to v24.10.1.1-new (b12a3677418) + +#### Backward Incompatible Change +* Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#68614](https://github.com/ClickHouse/ClickHouse/pull/68614) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Reordering of filter conditions from `[PRE]WHERE` clause is now allowed by default. It could be disabled by setting `allow_reorder_prewhere_conditions` to `false`. [#70657](https://github.com/ClickHouse/ClickHouse/pull/70657) ([Nikita Taranov](https://github.com/nickitat)). +* Fix `optimize_functions_to_subcolumns` optimization (previously could lead to `Invalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)` error), by preserving `LowCardinality` type in `mapKeys`/`mapValues`. [#70716](https://github.com/ClickHouse/ClickHouse/pull/70716) ([Azat Khuzhin](https://github.com/azat)). +* Remove the `idxd-config` library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. [#70987](https://github.com/ClickHouse/ClickHouse/pull/70987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB, remove support for deprecated old protocol, support for connection by URI, support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. [#63279](https://github.com/ClickHouse/ClickHouse/pull/63279) ([Kirill Nikiforov](https://github.com/allmazz)). +* A new `--progress-table` option in clickhouse-client prints a table with metrics changing during query execution; a new `--enable-progress-table-toggle` is associated with the `--progress-table` option, and toggles the rendering of the progress table by pressing the control key (Space). [#63689](https://github.com/ClickHouse/ClickHouse/pull/63689) ([Maria Khristenko](https://github.com/mariaKhr)). +* This allows to grant access to the wildcard prefixes. `GRANT SELECT ON db.table_pefix_* TO user`. [#65311](https://github.com/ClickHouse/ClickHouse/pull/65311) ([pufit](https://github.com/pufit)). +* Add system.query_metric_log which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. [#66532](https://github.com/ClickHouse/ClickHouse/pull/66532) ([Pablo Marcos](https://github.com/pamarcos)). +* A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g., `ch "1 + 2"`. This is controlled by a new setting, `implicit_select`. [#68502](https://github.com/ClickHouse/ClickHouse/pull/68502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support --copy mode for clickhouse local as a shortcut for format conversion [#68503](https://github.com/ClickHouse/ClickHouse/issues/68503). [#68583](https://github.com/ClickHouse/ClickHouse/pull/68583) ([Denis Hananein](https://github.com/denis-hananein)). +* Add support for `arrayUnion` function. [#68989](https://github.com/ClickHouse/ClickHouse/pull/68989) ([Peter Nguyen](https://github.com/petern48)). +* Support aggreate function `quantileExactWeightedInterpolated`, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a new `quantileExactWeightedInterpolated` since we already have `quantileExactInterpolatedWeighted`. The reason is the new one is more accurate than the old one. BTW, it is for spark compatiability in Apache Gluten. [#69619](https://github.com/ClickHouse/ClickHouse/pull/69619) ([李扬](https://github.com/taiyang-li)). +* Support function arrayElementOrNull. It returns null if array index is out of range or map key not found. [#69646](https://github.com/ClickHouse/ClickHouse/pull/69646) ([李扬](https://github.com/taiyang-li)). +* Allows users to specify regular expressions through new `message_regexp` and `message_regexp_negative` fields in the `config.xml` file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. [#69657](https://github.com/ClickHouse/ClickHouse/pull/69657) ([Peter Nguyen](https://github.com/petern48)). +* Support Dynamic type in most functions by executing them on internal types inside Dynamic. [#69691](https://github.com/ClickHouse/ClickHouse/pull/69691) ([Pavel Kruglov](https://github.com/Avogar)). +* Re-added `RIPEMD160` function, which computes the RIPEMD-160 cryptographic hash of a string. Example: `SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))` returns `37F332F68DB77BD9D7EDD4969571AD671CF9DD3B`. [#70087](https://github.com/ClickHouse/ClickHouse/pull/70087) ([Dergousov Maxim](https://github.com/m7kss1)). +* Allow to cache read files for object storage table engines and data lakes using hash from ETag + file path as cache key. [#70135](https://github.com/ClickHouse/ClickHouse/pull/70135) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support reading Iceberg tables on HDFS. [#70268](https://github.com/ClickHouse/ClickHouse/pull/70268) ([flynn](https://github.com/ucasfl)). +* Allow to read/write JSON type as binary string in RowBinary format under settings `input_format_binary_read_json_as_string/output_format_binary_write_json_as_string`. [#70288](https://github.com/ClickHouse/ClickHouse/pull/70288) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow to serialize/deserialize JSON column as single String column in Native format. For output use setting `output_format_native_write_json_as_string`. For input, use serialization version `1` before the column data. [#70312](https://github.com/ClickHouse/ClickHouse/pull/70312) ([Pavel Kruglov](https://github.com/Avogar)). +* Supports standard CTE, `with insert`, as previously only supports `insert ... with ...`. [#70593](https://github.com/ClickHouse/ClickHouse/pull/70593) ([Shichao Jin](https://github.com/jsc0218)). + +#### Performance Improvement +* Support minmax index for `pointInPolygon`. [#62085](https://github.com/ClickHouse/ClickHouse/pull/62085) ([JackyWoo](https://github.com/JackyWoo)). +* Add support for parquet bloom filters. [#62966](https://github.com/ClickHouse/ClickHouse/pull/62966) ([Arthur Passos](https://github.com/arthurpassos)). +* Lock-free parts rename to avoid INSERT affect SELECT (due to parts lock) (under normal circumstances with `fsync_part_directory`, QPS of SELECT with INSERT in parallel, increased 2x, under heavy load the effect is even bigger). Note, this only includes `ReplicatedMergeTree` for now. [#64955](https://github.com/ClickHouse/ClickHouse/pull/64955) ([Azat Khuzhin](https://github.com/azat)). +* Respect `ttl_only_drop_parts` on `materialize ttl`; only read necessary columns to recalculate TTL and drop parts by replacing them with an empty one. [#65488](https://github.com/ClickHouse/ClickHouse/pull/65488) ([Andrey Zvonov](https://github.com/zvonand)). +* Refactor `IDisk` and `IObjectStorage` for better performance. Tables from `plain` and `plain_rewritable` object storages will initialize faster. [#68146](https://github.com/ClickHouse/ClickHouse/pull/68146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Optimized thread creation in the ThreadPool to minimize lock contention. Thread creation is now performed outside of the critical section to avoid delays in job scheduling and thread management under high load conditions. This leads to a much more responsive ClickHouse under heavy concurrent load. [#68694](https://github.com/ClickHouse/ClickHouse/pull/68694) ([filimonov](https://github.com/filimonov)). +* Enable reading LowCardinality string columns from ORC. [#69481](https://github.com/ClickHouse/ClickHouse/pull/69481) ([李扬](https://github.com/taiyang-li)). +* Added an ability to parse data directly into sparse columns. [#69828](https://github.com/ClickHouse/ClickHouse/pull/69828) ([Anton Popov](https://github.com/CurtizJ)). +* Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. [#69862](https://github.com/ClickHouse/ClickHouse/pull/69862) ([LiuNeng](https://github.com/liuneng1994)). +* Improved performance of parsing formats with high number of missed values (e.g. `JSONEachRow`). [#69875](https://github.com/ClickHouse/ClickHouse/pull/69875) ([Anton Popov](https://github.com/CurtizJ)). +* Use `LowCardinality` for `ProfileEvents` in system logs such as `part_log`, `query_views_log`, `filesystem_cache_log`. [#70152](https://github.com/ClickHouse/ClickHouse/pull/70152) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve performance of FromUnixTimestamp/ToUnixTimestamp functions. [#71042](https://github.com/ClickHouse/ClickHouse/pull/71042) ([kevinyhzou](https://github.com/KevinyhZou)). + +#### Improvement +* Allow parametrised SQL aliases. [#50665](https://github.com/ClickHouse/ClickHouse/pull/50665) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Fixed [#57616](https://github.com/ClickHouse/ClickHouse/issues/57616) this problem occurs because all positive number arguments are automatically identified as `uint64` type, leading to an inability to match int type data in `summapfiltered`. the issue of non-matching is indeed confusing, as the `uint64` parameters are not specified by the user. additionally, if the arguments are `[1,2,3,toint8(-3)]`, due to the `getleastsupertype()`, these parameters will be uniformly treated as `int` type, causing `'1,2,3'` to also fail in matching the `uint` type data in `summapfiltered`. [#58408](https://github.com/ClickHouse/ClickHouse/pull/58408) ([Chen768959](https://github.com/Chen768959)). +* `ALTER TABLE .. REPLACE PARTITION` doesn't wait anymore for mutations/merges that happen in other partitions. [#59138](https://github.com/ClickHouse/ClickHouse/pull/59138) ([Vasily Nemkov](https://github.com/Enmk)). +* Refreshable materialized views are now supported in Replicated databases. [#60669](https://github.com/ClickHouse/ClickHouse/pull/60669) ([Michael Kolupaev](https://github.com/al13n321)). +* Symbolic links for tables in the `data/database_name/` directory are created for the actual paths to the table's data, depending on the storage policy, instead of the `store/...` directory on the default disk. [#61777](https://github.com/ClickHouse/ClickHouse/pull/61777) ([Kirill](https://github.com/kirillgarbar)). +* Apply configuration updates in global context object. It fixes issues like [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308). [#62944](https://github.com/ClickHouse/ClickHouse/pull/62944) ([Amos Bird](https://github.com/amosbird)). +* Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting `parallel_replicas_mode`, the default value for it is `read_tasks` Additionally, the toggle-switch setting `enable_parallel_replicas` has been added. [#63151](https://github.com/ClickHouse/ClickHouse/pull/63151) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `ReadSettings` not using user set values, because defaults were only used. [#65625](https://github.com/ClickHouse/ClickHouse/pull/65625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* While parsing an Enum field from JSON, a string containing an integer will be interpreted as the corresponding Enum element. This closes [#65119](https://github.com/ClickHouse/ClickHouse/issues/65119). [#66801](https://github.com/ClickHouse/ClickHouse/pull/66801) ([scanhex12](https://github.com/scanhex12)). +* Allow `TRIM` -ing `LEADING` or `TRAILING` empty string as a no-op. Closes [#67792](https://github.com/ClickHouse/ClickHouse/issues/67792). [#68455](https://github.com/ClickHouse/ClickHouse/pull/68455) ([Peter Nguyen](https://github.com/petern48)). +* Support creating a table with a query: `CREATE TABLE ... CLONE AS ...`. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of the `MergeTree` family Closes [#65015](https://github.com/ClickHouse/ClickHouse/issues/65015). [#69091](https://github.com/ClickHouse/ClickHouse/pull/69091) ([tuanpach](https://github.com/tuanpach)). +* In Gluten ClickHouse, Spark's timestamp type is mapped to ClickHouse's datetime64(6) type. When casting timestamp '2012-01-01 00:11:22' as a string, Spark returns '2012-01-01 00:11:22', while Gluten ClickHouse returns '2012-01-01 00:11:22.000000'. [#69179](https://github.com/ClickHouse/ClickHouse/pull/69179) ([Wenzheng Liu](https://github.com/lwz9103)). +* Always use the new analyzer to calculate constant expressions when `enable_analyzer` is set to `true`. Support calculation of `executable()` table function arguments without using `SELECT` query for constant expression. [#69292](https://github.com/ClickHouse/ClickHouse/pull/69292) ([Dmitry Novik](https://github.com/novikd)). +* Add `enable_secure_identifiers` to disallow insecure identifiers. [#69411](https://github.com/ClickHouse/ClickHouse/pull/69411) ([tuanpach](https://github.com/tuanpach)). +* Add `show_create_query_identifier_quoting_rule` to define identifier quoting behavior of the show create query result. Possible values: - `user_display`: When the identifiers is a keyword. - `when_necessary`: When the identifiers is one of `{"distinct", "all", "table"}`, or it can cause ambiguity: column names, dictionary attribute names. - `always`: Always quote identifiers. [#69448](https://github.com/ClickHouse/ClickHouse/pull/69448) ([tuanpach](https://github.com/tuanpach)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/69346 Point 4 described there will work now as well:. [#69563](https://github.com/ClickHouse/ClickHouse/pull/69563) ([Vitaly Baranov](https://github.com/vitlibar)). +* Implement generic SerDe between Avro Union and ClickHouse Variant type. Resolves [#69713](https://github.com/ClickHouse/ClickHouse/issues/69713). [#69712](https://github.com/ClickHouse/ClickHouse/pull/69712) ([Jiří Kozlovský](https://github.com/jirislav)). +* 1. CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses. Now it is supported only for the MergeTree family of table engines. 2. For example, the follow SQL statements will trigger exception in the past, but this PR fixes it: if the destination table do not provide an `ORDER BY` or `PRIMARY KEY` expression in the table definition, we will copy that from source table. [#69739](https://github.com/ClickHouse/ClickHouse/pull/69739) ([sakulali](https://github.com/sakulali)). +* Added user-level settings `min_free_disk_bytes_to_throw_insert` and `min_free_disk_ratio_to_throw_insert` to prevent insertions on disks that are almost full. [#69755](https://github.com/ClickHouse/ClickHouse/pull/69755) ([Marco Vilas Boas](https://github.com/marco-vb)). +* If you run `clickhouse-client` or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such as `SELECT`, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such as `SELECTClickHouse local version 24.10.1.1.` instead of `ClickHouse local version 24.10.1.1.`. Now it is fixed. This closes [#31696](https://github.com/ClickHouse/ClickHouse/issues/31696). [#69856](https://github.com/ClickHouse/ClickHouse/pull/69856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add new column readonly_duration to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. [#69871](https://github.com/ClickHouse/ClickHouse/pull/69871) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Change the join to sort settings type to unsigned int. [#69886](https://github.com/ClickHouse/ClickHouse/pull/69886) ([kevinyhzou](https://github.com/KevinyhZou)). +* Support 64-bit XID in Keeper. It can be enabled with `use_xid_64` config. [#69908](https://github.com/ClickHouse/ClickHouse/pull/69908) ([Antonio Andelic](https://github.com/antonio2368)). +* New function getSettingOrDefault() added to return the default value and avoid exception if a custom setting is not found in the current profile. [#69917](https://github.com/ClickHouse/ClickHouse/pull/69917) ([Shankar](https://github.com/shiyer7474)). +* Allow empty needle in function replace, the same behavior with PostgreSQL. [#69918](https://github.com/ClickHouse/ClickHouse/pull/69918) ([zhanglistar](https://github.com/zhanglistar)). +* Enhance OpenTelemetry span logging to include query settings. [#70011](https://github.com/ClickHouse/ClickHouse/pull/70011) ([sharathks118](https://github.com/sharathks118)). +* Allow empty needle in functions replaceRegexp*, like https://github.com/ClickHouse/ClickHouse/pull/69918. [#70053](https://github.com/ClickHouse/ClickHouse/pull/70053) ([zhanglistar](https://github.com/zhanglistar)). +* Add info to higher-order array functions if lambda result type is unexpected. [#70093](https://github.com/ClickHouse/ClickHouse/pull/70093) ([ttanay](https://github.com/ttanay)). +* Keeper improvement: less blocking during cluster changes. [#70275](https://github.com/ClickHouse/ClickHouse/pull/70275) ([Antonio Andelic](https://github.com/antonio2368)). +* Embedded documentation for settings will be strictly more detailed and complete than the documentation on the website. This is the first step before making the website documentation always auto-generated from the source code. This has long-standing implications: - it will be guaranteed to have every setting; - there is no chance of having default values obsolete; - we can generate this documentation for each ClickHouse version; - the documentation can be displayed by the server itself even without Internet access. Generate the docs on the website from the source code. [#70289](https://github.com/ClickHouse/ClickHouse/pull/70289) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `WITH IMPLICIT` and `FINAL` keywords to the `SHOW GRANTS` command. Fix a minor bug with implicit grants: [#70094](https://github.com/ClickHouse/ClickHouse/issues/70094). [#70293](https://github.com/ClickHouse/ClickHouse/pull/70293) ([pufit](https://github.com/pufit)). +* Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. [#70299](https://github.com/ClickHouse/ClickHouse/pull/70299) ([Antonio Andelic](https://github.com/antonio2368)). +* Respect `compatibility` for MergeTree settings. The `compatibility` value is taken from the `default` profile on server startup, and default MergeTree settings are changed accordingly. Further changes of the `compatibility` setting do not affect MergeTree settings. [#70322](https://github.com/ClickHouse/ClickHouse/pull/70322) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Clickhouse-client realtime metrics follow-up: restore cursor when ctrl-c cancels query; immediately stop intercepting keystrokes when the query is canceled; display the metrics table if `--progress-table` is on, and toggling is disabled. [#70423](https://github.com/ClickHouse/ClickHouse/pull/70423) ([Julia Kartseva](https://github.com/jkartseva)). +* Command-line arguments for Bool settings are set to true when no value is provided for the argument (e.g. `clickhouse-client --optimize_aggregation_in_order --query "SELECT 1"`). [#70459](https://github.com/ClickHouse/ClickHouse/pull/70459) ([davidtsuk](https://github.com/davidtsuk)). +* Avoid spamming the logs with large HTTP response bodies in case of errors during inter-server communication. [#70487](https://github.com/ClickHouse/ClickHouse/pull/70487) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Added a new setting `max_parts_to_move` to control the maximum number of parts that can be moved at once. [#70520](https://github.com/ClickHouse/ClickHouse/pull/70520) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Limit the frequency of certain log messages. [#70601](https://github.com/ClickHouse/ClickHouse/pull/70601) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Don't do validation when synchronizing user_directories from keeper. [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). +* Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the `merge_selector_use_blurry_base` MergeTree-level setting. [#70645](https://github.com/ClickHouse/ClickHouse/pull/70645) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* `CHECK TABLE` with `PART` qualifier was incorrectly formatted in the client. [#70660](https://github.com/ClickHouse/ClickHouse/pull/70660) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support write column index and offset index using parquet native writer. [#70669](https://github.com/ClickHouse/ClickHouse/pull/70669) ([LiuNeng](https://github.com/liuneng1994)). +* Support parse `DateTime64` for microseond and timezone in joda syntax. [#70737](https://github.com/ClickHouse/ClickHouse/pull/70737) ([kevinyhzou](https://github.com/KevinyhZou)). +* Changed an approach to figure out if a cloud storage supports [batch delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) or not. [#70786](https://github.com/ClickHouse/ClickHouse/pull/70786) ([Vitaly Baranov](https://github.com/vitlibar)). +* Support for Parquet page V2 on native reader. [#70807](https://github.com/ClickHouse/ClickHouse/pull/70807) ([Arthur Passos](https://github.com/arthurpassos)). +* Add an HTML page for visualizing merges. [#70821](https://github.com/ClickHouse/ClickHouse/pull/70821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#71234](https://github.com/ClickHouse/ClickHouse/issues/71234): Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* A check if table has both `storage_policy` and `disk` set after alter query is added. A check if a new storage policy is compatible with an old one when using `disk` setting is added. [#70839](https://github.com/ClickHouse/ClickHouse/pull/70839) ([Kirill](https://github.com/kirillgarbar)). +* Add system.s3_queue_settings and system.azure_queue_settings. [#70841](https://github.com/ClickHouse/ClickHouse/pull/70841) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Functions `base58Encode` and `base58Decode` now accept arguments of type `FixedString`. Example: `SELECT base58Encode(toFixedString('plaintext', 9));`. [#70846](https://github.com/ClickHouse/ClickHouse/pull/70846) ([Faizan Patel](https://github.com/faizan2786)). +* Add the `partition` column to every entry type of the part log. Previously, it was set only for some entries. This closes [#70819](https://github.com/ClickHouse/ClickHouse/issues/70819). [#70848](https://github.com/ClickHouse/ClickHouse/pull/70848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add merge start and mutate start events into `system.part_log` which helps with merges analysis and visualization. [#70850](https://github.com/ClickHouse/ClickHouse/pull/70850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. [#70852](https://github.com/ClickHouse/ClickHouse/pull/70852) ([Julia Kartseva](https://github.com/jkartseva)). +* Add a profile event about the number of merged source parts. It allows the monitoring of the fanout of the merge tree in production. [#70908](https://github.com/ClickHouse/ClickHouse/pull/70908) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). +* Background downloads to filesystem cache was enabled back. [#70929](https://github.com/ClickHouse/ClickHouse/pull/70929) ([Nikita Taranov](https://github.com/nickitat)). +* Add a new merge selector algorithm, named `Trivial`, for professional usage only. It is worse than the `Simple` merge selector. [#70969](https://github.com/ClickHouse/ClickHouse/pull/70969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix toHour-like conversion functions' monotonicity when optional time zone argument is passed. [#60264](https://github.com/ClickHouse/ClickHouse/pull/60264) ([Amos Bird](https://github.com/amosbird)). +* Relax `supportsPrewhere` check for StorageMerge. This fixes [#61064](https://github.com/ClickHouse/ClickHouse/issues/61064). It was hardened unnecessarily in [#60082](https://github.com/ClickHouse/ClickHouse/issues/60082). [#61091](https://github.com/ClickHouse/ClickHouse/pull/61091) ([Amos Bird](https://github.com/amosbird)). +* Fix `use_concurrency_control` setting handling for proper `concurrent_threads_soft_limit_num` limit enforcing. This enables concurrency control by default because previously it was broken. [#61473](https://github.com/ClickHouse/ClickHouse/pull/61473) ([Sergei Trifonov](https://github.com/serxa)). +* Fix incorrect JOIN ON section optimization in case of `IS NULL` check under any other function (like `NOT`) that may lead to wrong results. Closes [#67915](https://github.com/ClickHouse/ClickHouse/issues/67915). [#68049](https://github.com/ClickHouse/ClickHouse/pull/68049) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Prevent `ALTER` queries that would make the `CREATE` query of tables invalid. [#68574](https://github.com/ClickHouse/ClickHouse/pull/68574) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix inconsistent AST formatting for `negate` (`-`) and `NOT` functions with tuples and arrays. [#68600](https://github.com/ClickHouse/ClickHouse/pull/68600) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix insertion of incomplete type into Dynamic during deserialization. It could lead to `Parameter out of bound` errors. [#69291](https://github.com/ClickHouse/ClickHouse/pull/69291) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix inf loop after `restore replica` in the replicated merge tree with zero copy. [#69293](https://github.com/ClickHouse/ClickHouse/pull/69293) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Return back default value of `processing_threads_num` as number of cpu cores in storage `S3Queue`. [#69384](https://github.com/ClickHouse/ClickHouse/pull/69384) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns ( fixes [#41971](https://github.com/ClickHouse/ClickHouse/issues/41971) ). [#69556](https://github.com/ClickHouse/ClickHouse/pull/69556) ([Eliot Hautefeuille](https://github.com/hileef)). +* Fix vrash during insertion into FixedString column in PostgreSQL engine. [#69584](https://github.com/ClickHouse/ClickHouse/pull/69584) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix crash when executing `create view t as (with recursive 42 as ttt select ttt);`. [#69676](https://github.com/ClickHouse/ClickHouse/pull/69676) ([Han Fei](https://github.com/hanfei1991)). +* Added `strict_once` mode to aggregate function `windowFunnel` to avoid counting one event several times in case it matches multiple conditions, close [#21835](https://github.com/ClickHouse/ClickHouse/issues/21835). [#69738](https://github.com/ClickHouse/ClickHouse/pull/69738) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed `maxMapState` throwing 'Bad get' if value type is DateTime64. [#69787](https://github.com/ClickHouse/ClickHouse/pull/69787) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix `getSubcolumn` with `LowCardinality` columns by overriding `useDefaultImplementationForLowCardinalityColumns` to return `true`. [#69831](https://github.com/ClickHouse/ClickHouse/pull/69831) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Fix permanent blocked distributed sends if DROP of distributed table fails. [#69843](https://github.com/ClickHouse/ClickHouse/pull/69843) ([Azat Khuzhin](https://github.com/azat)). +* Fix non-cancellable queries containing WITH FILL with NaN keys. This closes [#69261](https://github.com/ClickHouse/ClickHouse/issues/69261). [#69845](https://github.com/ClickHouse/ClickHouse/pull/69845) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix analyzer default with old compatibility value. [#69895](https://github.com/ClickHouse/ClickHouse/pull/69895) ([Raúl Marín](https://github.com/Algunenano)). +* Don't check dependencies during CREATE OR REPLACE VIEW during DROP of old table. Previously CREATE OR REPLACE query failed when there are dependent tables of the recreated view. [#69907](https://github.com/ClickHouse/ClickHouse/pull/69907) ([Pavel Kruglov](https://github.com/Avogar)). +* Implement missing decimal cases for `zeroField`. Fixes [#69730](https://github.com/ClickHouse/ClickHouse/issues/69730). [#69978](https://github.com/ClickHouse/ClickHouse/pull/69978) ([Arthur Passos](https://github.com/arthurpassos)). +* Now SQL security will work with parameterized views correctly. [#69984](https://github.com/ClickHouse/ClickHouse/pull/69984) ([pufit](https://github.com/pufit)). +* Closes [#69752](https://github.com/ClickHouse/ClickHouse/issues/69752). [#69985](https://github.com/ClickHouse/ClickHouse/pull/69985) ([pufit](https://github.com/pufit)). +* Fixed a bug when the timezone could change the result of the query with a `Date` or `Date32` arguments. [#70036](https://github.com/ClickHouse/ClickHouse/pull/70036) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixes `Block structure mismatch` for queries with nested views and `WHERE` condition. Fixes [#66209](https://github.com/ClickHouse/ClickHouse/issues/66209). [#70054](https://github.com/ClickHouse/ClickHouse/pull/70054) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid reusing columns among different named tuples when evaluating `tuple` functions. This fixes [#70022](https://github.com/ClickHouse/ClickHouse/issues/70022). [#70103](https://github.com/ClickHouse/ClickHouse/pull/70103) ([Amos Bird](https://github.com/amosbird)). +* Fix wrong LOGICAL_ERROR when replacing literals in ranges. [#70122](https://github.com/ClickHouse/ClickHouse/pull/70122) ([Pablo Marcos](https://github.com/pamarcos)). +* Check for Nullable(Nothing) type during ALTER TABLE MODIFY COLUMN/QUERY to prevent tables with such data type. [#70123](https://github.com/ClickHouse/ClickHouse/pull/70123) ([Pavel Kruglov](https://github.com/Avogar)). +* Proper error message for illegal query `JOIN ... ON *` , close [#68650](https://github.com/ClickHouse/ClickHouse/issues/68650). [#70124](https://github.com/ClickHouse/ClickHouse/pull/70124) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix wrong result with skipping index. [#70127](https://github.com/ClickHouse/ClickHouse/pull/70127) ([Raúl Marín](https://github.com/Algunenano)). +* Fix data race in ColumnObject/ColumnTuple decompress method that could lead to heap use after free. [#70137](https://github.com/ClickHouse/ClickHouse/pull/70137) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix possible hung in ALTER COLUMN with Dynamic type. [#70144](https://github.com/ClickHouse/ClickHouse/pull/70144) ([Pavel Kruglov](https://github.com/Avogar)). +* Now ClickHouse will consider more errors as retriable and will not mark data parts as broken in case of such errors. [#70145](https://github.com/ClickHouse/ClickHouse/pull/70145) ([alesapin](https://github.com/alesapin)). +* Use correct `max_types` parameter during Dynamic type creation for JSON subcolumn. [#70147](https://github.com/ClickHouse/ClickHouse/pull/70147) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix the password being displayed in `system.query_log` for users with bcrypt password authentication method. [#70148](https://github.com/ClickHouse/ClickHouse/pull/70148) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix event counter for native interface (InterfaceNativeSendBytes). [#70153](https://github.com/ClickHouse/ClickHouse/pull/70153) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible crash in JSON column. [#70172](https://github.com/ClickHouse/ClickHouse/pull/70172) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix multiple issues with arrayMin and arrayMax. [#70207](https://github.com/ClickHouse/ClickHouse/pull/70207) ([Raúl Marín](https://github.com/Algunenano)). +* Respect setting allow_simdjson in JSON type parser. [#70218](https://github.com/ClickHouse/ClickHouse/pull/70218) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix server segfault on creating a materialized view with two selects and an `INTERSECT`, e.g. `CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);`. [#70264](https://github.com/ClickHouse/ClickHouse/pull/70264) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Don't modify global settings with startup scripts. Previously, changing a setting in a startup script would change it globally. [#70310](https://github.com/ClickHouse/ClickHouse/pull/70310) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ALTER of Dynamic type with reducing max_types parameter that could lead to server crash. [#70328](https://github.com/ClickHouse/ClickHouse/pull/70328) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix crash when using WITH FILL incorrectly. [#70338](https://github.com/ClickHouse/ClickHouse/pull/70338) ([Raúl Marín](https://github.com/Algunenano)). +* Fix possible use-after-free in `SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf`. [#70358](https://github.com/ClickHouse/ClickHouse/pull/70358) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash during GROUP BY JSON sub-object subcolumn. [#70374](https://github.com/ClickHouse/ClickHouse/pull/70374) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't prefetch parts for vertical merges if part has no rows. [#70452](https://github.com/ClickHouse/ClickHouse/pull/70452) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix crash in WHERE with lambda functions. [#70464](https://github.com/ClickHouse/ClickHouse/pull/70464) ([Raúl Marín](https://github.com/Algunenano)). +* Fix table creation with `CREATE ... AS table_function()` with database `Replicated` and unavailable table function source on secondary replica. [#70511](https://github.com/ClickHouse/ClickHouse/pull/70511) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Ignore all output on async insert with `wait_for_async_insert=1`. Closes [#62644](https://github.com/ClickHouse/ClickHouse/issues/62644). [#70530](https://github.com/ClickHouse/ClickHouse/pull/70530) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Ignore frozen_metadata.txt while traversing shadow directory from system.remote_data_paths. [#70590](https://github.com/ClickHouse/ClickHouse/pull/70590) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix creation of stateful window functions on misaligned memory. [#70631](https://github.com/ClickHouse/ClickHouse/pull/70631) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed rare crashes in `SELECT`-s and merges after adding a column of `Array` type with non-empty default expression. [#70695](https://github.com/ClickHouse/ClickHouse/pull/70695) ([Anton Popov](https://github.com/CurtizJ)). +* Insert into table function s3 respect query settings. [#70696](https://github.com/ClickHouse/ClickHouse/pull/70696) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix infinite recursion when infering a proto schema with skip unsupported fields enabled. [#70697](https://github.com/ClickHouse/ClickHouse/pull/70697) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71122](https://github.com/ClickHouse/ClickHouse/issues/71122): `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Disable enable_named_columns_in_function_tuple by default. [#70833](https://github.com/ClickHouse/ClickHouse/pull/70833) ([Raúl Marín](https://github.com/Algunenano)). +* Fix S3Queue table engine setting processing_threads_num not being effective in case it was deduced from the number of cpu cores on the server. [#70837](https://github.com/ClickHouse/ClickHouse/pull/70837) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Normalize named tuple arguments in aggregation states. This fixes [#69732](https://github.com/ClickHouse/ClickHouse/issues/69732) . [#70853](https://github.com/ClickHouse/ClickHouse/pull/70853) ([Amos Bird](https://github.com/amosbird)). +* Fix a logical error due to negative zeros in the two-level hash table. This closes [#70973](https://github.com/ClickHouse/ClickHouse/issues/70973). [#70979](https://github.com/ClickHouse/ClickHouse/pull/70979) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#71214](https://github.com/ClickHouse/ClickHouse/issues/71214): Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#71243](https://github.com/ClickHouse/ClickHouse/issues/71243): Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Backported in [#71157](https://github.com/ClickHouse/ClickHouse/issues/71157): Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). +* Backported in [#71265](https://github.com/ClickHouse/ClickHouse/issues/71265): Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#71331](https://github.com/ClickHouse/ClickHouse/issues/71331): Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement +* Docker in integration tests runner is updated to latest version. It was previously pinned u until patch release 24.0.3 was out. https://github.com/moby/moby/issues/45770#issuecomment-1618255130. - HDFS image was deprecated and not running with current docker version. Switched to newer version of a derivative image based on ubuntu. - HDFS tests were hardened to allow them to run with python-repeat. [#66867](https://github.com/ClickHouse/ClickHouse/pull/66867) ([Ilya Yatsishin](https://github.com/qoega)). +* Alpine docker images now use ubuntu 22.04 as glibc donor, results in upgrade of glibc version delivered with alpine images from 2.31 to 2.35. [#69033](https://github.com/ClickHouse/ClickHouse/pull/69033) ([filimonov](https://github.com/filimonov)). +* Makes dbms independent from clickhouse_functions. [#69914](https://github.com/ClickHouse/ClickHouse/pull/69914) ([Raúl Marín](https://github.com/Algunenano)). +* Fix FreeBSD compilation of the MariaDB connector. [#70007](https://github.com/ClickHouse/ClickHouse/pull/70007) ([Raúl Marín](https://github.com/Algunenano)). +* Building on Apple Mac OS X Darwin does not produce strange warnings anymore. [#70411](https://github.com/ClickHouse/ClickHouse/pull/70411) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix building with ARCH_NATIVE CMake flag. [#70585](https://github.com/ClickHouse/ClickHouse/pull/70585) ([Daniil Gentili](https://github.com/danog)). +* The universal installer will download Musl build on Alpine Linux. Some Docker containers are using Alpine Linux, but it was not possible to install ClickHouse there with `curl https://clickhouse.com/ | sh`. [#70767](https://github.com/ClickHouse/ClickHouse/pull/70767) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NO CL CATEGORY + +* Backported in [#71259](https://github.com/ClickHouse/ClickHouse/issues/71259):. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "JSONCompactWithProgress query output format"'. [#69989](https://github.com/ClickHouse/ClickHouse/pull/69989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Support CREATE OR REPLACE VIEW atomically"'. [#70535](https://github.com/ClickHouse/ClickHouse/pull/70535) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Support CREATE OR REPLACE VIEW atomically""'. [#70536](https://github.com/ClickHouse/ClickHouse/pull/70536) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Add projections size to system.projections"'. [#70858](https://github.com/ClickHouse/ClickHouse/pull/70858) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Allow writing argument of `has` or `hasAny` or `hasAll` as string values if array element type is `Enum`. [#56555](https://github.com/ClickHouse/ClickHouse/pull/56555) ([Duc Canh Le](https://github.com/canhld94)). +* Rename FileSegmentKind::Ephemeral and other changes. [#66600](https://github.com/ClickHouse/ClickHouse/pull/66600) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Closes [#67345](https://github.com/ClickHouse/ClickHouse/issues/67345). [#67346](https://github.com/ClickHouse/ClickHouse/pull/67346) ([KrJin](https://github.com/jincong8973)). +* Because it is too complicated to support. [#68410](https://github.com/ClickHouse/ClickHouse/pull/68410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix 01600_parts_states_metrics_long flakiness. [#68521](https://github.com/ClickHouse/ClickHouse/pull/68521) ([Azat Khuzhin](https://github.com/azat)). +* Reduce client start time in debug/sanitizer mode. [#68980](https://github.com/ClickHouse/ClickHouse/pull/68980) ([Raúl Marín](https://github.com/Algunenano)). +* Closes [#69038](https://github.com/ClickHouse/ClickHouse/issues/69038). [#69040](https://github.com/ClickHouse/ClickHouse/pull/69040) ([Nikolay Degterinsky](https://github.com/evillique)). +* Better exception for unsupported full_text index with non-full parts. [#69067](https://github.com/ClickHouse/ClickHouse/pull/69067) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Catch additional zk connection erros while creating table and make sure to cleanup dirs if necessary for retries. [#69093](https://github.com/ClickHouse/ClickHouse/pull/69093) ([Sumit](https://github.com/sum12)). +* Update version_date.tsv and changelog after v24.7.5.37-stable. [#69185](https://github.com/ClickHouse/ClickHouse/pull/69185) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* DOCS: Replace live view with refreshable since the former is deprecated. [#69392](https://github.com/ClickHouse/ClickHouse/pull/69392) ([Damian Kula](https://github.com/heavelock)). +* Update ORC to the current HEAD. [#69473](https://github.com/ClickHouse/ClickHouse/pull/69473) ([Nikita Taranov](https://github.com/nickitat)). +* Make a test ready for flaky check. [#69586](https://github.com/ClickHouse/ClickHouse/pull/69586) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support antlr parser to parse sql with some keywords as alias, make the behaviour same as the clickhouse-server - remove redundant `for` in the `keyword` field. [#69614](https://github.com/ClickHouse/ClickHouse/pull/69614) ([Z.H.](https://github.com/onlyacat)). +* Allow default implementations for null in function mapFromArrays for spark compatiability in apache gluten. Current change doesn't have any side effects on clickhouse in theory. [#69715](https://github.com/ClickHouse/ClickHouse/pull/69715) ([李扬](https://github.com/taiyang-li)). +* Fix exception message in AzureBlobStorage. [#69728](https://github.com/ClickHouse/ClickHouse/pull/69728) ([Pavel Kruglov](https://github.com/Avogar)). +* Add test parsing s3 URL with a bucket name including a dot. [#69743](https://github.com/ClickHouse/ClickHouse/pull/69743) ([Kaushik Iska](https://github.com/iskakaushik)). +* Make `clang-tidy` happy. [#69765](https://github.com/ClickHouse/ClickHouse/pull/69765) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Prepare to enable `clang-tidy` `readability-else-after-return`. [#69768](https://github.com/ClickHouse/ClickHouse/pull/69768) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* S3Queue: support having deprecated settings to not fail server startup. [#69769](https://github.com/ClickHouse/ClickHouse/pull/69769) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use only adaptive heuristic to choose task sizes for remote reading. [#69778](https://github.com/ClickHouse/ClickHouse/pull/69778) ([Nikita Taranov](https://github.com/nickitat)). +* Remove unused buggy code. [#69780](https://github.com/ClickHouse/ClickHouse/pull/69780) ([Raúl Marín](https://github.com/Algunenano)). +* Fix bugfix check. [#69789](https://github.com/ClickHouse/ClickHouse/pull/69789) ([Antonio Andelic](https://github.com/antonio2368)). +* Followup for [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69790](https://github.com/ClickHouse/ClickHouse/pull/69790) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update version after release. [#69816](https://github.com/ClickHouse/ClickHouse/pull/69816) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update ext-dict-functions.md. [#69819](https://github.com/ClickHouse/ClickHouse/pull/69819) ([kurikuQwQ](https://github.com/kurikuQwQ)). +* Allow cyrillic characters in generated contributor names. [#69820](https://github.com/ClickHouse/ClickHouse/pull/69820) ([Raúl Marín](https://github.com/Algunenano)). +* CI: praktika integration 1. [#69822](https://github.com/ClickHouse/ClickHouse/pull/69822) ([Max Kainov](https://github.com/maxknv)). +* Fix `test_delayed_replica_failover`. [#69826](https://github.com/ClickHouse/ClickHouse/pull/69826) ([Antonio Andelic](https://github.com/antonio2368)). +* minor change, less conflicts. [#69830](https://github.com/ClickHouse/ClickHouse/pull/69830) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Improve error message DDLWorker.cpp. [#69835](https://github.com/ClickHouse/ClickHouse/pull/69835) ([Denny Crane](https://github.com/den-crane)). +* Fix typo in description: mutation_sync -> mutations_sync. [#69838](https://github.com/ClickHouse/ClickHouse/pull/69838) ([Alexander Gololobov](https://github.com/davenger)). +* Fix changelog. [#69841](https://github.com/ClickHouse/ClickHouse/pull/69841) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#49940](https://github.com/ClickHouse/ClickHouse/issues/49940). [#69842](https://github.com/ClickHouse/ClickHouse/pull/69842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#51036](https://github.com/ClickHouse/ClickHouse/issues/51036). [#69844](https://github.com/ClickHouse/ClickHouse/pull/69844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update README.md - Update meetups. [#69849](https://github.com/ClickHouse/ClickHouse/pull/69849) ([Tanya Bragin](https://github.com/tbragin)). +* Revert [#69790](https://github.com/ClickHouse/ClickHouse/issues/69790) and [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69850](https://github.com/ClickHouse/ClickHouse/pull/69850) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* See [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69851](https://github.com/ClickHouse/ClickHouse/pull/69851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#50928](https://github.com/ClickHouse/ClickHouse/issues/50928). [#69852](https://github.com/ClickHouse/ClickHouse/pull/69852) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#55981](https://github.com/ClickHouse/ClickHouse/issues/55981). [#69853](https://github.com/ClickHouse/ClickHouse/pull/69853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#56823](https://github.com/ClickHouse/ClickHouse/issues/56823). [#69854](https://github.com/ClickHouse/ClickHouse/pull/69854) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* This closes [#62350](https://github.com/ClickHouse/ClickHouse/issues/62350). [#69855](https://github.com/ClickHouse/ClickHouse/pull/69855) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Refactor functions and variables in statistics code. [#69860](https://github.com/ClickHouse/ClickHouse/pull/69860) ([Robert Schulze](https://github.com/rschu1ze)). +* Resubmit [#63279](https://github.com/ClickHouse/ClickHouse/issues/63279). [#69861](https://github.com/ClickHouse/ClickHouse/pull/69861) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Improve stateless test runner. [#69864](https://github.com/ClickHouse/ClickHouse/pull/69864) ([Alexey Katsman](https://github.com/alexkats)). +* Adjust fast test time limit a bit. [#69874](https://github.com/ClickHouse/ClickHouse/pull/69874) ([Raúl Marín](https://github.com/Algunenano)). +* Add initial 24.9 CHANGELOG. [#69876](https://github.com/ClickHouse/ClickHouse/pull/69876) ([Raúl Marín](https://github.com/Algunenano)). +* Fix test `01278_random_string_utf8`. [#69878](https://github.com/ClickHouse/ClickHouse/pull/69878) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix minor fuzzer issue with experimental statistics. [#69881](https://github.com/ClickHouse/ClickHouse/pull/69881) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix linking after settings refactoring. [#69882](https://github.com/ClickHouse/ClickHouse/pull/69882) ([Robert Schulze](https://github.com/rschu1ze)). +* Add Proj Obsolete Setting. [#69883](https://github.com/ClickHouse/ClickHouse/pull/69883) ([Shichao Jin](https://github.com/jsc0218)). +* Improve remote queries startup time. [#69884](https://github.com/ClickHouse/ClickHouse/pull/69884) ([Igor Nikonov](https://github.com/devcrafter)). +* Revert "Merge pull request [#69032](https://github.com/ClickHouse/ClickHouse/issues/69032) from alexon1234/include_real_time_execution_in_http_header". [#69885](https://github.com/ClickHouse/ClickHouse/pull/69885) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A dedicated commits from https://github.com/ClickHouse/ClickHouse/pull/61473. [#69896](https://github.com/ClickHouse/ClickHouse/pull/69896) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added aliases `time_bucket`(from TimescaleDB) and `date_bin`(from PostgreSQL) for `toStartOfInterval`. [#69900](https://github.com/ClickHouse/ClickHouse/pull/69900) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* RIPE is an acronym and thus should be capital. RIPE stands for **R**ACE **I**ntegrity **P**rimitives **E**valuation and RACE stands for **R**esearch and Development in **A**dvanced **C**ommunications **T**echnologies in **E**urope. [#69901](https://github.com/ClickHouse/ClickHouse/pull/69901) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Replace error codes with error names in stateless tests. [#69906](https://github.com/ClickHouse/ClickHouse/pull/69906) ([Dmitry Novik](https://github.com/novikd)). +* Move setting to 24.10. [#69913](https://github.com/ClickHouse/ClickHouse/pull/69913) ([Raúl Marín](https://github.com/Algunenano)). +* Minor: Reduce diff between public and private repo. [#69928](https://github.com/ClickHouse/ClickHouse/pull/69928) ([Robert Schulze](https://github.com/rschu1ze)). +* Followup for [#69861](https://github.com/ClickHouse/ClickHouse/issues/69861). [#69930](https://github.com/ClickHouse/ClickHouse/pull/69930) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix test_dictionaries_all_layouts_separate_sources. [#69962](https://github.com/ClickHouse/ClickHouse/pull/69962) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix test_keeper_mntr_data_size. [#69965](https://github.com/ClickHouse/ClickHouse/pull/69965) ([Antonio Andelic](https://github.com/antonio2368)). +* This closes [#49823](https://github.com/ClickHouse/ClickHouse/issues/49823). [#69981](https://github.com/ClickHouse/ClickHouse/pull/69981) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add changelog for 24.9. [#69982](https://github.com/ClickHouse/ClickHouse/pull/69982) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#45303](https://github.com/ClickHouse/ClickHouse/issues/45303). [#69987](https://github.com/ClickHouse/ClickHouse/pull/69987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#69988](https://github.com/ClickHouse/ClickHouse/pull/69988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update README.md. [#69991](https://github.com/ClickHouse/ClickHouse/pull/69991) ([Tyler Hannan](https://github.com/tylerhannan)). +* Disable `03215_parallel_replicas_crash_after_refactoring.sql` for Azure. [#69992](https://github.com/ClickHouse/ClickHouse/pull/69992) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Update CHANGELOG.md. [#69993](https://github.com/ClickHouse/ClickHouse/pull/69993) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#70004](https://github.com/ClickHouse/ClickHouse/pull/70004) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Add RIPEMD160 function". [#70005](https://github.com/ClickHouse/ClickHouse/pull/70005) ([Robert Schulze](https://github.com/rschu1ze)). +* Update CHANGELOG.md. [#70009](https://github.com/ClickHouse/ClickHouse/pull/70009) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update CHANGELOG.md. [#70010](https://github.com/ClickHouse/ClickHouse/pull/70010) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make the pylint stricter. [#70013](https://github.com/ClickHouse/ClickHouse/pull/70013) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added a setting `restore_replace_external_dictionary_source_to_null` which enables replacing dictionary source with Null on restore for external dictionaries (useful for testing). [#70032](https://github.com/ClickHouse/ClickHouse/pull/70032) ([Alexander Tokmakov](https://github.com/tavplubix)). +* `isort` is a simple import sorter for the python to comply [pep-8](https://peps.python.org/pep-0008/#imports) requirements. It will allow to decrease conflicts during sync and beautify the code. The import block is divided into three sub-blocks: `standard library` -> `third-party libraries` -> `local imports` -> `.local imports`. Each sub-block is ordered alphabetically with sub-sub-blocks `import X` -> `from X import Y`. [#70038](https://github.com/ClickHouse/ClickHouse/pull/70038) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update version_date.tsv and changelog after v24.9.1.3278-stable. [#70049](https://github.com/ClickHouse/ClickHouse/pull/70049) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Despite the fact that we set the org-level workflow parameter `PYTHONUNBUFFERED`, it's not inherited in workflows. [#70050](https://github.com/ClickHouse/ClickHouse/pull/70050) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix ubsan issue in function sqid. [#70061](https://github.com/ClickHouse/ClickHouse/pull/70061) ([Robert Schulze](https://github.com/rschu1ze)). +* Delete a setting change. [#70071](https://github.com/ClickHouse/ClickHouse/pull/70071) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `test_distributed_ddl`. [#70075](https://github.com/ClickHouse/ClickHouse/pull/70075) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused placeholder from exception message string. [#70086](https://github.com/ClickHouse/ClickHouse/pull/70086) ([Alsu Giliazova](https://github.com/alsugiliazova)). +* Better exception message when some of the permission is missing. [#70088](https://github.com/ClickHouse/ClickHouse/pull/70088) ([pufit](https://github.com/pufit)). +* Make vector similarity indexes work with adaptive granularity. [#70101](https://github.com/ClickHouse/ClickHouse/pull/70101) ([Robert Schulze](https://github.com/rschu1ze)). +* Add missing columns `total_rows`, `data_compressed_bytes`, and `data_uncompressed_bytes` to `system.projections`. Part of https://github.com/ClickHouse/ClickHouse/pull/68901. [#70106](https://github.com/ClickHouse/ClickHouse/pull/70106) ([Jordi Villar](https://github.com/jrdi)). +* Make `00938_fix_rwlock_segfault_long` non flaky. [#70109](https://github.com/ClickHouse/ClickHouse/pull/70109) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove TODO. [#70110](https://github.com/ClickHouse/ClickHouse/pull/70110) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Change the default threshold to enable hyper threading. [#70111](https://github.com/ClickHouse/ClickHouse/pull/70111) ([Jiebin Sun](https://github.com/jiebinn)). +* Fixed [#69092](https://github.com/ClickHouse/ClickHouse/issues/69092): if `materialized_postgresql_tables_list=table1(id, code),table(id,name)` (`table1` has name that is a substring for `table`) `getTableAllowedColumns` method returns `[id, code]` for `table` before this fix. [#70114](https://github.com/ClickHouse/ClickHouse/pull/70114) ([Kruglov Kirill](https://github.com/1on)). +* Reduce log level. [#70117](https://github.com/ClickHouse/ClickHouse/pull/70117) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Rename `getNumberOfPhysicalCPUCores` and fix its decription. [#70130](https://github.com/ClickHouse/ClickHouse/pull/70130) ([Nikita Taranov](https://github.com/nickitat)). +* Adding 24.10. [#70132](https://github.com/ClickHouse/ClickHouse/pull/70132) ([Tyler Hannan](https://github.com/tylerhannan)). +* (Re?)-enable libcxx asserts for debug builds. [#70134](https://github.com/ClickHouse/ClickHouse/pull/70134) ([Robert Schulze](https://github.com/rschu1ze)). +* Refactor reading from object storage. [#70141](https://github.com/ClickHouse/ClickHouse/pull/70141) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Silence UBSAN for integer overflows in some datetime functions. [#70142](https://github.com/ClickHouse/ClickHouse/pull/70142) ([Michael Kolupaev](https://github.com/al13n321)). +* Improve pipdeptree generator for docker images. - Update requirements.txt for the integration tests runner container - Remove some small dependencies, improve `helpers/retry_decorator.py` - Upgrade docker-compose from EOL version 1 to version 2. [#70146](https://github.com/ClickHouse/ClickHouse/pull/70146) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix 'QueryPlan was not initialized' in 'loop' with empty MergeTree. [#70149](https://github.com/ClickHouse/ClickHouse/pull/70149) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove QueryPlan DataStream. [#70158](https://github.com/ClickHouse/ClickHouse/pull/70158) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_s3_queue/test.py. [#70159](https://github.com/ClickHouse/ClickHouse/pull/70159) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Small docs fix. [#70160](https://github.com/ClickHouse/ClickHouse/pull/70160) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Test: PR local plan, non-constant in source stream. [#70173](https://github.com/ClickHouse/ClickHouse/pull/70173) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix performance checks. [#70175](https://github.com/ClickHouse/ClickHouse/pull/70175) ([Antonio Andelic](https://github.com/antonio2368)). +* Simplify test 03246_range_literal_replacement_works. [#70176](https://github.com/ClickHouse/ClickHouse/pull/70176) ([Pablo Marcos](https://github.com/pamarcos)). +* Update 01079_parallel_alter_add_drop_column_zookeeper.sh. [#70196](https://github.com/ClickHouse/ClickHouse/pull/70196) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Require bugfix job for a set of labels. [#70197](https://github.com/ClickHouse/ClickHouse/pull/70197) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* CI: Praktika integration, fast test. [#70239](https://github.com/ClickHouse/ClickHouse/pull/70239) ([Max Kainov](https://github.com/maxknv)). +* Avoid `Cannot schedule a task` error when loading parts. [#70257](https://github.com/ClickHouse/ClickHouse/pull/70257) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Bump usearch to v2.15.2 and SimSIMD to v5.0.0. [#70270](https://github.com/ClickHouse/ClickHouse/pull/70270) ([Robert Schulze](https://github.com/rschu1ze)). +* Instead of balancing tests by `crc32(file_name)` we'll use `add tests to a group with a minimal number of tests`. [#70272](https://github.com/ClickHouse/ClickHouse/pull/70272) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Closes [#70263](https://github.com/ClickHouse/ClickHouse/issues/70263). [#70273](https://github.com/ClickHouse/ClickHouse/pull/70273) ([flynn](https://github.com/ucasfl)). +* Hide MergeTreeSettings implementation. [#70285](https://github.com/ClickHouse/ClickHouse/pull/70285) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Remove await feature from release branches. [#70294](https://github.com/ClickHouse/ClickHouse/pull/70294) ([Max Kainov](https://github.com/maxknv)). +* Fix `test_keeper_four_word_command`. [#70298](https://github.com/ClickHouse/ClickHouse/pull/70298) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelog after v24.9.2.42-stable. [#70301](https://github.com/ClickHouse/ClickHouse/pull/70301) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Synchronize settings with private. [#70320](https://github.com/ClickHouse/ClickHouse/pull/70320) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add Ignore Option In DeduplicateMergeProjectionMode. [#70327](https://github.com/ClickHouse/ClickHouse/pull/70327) ([Shichao Jin](https://github.com/jsc0218)). +* CI: Enable Integration Tests for backport PRs. [#70329](https://github.com/ClickHouse/ClickHouse/pull/70329) ([Max Kainov](https://github.com/maxknv)). +* There is [a failed CI job](https://s3.amazonaws.com/clickhouse-test-reports/69778/2d81c38874958bd9d54a25524173bdb1ddf2b75c/stateless_tests__release_.html) which is triggered by [03237_create_or_replace_view_atomically_with_atomic_engine](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/03237_create_or_replace_view_atomically_with_atomic_engine.sh). [#70330](https://github.com/ClickHouse/ClickHouse/pull/70330) ([tuanpach](https://github.com/tuanpach)). +* Fix flaky test `03237_insert_sparse_columns_mem`. [#70333](https://github.com/ClickHouse/ClickHouse/pull/70333) ([Anton Popov](https://github.com/CurtizJ)). +* Rename enable_secure_identifiers -> enforce_strict_identifier_format. [#70335](https://github.com/ClickHouse/ClickHouse/pull/70335) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Attempt to fix flaky RabbitMQ tests. Maybe closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#70336](https://github.com/ClickHouse/ClickHouse/pull/70336) ([filimonov](https://github.com/filimonov)). +* Don't fail the stateless check script if we can't collect minio logs. [#70350](https://github.com/ClickHouse/ClickHouse/pull/70350) ([Raúl Marín](https://github.com/Algunenano)). +* Fix tiny mistake, responsible for some of kafka test flaps. Example [report](https://s3.amazonaws.com/clickhouse-test-reports/0/3198aafac59c368993e7b5f49d95674cc1b1be18/integration_tests__release__[2_4].html). [#70352](https://github.com/ClickHouse/ClickHouse/pull/70352) ([filimonov](https://github.com/filimonov)). +* Closes [#69634](https://github.com/ClickHouse/ClickHouse/issues/69634). [#70354](https://github.com/ClickHouse/ClickHouse/pull/70354) ([pufit](https://github.com/pufit)). +* Fix 02346_fulltext_index_bug52019. [#70357](https://github.com/ClickHouse/ClickHouse/pull/70357) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Use new JSON for collecting minio logs. [#70359](https://github.com/ClickHouse/ClickHouse/pull/70359) ([Antonio Andelic](https://github.com/antonio2368)). +* Update comments in VectorSimilarityCondition (WHERE is not supported). [#70360](https://github.com/ClickHouse/ClickHouse/pull/70360) ([Azat Khuzhin](https://github.com/azat)). +* Remove 02492_clickhouse_local_context_uaf test. [#70363](https://github.com/ClickHouse/ClickHouse/pull/70363) ([Azat Khuzhin](https://github.com/azat)). +* Fix `clang-19` build issues. [#70412](https://github.com/ClickHouse/ClickHouse/pull/70412) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Ignore "Invalid multibyte data detected" error during completion. [#70422](https://github.com/ClickHouse/ClickHouse/pull/70422) ([Azat Khuzhin](https://github.com/azat)). +* Make QueryPlan explain methods const. [#70444](https://github.com/ClickHouse/ClickHouse/pull/70444) ([Alexander Gololobov](https://github.com/davenger)). +* Fix 0.1 second delay for interactive queries (due to keystroke interceptor). [#70445](https://github.com/ClickHouse/ClickHouse/pull/70445) ([Azat Khuzhin](https://github.com/azat)). +* Increase lock timeout in attempt to fix 02125_many_mutations. [#70448](https://github.com/ClickHouse/ClickHouse/pull/70448) ([Azat Khuzhin](https://github.com/azat)). +* Fix order in 03249_dynamic_alter_consistency. [#70453](https://github.com/ClickHouse/ClickHouse/pull/70453) ([Alexander Gololobov](https://github.com/davenger)). +* Fix refreshable MV in system database breaking server startup. [#70460](https://github.com/ClickHouse/ClickHouse/pull/70460) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix flaky test_refreshable_mv_in_replicated_db. [#70462](https://github.com/ClickHouse/ClickHouse/pull/70462) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelog after v24.8.5.115-lts. [#70463](https://github.com/ClickHouse/ClickHouse/pull/70463) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Decrease probability of "Server died" due to 00913_many_threads. [#70473](https://github.com/ClickHouse/ClickHouse/pull/70473) ([Azat Khuzhin](https://github.com/azat)). +* Fixes for killing leftovers in clikhouse-test. [#70474](https://github.com/ClickHouse/ClickHouse/pull/70474) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelog after v24.3.12.75-lts. [#70485](https://github.com/ClickHouse/ClickHouse/pull/70485) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Use logging instead of print. [#70505](https://github.com/ClickHouse/ClickHouse/pull/70505) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Remove slow poll() logs in keeper. [#70508](https://github.com/ClickHouse/ClickHouse/pull/70508) ([Raúl Marín](https://github.com/Algunenano)). +* Add timeouts for retry loops in test_storage_rabbitmq. It should prevent cascading failures of the whole test suite caused by deadloop in one of the test scenarios. Also added small sleeps in a 'tight' loops to make retries bit less agressive. [#70510](https://github.com/ClickHouse/ClickHouse/pull/70510) ([filimonov](https://github.com/filimonov)). +* CI: Fix for canceled Sync workflow. [#70521](https://github.com/ClickHouse/ClickHouse/pull/70521) ([Max Kainov](https://github.com/maxknv)). +* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/70412, don't know why it's ok in release build, simply changing `_` to `_1` is ok for both release and debug build. [#70532](https://github.com/ClickHouse/ClickHouse/pull/70532) ([Chang chen](https://github.com/baibaichen)). +* Refreshable materialized views are not experimental anymore. [#70550](https://github.com/ClickHouse/ClickHouse/pull/70550) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix 24.9 setting compatibility `database_replicated_allow_explicit_uuid`. [#70565](https://github.com/ClickHouse/ClickHouse/pull/70565) ([Nikita Fomichev](https://github.com/fm4v)). +* Fix typos. [#70588](https://github.com/ClickHouse/ClickHouse/pull/70588) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Vector search: allow to specify HNSW parameter `ef_search` at query time. [#70616](https://github.com/ClickHouse/ClickHouse/pull/70616) ([Robert Schulze](https://github.com/rschu1ze)). +* Increase max_rows_to_read limit in some tests. [#70617](https://github.com/ClickHouse/ClickHouse/pull/70617) ([Raúl Marín](https://github.com/Algunenano)). +* Reduce sync efforts with private. [#70634](https://github.com/ClickHouse/ClickHouse/pull/70634) ([Raúl Marín](https://github.com/Algunenano)). +* Fix parsing of some formats into sparse columns. [#70635](https://github.com/ClickHouse/ClickHouse/pull/70635) ([Anton Popov](https://github.com/CurtizJ)). +* Fix typos. [#70637](https://github.com/ClickHouse/ClickHouse/pull/70637) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Try fix 00180_no_seek_avoiding_when_reading_from_cache. [#70640](https://github.com/ClickHouse/ClickHouse/pull/70640) ([Kseniia Sumarokova](https://github.com/kssenii)). +* When the `PR Check` status is set, it's a valid RunConfig job failure. [#70643](https://github.com/ClickHouse/ClickHouse/pull/70643) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix timeout in materialized pg tests. [#70646](https://github.com/ClickHouse/ClickHouse/pull/70646) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Introduced MergeTree setting which allow to change merge selecting algorithm. However we still have only one algorithm and it's mostly for future experiments. [#70647](https://github.com/ClickHouse/ClickHouse/pull/70647) ([alesapin](https://github.com/alesapin)). +* Docs: Follow-up for [#70585](https://github.com/ClickHouse/ClickHouse/issues/70585). [#70654](https://github.com/ClickHouse/ClickHouse/pull/70654) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove strange file. [#70662](https://github.com/ClickHouse/ClickHouse/pull/70662) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Locally I had lots of errors like `'AllocList' does not refer to a value` around places which used `offsetof`. Changing it to `__builtin_offsetof ` helped and I didn't debug any further. [#70671](https://github.com/ClickHouse/ClickHouse/pull/70671) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Adding the report link to a test result and files' list. [#70677](https://github.com/ClickHouse/ClickHouse/pull/70677) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* materialized postgres: minor fixes. [#70710](https://github.com/ClickHouse/ClickHouse/pull/70710) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Probably fix flaky test_refreshable_mv_in_replicated_db. [#70714](https://github.com/ClickHouse/ClickHouse/pull/70714) ([Michael Kolupaev](https://github.com/al13n321)). +* Move more setting structs to pImpl. [#70739](https://github.com/ClickHouse/ClickHouse/pull/70739) ([Raúl Marín](https://github.com/Algunenano)). +* Reduce sync effort. [#70747](https://github.com/ClickHouse/ClickHouse/pull/70747) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71198](https://github.com/ClickHouse/ClickHouse/issues/71198): Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). +* Add s3queue settings check for cloud. [#70750](https://github.com/ClickHouse/ClickHouse/pull/70750) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix readiness/health check for OpenLDAP container. [#70755](https://github.com/ClickHouse/ClickHouse/pull/70755) ([Julian Maicher](https://github.com/jmaicher)). +* Allow update plan headers for all the steps. [#70761](https://github.com/ClickHouse/ClickHouse/pull/70761) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Autogenerate documentation for settings. [#70768](https://github.com/ClickHouse/ClickHouse/pull/70768) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Not a logical error. [#70770](https://github.com/ClickHouse/ClickHouse/pull/70770) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Aarch64 build with Asan. [#70778](https://github.com/ClickHouse/ClickHouse/pull/70778) ([Max Kainov](https://github.com/maxknv)). +* Minor fix. [#70783](https://github.com/ClickHouse/ClickHouse/pull/70783) ([Anton Popov](https://github.com/CurtizJ)). +* The docs for settings should be located in the source code. Now, the CI supports that. [#70784](https://github.com/ClickHouse/ClickHouse/pull/70784) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update style-test image. [#70785](https://github.com/ClickHouse/ClickHouse/pull/70785) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid double finalization of `WriteBuffer` in library bridge. [#70799](https://github.com/ClickHouse/ClickHouse/pull/70799) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Make Array Field serialization consistent. [#70803](https://github.com/ClickHouse/ClickHouse/pull/70803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* A follow-up for [#70785](https://github.com/ClickHouse/ClickHouse/issues/70785), [jwt](https://pypi.org/project/jwt/#history) looks very outdated, and we have issue with conflicting paths. [#70815](https://github.com/ClickHouse/ClickHouse/pull/70815) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove inneficient code. [#70816](https://github.com/ClickHouse/ClickHouse/pull/70816) ([Raúl Marín](https://github.com/Algunenano)). +* Allow large object files if OMIT_HEAVY_DEBUG_SYMBOLS = 0. [#70818](https://github.com/ClickHouse/ClickHouse/pull/70818) ([Michael Kolupaev](https://github.com/al13n321)). +* Add test with distributed queries for 15768. [#70834](https://github.com/ClickHouse/ClickHouse/pull/70834) ([Nikita Taranov](https://github.com/nickitat)). +* More setting structs to pImpl and reuse code. [#70840](https://github.com/ClickHouse/ClickHouse/pull/70840) ([Raúl Marín](https://github.com/Algunenano)). +* Update default HNSW parameter settings. [#70873](https://github.com/ClickHouse/ClickHouse/pull/70873) ([Robert Schulze](https://github.com/rschu1ze)). +* Limiting logging some lines about configs. [#70879](https://github.com/ClickHouse/ClickHouse/pull/70879) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix `limit by`, `limit with ties` for distributed and parallel replicas. [#70880](https://github.com/ClickHouse/ClickHouse/pull/70880) ([Nikita Taranov](https://github.com/nickitat)). +* Fix darwin build. [#70894](https://github.com/ClickHouse/ClickHouse/pull/70894) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add dots for consistency. [#70909](https://github.com/ClickHouse/ClickHouse/pull/70909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Logical error fix for substrings, found by fuzzer. [#70914](https://github.com/ClickHouse/ClickHouse/pull/70914) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* More setting structs to pImpl. [#70942](https://github.com/ClickHouse/ClickHouse/pull/70942) ([Raúl Marín](https://github.com/Algunenano)). +* Add logging for mock HTTP servers used in minio integration tests. [#70943](https://github.com/ClickHouse/ClickHouse/pull/70943) ([Vitaly Baranov](https://github.com/vitlibar)). +* Minor fixups of [#70011](https://github.com/ClickHouse/ClickHouse/issues/70011) and [#69918](https://github.com/ClickHouse/ClickHouse/issues/69918). [#70959](https://github.com/ClickHouse/ClickHouse/pull/70959) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Do not skip Build report and status fix. [#70965](https://github.com/ClickHouse/ClickHouse/pull/70965) ([Max Kainov](https://github.com/maxknv)). +* Fix Keeper entry serialization compatibility. [#70972](https://github.com/ClickHouse/ClickHouse/pull/70972) ([Antonio Andelic](https://github.com/antonio2368)). +* Update exception message. [#70975](https://github.com/ClickHouse/ClickHouse/pull/70975) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix `utils/c++expr` option `-b`. [#70978](https://github.com/ClickHouse/ClickHouse/pull/70978) ([Sergei Trifonov](https://github.com/serxa)). +* Fix `test_keeper_broken_logs`. [#70982](https://github.com/ClickHouse/ClickHouse/pull/70982) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix `01039_test_setting_parse`. [#70986](https://github.com/ClickHouse/ClickHouse/pull/70986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tests for languages support for Embedded Dictionaries. [#71004](https://github.com/ClickHouse/ClickHouse/pull/71004) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Required for internal test runs with the same image build in public CI. [#71008](https://github.com/ClickHouse/ClickHouse/pull/71008) ([Ilya Yatsishin](https://github.com/qoega)). +* Move remaining settings objects to pImpl and start simplification. [#71019](https://github.com/ClickHouse/ClickHouse/pull/71019) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Rearrange directories for praktika ci. [#71029](https://github.com/ClickHouse/ClickHouse/pull/71029) ([Max Kainov](https://github.com/maxknv)). +* Fix assert in RemoteSource::onAsyncJobReady(). [#71034](https://github.com/ClickHouse/ClickHouse/pull/71034) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix showing error message in ReadBufferFromS3 when retrying. Without this PR information about a retryable failure in `ReadBufferFromS3` could look like this:. [#71038](https://github.com/ClickHouse/ClickHouse/pull/71038) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `test_truncate_database`. [#71057](https://github.com/ClickHouse/ClickHouse/pull/71057) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix clickhouse-test useless 5 second delay in case of multiple threads are used. [#71069](https://github.com/ClickHouse/ClickHouse/pull/71069) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#71142](https://github.com/ClickHouse/ClickHouse/issues/71142): Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Backported in [#71189](https://github.com/ClickHouse/ClickHouse/issues/71189): Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#71222](https://github.com/ClickHouse/ClickHouse/issues/71222): Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). +* Backported in [#71205](https://github.com/ClickHouse/ClickHouse/issues/71205): Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#71277](https://github.com/ClickHouse/ClickHouse/issues/71277): Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71253](https://github.com/ClickHouse/ClickHouse/issues/71253): Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71303](https://github.com/ClickHouse/ClickHouse/issues/71303): Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). +* Backported in [#71317](https://github.com/ClickHouse/ClickHouse/issues/71317): Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Not for changeling + +* Reverted. [#69812](https://github.com/ClickHouse/ClickHouse/pull/69812) ([tuanpach](https://github.com/tuanpach)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 10c55aa4bf5..da7ad3ebd88 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.10.1.2812-stable 2024-11-01 v24.9.2.42-stable 2024-10-03 v24.9.1.3278-stable 2024-09-26 v24.8.5.115-lts 2024-10-08 From 4b04604f5bed39613b0c26da1199caa9eaa5ae89 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Nov 2024 08:07:32 +0000 Subject: [PATCH 709/816] Bump USearch to 2.16.0 and add more tests --- contrib/SimSIMD | 2 +- contrib/usearch | 2 +- .../02354_vector_search_queries.reference | 102 +++++++++++- .../02354_vector_search_queries.sql | 145 ++++++++++++++---- 4 files changed, 217 insertions(+), 34 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index ff51434d90c..935fef2964b 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit ff51434d90c66f916e94ff05b24530b127aa4cff +Subproject commit 935fef2964bc38e995c5f465b42259a35b8cf0d3 diff --git a/contrib/usearch b/contrib/usearch index 1706420acaf..53799b84ca9 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 1706420acafbd83d852c512dcf343af0a4059e48 +Subproject commit 53799b84ca9ad708b060d0b1cfa5f039371721cd diff --git a/tests/queries/0_stateless/02354_vector_search_queries.reference b/tests/queries/0_stateless/02354_vector_search_queries.reference index 223a18b57bf..cf80f46f53c 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.reference +++ b/tests/queries/0_stateless/02354_vector_search_queries.reference @@ -67,7 +67,7 @@ Expression (Projection) Condition: true Parts: 1/1 Granules: 4/4 --- Non-default quantization +-- Test all distance metrics x all quantization 1 [2,3.2] 2.3323807824711897 4 [2.4,5.2] 3.9999999046325727 2 [4.2,3.4] 4.427188573446585 @@ -75,7 +75,7 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - ReadFromMergeTree (default.tab_f64) + ReadFromMergeTree (default.tab_l2_f64) Indexes: PrimaryKey Condition: true @@ -93,7 +93,7 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - ReadFromMergeTree (default.tab_f32) + ReadFromMergeTree (default.tab_l2_f32) Indexes: PrimaryKey Condition: true @@ -111,7 +111,7 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - ReadFromMergeTree (default.tab_f16) + ReadFromMergeTree (default.tab_l2_f16) Indexes: PrimaryKey Condition: true @@ -129,7 +129,7 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - ReadFromMergeTree (default.tab_bf16) + ReadFromMergeTree (default.tab_l2_bf16) Indexes: PrimaryKey Condition: true @@ -147,7 +147,97 @@ Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - ReadFromMergeTree (default.tab_i8) + ReadFromMergeTree (default.tab_l2_i8) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 3/4 +6 [1,9.3] 0.005731362878640178 +4 [2.4,5.2] 0.09204062768384846 +1 [2,3.2] 0.15200169244542905 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_cos_f64) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 3/4 +6 [1,9.3] 0.005731362878640178 +4 [2.4,5.2] 0.09204062768384846 +1 [2,3.2] 0.15200169244542905 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_cos_f32) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 3/4 +6 [1,9.3] 0.005731362878640178 +4 [2.4,5.2] 0.09204062768384846 +1 [2,3.2] 0.15200169244542905 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_cos_f16) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 3/4 +6 [1,9.3] 0.005731362878640178 +4 [2.4,5.2] 0.09204062768384846 +1 [2,3.2] 0.15200169244542905 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_cos_bf16) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: idx + Description: vector_similarity GRANULARITY 2 + Parts: 1/1 + Granules: 3/4 +6 [1,9.3] 0.005731362878640178 +4 [2.4,5.2] 0.09204062768384846 +1 [2,3.2] 0.15200169244542905 +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab_cos_i8) Indexes: PrimaryKey Condition: true diff --git a/tests/queries/0_stateless/02354_vector_search_queries.sql b/tests/queries/0_stateless/02354_vector_search_queries.sql index 71b8a1e520a..0941f9a43d6 100644 --- a/tests/queries/0_stateless/02354_vector_search_queries.sql +++ b/tests/queries/0_stateless/02354_vector_search_queries.sql @@ -81,88 +81,181 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind DROP TABLE tab; -SELECT '-- Non-default quantization'; -CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -INSERT INTO tab_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -INSERT INTO tab_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); -INSERT INTO tab_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +SELECT '-- Test all distance metrics x all quantization'; + +DROP TABLE IF EXISTS tab_l2_f64; +DROP TABLE IF EXISTS tab_l2_f32; +DROP TABLE IF EXISTS tab_l2_f16; +DROP TABLE IF EXISTS tab_l2_bf16; +DROP TABLE IF EXISTS tab_l2_i8; +DROP TABLE IF EXISTS tab_cos_f64; +DROP TABLE IF EXISTS tab_cos_f32; +DROP TABLE IF EXISTS tab_cos_f16; +DROP TABLE IF EXISTS tab_cos_bf16; +DROP TABLE IF EXISTS tab_cos_i8; + +CREATE TABLE tab_l2_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_l2_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_l2_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_l2_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_l2_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_cos_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_cos_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_cos_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_cos_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +CREATE TABLE tab_cos_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; + +INSERT INTO tab_l2_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_l2_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_l2_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_l2_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_l2_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_cos_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_cos_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_cos_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_cos_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); +INSERT INTO tab_cos_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f64 +FROM tab_l2_f64 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f64 +FROM tab_l2_f64 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f32 +FROM tab_l2_f32 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f32 +FROM tab_l2_f32 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f16 +FROM tab_l2_f16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_f16 +FROM tab_l2_f16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_bf16 +FROM tab_l2_bf16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_bf16 +FROM tab_l2_bf16 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_i8 +FROM tab_l2_i8 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; EXPLAIN indexes = 1 WITH [0.0, 2.0] AS reference_vec SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab_i8 +FROM tab_l2_i8 ORDER BY L2Distance(vec, reference_vec) LIMIT 3; -DROP TABLE tab_f64; -DROP TABLE tab_f32; -DROP TABLE tab_f16; -DROP TABLE tab_bf16; -DROP TABLE tab_i8; +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f64 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f64 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f32 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f32 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f16 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_f16 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_bf16 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_bf16 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_i8 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +EXPLAIN indexes = 1 +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, cosineDistance(vec, reference_vec) +FROM tab_cos_i8 +ORDER BY cosineDistance(vec, reference_vec) +LIMIT 3; + +DROP TABLE tab_l2_f64; +DROP TABLE tab_l2_f32; +DROP TABLE tab_l2_f16; +DROP TABLE tab_l2_bf16; +DROP TABLE tab_l2_i8; +DROP TABLE tab_cos_f64; +DROP TABLE tab_cos_f32; +DROP TABLE tab_cos_f16; +DROP TABLE tab_cos_bf16; +DROP TABLE tab_cos_i8; SELECT '-- Index on Array(Float64) column'; CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; From c6184440c4b036809de98d76efd3e177d4d8483e Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Fri, 1 Nov 2024 10:39:14 +0100 Subject: [PATCH 710/816] check-doc-aspell: Print full path to script in CI report --- utils/check-style/check-doc-aspell | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-doc-aspell b/utils/check-style/check-doc-aspell index b5a3958e6cf..0406b337575 100755 --- a/utils/check-style/check-doc-aspell +++ b/utils/check-style/check-doc-aspell @@ -53,7 +53,7 @@ done if (( STATUS != 0 )); then echo "====== Errors found ======" echo "To exclude some words add them to the dictionary file \"${ASPELL_IGNORE_PATH}/aspell-dict.txt\"" - echo "You can also run ${0} -i to see the errors interactively and fix them or add to the dictionary file" + echo "You can also run '$(realpath --relative-base=${ROOT_PATH} ${0}) -i' to see the errors interactively and fix them or add to the dictionary file" fi exit ${STATUS} From e4aa477c42e7a05e7de20187496e1b266b5b3187 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Thu, 31 Oct 2024 11:29:08 +0000 Subject: [PATCH 711/816] make integration tests that use hardcoded ip addresses sequential --- tests/integration/parallel_skip.json | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 507894534d4..d293cae4dfd 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -170,6 +170,18 @@ "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_request_new_ticket_after_expiration", "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection" + "test_storage_kerberized_kafka/test.py::test_kafka_config_from_sql_named_collection", + "test_dns_cache/test.py::test_ip_change_drop_dns_cache", + "test_dns_cache/test.py::test_ip_change_update_dns_cache", + "test_dns_cache/test.py::test_dns_cache_update", + "test_dns_cache/test.py::test_user_access_ip_change", + "test_dns_cache/test.py::test_host_is_drop_from_cache_after_consecutive_failures", + "test_dns_cache/test.py::test_dns_resolver_filter", + + "test_https_replication/test_change_ip.py::test_replication_when_node_ip_changed", + + "test_host_regexp_multiple_ptr_records/test.py::test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution", + "test_host_regexp_multiple_ptr_records/test.py::test_host_regexp_multiple_ptr_v4", + "test_host_regexp_multiple_ptr_records/test.py::test_host_regexp_multiple_ptr_v6" ] From d0394719c6da6c3a7d647332b7ae977f703636b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Nov 2024 12:11:07 +0100 Subject: [PATCH 712/816] More assertions --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 1 + src/Interpreters/Cache/FileCache.cpp | 2 + src/Interpreters/Cache/FileSegment.cpp | 91 ++++++++++++++----- src/Interpreters/Cache/FileSegment.h | 2 +- src/Interpreters/Cache/Metadata.cpp | 21 +++-- 5 files changed, 89 insertions(+), 28 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 51c6045cb68..0f0cc4c4139 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -784,6 +784,7 @@ bool CachedOnDiskReadBufferFromFile::writeCache(char * data, size_t size, size_t LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); return false; } + chassert(file_segment.state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); throw; } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index f7b7ffc5aea..ae3c9c58fc5 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1438,6 +1438,8 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) "cached file `{}` does not fit in cache anymore (size: {})", size_limit, offset_it->path().string(), size); + chassert(false); /// TODO: remove before merge. + fs::remove(offset_it->path()); } } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index c356800fa57..f5a7011833a 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -139,7 +139,7 @@ FileSegmentGuard::Lock FileSegment::lock() const void FileSegment::setDownloadState(State state, const FileSegmentGuard::Lock & lock) { - if (isCompleted(false) && state != State::DETACHED) + if (isCompleted(false)) { throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -700,6 +700,8 @@ void FileSegment::complete() case State::PARTIALLY_DOWNLOADED: { chassert(current_downloaded_size > 0); + chassert(fs::exists(getPath())); + chassert(fs::file_size(getPath()) > 0); if (is_last_holder) { @@ -841,29 +843,60 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) } } - if (download_state == State::DOWNLOADED) + switch (download_state.load()) { - chassert(downloader_id.empty()); - chassert(downloaded_size == reserved_size); - chassert(downloaded_size == range().size()); - chassert(downloaded_size > 0); - chassert(std::filesystem::file_size(getPath()) > 0); - check_iterator(queue_iterator); - } - else - { - if (download_state == State::DOWNLOADING) - { - chassert(!downloader_id.empty()); - } - else if (download_state == State::PARTIALLY_DOWNLOADED - || download_state == State::EMPTY) + case State::EMPTY: { chassert(downloader_id.empty()); + chassert(!fs::exists(getPath())); + chassert(!queue_iterator); + break; } + case State::DOWNLOADED: + { + chassert(downloader_id.empty()); - chassert(reserved_size >= downloaded_size); - check_iterator(queue_iterator); + chassert(downloaded_size == reserved_size); + chassert(downloaded_size == range().size()); + chassert(downloaded_size > 0); + chassert(fs::file_size(getPath()) > 0); + + chassert(queue_iterator); + check_iterator(queue_iterator); + break; + } + case State::DOWNLOADING: + { + chassert(!downloader_id.empty()); + if (downloaded_size) + { + chassert(queue_iterator); + chassert(fs::file_size(getPath()) > 0); + } + break; + } + case State::PARTIALLY_DOWNLOADED: + { + chassert(downloader_id.empty()); + + chassert(reserved_size >= downloaded_size); + chassert(downloaded_size > 0); + chassert(fs::file_size(getPath()) > 0); + + chassert(queue_iterator); + check_iterator(queue_iterator); + break; + } + case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: + { + chassert(reserved_size >= downloaded_size); + check_iterator(queue_iterator); + break; + } + case State::DETACHED: + { + break; + } } return true; @@ -991,7 +1024,12 @@ FileSegmentsHolder::FileSegmentsHolder(FileSegments && file_segments_) FileSegmentPtr FileSegmentsHolder::getSingleFileSegment() const { if (file_segments.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single file segment, got: {} in holder {}", file_segments.size(), toString()); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected single file segment, got: {} in holder {}", + file_segments.size(), toString()); + } return file_segments.front(); } @@ -1001,7 +1039,18 @@ void FileSegmentsHolder::reset() ProfileEvents::increment(ProfileEvents::FilesystemCacheUnusedHoldFileSegments, file_segments.size()); for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end();) - file_segment_it = completeAndPopFrontImpl(); + { + try + { + file_segment_it = completeAndPopFrontImpl(); + } + catch (...) + { + chassert(false); + tryLogCurrentException(__PRETTY_FUNCTION__); + continue; + } + } file_segments.clear(); } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index ee9aee1e354..79adc342329 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -254,7 +254,7 @@ private: const FileSegmentKind segment_kind; /// Size of the segment is not known until it is downloaded and /// can be bigger than max_file_segment_size. - const bool is_unbound = false; + const bool is_unbound; const bool background_download_enabled; std::atomic download_state; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 99ea01aa4f1..49dbbc71fa2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -940,7 +940,16 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( if (file_segment->queue_iterator && invalidate_queue_entry) file_segment->queue_iterator->invalidate(); - file_segment->detach(segment_lock, *this); + try + { + file_segment->detach(segment_lock, *this); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + /// Do not rethrow, we much delete the file below. + } try { @@ -990,8 +999,8 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( * because of no space left in cache, we need to be able to cut file segment's size to downloaded_size. */ - auto metadata = getByOffset(offset); - const auto & file_segment = metadata->file_segment; + auto file_segment_metadata = getByOffset(offset); + const auto & file_segment = file_segment_metadata->file_segment; chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); const size_t downloaded_size = file_segment->getDownloadedSize(); @@ -1006,15 +1015,15 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( chassert(file_segment->reserved_size >= downloaded_size); int64_t diff = file_segment->reserved_size - downloaded_size; - metadata->file_segment = std::make_shared( + file_segment_metadata->file_segment = std::make_shared( getKey(), offset, downloaded_size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(file_segment->getKind()), false, file_segment->cache, key_metadata, file_segment->queue_iterator); if (diff) - metadata->getQueueIterator()->decrementSize(diff); + file_segment_metadata->getQueueIterator()->decrementSize(diff); - chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); + chassert(file_segment_metadata->file_segment->assertCorrectnessUnlocked(segment_lock)); } bool LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &) From fad6e8869182dad498a090e4ec442b949d619acc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 15 Oct 2024 12:38:12 +0000 Subject: [PATCH 713/816] Bump --- contrib/arrow | 2 +- contrib/arrow-cmake/CMakeLists.txt | 1 - contrib/flatbuffers | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/arrow b/contrib/arrow index 5cfccd8ea65..3264fdad38b 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 5cfccd8ea65f33d4517e7409815d761c7650b45d +Subproject commit 3264fdad38b2a1628f296cd574a9dd03f4928aea diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 96d1f4adda7..b1c5154a0fe 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -390,7 +390,6 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/vector_selection_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_filter_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_take_internal.cc" - "${LIBRARY_DIR}/compute/light_array.cc" "${LIBRARY_DIR}/compute/registry.cc" "${LIBRARY_DIR}/compute/expression.cc" "${LIBRARY_DIR}/compute/ordering.cc" diff --git a/contrib/flatbuffers b/contrib/flatbuffers index eb3f8279482..0100f6a5779 160000 --- a/contrib/flatbuffers +++ b/contrib/flatbuffers @@ -1 +1 @@ -Subproject commit eb3f827948241ce0e701516f16cd67324802bce9 +Subproject commit 0100f6a5779831fa7a651e4b67ef389a8752bd9b From c6f4ae696be83ea40aeb83f99e6f303051be0158 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 15 Oct 2024 13:33:08 +0000 Subject: [PATCH 714/816] Sort lines in CMake --- contrib/arrow-cmake/CMakeLists.txt | 218 ++++++++++++++--------------- 1 file changed, 108 insertions(+), 110 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index b1c5154a0fe..a35a9b80cb9 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -220,6 +220,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/array/array_dict.cc" "${LIBRARY_DIR}/array/array_nested.cc" "${LIBRARY_DIR}/array/array_primitive.cc" + "${LIBRARY_DIR}/array/array_run_end.cc" "${LIBRARY_DIR}/array/builder_adaptive.cc" "${LIBRARY_DIR}/array/builder_base.cc" "${LIBRARY_DIR}/array/builder_binary.cc" @@ -227,124 +228,25 @@ set(ARROW_SRCS "${LIBRARY_DIR}/array/builder_dict.cc" "${LIBRARY_DIR}/array/builder_nested.cc" "${LIBRARY_DIR}/array/builder_primitive.cc" - "${LIBRARY_DIR}/array/builder_union.cc" "${LIBRARY_DIR}/array/builder_run_end.cc" - "${LIBRARY_DIR}/array/array_run_end.cc" + "${LIBRARY_DIR}/array/builder_union.cc" "${LIBRARY_DIR}/array/concatenate.cc" "${LIBRARY_DIR}/array/data.cc" "${LIBRARY_DIR}/array/diff.cc" "${LIBRARY_DIR}/array/util.cc" "${LIBRARY_DIR}/array/validate.cc" - "${LIBRARY_DIR}/builder.cc" "${LIBRARY_DIR}/buffer.cc" - "${LIBRARY_DIR}/chunked_array.cc" - "${LIBRARY_DIR}/chunk_resolver.cc" - "${LIBRARY_DIR}/compare.cc" - "${LIBRARY_DIR}/config.cc" - "${LIBRARY_DIR}/datum.cc" - "${LIBRARY_DIR}/device.cc" - "${LIBRARY_DIR}/extension_type.cc" - "${LIBRARY_DIR}/memory_pool.cc" - "${LIBRARY_DIR}/pretty_print.cc" - "${LIBRARY_DIR}/record_batch.cc" - "${LIBRARY_DIR}/result.cc" - "${LIBRARY_DIR}/scalar.cc" - "${LIBRARY_DIR}/sparse_tensor.cc" - "${LIBRARY_DIR}/status.cc" - "${LIBRARY_DIR}/table.cc" - "${LIBRARY_DIR}/table_builder.cc" - "${LIBRARY_DIR}/tensor.cc" - "${LIBRARY_DIR}/tensor/coo_converter.cc" - "${LIBRARY_DIR}/tensor/csf_converter.cc" - "${LIBRARY_DIR}/tensor/csx_converter.cc" - "${LIBRARY_DIR}/type.cc" - "${LIBRARY_DIR}/visitor.cc" + "${LIBRARY_DIR}/builder.cc" "${LIBRARY_DIR}/c/bridge.cc" - "${LIBRARY_DIR}/io/buffered.cc" - "${LIBRARY_DIR}/io/caching.cc" - "${LIBRARY_DIR}/io/compressed.cc" - "${LIBRARY_DIR}/io/file.cc" - "${LIBRARY_DIR}/io/hdfs.cc" - "${LIBRARY_DIR}/io/hdfs_internal.cc" - "${LIBRARY_DIR}/io/interfaces.cc" - "${LIBRARY_DIR}/io/memory.cc" - "${LIBRARY_DIR}/io/slow.cc" - "${LIBRARY_DIR}/io/stdio.cc" - "${LIBRARY_DIR}/io/transform.cc" - "${LIBRARY_DIR}/util/async_util.cc" - "${LIBRARY_DIR}/util/basic_decimal.cc" - "${LIBRARY_DIR}/util/bit_block_counter.cc" - "${LIBRARY_DIR}/util/bit_run_reader.cc" - "${LIBRARY_DIR}/util/bit_util.cc" - "${LIBRARY_DIR}/util/bitmap.cc" - "${LIBRARY_DIR}/util/bitmap_builders.cc" - "${LIBRARY_DIR}/util/bitmap_ops.cc" - "${LIBRARY_DIR}/util/bpacking.cc" - "${LIBRARY_DIR}/util/cancel.cc" - "${LIBRARY_DIR}/util/compression.cc" - "${LIBRARY_DIR}/util/counting_semaphore.cc" - "${LIBRARY_DIR}/util/cpu_info.cc" - "${LIBRARY_DIR}/util/decimal.cc" - "${LIBRARY_DIR}/util/delimiting.cc" - "${LIBRARY_DIR}/util/formatting.cc" - "${LIBRARY_DIR}/util/future.cc" - "${LIBRARY_DIR}/util/int_util.cc" - "${LIBRARY_DIR}/util/io_util.cc" - "${LIBRARY_DIR}/util/logging.cc" - "${LIBRARY_DIR}/util/key_value_metadata.cc" - "${LIBRARY_DIR}/util/memory.cc" - "${LIBRARY_DIR}/util/mutex.cc" - "${LIBRARY_DIR}/util/string.cc" - "${LIBRARY_DIR}/util/string_builder.cc" - "${LIBRARY_DIR}/util/task_group.cc" - "${LIBRARY_DIR}/util/tdigest.cc" - "${LIBRARY_DIR}/util/thread_pool.cc" - "${LIBRARY_DIR}/util/time.cc" - "${LIBRARY_DIR}/util/trie.cc" - "${LIBRARY_DIR}/util/unreachable.cc" - "${LIBRARY_DIR}/util/uri.cc" - "${LIBRARY_DIR}/util/utf8.cc" - "${LIBRARY_DIR}/util/value_parsing.cc" - "${LIBRARY_DIR}/util/byte_size.cc" - "${LIBRARY_DIR}/util/debug.cc" - "${LIBRARY_DIR}/util/tracing.cc" - "${LIBRARY_DIR}/util/atfork_internal.cc" - "${LIBRARY_DIR}/util/crc32.cc" - "${LIBRARY_DIR}/util/hashing.cc" - "${LIBRARY_DIR}/util/ree_util.cc" - "${LIBRARY_DIR}/util/union_util.cc" - "${LIBRARY_DIR}/vendored/base64.cpp" - "${LIBRARY_DIR}/vendored/datetime/tz.cpp" - "${LIBRARY_DIR}/vendored/musl/strptime.c" - "${LIBRARY_DIR}/vendored/uriparser/UriCommon.c" - "${LIBRARY_DIR}/vendored/uriparser/UriCompare.c" - "${LIBRARY_DIR}/vendored/uriparser/UriEscape.c" - "${LIBRARY_DIR}/vendored/uriparser/UriFile.c" - "${LIBRARY_DIR}/vendored/uriparser/UriIp4Base.c" - "${LIBRARY_DIR}/vendored/uriparser/UriIp4.c" - "${LIBRARY_DIR}/vendored/uriparser/UriMemory.c" - "${LIBRARY_DIR}/vendored/uriparser/UriNormalizeBase.c" - "${LIBRARY_DIR}/vendored/uriparser/UriNormalize.c" - "${LIBRARY_DIR}/vendored/uriparser/UriParseBase.c" - "${LIBRARY_DIR}/vendored/uriparser/UriParse.c" - "${LIBRARY_DIR}/vendored/uriparser/UriQuery.c" - "${LIBRARY_DIR}/vendored/uriparser/UriRecompose.c" - "${LIBRARY_DIR}/vendored/uriparser/UriResolve.c" - "${LIBRARY_DIR}/vendored/uriparser/UriShorten.c" - "${LIBRARY_DIR}/vendored/double-conversion/bignum.cc" - "${LIBRARY_DIR}/vendored/double-conversion/bignum-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/cached-powers.cc" - "${LIBRARY_DIR}/vendored/double-conversion/double-to-string.cc" - "${LIBRARY_DIR}/vendored/double-conversion/fast-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/fixed-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/string-to-double.cc" - "${LIBRARY_DIR}/vendored/double-conversion/strtod.cc" - + "${LIBRARY_DIR}/chunk_resolver.cc" + "${LIBRARY_DIR}/chunked_array.cc" + "${LIBRARY_DIR}/compare.cc" "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_vector.cc" "${LIBRARY_DIR}/compute/cast.cc" "${LIBRARY_DIR}/compute/exec.cc" + "${LIBRARY_DIR}/compute/expression.cc" "${LIBRARY_DIR}/compute/function.cc" "${LIBRARY_DIR}/compute/function_internal.cc" "${LIBRARY_DIR}/compute/kernel.cc" @@ -386,18 +288,31 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/vector_replace.cc" "${LIBRARY_DIR}/compute/kernels/vector_select_k.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection.cc" - "${LIBRARY_DIR}/compute/kernels/vector_sort.cc" - "${LIBRARY_DIR}/compute/kernels/vector_selection_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_filter_internal.cc" + "${LIBRARY_DIR}/compute/kernels/vector_selection_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_take_internal.cc" - "${LIBRARY_DIR}/compute/registry.cc" - "${LIBRARY_DIR}/compute/expression.cc" + "${LIBRARY_DIR}/compute/kernels/vector_sort.cc" "${LIBRARY_DIR}/compute/ordering.cc" + "${LIBRARY_DIR}/compute/registry.cc" "${LIBRARY_DIR}/compute/row/compare_internal.cc" "${LIBRARY_DIR}/compute/row/encode_internal.cc" "${LIBRARY_DIR}/compute/row/grouper.cc" "${LIBRARY_DIR}/compute/row/row_internal.cc" - + "${LIBRARY_DIR}/config.cc" + "${LIBRARY_DIR}/datum.cc" + "${LIBRARY_DIR}/device.cc" + "${LIBRARY_DIR}/extension_type.cc" + "${LIBRARY_DIR}/io/buffered.cc" + "${LIBRARY_DIR}/io/caching.cc" + "${LIBRARY_DIR}/io/compressed.cc" + "${LIBRARY_DIR}/io/file.cc" + "${LIBRARY_DIR}/io/hdfs.cc" + "${LIBRARY_DIR}/io/hdfs_internal.cc" + "${LIBRARY_DIR}/io/interfaces.cc" + "${LIBRARY_DIR}/io/memory.cc" + "${LIBRARY_DIR}/io/slow.cc" + "${LIBRARY_DIR}/io/stdio.cc" + "${LIBRARY_DIR}/io/transform.cc" "${LIBRARY_DIR}/ipc/dictionary.cc" "${LIBRARY_DIR}/ipc/feather.cc" "${LIBRARY_DIR}/ipc/message.cc" @@ -405,6 +320,89 @@ set(ARROW_SRCS "${LIBRARY_DIR}/ipc/options.cc" "${LIBRARY_DIR}/ipc/reader.cc" "${LIBRARY_DIR}/ipc/writer.cc" + "${LIBRARY_DIR}/memory_pool.cc" + "${LIBRARY_DIR}/pretty_print.cc" + "${LIBRARY_DIR}/record_batch.cc" + "${LIBRARY_DIR}/result.cc" + "${LIBRARY_DIR}/scalar.cc" + "${LIBRARY_DIR}/sparse_tensor.cc" + "${LIBRARY_DIR}/status.cc" + "${LIBRARY_DIR}/table.cc" + "${LIBRARY_DIR}/table_builder.cc" + "${LIBRARY_DIR}/tensor.cc" + "${LIBRARY_DIR}/tensor/coo_converter.cc" + "${LIBRARY_DIR}/tensor/csf_converter.cc" + "${LIBRARY_DIR}/tensor/csx_converter.cc" + "${LIBRARY_DIR}/type.cc" + "${LIBRARY_DIR}/util/async_util.cc" + "${LIBRARY_DIR}/util/atfork_internal.cc" + "${LIBRARY_DIR}/util/basic_decimal.cc" + "${LIBRARY_DIR}/util/bit_block_counter.cc" + "${LIBRARY_DIR}/util/bit_run_reader.cc" + "${LIBRARY_DIR}/util/bit_util.cc" + "${LIBRARY_DIR}/util/bitmap.cc" + "${LIBRARY_DIR}/util/bitmap_builders.cc" + "${LIBRARY_DIR}/util/bitmap_ops.cc" + "${LIBRARY_DIR}/util/bpacking.cc" + "${LIBRARY_DIR}/util/byte_size.cc" + "${LIBRARY_DIR}/util/cancel.cc" + "${LIBRARY_DIR}/util/compression.cc" + "${LIBRARY_DIR}/util/counting_semaphore.cc" + "${LIBRARY_DIR}/util/cpu_info.cc" + "${LIBRARY_DIR}/util/crc32.cc" + "${LIBRARY_DIR}/util/debug.cc" + "${LIBRARY_DIR}/util/decimal.cc" + "${LIBRARY_DIR}/util/delimiting.cc" + "${LIBRARY_DIR}/util/formatting.cc" + "${LIBRARY_DIR}/util/future.cc" + "${LIBRARY_DIR}/util/hashing.cc" + "${LIBRARY_DIR}/util/int_util.cc" + "${LIBRARY_DIR}/util/io_util.cc" + "${LIBRARY_DIR}/util/key_value_metadata.cc" + "${LIBRARY_DIR}/util/logging.cc" + "${LIBRARY_DIR}/util/memory.cc" + "${LIBRARY_DIR}/util/mutex.cc" + "${LIBRARY_DIR}/util/ree_util.cc" + "${LIBRARY_DIR}/util/string.cc" + "${LIBRARY_DIR}/util/string_builder.cc" + "${LIBRARY_DIR}/util/task_group.cc" + "${LIBRARY_DIR}/util/tdigest.cc" + "${LIBRARY_DIR}/util/thread_pool.cc" + "${LIBRARY_DIR}/util/time.cc" + "${LIBRARY_DIR}/util/tracing.cc" + "${LIBRARY_DIR}/util/trie.cc" + "${LIBRARY_DIR}/util/union_util.cc" + "${LIBRARY_DIR}/util/unreachable.cc" + "${LIBRARY_DIR}/util/uri.cc" + "${LIBRARY_DIR}/util/utf8.cc" + "${LIBRARY_DIR}/util/value_parsing.cc" + "${LIBRARY_DIR}/vendored/base64.cpp" + "${LIBRARY_DIR}/vendored/datetime/tz.cpp" + "${LIBRARY_DIR}/vendored/double-conversion/bignum-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/bignum.cc" + "${LIBRARY_DIR}/vendored/double-conversion/cached-powers.cc" + "${LIBRARY_DIR}/vendored/double-conversion/double-to-string.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fast-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fixed-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/string-to-double.cc" + "${LIBRARY_DIR}/vendored/double-conversion/strtod.cc" + "${LIBRARY_DIR}/vendored/musl/strptime.c" + "${LIBRARY_DIR}/vendored/uriparser/UriCommon.c" + "${LIBRARY_DIR}/vendored/uriparser/UriCompare.c" + "${LIBRARY_DIR}/vendored/uriparser/UriEscape.c" + "${LIBRARY_DIR}/vendored/uriparser/UriFile.c" + "${LIBRARY_DIR}/vendored/uriparser/UriIp4.c" + "${LIBRARY_DIR}/vendored/uriparser/UriIp4Base.c" + "${LIBRARY_DIR}/vendored/uriparser/UriMemory.c" + "${LIBRARY_DIR}/vendored/uriparser/UriNormalize.c" + "${LIBRARY_DIR}/vendored/uriparser/UriNormalizeBase.c" + "${LIBRARY_DIR}/vendored/uriparser/UriParse.c" + "${LIBRARY_DIR}/vendored/uriparser/UriParseBase.c" + "${LIBRARY_DIR}/vendored/uriparser/UriQuery.c" + "${LIBRARY_DIR}/vendored/uriparser/UriRecompose.c" + "${LIBRARY_DIR}/vendored/uriparser/UriResolve.c" + "${LIBRARY_DIR}/vendored/uriparser/UriShorten.c" + "${LIBRARY_DIR}/visitor.cc" "${ARROW_SRC_DIR}/arrow/adapters/orc/adapter.cc" "${ARROW_SRC_DIR}/arrow/adapters/orc/util.cc" From f38e07a027c7868a05f88d6356a3d465e7a5d87c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 15 Oct 2024 14:43:28 +0000 Subject: [PATCH 715/816] Fix build --- contrib/arrow-cmake/CMakeLists.txt | 42 +++++++++++++++++++++++++++--- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index a35a9b80cb9..06de5135ad2 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -213,7 +213,12 @@ target_include_directories(_orc SYSTEM PRIVATE set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/arrow") # arrow/cpp/src/arrow/CMakeLists.txt (ARROW_SRCS + ARROW_COMPUTE + ARROW_IPC) +# find . \( -iname \*.cc -o -iname \*.cpp -o -iname \*.c \) | sort | awk '{print "\"${LIBRARY_DIR}" substr($1,2) "\"" }' | grep -v 'test.cc' | grep -v 'json' | grep -v 'flight' \| +# grep -v 'csv' | grep -v 'acero' | grep -v 'dataset' | grep -v 'testing' | grep -v 'gpu' | grep -v 'engine' | grep -v 'filesystem' | grep -v 'benchmark.cc' set(ARROW_SRCS + "${LIBRARY_DIR}/adapters/orc/adapter.cc" + "${LIBRARY_DIR}/adapters/orc/options.cc" + "${LIBRARY_DIR}/adapters/orc/util.cc" "${LIBRARY_DIR}/array/array_base.cc" "${LIBRARY_DIR}/array/array_binary.cc" "${LIBRARY_DIR}/array/array_decimal.cc" @@ -238,6 +243,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/buffer.cc" "${LIBRARY_DIR}/builder.cc" "${LIBRARY_DIR}/c/bridge.cc" + "${LIBRARY_DIR}/c/dlpack.cc" "${LIBRARY_DIR}/chunk_resolver.cc" "${LIBRARY_DIR}/chunked_array.cc" "${LIBRARY_DIR}/compare.cc" @@ -257,6 +263,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/aggregate_var_std.cc" "${LIBRARY_DIR}/compute/kernels/codegen_internal.cc" "${LIBRARY_DIR}/compute/kernels/hash_aggregate.cc" + "${LIBRARY_DIR}/compute/kernels/ree_util_internal.cc" "${LIBRARY_DIR}/compute/kernels/row_encoder.cc" "${LIBRARY_DIR}/compute/kernels/scalar_arithmetic.cc" "${LIBRARY_DIR}/compute/kernels/scalar_boolean.cc" @@ -284,24 +291,31 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/vector_cumulative_ops.cc" "${LIBRARY_DIR}/compute/kernels/vector_hash.cc" "${LIBRARY_DIR}/compute/kernels/vector_nested.cc" + "${LIBRARY_DIR}/compute/kernels/vector_pairwise.cc" "${LIBRARY_DIR}/compute/kernels/vector_rank.cc" "${LIBRARY_DIR}/compute/kernels/vector_replace.cc" + "${LIBRARY_DIR}/compute/kernels/vector_run_end_encode.cc" "${LIBRARY_DIR}/compute/kernels/vector_select_k.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_filter_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_selection_take_internal.cc" "${LIBRARY_DIR}/compute/kernels/vector_sort.cc" + "${LIBRARY_DIR}/compute/key_hash_internal.cc" + "${LIBRARY_DIR}/compute/key_map_internal.cc" + "${LIBRARY_DIR}/compute/light_array_internal.cc" "${LIBRARY_DIR}/compute/ordering.cc" "${LIBRARY_DIR}/compute/registry.cc" "${LIBRARY_DIR}/compute/row/compare_internal.cc" "${LIBRARY_DIR}/compute/row/encode_internal.cc" "${LIBRARY_DIR}/compute/row/grouper.cc" "${LIBRARY_DIR}/compute/row/row_internal.cc" + "${LIBRARY_DIR}/compute/util.cc" "${LIBRARY_DIR}/config.cc" "${LIBRARY_DIR}/datum.cc" "${LIBRARY_DIR}/device.cc" "${LIBRARY_DIR}/extension_type.cc" + "${LIBRARY_DIR}/integration/c_data_integration_internal.cc" "${LIBRARY_DIR}/io/buffered.cc" "${LIBRARY_DIR}/io/caching.cc" "${LIBRARY_DIR}/io/compressed.cc" @@ -315,10 +329,12 @@ set(ARROW_SRCS "${LIBRARY_DIR}/io/transform.cc" "${LIBRARY_DIR}/ipc/dictionary.cc" "${LIBRARY_DIR}/ipc/feather.cc" + "${LIBRARY_DIR}/ipc/file_to_stream.cc" "${LIBRARY_DIR}/ipc/message.cc" "${LIBRARY_DIR}/ipc/metadata_internal.cc" "${LIBRARY_DIR}/ipc/options.cc" "${LIBRARY_DIR}/ipc/reader.cc" + "${LIBRARY_DIR}/ipc/stream_to_file.cc" "${LIBRARY_DIR}/ipc/writer.cc" "${LIBRARY_DIR}/memory_pool.cc" "${LIBRARY_DIR}/pretty_print.cc" @@ -334,6 +350,8 @@ set(ARROW_SRCS "${LIBRARY_DIR}/tensor/csf_converter.cc" "${LIBRARY_DIR}/tensor/csx_converter.cc" "${LIBRARY_DIR}/type.cc" + "${LIBRARY_DIR}/type_traits.cc" + "${LIBRARY_DIR}/util/align_util.cc" "${LIBRARY_DIR}/util/async_util.cc" "${LIBRARY_DIR}/util/atfork_internal.cc" "${LIBRARY_DIR}/util/basic_decimal.cc" @@ -353,12 +371,15 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/debug.cc" "${LIBRARY_DIR}/util/decimal.cc" "${LIBRARY_DIR}/util/delimiting.cc" + "${LIBRARY_DIR}/util/dict_util.cc" + "${LIBRARY_DIR}/util/float16.cc" "${LIBRARY_DIR}/util/formatting.cc" "${LIBRARY_DIR}/util/future.cc" "${LIBRARY_DIR}/util/hashing.cc" "${LIBRARY_DIR}/util/int_util.cc" "${LIBRARY_DIR}/util/io_util.cc" "${LIBRARY_DIR}/util/key_value_metadata.cc" + "${LIBRARY_DIR}/util/list_util.cc" "${LIBRARY_DIR}/util/logging.cc" "${LIBRARY_DIR}/util/memory.cc" "${LIBRARY_DIR}/util/mutex.cc" @@ -462,22 +483,38 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/arrow/schema.cc" "${LIBRARY_DIR}/arrow/schema_internal.cc" "${LIBRARY_DIR}/arrow/writer.cc" + "${LIBRARY_DIR}/benchmark_util.cc" "${LIBRARY_DIR}/bloom_filter.cc" + "${LIBRARY_DIR}/bloom_filter_reader.cc" "${LIBRARY_DIR}/column_reader.cc" "${LIBRARY_DIR}/column_scanner.cc" "${LIBRARY_DIR}/column_writer.cc" "${LIBRARY_DIR}/encoding.cc" + "${LIBRARY_DIR}/encryption/crypto_factory.cc" "${LIBRARY_DIR}/encryption/encryption.cc" "${LIBRARY_DIR}/encryption/encryption_internal.cc" + "${LIBRARY_DIR}/encryption/encryption_internal_nossl.cc" + "${LIBRARY_DIR}/encryption/file_key_unwrapper.cc" + "${LIBRARY_DIR}/encryption/file_key_wrapper.cc" + "${LIBRARY_DIR}/encryption/file_system_key_material_store.cc" "${LIBRARY_DIR}/encryption/internal_file_decryptor.cc" "${LIBRARY_DIR}/encryption/internal_file_encryptor.cc" + "${LIBRARY_DIR}/encryption/key_material.cc" + "${LIBRARY_DIR}/encryption/key_metadata.cc" + "${LIBRARY_DIR}/encryption/key_toolkit.cc" + "${LIBRARY_DIR}/encryption/key_toolkit_internal.cc" + "${LIBRARY_DIR}/encryption/kms_client.cc" + "${LIBRARY_DIR}/encryption/local_wrap_kms_client.cc" + "${LIBRARY_DIR}/encryption/openssl_internal.cc" "${LIBRARY_DIR}/exception.cc" "${LIBRARY_DIR}/file_reader.cc" "${LIBRARY_DIR}/file_writer.cc" - "${LIBRARY_DIR}/page_index.cc" - "${LIBRARY_DIR}/level_conversion.cc" "${LIBRARY_DIR}/level_comparison.cc" + "${LIBRARY_DIR}/level_comparison_avx2.cc" + "${LIBRARY_DIR}/level_conversion.cc" + "${LIBRARY_DIR}/level_conversion_bmi2.cc" "${LIBRARY_DIR}/metadata.cc" + "${LIBRARY_DIR}/page_index.cc" "${LIBRARY_DIR}/platform.cc" "${LIBRARY_DIR}/printer.cc" "${LIBRARY_DIR}/properties.cc" @@ -486,7 +523,6 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/stream_reader.cc" "${LIBRARY_DIR}/stream_writer.cc" "${LIBRARY_DIR}/types.cc" - "${LIBRARY_DIR}/bloom_filter_reader.cc" "${LIBRARY_DIR}/xxhasher.cc" "${GEN_LIBRARY_DIR}/parquet_constants.cpp" From 6923b9ec3fff0891ceff75d2515d7d5c75de1293 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 16 Oct 2024 13:42:40 +0000 Subject: [PATCH 716/816] Update the submodule --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 3264fdad38b..60896c89713 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 3264fdad38b2a1628f296cd574a9dd03f4928aea +Subproject commit 60896c89713c2c1ed4bbc1e22e8eaeb6b7d7f9d5 From c895e855851972c0efd3dc742258ccd7dc234710 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 16 Oct 2024 15:28:41 +0000 Subject: [PATCH 717/816] Bump --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 60896c89713..6e2574f5013 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 60896c89713c2c1ed4bbc1e22e8eaeb6b7d7f9d5 +Subproject commit 6e2574f5013a005c050c9a7787d341aef09d0063 From fe2514955c0eb98e017c63adea1f4b4cdec57e70 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 31 Oct 2024 17:55:31 +0000 Subject: [PATCH 718/816] Enable threading --- contrib/arrow-cmake/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 06de5135ad2..208d48df178 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -553,6 +553,9 @@ endif () add_definitions(-DPARQUET_THRIFT_VERSION_MAJOR=0) add_definitions(-DPARQUET_THRIFT_VERSION_MINOR=16) +# As per https://github.com/apache/arrow/pull/35672 you need to enable it explicitly. +add_definitions(-DARROW_ENABLE_THREADING) + # === tools set(TOOLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet") From 9d0da01ddbfe6648c1d4c0e5f958790d70861bc1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Nov 2024 12:43:38 +0100 Subject: [PATCH 719/816] Update src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h --- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index d486850a9db..2c4b7c4f3bc 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -492,7 +492,7 @@ public: nodes.push_back(impl.semaphore); if (impl.branch.queue) nodes.push_back(impl.branch.queue); - for (auto & [_, branch] : impl.branch.branch.branches) + for (auto & [_0, branch] : impl.branch.branch.branches) { for (auto & [_1, child] : branch.children) child->addRawPointerNodes(nodes); From e83cff7360e1a7ec0459a09bf95c954263b4c27c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Nov 2024 12:47:03 +0100 Subject: [PATCH 720/816] Fix typo --- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f5a7011833a..080b54feb06 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1046,8 +1046,8 @@ void FileSegmentsHolder::reset() } catch (...) { - chassert(false); tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); continue; } } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 49dbbc71fa2..231545212cd 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -948,7 +948,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( { tryLogCurrentException(__PRETTY_FUNCTION__); chassert(false); - /// Do not rethrow, we much delete the file below. + /// Do not rethrow, we must delete the file below. } try From 24e7fc2714f2ccd3391c24b77e07ecaad8608d96 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 1 Nov 2024 13:15:15 +0100 Subject: [PATCH 721/816] Add try catch to data part destructors --- src/Storages/MergeTree/MergeTreeDataPartCompact.cpp | 9 ++++++++- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 9 ++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 22f3c379398..14c2da82de1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -256,7 +256,14 @@ bool MergeTreeDataPartCompact::isStoredOnRemoteDiskWithZeroCopySupport() const MergeTreeDataPartCompact::~MergeTreeDataPartCompact() { - removeIfNeeded(); + try + { + removeIfNeeded(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index d6f213463f2..c515d645253 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -241,7 +241,14 @@ bool MergeTreeDataPartWide::isStoredOnRemoteDiskWithZeroCopySupport() const MergeTreeDataPartWide::~MergeTreeDataPartWide() { - removeIfNeeded(); + try + { + removeIfNeeded(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const From 00cd06838999b775877e2beef1550b17a399f6ca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Nov 2024 12:22:22 +0000 Subject: [PATCH 722/816] Remove upstream SimSIMD submodule --- .gitmodules | 3 --- contrib/SimSIMD | 1 - 2 files changed, 4 deletions(-) delete mode 160000 contrib/SimSIMD diff --git a/.gitmodules b/.gitmodules index bbc8fc7d06c..ac1c4d05e1a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -330,9 +330,6 @@ [submodule "contrib/usearch"] path = contrib/usearch url = https://github.com/ClickHouse/usearch.git -[submodule "contrib/SimSIMD"] - path = contrib/SimSIMD - url = https://github.com/ashvardanian/SimSIMD.git [submodule "contrib/FP16"] path = contrib/FP16 url = https://github.com/Maratyszcza/FP16.git diff --git a/contrib/SimSIMD b/contrib/SimSIMD deleted file mode 160000 index 935fef2964b..00000000000 --- a/contrib/SimSIMD +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 935fef2964bc38e995c5f465b42259a35b8cf0d3 From 3e2d5e508b4c75537dd935bf380019f534aa6351 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Nov 2024 12:22:51 +0000 Subject: [PATCH 723/816] Add forked SimSIMD submodule --- .gitmodules | 3 +++ contrib/SimSIMD | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/SimSIMD diff --git a/.gitmodules b/.gitmodules index ac1c4d05e1a..a3b6450032a 100644 --- a/.gitmodules +++ b/.gitmodules @@ -330,6 +330,9 @@ [submodule "contrib/usearch"] path = contrib/usearch url = https://github.com/ClickHouse/usearch.git +[submodule "contrib/SimSIMD"] + path = contrib/SimSIMD + url = https://github.com/ClickHouse/SimSIMD.git [submodule "contrib/FP16"] path = contrib/FP16 url = https://github.com/Maratyszcza/FP16.git diff --git a/contrib/SimSIMD b/contrib/SimSIMD new file mode 160000 index 00000000000..935fef2964b --- /dev/null +++ b/contrib/SimSIMD @@ -0,0 +1 @@ +Subproject commit 935fef2964bc38e995c5f465b42259a35b8cf0d3 From 93acf134f68cfc091f8d5f7996e7b6ca21c17298 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 1 Nov 2024 14:02:43 +0100 Subject: [PATCH 724/816] Fix Fedora version for testing RPMs installations --- tests/ci/artifactory.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index c66659d4e93..00a7eeebb35 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -200,6 +200,7 @@ class RpmArtifactory: ) _PROD_REPO_URL = "https://packages.clickhouse.com/rpm/clickhouse.repo" _SIGN_KEY = "885E2BDCF96B0B45ABF058453E4AD4719DDE9A38" + FEDORA_VERSION = 40 def __init__(self, release_info: ReleaseInfo, dry_run: bool): self.release_info = release_info @@ -249,16 +250,16 @@ class RpmArtifactory: Shell.check("sync") def test_packages(self): - Shell.check("docker pull fedora:latest", strict=True) + Shell.check(f"docker pull fedora:{self.FEDORA_VERSION}", strict=True) print(f"Test package installation, version [{self.version}]") rpm_command = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1" - cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' + cmd = f'docker run --rm fedora:{self.FEDORA_VERSION} /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) print("Test package installation, version [latest]") rpm_command_2 = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client" - cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command_2}"' + cmd = f'docker run --rm fedora:{self.FEDORA_VERSION} /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command_2}"' print("Running test command:") print(f" {cmd}") assert Shell.check(cmd) From 2bafaa2fc675132d70d7683e16db4571dcddbd0e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Fri, 1 Nov 2024 14:08:45 +0100 Subject: [PATCH 725/816] Update 03261_tuple_map_object_to_json_cast.sql --- .../queries/0_stateless/03261_tuple_map_object_to_json_cast.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql index fcec7eb3af4..c0199452843 100644 --- a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + set allow_experimental_json_type = 1; set allow_experimental_object_type = 1; set allow_experimental_variant_type = 1; From 603cb16e986a7330693be95b23c30dc98ed307e0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Nov 2024 15:06:52 +0100 Subject: [PATCH 726/816] Fix build --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 1b7a559698c..4a2e867e2e2 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -48,9 +48,9 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) /// Returns a type of a workload entity `ptr` WorkloadEntityType getEntityType(const ASTPtr & ptr) { - if (auto * res = typeid_cast(ptr.get())) + if (auto * res = typeid_cast(ptr.get()); res) return WorkloadEntityType::Workload; - if (auto * res = typeid_cast(ptr.get())) + if (auto * res = typeid_cast(ptr.get()); res) return WorkloadEntityType::Resource; chassert(false); return WorkloadEntityType::MAX; @@ -106,7 +106,7 @@ void forEachReference( for (const String & resource : resources) func(resource, res->getWorkloadName(), ReferenceType::ForResource); } - if (auto * res = typeid_cast(source_entity.get())) + if (auto * res = typeid_cast(source_entity.get()); res) { // RESOURCE has no references to be validated, we allow mentioned disks to be created later } From 7a34fbc5b2f322341ae6c378920670f6a2258698 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 1 Nov 2024 14:20:37 +0000 Subject: [PATCH 727/816] allow to prewarm mark cache without enabled setting --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 ++++- src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../03254_system_prewarm_mark_cache.reference | 4 +++ .../03254_system_prewarm_mark_cache.sql | 27 +++++++++++++++++++ 7 files changed, 41 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03254_system_prewarm_mark_cache.reference create mode 100644 tests/queries/0_stateless/03254_system_prewarm_mark_cache.sql diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 45636ab40b9..4c875026ace 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1310,7 +1310,7 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks() void InterpreterSystemQuery::prewarmMarkCache() { if (table_id.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for prewarming marks cache"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM MARK CACHE command"); getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4ed8c67469d..69979809c31 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2343,11 +2343,16 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() } } -void MergeTreeData::prewarmMarkCache(ThreadPool & pool) +void MergeTreeData::prewarmMarkCacheIfNeeded(ThreadPool & pool) { if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) return; + prewarmMarkCache(pool); +} + +void MergeTreeData::prewarmMarkCache(ThreadPool & pool) +{ auto * mark_cache = getContext()->getMarkCache().get(); if (!mark_cache) return; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a32106f76bb..8da4329a93b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -508,6 +508,7 @@ public: /// Prewarm mark cache for the most recent data parts. void prewarmMarkCache(ThreadPool & pool); + void prewarmMarkCacheIfNeeded(ThreadPool & pool); String getLogName() const { return log.loadName(); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 40cd6e01dba..1ba0617d8ae 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -155,7 +155,7 @@ StorageMergeTree::StorageMergeTree( loadMutations(); loadDeduplicationLog(); - prewarmMarkCache(getActivePartsLoadingThreadPool().get()); + prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get()); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bbfedb2f355..15341cca976 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -509,7 +509,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } loadDataParts(skip_sanity_checks, expected_parts_on_this_replica); - prewarmMarkCache(getActivePartsLoadingThreadPool().get()); + prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get()); if (LoadingStrictnessLevel::ATTACH <= mode) { diff --git a/tests/queries/0_stateless/03254_system_prewarm_mark_cache.reference b/tests/queries/0_stateless/03254_system_prewarm_mark_cache.reference new file mode 100644 index 00000000000..86674e7765a --- /dev/null +++ b/tests/queries/0_stateless/03254_system_prewarm_mark_cache.reference @@ -0,0 +1,4 @@ +20000 +20000 +1 +0 diff --git a/tests/queries/0_stateless/03254_system_prewarm_mark_cache.sql b/tests/queries/0_stateless/03254_system_prewarm_mark_cache.sql new file mode 100644 index 00000000000..f9e77365836 --- /dev/null +++ b/tests/queries/0_stateless/03254_system_prewarm_mark_cache.sql @@ -0,0 +1,27 @@ +-- Tags: no-parallel, no-shared-merge-tree + +DROP TABLE IF EXISTS t_prewarm_cache; + +CREATE TABLE t_prewarm_cache (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03254_prewarm_mark_cache_smt/t_prewarm_cache', '1') +ORDER BY a SETTINGS prewarm_mark_cache = 0; + +SYSTEM DROP MARK CACHE; + +INSERT INTO t_prewarm_cache SELECT number, rand(), rand() FROM numbers(20000); + +SELECT count() FROM t_prewarm_cache WHERE NOT ignore(*); + +SYSTEM DROP MARK CACHE; + +SYSTEM PREWARM MARK CACHE t_prewarm_cache; + +SELECT count() FROM t_prewarm_cache WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['LoadedMarksCount'] > 0 FROM system.query_log +WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache%' +ORDER BY event_time_microseconds; + +DROP TABLE IF EXISTS t_prewarm_cache; From 47ddd7fb6b230e0d9b0d2341e118bd88ba871d07 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 1 Nov 2024 14:33:03 +0000 Subject: [PATCH 728/816] Check suspicious and experimental types in JSON type hints --- src/DataTypes/DataTypeObject.cpp | 9 +++++++++ src/DataTypes/DataTypeObject.h | 2 ++ .../0_stateless/03261_json_hints_types_check.reference | 0 .../queries/0_stateless/03261_json_hints_types_check.sql | 9 +++++++++ 4 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03261_json_hints_types_check.reference create mode 100644 tests/queries/0_stateless/03261_json_hints_types_check.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 18bfed9c5c3..69ae9b8e906 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -230,6 +230,15 @@ MutableColumnPtr DataTypeObject::createColumn() const return ColumnObject::create(std::move(typed_path_columns), max_dynamic_paths, max_dynamic_types); } +void DataTypeObject::forEachChild(const ChildCallback & callback) const +{ + for (const auto & [path, type] : typed_paths) + { + callback(*type); + type->forEachChild(callback); + } +} + namespace { diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 7eb2e7729de..9321570fb75 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -50,6 +50,8 @@ public: bool equals(const IDataType & rhs) const override; + void forEachChild(const ChildCallback &) const override; + bool hasDynamicSubcolumnsData() const override { return true; } std::unique_ptr getDynamicSubcolumnData(std::string_view subcolumn_name, const SubstreamData & data, bool throw_if_null) const override; diff --git a/tests/queries/0_stateless/03261_json_hints_types_check.reference b/tests/queries/0_stateless/03261_json_hints_types_check.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03261_json_hints_types_check.sql b/tests/queries/0_stateless/03261_json_hints_types_check.sql new file mode 100644 index 00000000000..a407aa9474b --- /dev/null +++ b/tests/queries/0_stateless/03261_json_hints_types_check.sql @@ -0,0 +1,9 @@ +set allow_experimental_json_type=1; +set allow_experimental_variant_type=0; +set allow_experimental_object_type=0; + +select '{}'::JSON(a LowCardinality(Int128)); -- {serverError SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY} +select '{}'::JSON(a FixedString(100000)); -- {serverError ILLEGAL_COLUMN} +select '{}'::JSON(a Variant(Int32)); -- {serverError ILLEGAL_COLUMN} +select '{}'::JSON(a Object('json')); -- {serverError ILLEGAL_COLUMN} + From 3fb4836f635a92ca59eda9dda519c8a466428bf9 Mon Sep 17 00:00:00 2001 From: Alexandre Snarskii Date: Fri, 1 Nov 2024 19:21:54 +0300 Subject: [PATCH 729/816] memory_worker shall be started on non-Linux OS too --- programs/server/Server.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1f481381b2b..5159f95419e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1353,9 +1353,11 @@ try } FailPointInjection::enableFromGlobalConfig(config()); +#endif memory_worker.start(); +#if defined(OS_LINUX) int default_oom_score = 0; #if !defined(NDEBUG) From e23dc25863f49b419bc6ce28463a13bd8ad38277 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 1 Nov 2024 16:22:16 +0100 Subject: [PATCH 730/816] Done --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7ea388f18dd..49f6acff57b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -65,6 +65,7 @@ static std::initializer_list Date: Fri, 1 Nov 2024 17:09:00 +0000 Subject: [PATCH 731/816] Fix tests --- tests/queries/0_stateless/03214_json_typed_dynamic_path.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql b/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql index 1f6a025825a..eee3d70b8da 100644 --- a/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql +++ b/tests/queries/0_stateless/03214_json_typed_dynamic_path.sql @@ -1,6 +1,7 @@ -- Tags: no-fasttest set allow_experimental_json_type = 1; +set allow_experimental_dynamic_type = 1; drop table if exists test; create table test (json JSON(a Dynamic)) engine=MergeTree order by tuple() settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; insert into test select '{"a" : 42}'; From 22e48f6852adcb3b3092b9b5a9e78674d52c0997 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Fri, 1 Nov 2024 18:16:16 +0100 Subject: [PATCH 732/816] Update 03261_tuple_map_object_to_json_cast.sql --- .../queries/0_stateless/03261_tuple_map_object_to_json_cast.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql index c0199452843..91d3f504f92 100644 --- a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql @@ -4,6 +4,7 @@ set allow_experimental_json_type = 1; set allow_experimental_object_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; +set enable_named_columns_in_function_tuple = 1; select 'Map to JSON'; select map('a', number::UInt32, 'b', toDate(number), 'c', range(number), 'd', [map('e', number::UInt32)])::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); From bbde6ba51224c43cf88978adb92cd1a72b767313 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 1 Nov 2024 17:53:32 +0000 Subject: [PATCH 733/816] update test --- ...rallel_replicas_join_algo_and_analyzer_1.sh | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh index 1d43f540138..8d54c2eed13 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh @@ -27,6 +27,8 @@ inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 SETTINGS allow_experimental_analyzer=1" +PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join = 0" + ############## echo echo "simple (global) join with analyzer and parallel replicas" @@ -35,17 +37,13 @@ $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0, parallel_replicas_local_plan=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level='trace', -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0, parallel_replicas_local_plan=0" 2>&1 | +SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -57,17 +55,13 @@ $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0, parallel_replicas_local_plan=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=1" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS allow_experimental_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, send_logs_level='trace', -max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0, parallel_replicas_local_plan=1" 2>&1 | +SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' From 7315ad482052f50a98ea2eda433df34353e8d8d0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 1 Nov 2024 17:55:49 +0000 Subject: [PATCH 734/816] Polishing --- .../02967_parallel_replicas_join_algo_and_analyzer_1.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh index 8d54c2eed13..d315257dbac 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh @@ -27,7 +27,7 @@ inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 SETTINGS allow_experimental_analyzer=1" -PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join = 0" +PARALLEL_REPLICAS_SETTINGS="enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join = 0" ############## echo From b3b245e3b859fcd96cf3178afddaca1847ac5dcb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 1 Nov 2024 15:37:48 -0300 Subject: [PATCH 735/816] Update anylast.md --- .../en/sql-reference/aggregate-functions/reference/anylast.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index 202d2e9fb10..4fe21531c76 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -17,7 +17,7 @@ anyLast(column) [RESPECT NULLS] - `column`: The column name. :::note -Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. +Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the last value passed, regardless of whether it is `NULL` or not. ::: **Returned value** @@ -40,4 +40,4 @@ SELECT anyLast(city) FROM any_last_nulls; ┌─anyLast(city)─┐ │ Valencia │ └───────────────┘ -``` \ No newline at end of file +``` From 9d0f256dfe87d0b914655570513e64f167cadeb0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Nov 2024 12:17:40 +0000 Subject: [PATCH 736/816] Enable SimSIMD backend in Usearch --- contrib/SimSIMD | 2 +- contrib/SimSIMD-cmake/CMakeLists.txt | 10 +++-- contrib/usearch-cmake/CMakeLists.txt | 64 +++++++++++++++++++++++++--- 3 files changed, 64 insertions(+), 12 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 935fef2964b..d7798ac6cb7 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 935fef2964bc38e995c5f465b42259a35b8cf0d3 +Subproject commit d7798ac6cb78ac1cb1cdc590f391643f983a2fd7 diff --git a/contrib/SimSIMD-cmake/CMakeLists.txt b/contrib/SimSIMD-cmake/CMakeLists.txt index f5dc4d63604..1d434490c7c 100644 --- a/contrib/SimSIMD-cmake/CMakeLists.txt +++ b/contrib/SimSIMD-cmake/CMakeLists.txt @@ -1,4 +1,6 @@ -set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") - -add_library(_simsimd INTERFACE) -target_include_directories(_simsimd SYSTEM INTERFACE "${SIMSIMD_PROJECT_DIR}/include") +# See contrib/usearch-cmake/CMakeLists.txt, why only enabled on x86 +if (ARCH_AMD64) + set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") + add_library(_simsimd INTERFACE) + target_include_directories(_simsimd SYSTEM INTERFACE "${SIMSIMD_PROJECT_DIR}/include") +endif() diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 25f6ca82a74..69a986de192 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -6,12 +6,62 @@ target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/incl target_link_libraries(_usearch INTERFACE _fp16) target_compile_definitions(_usearch INTERFACE USEARCH_USE_FP16LIB) -# target_compile_definitions(_usearch INTERFACE USEARCH_USE_SIMSIMD) -# ^^ simsimd is not enabled at the moment. Reasons: -# - Vectorization is important for raw scans but not so much for HNSW. We use usearch only for HNSW. -# - Simsimd does compile-time dispatch (choice of SIMD kernels determined by capabilities of the build machine) or dynamic dispatch (SIMD -# kernels chosen at runtime based on cpuid instruction). Since current builds are limited to SSE 4.2 (x86) and NEON (ARM), the speedup of -# the former would be moderate compared to AVX-512 / SVE. The latter is at the moment too fragile with respect to portability across x86 -# and ARM machines ... certain conbinations of quantizations / distance functions / SIMD instructions are not implemented at the moment. +# Only x86 for now. On ARM, the linker goes down in flames. To make SimSIMD compile, I had to remove a macro checks in SimSIMD +# for AVX512 (x86, worked nicely) and __ARM_BF16_FORMAT_ALTERNATIVE. It is probably because of that. +if (ARCH_AMD64) + target_link_libraries(_usearch INTERFACE _simsimd) + target_compile_definitions(_usearch INTERFACE USEARCH_USE_SIMSIMD) + + target_compile_definitions(_usearch INTERFACE USEARCH_CAN_COMPILE_FLOAT16) + target_compile_definitions(_usearch INTERFACE USEARCH_CAN_COMPILE_BF16) +endif () add_library(ch_contrib::usearch ALIAS _usearch) + + + +# LLVM ERROR: Cannot select: 0x7996e7a73150: f32,ch = load<(load (s16) from %ir.22, !tbaa !54231), anyext from bf16> 0x79961cb737c0, 0x7996e7a1a500, undef:i64, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 +# 0x7996e7a1a500: i64 = add 0x79961e770d00, Constant:i64<-16>, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 +# 0x79961e770d00: i64,ch = CopyFromReg 0x79961cb737c0, Register:i64 %4, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 +# 0x7996e7a1ae10: i64 = Register %4 +# 0x7996e7a1b5f0: i64 = Constant<-16> +# 0x7996e7a1a730: i64 = undef +# In function: _ZL23simsimd_dot_bf16_serialPKu6__bf16S0_yPd +# PLEASE submit a bug report to https://github.com/llvm/llvm-project/issues/ and include the crash backtrace. +# Stack dump: +# 0. Running pass 'Function Pass Manager' on module 'src/libdbms.a(MergeTreeIndexVectorSimilarity.cpp.o at 2312737440)'. +# 1. Running pass 'AArch64 Instruction Selection' on function '@_ZL23simsimd_dot_bf16_serialPKu6__bf16S0_yPd' +# #0 0x00007999e83a63bf llvm::sys::PrintStackTrace(llvm::raw_ostream&, int) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda63bf) +# #1 0x00007999e83a44f9 llvm::sys::RunSignalHandlers() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda44f9) +# #2 0x00007999e83a6b00 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda6b00) +# #3 0x00007999e6e45320 (/lib/x86_64-linux-gnu/libc.so.6+0x45320) +# #4 0x00007999e6e9eb1c pthread_kill (/lib/x86_64-linux-gnu/libc.so.6+0x9eb1c) +# #5 0x00007999e6e4526e raise (/lib/x86_64-linux-gnu/libc.so.6+0x4526e) +# #6 0x00007999e6e288ff abort (/lib/x86_64-linux-gnu/libc.so.6+0x288ff) +# #7 0x00007999e82fe0c2 llvm::report_fatal_error(llvm::Twine const&, bool) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xcfe0c2) +# #8 0x00007999e8c2f8e3 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162f8e3) +# #9 0x00007999e8c2ed76 llvm::SelectionDAGISel::SelectCodeCommon(llvm::SDNode*, unsigned char const*, unsigned int) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162ed76) +# #10 0x00007999ea1adbcb (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x2badbcb) +# #11 0x00007999e8c2611f llvm::SelectionDAGISel::DoInstructionSelection() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162611f) +# #12 0x00007999e8c25790 llvm::SelectionDAGISel::CodeGenAndEmitDAG() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x1625790) +# #13 0x00007999e8c248de llvm::SelectionDAGISel::SelectAllBasicBlocks(llvm::Function const&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x16248de) +# #14 0x00007999e8c22934 llvm::SelectionDAGISel::runOnMachineFunction(llvm::MachineFunction&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x1622934) +# #15 0x00007999e87826b9 llvm::MachineFunctionPass::runOnFunction(llvm::Function&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x11826b9) +# #16 0x00007999e84f7772 llvm::FPPassManager::runOnFunction(llvm::Function&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xef7772) +# #17 0x00007999e84fd2f4 llvm::FPPassManager::runOnModule(llvm::Module&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xefd2f4) +# #18 0x00007999e84f7e9f llvm::legacy::PassManagerImpl::run(llvm::Module&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xef7e9f) +# #19 0x00007999e99f7d61 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f7d61) +# #20 0x00007999e99f8c91 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f8c91) +# #21 0x00007999e99f8b10 llvm::lto::thinBackend(llvm::lto::Config const&, unsigned int, std::function>> (unsigned int, llvm::Twine const&)>, llvm::Module&, llvm::ModuleSummaryIndex const&, llvm::DenseMap, std::equal_to, std::allocator>, llvm::DenseMapInfo, llvm::detail::DenseMapPair, std::equal_to, std::allocator>>> const&, llvm::DenseMap, llvm::detail::DenseMapPair> const&, llvm::MapVector, llvm::detail::DenseMapPair>, llvm::SmallVector, 0u>>*, std::vector> const&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f8b10) +# #22 0x00007999e99f248d (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f248d) +# #23 0x00007999e99f1cd6 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f1cd6) +# #24 0x00007999e82c9beb (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xcc9beb) +# #25 0x00007999e834ebe3 llvm::ThreadPool::processTasks(llvm::ThreadPoolTaskGroup*) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xd4ebe3) +# #26 0x00007999e834f704 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xd4f704) +# #27 0x00007999e6e9ca94 (/lib/x86_64-linux-gnu/libc.so.6+0x9ca94) +# #28 0x00007999e6f29c3c (/lib/x86_64-linux-gnu/libc.so.6+0x129c3c) +# clang++-18: error: unable to execute command: Aborted (core dumped) +# clang++-18: error: linker command failed due to signal (use -v to see invocation) +# ^[[A^Cninja: build stopped: interrupted by user. From 3a042c080473957ffe40c5e299b06714868ab841 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 1 Nov 2024 12:55:02 +0000 Subject: [PATCH 737/816] Enable dynamic dispatch in SimSIMD --- contrib/SimSIMD-cmake/CMakeLists.txt | 6 ++++-- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD-cmake/CMakeLists.txt b/contrib/SimSIMD-cmake/CMakeLists.txt index 1d434490c7c..8350417479a 100644 --- a/contrib/SimSIMD-cmake/CMakeLists.txt +++ b/contrib/SimSIMD-cmake/CMakeLists.txt @@ -1,6 +1,8 @@ # See contrib/usearch-cmake/CMakeLists.txt, why only enabled on x86 if (ARCH_AMD64) set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") - add_library(_simsimd INTERFACE) - target_include_directories(_simsimd SYSTEM INTERFACE "${SIMSIMD_PROJECT_DIR}/include") + set(SIMSIMD_SRCS ${SIMSIMD_PROJECT_DIR}/c/lib.c) + add_library(_simsimd ${SIMSIMD_SRCS}) + target_include_directories(_simsimd SYSTEM PUBLIC "${SIMSIMD_PROJECT_DIR}/include") + target_compile_definitions(_simsimd PUBLIC SIMSIMD_DYNAMIC_DISPATCH) endif() diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 5a725922e14..0b5ffa659dc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -118,6 +118,8 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( if (!result) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release())); swap(result.index); + + /// LOG_TRACE(getLogger("XXX"), "{}", simsimd_uses_dynamic_dispatch()); } void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const From a4e576924b16ed199e3726313f96c241b604d4b6 Mon Sep 17 00:00:00 2001 From: 0xMihalich Date: Sat, 2 Nov 2024 18:48:57 +1000 Subject: [PATCH 738/816] Fix: ERROR: column "attgenerated" does not exist for old PostgreSQL databases Restore support for GreenPlum and older versions of PostgreSQL without affecting existing functionality. --- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 45fd52f27ab..5268dbcb59f 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -307,6 +307,13 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( if (!columns.empty()) columns_part = fmt::format(" AND attname IN ('{}')", boost::algorithm::join(columns, "','")); + /// Bypassing the error of the missing column `attgenerated` in the system table `pg_attribute` for PostgreSQL versions below 12. + /// This trick involves executing a special query to the DBMS in advance to obtain the correct line with comment /// if column has GENERATED. + /// The result of the query will be the name of the column `attgenerated` or an empty string declaration for PostgreSQL version 11 and below. + /// This change does not degrade the function's performance but restores support for older versions and fix ERROR: column "attgenerated" does not exist. + pqxx::result gen_result{tx.exec("select case when current_setting('server_version_num')::int < 120000 then '''''' else 'attgenerated' end as generated")}; + std::string generated = gen_result[0][0].as(); + std::string query = fmt::format( "SELECT attname AS name, " /// column name "format_type(atttypid, atttypmod) AS type, " /// data type @@ -315,11 +322,11 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "atttypid as type_id, " "atttypmod as type_modifier, " "attnum as att_num, " - "attgenerated as generated " /// if column has GENERATED + "{} as generated " /// if column has GENERATED "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) {}" "AND NOT attisdropped AND attnum > 0 " - "ORDER BY attnum ASC", where, columns_part); + "ORDER BY attnum ASC", generated, where, columns_part); /// Now we use variable `generated` to form query string. End of trick. auto postgres_table_with_schema = postgres_schema.empty() ? postgres_table : doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(postgres_table); table.physical_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, false); From b876d52e89ba6f28a71acbb7af3d43c7879c7dc4 Mon Sep 17 00:00:00 2001 From: Plasmaion <150329062+Plasmaion@users.noreply.github.com> Date: Sat, 2 Nov 2024 15:40:18 +0300 Subject: [PATCH 739/816] Update install.md (comment) typo in word "or" :) --- docs/ru/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index f8a660fbec9..083ddc8c39c 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -95,7 +95,7 @@ sudo yum install -y clickhouse-server clickhouse-client sudo systemctl enable clickhouse-server sudo systemctl start clickhouse-server sudo systemctl status clickhouse-server -clickhouse-client # илм "clickhouse-client --password" если установлен пароль +clickhouse-client # или "clickhouse-client --password" если установлен пароль ``` Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`. From 64b405254c0c7dbe2217bd6251f3767556d01d75 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 2 Nov 2024 19:50:45 +0000 Subject: [PATCH 740/816] Fix --- .../02967_parallel_replicas_join_algo_and_analyzer_1.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh index d315257dbac..a6e755ebc35 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_1.sh @@ -37,13 +37,13 @@ $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_local_plan=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=0" 2>&1 | +SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_local_plan=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -55,13 +55,13 @@ $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=1" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_local_plan=1" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2) r on l.key = r.key order by l.key limit 10 offset 700000 -SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTING, parallel_replicas_local_plan=1" 2>&1 | +SETTINGS enable_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_local_plan=1" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' From 1d83bb2ddaeab407af0fa7d93307bb2465568b2b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 3 Nov 2024 07:39:38 +0000 Subject: [PATCH 741/816] Update settings changes history --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 317037070fc..9f314788505 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -71,6 +71,7 @@ static std::initializer_list Date: Sun, 3 Nov 2024 13:20:27 +0100 Subject: [PATCH 742/816] Docs: An attempt to fix the missing sidebar for TPC-H/DS and SSB benchmark docs See https://github.com/ClickHouse/clickhouse-docs/issues/2721 --- docs/en/getting-started/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/getting-started/index.md b/docs/en/getting-started/index.md index b520220984c..7898ca01129 100644 --- a/docs/en/getting-started/index.md +++ b/docs/en/getting-started/index.md @@ -23,6 +23,7 @@ functions in ClickHouse. The sample datasets include: - The [NYPD Complaint Data](../getting-started/example-datasets/nypd_complaint_data.md) demonstrates how to use data inference to simplify creating tables - The ["What's on the Menu?" dataset](../getting-started/example-datasets/menus.md) has an example of denormalizing data - The [Laion dataset](../getting-started/example-datasets/laion.md) has an example of [Approximate nearest neighbor search indexes](../engines/table-engines/mergetree-family/annindexes.md) usage +- The [TPC-H](../getting-started/example-datasets/tpch.md), [TPC-DS](../getting-started/example-datasets/tpcds.md), and [Star Schema (SSB)](../getting-started/example-datasets/star-schema.md) industry benchmarks for analytics databases - [Getting Data Into ClickHouse - Part 1](https://clickhouse.com/blog/getting-data-into-clickhouse-part-1) provides examples of defining a schema and loading a small Hacker News dataset - [Getting Data Into ClickHouse - Part 3 - Using S3](https://clickhouse.com/blog/getting-data-into-clickhouse-part-3-s3) has examples of loading data from s3 - [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse) shows how to generate random data if none of the above fit your needs. From a801ece2804801f640f0bfaed2d11974bd3fb376 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 3 Nov 2024 15:10:26 +0000 Subject: [PATCH 743/816] Fix test --- .../0_stateless/02354_vector_search_expansion_search.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02354_vector_search_expansion_search.sql b/tests/queries/0_stateless/02354_vector_search_expansion_search.sql index fcbe9ee42b9..f0cd5374be7 100644 --- a/tests/queries/0_stateless/02354_vector_search_expansion_search.sql +++ b/tests/queries/0_stateless/02354_vector_search_expansion_search.sql @@ -14,7 +14,7 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar -- Generate random values but with a fixed seed (conceptually), so that the data is deterministic. -- Unfortunately, no random functions in ClickHouse accepts a seed. Instead, abuse the numbers table + hash functions to provide -- deterministic randomness. -INSERT INTO tab SELECT number, [sipHash64(number)/18446744073709551615, wyHash64(number)/18446744073709551615] FROM numbers(370000); -- 18446744073709551615 is the biggest UInt64 +INSERT INTO tab SELECT number, [sipHash64(number)/18446744073709551615, wyHash64(number)/18446744073709551615] FROM numbers(660000); -- 18446744073709551615 is the biggest UInt64 -- hnsw_candidate_list_size_for_search = 0 is illegal WITH [0.5, 0.5] AS reference_vec From 27241b484f8c26197ec4329212a8a5ef11d02007 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 3 Nov 2024 16:00:33 +0000 Subject: [PATCH 744/816] Fix linker warning --- contrib/usearch | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/usearch b/contrib/usearch index 53799b84ca9..7efe8b710c9 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 53799b84ca9ad708b060d0b1cfa5f039371721cd +Subproject commit 7efe8b710c9831bfe06573b1df0fad001b04a2b5 From 27049f2cb599b4f93ae327783ab0cc588bef7dd1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 3 Nov 2024 19:16:35 +0000 Subject: [PATCH 745/816] Demote log level for failed authentication --- src/Access/AccessControl.cpp | 7 ++++--- src/Common/Exception.cpp | 31 +++++++++++++++++++++---------- src/Common/Exception.h | 9 +++++---- src/Server/TCPHandler.cpp | 3 ++- 4 files changed, 32 insertions(+), 18 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index e8ee363be1a..9b3b8d2a977 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -608,7 +608,7 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po } catch (...) { - tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed"); + tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::information); WriteBufferFromOwnString message; message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name."; @@ -622,8 +622,9 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po << "and deleting this file will reset the password.\n" << "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed.\n\n"; - /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons, - /// only the log will show the exact reason. + /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons. + /// Only the log ((*), above) will show the exact reason. Note that (*) logs at information level instead of the default error level as + /// authentication failures are not an unusual event. throw Exception(PreformattedMessage{message.str(), "{}: Authentication failed: password is incorrect, or there is no user with such name", std::vector{credentials.getUserName()}}, diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 320fc06cb2f..644c9a19738 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -251,7 +251,7 @@ void Exception::setThreadFramePointers(ThreadFramePointersBase frame_pointers) thread_frame_pointers.frame_pointers = std::move(frame_pointers); } -static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message) +static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message, LogsLevel level) { if (!isLoggingEnabled()) return; @@ -262,14 +262,25 @@ static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string if (!start_of_message.empty()) message.text = fmt::format("{}: {}", start_of_message, message.text); - LOG_ERROR(logger, message); + switch (level) + { + case LogsLevel::none: break; + case LogsLevel::test: LOG_TEST(logger, message); break; + case LogsLevel::trace: LOG_TRACE(logger, message); break; + case LogsLevel::debug: LOG_DEBUG(logger, message); break; + case LogsLevel::information: LOG_INFO(logger, message); break; + case LogsLevel::warning: LOG_WARNING(logger, message); break; + case LogsLevel::error: LOG_ERROR(logger, message); break; + case LogsLevel::fatal: LOG_FATAL(logger, message); break; + } + } catch (...) // NOLINT(bugprone-empty-catch) { } } -void tryLogCurrentException(const char * log_name, const std::string & start_of_message) +void tryLogCurrentException(const char * log_name, const std::string & start_of_message, LogsLevel level) { if (!isLoggingEnabled()) return; @@ -283,10 +294,10 @@ void tryLogCurrentException(const char * log_name, const std::string & start_of_ /// getLogger can allocate memory too auto logger = getLogger(log_name); - tryLogCurrentExceptionImpl(logger.get(), start_of_message); + tryLogCurrentExceptionImpl(logger.get(), start_of_message, level); } -void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message) +void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message, LogsLevel level) { /// Under high memory pressure, new allocations throw a /// MEMORY_LIMIT_EXCEEDED exception. @@ -295,17 +306,17 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_ /// MemoryTracker until the exception will be logged. LockMemoryExceptionInThread lock_memory_tracker(VariableContext::Global); - tryLogCurrentExceptionImpl(logger, start_of_message); + tryLogCurrentExceptionImpl(logger, start_of_message, level); } -void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message) +void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message, LogsLevel level) { - tryLogCurrentException(logger.get(), start_of_message); + tryLogCurrentException(logger.get(), start_of_message, level); } -void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message) +void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message, LogsLevel level) { - tryLogCurrentException(logger.load(), start_of_message); + tryLogCurrentException(logger.load(), start_of_message, level); } static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 8ec640ff642..edc1b95bca4 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -276,10 +277,10 @@ using Exceptions = std::vector; * Can be used in destructors in the catch-all block. */ /// TODO: Logger leak constexpr overload -void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); -void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); -void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = ""); -void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = ""); +void tryLogCurrentException(const char * log_name, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error); +void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error); +void tryLogCurrentException(LoggerPtr logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error); +void tryLogCurrentException(const AtomicLogger & logger, const std::string & start_of_message = "", LogsLevel level = LogsLevel::error); /** Prints current exception in canonical format. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e7e4ae25a68..ea5507c3155 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1614,7 +1614,8 @@ void TCPHandler::receiveHello() if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED) throw; - tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication"); + tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication", LogsLevel::debug); + /// ^^ Log at debug level instead of default error level as authentication failures are not an unusual event. } } } From 7f1ccc30c9e192a00ca624bcfcd05c9b2837d27d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 3 Nov 2024 21:19:27 +0000 Subject: [PATCH 746/816] Try to suppress msan warnings --- contrib/SimSIMD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index d7798ac6cb7..c03d065a766 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit d7798ac6cb78ac1cb1cdc590f391643f983a2fd7 +Subproject commit c03d065a7661004a9a18fe52753efafa170c67f9 From 5aba66e50a98f040daaa3c2235310e68cfa45e55 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 4 Nov 2024 03:13:42 +0000 Subject: [PATCH 747/816] adjust CI timeout, use TIMEOUT variable for setting fuzzers timeout --- docker/test/libfuzzer/Dockerfile | 2 -- tests/ci/ci_config.py | 2 +- tests/ci/libfuzzer_test_check.py | 3 +++ tests/fuzz/runner.py | 8 ++------ 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index 3ffae0cd921..46e305c90ab 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -33,8 +33,6 @@ RUN apt-get update \ COPY requirements.txt / RUN pip3 install --no-cache-dir -r /requirements.txt -ENV FUZZER_ARGS="-max_total_time=60" - SHELL ["/bin/bash", "-c"] # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b4b7dbee59c..80da822652f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -530,7 +530,7 @@ class CI: JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], run_by_labels=[Tags.libFuzzer], - timeout=10800, + timeout=5400, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, ), diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 379d681cb3e..d0936eb2323 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -22,6 +22,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen NO_CHANGES_MSG = "Nothing to run" +TIMEOUT = 60 s3 = S3Helper() @@ -264,6 +265,8 @@ def main(): check_name, run_by_hash_num, run_by_hash_total ) + additional_envs.append(f"TIMEOUT={TIMEOUT}") + ci_logs_credentials = CiLogsCredentials(Path(temp_path) / "export-logs-config.sh") ci_logs_args = ci_logs_credentials.get_docker_arguments( pr_info, stopwatch.start_time_str, check_name diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index af73a989ec3..0880940aabd 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -9,7 +9,7 @@ import subprocess from pathlib import Path DEBUGGER = os.getenv("DEBUGGER", "") -FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") +TIMEOUT = int(os.getenv("TIMEOUT", "0")) OUTPUT = "/test_output" @@ -150,11 +150,7 @@ def main(): subprocess.check_call("ls -al", shell=True) - timeout = 60 - - match = re.search(r"(^|\s+)-max_total_time=(\d+)($|\s)", FUZZER_ARGS) - if match: - timeout = int(match.group(2)) + timeout = 30 if TIMEOUT == 0 else TIMEOUT with Path() as current: for fuzzer in current.iterdir(): From e2d64ea30254ce7e126c4442fe393429cfbd1c21 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 4 Nov 2024 03:37:46 +0000 Subject: [PATCH 748/816] fix style --- tests/fuzz/runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 0880940aabd..f4c66e00117 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -4,7 +4,6 @@ import configparser import datetime import logging import os -import re import subprocess from pathlib import Path From a6c98a4a7f6c650c84dc750972176427a6e8c479 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 4 Nov 2024 05:17:46 +0000 Subject: [PATCH 749/816] take some changes from private --- tests/ci/s3_helper.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 46c206f0540..ced6d29e5c7 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -322,17 +322,23 @@ class S3Helper: return result def list_prefix_non_recursive( - self, s3_prefix_path: str, bucket: str = S3_BUILDS_BUCKET + self, + s3_prefix_path: str, + bucket: str = S3_BUILDS_BUCKET, + only_dirs: bool = False, ) -> List[str]: paginator = self.client.get_paginator("list_objects_v2") - pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path) + pages = paginator.paginate( + Bucket=bucket, Prefix=s3_prefix_path, Delimiter="/" + ) result = [] for page in pages: - if "Contents" in page: + if not only_dirs and "Contents" in page: for obj in page["Contents"]: - if "/" not in obj["Key"][len(s3_prefix_path) + 1 :]: - result.append(obj["Key"]) - + result.append(obj["Key"]) + if "CommonPrefixes" in page: + for obj in page["CommonPrefixes"]: + result.append(obj["Prefix"]) return result def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str: From 94c8e6e6c201194fc6eea0784e9200fdf5d639a4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 4 Nov 2024 05:31:15 +0000 Subject: [PATCH 750/816] Automatic style fix --- tests/ci/s3_helper.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index ced6d29e5c7..d0aa034258a 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -328,9 +328,7 @@ class S3Helper: only_dirs: bool = False, ) -> List[str]: paginator = self.client.get_paginator("list_objects_v2") - pages = paginator.paginate( - Bucket=bucket, Prefix=s3_prefix_path, Delimiter="/" - ) + pages = paginator.paginate(Bucket=bucket, Prefix=s3_prefix_path, Delimiter="/") result = [] for page in pages: if not only_dirs and "Contents" in page: From 12c21dc7df4ea2a538a1c59bfa7eb05dd76df08d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 4 Nov 2024 09:00:01 +0000 Subject: [PATCH 751/816] Minor fixups --- contrib/SimSIMD | 2 +- .../0_stateless/02354_vector_search_expansion_search.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index c03d065a766..ee3c9c9c00b 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit c03d065a7661004a9a18fe52753efafa170c67f9 +Subproject commit ee3c9c9c00b51645f62a1a9e99611b78c0052a21 diff --git a/tests/queries/0_stateless/02354_vector_search_expansion_search.sql b/tests/queries/0_stateless/02354_vector_search_expansion_search.sql index f0cd5374be7..427148b829f 100644 --- a/tests/queries/0_stateless/02354_vector_search_expansion_search.sql +++ b/tests/queries/0_stateless/02354_vector_search_expansion_search.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, long, no-asan, no-asan, no-ubsan, no-debug +-- Tags: no-fasttest, long, no-asan, no-ubsan, no-debug -- ^^ Disable test for slow builds: generating data takes time but a sufficiently large data set -- is necessary for different hnsw_candidate_list_size_for_search settings to make a difference From 6471034082e931a602fafd2530b218d4b1d386b3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 4 Nov 2024 13:02:58 +0100 Subject: [PATCH 752/816] impl --- base/base/StringRef.h | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index af3441c2a75..ee62be2c4eb 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -86,7 +86,7 @@ using StringRefs = std::vector; * For more information, see hash_map_string_2.cpp */ -inline bool compare8(const char * p1, const char * p2) +inline bool compare16(const char * p1, const char * p2) { return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8( _mm_loadu_si128(reinterpret_cast(p1)), @@ -115,7 +115,7 @@ inline bool compare64(const char * p1, const char * p2) #elif defined(__aarch64__) && defined(__ARM_NEON) -inline bool compare8(const char * p1, const char * p2) +inline bool compare16(const char * p1, const char * p2) { uint64_t mask = getNibbleMask(vceqq_u8( vld1q_u8(reinterpret_cast(p1)), vld1q_u8(reinterpret_cast(p2)))); @@ -185,13 +185,22 @@ inline bool memequalWide(const char * p1, const char * p2, size_t size) switch (size / 16) // NOLINT(bugprone-switch-missing-default-case) { - case 3: if (!compare8(p1 + 32, p2 + 32)) return false; [[fallthrough]]; - case 2: if (!compare8(p1 + 16, p2 + 16)) return false; [[fallthrough]]; - case 1: if (!compare8(p1, p2)) return false; [[fallthrough]]; + case 3: + if (!compare16(p1 + 32, p2 + 32)) + return false; + [[fallthrough]]; + case 2: + if (!compare16(p1 + 16, p2 + 16)) + return false; + [[fallthrough]]; + case 1: + if (!compare16(p1, p2)) + return false; + [[fallthrough]]; default: ; } - return compare8(p1 + size - 16, p2 + size - 16); + return compare16(p1 + size - 16, p2 + size - 16); } #endif From a37c1134b99e75df1df7320c1cad6420d2014a04 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 4 Nov 2024 12:32:14 +0000 Subject: [PATCH 753/816] Resolve issues --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index a72fd16abc2..fd2fe0400bb 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -102,7 +102,7 @@ StorageObjectStorage::StorageObjectStorage( } else { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } From c3471ef20d5a3c375d632bd600438d555cd51595 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 4 Nov 2024 13:33:34 +0100 Subject: [PATCH 754/816] Update AccessControl.cpp --- src/Access/AccessControl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 9b3b8d2a977..647fb238d48 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -608,7 +608,7 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po } catch (...) { - tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::information); + tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::debug); WriteBufferFromOwnString message; message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name."; From 24a7e0f4ee52e47cadd00a41bff80eb3ac614960 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Nov 2024 13:44:36 +0100 Subject: [PATCH 755/816] Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail def test_insert_quorum_with_keeper_loss_connection(): > zero.query( "DROP TABLE IF EXISTS test_insert_quorum_with_keeper_fail ON CLUSTER cluster" ) def query( > return self.client.query( E AttributeError: 'NoneType' object has no attribute 'query' CI: https://s3.amazonaws.com/clickhouse-test-reports/71406/8b3ce129456a1f85839a48538780639e2e3c3020/integration_tests__asan__old_analyzer__[6_6]//home/ubuntu/actions-runner/_work/_temp/test/output_dir/integration_run_parallel3_0.log Signed-off-by: Azat Khuzhin --- tests/integration/test_quorum_inserts/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index eefc4882e8e..66f96d61b3e 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -366,7 +366,7 @@ def test_insert_quorum_with_ttl(started_cluster): zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") -def test_insert_quorum_with_keeper_loss_connection(): +def test_insert_quorum_with_keeper_loss_connection(started_cluster): zero.query( "DROP TABLE IF EXISTS test_insert_quorum_with_keeper_fail ON CLUSTER cluster" ) From 097b45bf5af2d32c4a816c9208c65dab60f2da18 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 4 Nov 2024 13:56:40 +0000 Subject: [PATCH 756/816] small refactoring --- tests/ci/libfuzzer_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index d0936eb2323..2616fbe3f5d 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -21,8 +21,8 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen -NO_CHANGES_MSG = "Nothing to run" TIMEOUT = 60 +NO_CHANGES_MSG = "Nothing to run" s3 = S3Helper() From 978cf9a90525e7162f7841d794ec20d1096c84ed Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 4 Nov 2024 15:32:55 +0100 Subject: [PATCH 757/816] Add per host dashboards to advanced dashboard --- .../System/StorageSystemDashboards.cpp | 490 +++++++++++++++++- 1 file changed, 489 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp index 96ba7e59cf2..340117d1494 100644 --- a/src/Storages/System/StorageSystemDashboards.cpp +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -227,6 +227,194 @@ FROM merge('system', '^metric_log') WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + /// Default per host dashboard for self-managed ClickHouse + { + { "dashboard", "Overview (host)" }, + { "title", "Queries/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_Query) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "CPU Usage (cores)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUVirtualTimeMicroseconds) / 1000000 +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Queries Running" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Query) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Merges Running" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_Merge) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Selected Bytes/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedBytes) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "IO Wait" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSIOWaitMicroseconds) / 1000000 +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "CPU Wait" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "OS CPU Usage (Userspace)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value) +FROM merge('system', '^asynchronous_metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSUserTimeNormalized' +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "OS CPU Usage (Kernel)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value) +FROM merge('system', '^asynchronous_metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'OSSystemTimeNormalized' +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Read From Disk" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadBytes) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Read From Filesystem" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_OSReadChars) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Memory (tracked)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(CurrentMetric_MemoryTracking) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Load Average (15 minutes)" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value) +FROM merge('system', '^asynchronous_metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'LoadAverage15' +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Selected Rows/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_SelectedRows) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Inserted Rows/second" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(ProfileEvent_InsertedRows) +FROM merge('system', '^metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Total MergeTree Parts" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, avg(value) +FROM merge('system', '^asynchronous_metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'TotalPartsOfMergeTreeTables' +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} +)EOQ") } + }, + { + { "dashboard", "Overview (host)" }, + { "title", "Max Parts For Partition" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t, hostname, max(value) +FROM merge('system', '^asynchronous_metric_log') +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} AND metric = 'MaxPartCountForPartition' +GROUP BY t, hostname +ORDER BY t WITH FILL STEP {rounding:UInt32} )EOQ") } }, /// Default dashboard for ClickHouse Cloud @@ -369,7 +557,307 @@ ORDER BY t WITH FILL STEP {rounding:UInt32} { "dashboard", "Cloud overview" }, { "title", "Concurrent network connections" }, { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(TCP_Connections), max(MySQL_Connections), max(HTTP_Connections) FROM (SELECT event_time, sum(CurrentMetric_TCPConnection) AS TCP_Connections, sum(CurrentMetric_MySQLConnection) AS MySQL_Connections, sum(CurrentMetric_HTTPConnection) AS HTTP_Connections FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - } + }, + /// Default per host dashboard for ClickHouse Cloud + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Queries/second" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "CPU Usage (cores)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(metric) / 1000000\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUVirtualTimeMicroseconds) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32} GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Queries Running" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Query) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Merges Running" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_Merge) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Selected Bytes/second" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "IO Wait (local fs)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSIOWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "S3 read wait" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Microseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "S3 read errors/sec" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3RequestsErrors) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "CPU Wait" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSCPUWaitMicroseconds) / 1000000 AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "OS CPU Usage (Userspace, normalized)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSUserTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "OS CPU Usage (Kernel, normalized)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'OSSystemTimeNormalized'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Read From Disk (bytes/sec)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadBytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Read From Filesystem (bytes/sec)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Memory (tracked, bytes)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_MemoryTracking) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Load Average (15 minutes)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric = 'LoadAverage15'\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Selected Rows/sec" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_SelectedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Inserted Rows/sec" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_InsertedRows) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Total MergeTree Parts" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'TotalPartsOfMergeTreeTables'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Max Parts For Partition" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, max(value)\nFROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\nWHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32}\nAND metric = 'MaxPartCountForPartition'\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Read From S3 (bytes/sec)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Filesystem Cache Size" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(CurrentMetric_FilesystemCacheSize) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Disk S3 write req/sec" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT as t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3PutObject + ProfileEvent_DiskS3UploadPart + ProfileEvent_DiskS3CreateMultipartUpload + ProfileEvent_DiskS3CompleteMultipartUpload) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\n GROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Disk S3 read req/sec" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_DiskS3GetObject + ProfileEvent_DiskS3HeadObject + ProfileEvent_DiskS3ListObjects) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t\nWITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "FS cache hit rate" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) / (sum(ProfileEvent_CachedReadBufferReadFromCacheBytes) + sum(ProfileEvent_CachedReadBufferReadFromSourceBytes)) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Page cache hit rate" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n hostname,\n avg(metric)\nFROM (\n SELECT event_time, hostname, greatest(0, (sum(ProfileEvent_OSReadChars) - sum(ProfileEvent_OSReadBytes)) / (sum(ProfileEvent_OSReadChars) + sum(ProfileEvent_ReadBufferFromS3Bytes))) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Network receive bytes/sec" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkReceiveBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Cloud overview (host)" }, + { "title", "Network send bytes/sec" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + /// Distributed cache client metrics start + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Read from Distributed Cache (bytes/sec)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadBytesFromCache) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Read from Distributed Cache fallback buffer (bytes/sec)" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadBytesFromFallbackBuffer) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Read From Filesystem (no Distributed Cache) (bytes/sec)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Read From S3 (no Distributed Cache) (bytes/sec)" }, + { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache read requests" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheReadRequests) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache write requests" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheWriteRequests) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache open connections" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheOpenedConnections) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache registered servers" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheRegisteredServersCurrentAZ) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache read errors" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadErrors) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache make request errors" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheMakeRequestErrors) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache receive response errors" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReceiveResponseErrors) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache registry updates" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheHashRingRebuilds) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache packets" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCachePackets) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + { + { "dashboard", "Distributed cache client overview" }, + { "title", "Distributed Cache unused packets" }, + { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheUnusedPackets) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } + }, + /// Distributed cache client metrics end + /// + /// Distributed cache server metrics start + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache open connections" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(CurrentMetric_DistrCacheServerConnections) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache StartRequest packets" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerStartRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache ContinueRequest packets" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerContinueRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache EndRequest packets" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerEndRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache AckRequest packets" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerAckRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache reused s3 clients" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerReusedS3CachedClients) AS metric FROM clusterAllReplicas(default, system.metric_log) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + { + { "dashboard", "Distributed cache server overview" }, + { "title", "Distributed Cache new s3 clients" }, + { "query", trim(R"EOQ( +SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) +FROM (SELECT event_time, sum(ProfileEvent_DistrCacheNewS3CachedClients) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) +WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} +GROUP BY event_time) +GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 +)EOQ") } + }, + /// Distributed cache server metrics end }; auto add_dashboards = [&](const auto & dashboards) From 1976c399ca8f58283ac97d1a47749cb5e6072649 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 4 Nov 2024 15:34:30 +0100 Subject: [PATCH 758/816] Remove redundant changes --- .../System/StorageSystemDashboards.cpp | 164 ------------------ 1 file changed, 164 deletions(-) diff --git a/src/Storages/System/StorageSystemDashboards.cpp b/src/Storages/System/StorageSystemDashboards.cpp index 340117d1494..27579da4bfe 100644 --- a/src/Storages/System/StorageSystemDashboards.cpp +++ b/src/Storages/System/StorageSystemDashboards.cpp @@ -694,170 +694,6 @@ ORDER BY t WITH FILL STEP {rounding:UInt32} { "title", "Network send bytes/sec" }, { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, hostname, avg(value)\nFROM (\n SELECT event_time, hostname, sum(value) AS value\n FROM clusterAllReplicas(default, merge('system', '^asynchronous_metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n AND metric LIKE 'NetworkSendBytes%'\n GROUP BY event_time, hostname)\nGROUP BY t, hostname\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } }, - /// Distributed cache client metrics start - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Read from Distributed Cache (bytes/sec)" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadBytesFromCache) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Read from Distributed Cache fallback buffer (bytes/sec)" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadBytesFromFallbackBuffer) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Read From Filesystem (no Distributed Cache) (bytes/sec)" }, - { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_OSReadChars) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Read From S3 (no Distributed Cache) (bytes/sec)" }, - { "query", "SELECT \n toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t,\n avg(metric)\nFROM (\n SELECT event_time, sum(ProfileEvent_ReadBufferFromS3Bytes) AS metric \n FROM clusterAllReplicas(default, merge('system', '^metric_log'))\n WHERE event_date >= toDate(now() - {seconds:UInt32})\n AND event_time >= now() - {seconds:UInt32}\n GROUP BY event_time)\nGROUP BY t\nORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache read requests" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheReadRequests) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache write requests" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheWriteRequests) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache open connections" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheOpenedConnections) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache registered servers" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(CurrentMetric_DistrCacheRegisteredServersCurrentAZ) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache read errors" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReadErrors) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache make request errors" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheMakeRequestErrors) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache receive response errors" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheReceiveResponseErrors) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache registry updates" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, max(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheHashRingRebuilds) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache packets" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCachePackets) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - { - { "dashboard", "Distributed cache client overview" }, - { "title", "Distributed Cache unused packets" }, - { "query", "SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) FROM (SELECT event_time, sum(ProfileEvent_DistrCacheUnusedPackets) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} GROUP BY event_time) GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1" } - }, - /// Distributed cache client metrics end - /// - /// Distributed cache server metrics start - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache open connections" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(CurrentMetric_DistrCacheServerConnections) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache StartRequest packets" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerStartRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache ContinueRequest packets" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerContinueRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache EndRequest packets" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerEndRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache AckRequest packets" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerAckRequestPackets) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache reused s3 clients" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheServerReusedS3CachedClients) AS metric FROM clusterAllReplicas(default, system.metric_log) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - { - { "dashboard", "Distributed cache server overview" }, - { "title", "Distributed Cache new s3 clients" }, - { "query", trim(R"EOQ( -SELECT toStartOfInterval(event_time, INTERVAL {rounding:UInt32} SECOND)::INT AS t, avg(metric) -FROM (SELECT event_time, sum(ProfileEvent_DistrCacheNewS3CachedClients) AS metric FROM clusterAllReplicas(default, merge('system', '^metric_log')) -WHERE event_date >= toDate(now() - {seconds:UInt32}) AND event_time >= now() - {seconds:UInt32} -GROUP BY event_time) -GROUP BY t ORDER BY t WITH FILL STEP {rounding:UInt32} SETTINGS skip_unavailable_shards = 1 -)EOQ") } - }, - /// Distributed cache server metrics end }; auto add_dashboards = [&](const auto & dashboards) From 1d888bc1ebc762faf1136d6910fef8641216fb6e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Nov 2024 16:40:26 +0100 Subject: [PATCH 759/816] Fix wrong change --- src/Interpreters/Cache/FileSegment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 080b54feb06..9c8f041fabf 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -139,7 +139,7 @@ FileSegmentGuard::Lock FileSegment::lock() const void FileSegment::setDownloadState(State state, const FileSegmentGuard::Lock & lock) { - if (isCompleted(false)) + if (isCompleted(false) && state != State::DETACHED) { throw Exception( ErrorCodes::LOGICAL_ERROR, From 929da1411e5357d7a99210a4b6f617a2f66f933e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Nov 2024 16:06:20 +0000 Subject: [PATCH 760/816] Fix crash in mongodb table function --- src/TableFunctions/TableFunctionMongoDB.cpp | 10 +++++++--- .../TableFunctionMongoDBPocoLegacy.cpp | 8 +++++--- .../03261_mongodb_argumetns_crash.reference | 0 .../0_stateless/03261_mongodb_argumetns_crash.sql | 13 +++++++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03261_mongodb_argumetns_crash.reference create mode 100644 tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql diff --git a/src/TableFunctions/TableFunctionMongoDB.cpp b/src/TableFunctions/TableFunctionMongoDB.cpp index e13427c1557..966ce858875 100644 --- a/src/TableFunctions/TableFunctionMongoDB.cpp +++ b/src/TableFunctions/TableFunctionMongoDB.cpp @@ -118,14 +118,18 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt if (const auto * ast_func = typeid_cast(args[i].get())) { const auto * args_expr = assert_cast(ast_func->arguments.get()); - auto function_args = args_expr->children; - if (function_args.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); + const auto & function_args = args_expr->children; + if (function_args.size() != 2 || ast_func->name != "equals" || function_args[0]->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); auto arg_name = function_args[0]->as()->name(); if (arg_name == "structure") structure = checkAndGetLiteralArgument(function_args[1], "structure"); + else if (arg_name == "options") + main_arguments.push_back(function_args[1]); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); } else if (i == 2) { diff --git a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp b/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp index dc1df7fcad8..70b28ddfaf0 100644 --- a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp +++ b/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp @@ -98,9 +98,9 @@ void TableFunctionMongoDBPocoLegacy::parseArguments(const ASTPtr & ast_function, if (const auto * ast_func = typeid_cast(args[i].get())) { const auto * args_expr = assert_cast(ast_func->arguments.get()); - auto function_args = args_expr->children; - if (function_args.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); + const auto & function_args = args_expr->children; + if (function_args.size() != 2 || ast_func->name != "equals" || function_args[0]->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); auto arg_name = function_args[0]->as()->name(); @@ -108,6 +108,8 @@ void TableFunctionMongoDBPocoLegacy::parseArguments(const ASTPtr & ast_function, structure = checkAndGetLiteralArgument(function_args[1], "structure"); else if (arg_name == "options") main_arguments.push_back(function_args[1]); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); } else if (i == 5) { diff --git a/tests/queries/0_stateless/03261_mongodb_argumetns_crash.reference b/tests/queries/0_stateless/03261_mongodb_argumetns_crash.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql b/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql new file mode 100644 index 00000000000..830d3995bd5 --- /dev/null +++ b/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest + +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', NULL, 'my_collection', 'test_user', 'password', 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', NULL, 'test_user', 'password', 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', NULL, 'password', 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', NULL, 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', NULL); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', materialize(1) + 1); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', 'x Int32', NULL); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', NULL, 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', NULL, 'x Int32'); -- { serverError BAD_ARGUMENTS } +SELECT * FROM mongodb(NULL, 'test', 'my_collection', 'test_user', 'password', 'x Int32'); -- { serverError BAD_ARGUMENTS } + From 24017bb7add084f38022c2cf1a1fa9a96788ebc9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 4 Nov 2024 17:31:39 +0100 Subject: [PATCH 761/816] add parallel_replicas_prefer_local_join --- ...eplicas_join_algo_and_analyzer_4.reference | 58 +++++++++++++++++++ ...allel_replicas_join_algo_and_analyzer_4.sh | 34 ++++++----- 2 files changed, 77 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference index 8464317f7e6..52c4e872f84 100644 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference @@ -56,3 +56,61 @@ SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP 500030000 500040000 SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` ALL LEFT JOIN (SELECT `__table4`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table4`) AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` GROUP BY `__table1`.`item_id` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP BY `__table1`.`item_id` +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_4551627371769371400_3093038500622465792` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` GROUP BY `__table1`.`item_id` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` +4999950000 +4999950000 +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` +SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP BY `__table1`.`item_id` +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +499950000 +499960000 +499970000 +499980000 +499990000 +500000000 +500010000 +500020000 +500030000 +500040000 +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_4551627371769371400_3093038500622465792` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh index 0e1f07b6ac5..18a2fbd317b 100755 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh @@ -75,23 +75,27 @@ query3=" ORDER BY price_sold " -for prefer_local_plan in {0..1}; do - for query in "${query1}" "${query2}" "${query3}"; do - for enable_parallel_replicas in {0..1}; do - ${CLICKHOUSE_CLIENT} --query=" - set enable_analyzer=1; - set parallel_replicas_local_plan=${prefer_local_plan}; - set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; +for parallel_replicas_prefer_local_join in 1 0; do + for prefer_local_plan in {0..1}; do + for query in "${query1}" "${query2}" "${query3}"; do + for enable_parallel_replicas in {0..1}; do + ${CLICKHOUSE_CLIENT} --query=" + set enable_analyzer=1; + set parallel_replicas_prefer_local_join=${parallel_replicas_prefer_local_join}; + set parallel_replicas_local_plan=${prefer_local_plan}; + set allow_experimental_parallel_reading_from_replicas=${enable_parallel_replicas}, cluster_for_parallel_replicas='parallel_replicas', max_parallel_replicas=100, parallel_replicas_for_non_replicated_merge_tree=1; - ${query}; + --SELECT '----- enable_parallel_replicas=$enable_parallel_replicas prefer_local_plan=$prefer_local_plan parallel_replicas_prefer_local_join=$parallel_replicas_prefer_local_join -----'; + ${query}; - SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') - FROM - ( - EXPLAIN actions=1 ${query} - ) - WHERE explain LIKE '%ParallelReplicas%'; - " + SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') + FROM + ( + EXPLAIN actions=1 ${query} + ) + WHERE explain LIKE '%ParallelReplicas%'; + " + done done done done From 6b4d44be2894bf99897fca011817c9d77bfbabdf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 4 Nov 2024 16:42:06 +0000 Subject: [PATCH 762/816] Update version_date.tsv and changelogs after v24.8.6.70-lts --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.6.70-lts.md | 50 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 6 files changed, 57 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v24.8.6.70-lts.md diff --git a/SECURITY.md b/SECURITY.md index db302da8ecd..1b0648dc489 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,9 +14,10 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.10 | ✔️ | | 24.9 | ✔️ | | 24.8 | ✔️ | -| 24.7 | ✔️ | +| 24.7 | ❌ | | 24.6 | ❌ | | 24.5 | ❌ | | 24.4 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index dfe6a420260..bc76bdbb619 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 991c25ad142..93acf1a5773 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -35,7 +35,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5dc88b49e31..506a627b11c 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.9.2.42" +ARG VERSION="24.10.1.2812" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.6.70-lts.md b/docs/changelogs/v24.8.6.70-lts.md new file mode 100644 index 00000000000..81fa4db1458 --- /dev/null +++ b/docs/changelogs/v24.8.6.70-lts.md @@ -0,0 +1,50 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.6.70-lts (ddb8c219771) FIXME as compared to v24.8.5.115-lts (8c4cb00a384) + +#### Backward Incompatible Change +* Backported in [#71359](https://github.com/ClickHouse/ClickHouse/issues/71359): Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). + +#### Improvement +* Backported in [#70680](https://github.com/ClickHouse/ClickHouse/issues/70680): Don't do validation when synchronizing user_directories from keeper. [#70644](https://github.com/ClickHouse/ClickHouse/pull/70644) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71395](https://github.com/ClickHouse/ClickHouse/issues/71395): Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* Backported in [#71287](https://github.com/ClickHouse/ClickHouse/issues/71287): Reduce the number of object storage HEAD API requests in the plain_rewritable disk. [#70915](https://github.com/ClickHouse/ClickHouse/pull/70915) ([Julia Kartseva](https://github.com/jkartseva)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#70934](https://github.com/ClickHouse/ClickHouse/issues/70934): Fix incorrect JOIN ON section optimization in case of `IS NULL` check under any other function (like `NOT`) that may lead to wrong results. Closes [#67915](https://github.com/ClickHouse/ClickHouse/issues/67915). [#68049](https://github.com/ClickHouse/ClickHouse/pull/68049) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Backported in [#70735](https://github.com/ClickHouse/ClickHouse/issues/70735): Fix unexpected exception when passing empty tuple in array. This fixes [#68618](https://github.com/ClickHouse/ClickHouse/issues/68618). [#68848](https://github.com/ClickHouse/ClickHouse/pull/68848) ([Amos Bird](https://github.com/amosbird)). +* Backported in [#71138](https://github.com/ClickHouse/ClickHouse/issues/71138): Fix propogating structure argument in s3Cluster. Previously the `DEFAULT` expression of the column could be lost when sending the query to the replicas in s3Cluster. [#69147](https://github.com/ClickHouse/ClickHouse/pull/69147) ([Pavel Kruglov](https://github.com/Avogar)). +* Backported in [#70561](https://github.com/ClickHouse/ClickHouse/issues/70561): Fix `getSubcolumn` with `LowCardinality` columns by overriding `useDefaultImplementationForLowCardinalityColumns` to return `true`. [#69831](https://github.com/ClickHouse/ClickHouse/pull/69831) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#70903](https://github.com/ClickHouse/ClickHouse/issues/70903): Avoid reusing columns among different named tuples when evaluating `tuple` functions. This fixes [#70022](https://github.com/ClickHouse/ClickHouse/issues/70022). [#70103](https://github.com/ClickHouse/ClickHouse/pull/70103) ([Amos Bird](https://github.com/amosbird)). +* Backported in [#70623](https://github.com/ClickHouse/ClickHouse/issues/70623): Fix server segfault on creating a materialized view with two selects and an `INTERSECT`, e.g. `CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);`. [#70264](https://github.com/ClickHouse/ClickHouse/pull/70264) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#70688](https://github.com/ClickHouse/ClickHouse/issues/70688): Fix possible use-after-free in `SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf`. [#70358](https://github.com/ClickHouse/ClickHouse/pull/70358) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#70494](https://github.com/ClickHouse/ClickHouse/issues/70494): Fix crash during GROUP BY JSON sub-object subcolumn. [#70374](https://github.com/ClickHouse/ClickHouse/pull/70374) ([Pavel Kruglov](https://github.com/Avogar)). +* Backported in [#70482](https://github.com/ClickHouse/ClickHouse/issues/70482): Don't prefetch parts for vertical merges if part has no rows. [#70452](https://github.com/ClickHouse/ClickHouse/pull/70452) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#70556](https://github.com/ClickHouse/ClickHouse/issues/70556): Fix crash in WHERE with lambda functions. [#70464](https://github.com/ClickHouse/ClickHouse/pull/70464) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#70878](https://github.com/ClickHouse/ClickHouse/issues/70878): Fix table creation with `CREATE ... AS table_function()` with database `Replicated` and unavailable table function source on secondary replica. [#70511](https://github.com/ClickHouse/ClickHouse/pull/70511) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#70575](https://github.com/ClickHouse/ClickHouse/issues/70575): Ignore all output on async insert with `wait_for_async_insert=1`. Closes [#62644](https://github.com/ClickHouse/ClickHouse/issues/62644). [#70530](https://github.com/ClickHouse/ClickHouse/pull/70530) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#71052](https://github.com/ClickHouse/ClickHouse/issues/71052): Ignore frozen_metadata.txt while traversing shadow directory from system.remote_data_paths. [#70590](https://github.com/ClickHouse/ClickHouse/pull/70590) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#70651](https://github.com/ClickHouse/ClickHouse/issues/70651): Fix creation of stateful window functions on misaligned memory. [#70631](https://github.com/ClickHouse/ClickHouse/pull/70631) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#70757](https://github.com/ClickHouse/ClickHouse/issues/70757): Fixed rare crashes in `SELECT`-s and merges after adding a column of `Array` type with non-empty default expression. [#70695](https://github.com/ClickHouse/ClickHouse/pull/70695) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#70763](https://github.com/ClickHouse/ClickHouse/issues/70763): Fix infinite recursion when infering a proto schema with skip unsupported fields enabled. [#70697](https://github.com/ClickHouse/ClickHouse/pull/70697) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#71118](https://github.com/ClickHouse/ClickHouse/issues/71118): `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Backported in [#70896](https://github.com/ClickHouse/ClickHouse/issues/70896): Disable enable_named_columns_in_function_tuple by default. [#70833](https://github.com/ClickHouse/ClickHouse/pull/70833) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#70994](https://github.com/ClickHouse/ClickHouse/issues/70994): Fix a logical error due to negative zeros in the two-level hash table. This closes [#70973](https://github.com/ClickHouse/ClickHouse/issues/70973). [#70979](https://github.com/ClickHouse/ClickHouse/pull/70979) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#71210](https://github.com/ClickHouse/ClickHouse/issues/71210): Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#71248](https://github.com/ClickHouse/ClickHouse/issues/71248): Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Backported in [#71375](https://github.com/ClickHouse/ClickHouse/issues/71375): Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#71026](https://github.com/ClickHouse/ClickHouse/issues/71026): Fix dropping of file cache in CHECK query in case of enabled transactions. [#69256](https://github.com/ClickHouse/ClickHouse/pull/69256) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#70388](https://github.com/ClickHouse/ClickHouse/issues/70388): CI: Enable Integration Tests for backport PRs. [#70329](https://github.com/ClickHouse/ClickHouse/pull/70329) ([Max Kainov](https://github.com/maxknv)). +* Backported in [#70701](https://github.com/ClickHouse/ClickHouse/issues/70701): Fix order in 03249_dynamic_alter_consistency. [#70453](https://github.com/ClickHouse/ClickHouse/pull/70453) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#70542](https://github.com/ClickHouse/ClickHouse/issues/70542): Remove slow poll() logs in keeper. [#70508](https://github.com/ClickHouse/ClickHouse/pull/70508) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#70804](https://github.com/ClickHouse/ClickHouse/issues/70804): When the `PR Check` status is set, it's a valid RunConfig job failure. [#70643](https://github.com/ClickHouse/ClickHouse/pull/70643) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#71229](https://github.com/ClickHouse/ClickHouse/issues/71229): Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 10c55aa4bf5..cf28db5d49a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,7 @@ +v24.10.1.2812-stable 2024-11-01 v24.9.2.42-stable 2024-10-03 v24.9.1.3278-stable 2024-09-26 +v24.8.6.70-lts 2024-11-04 v24.8.5.115-lts 2024-10-08 v24.8.4.13-lts 2024-09-06 v24.8.3.59-lts 2024-09-03 From de751c7e4d3e6445348cd6e5d92a09dc7c41e0ab Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 4 Nov 2024 18:25:27 +0100 Subject: [PATCH 763/816] Update AccessControl.cpp --- src/Access/AccessControl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 647fb238d48..9b3b8d2a977 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -608,7 +608,7 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po } catch (...) { - tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::debug); + tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed", LogsLevel::information); WriteBufferFromOwnString message; message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name."; From a612e9248c44bd41db761eb88e152a7d2ce6218c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 4 Nov 2024 18:26:02 +0100 Subject: [PATCH 764/816] Update TCPHandler.cpp --- src/Server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ea5507c3155..4f54918445f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1614,7 +1614,7 @@ void TCPHandler::receiveHello() if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED) throw; - tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication", LogsLevel::debug); + tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication", LogsLevel::information); /// ^^ Log at debug level instead of default error level as authentication failures are not an unusual event. } } From c1ce74f52f9b5b53db7bcf43aa0a1a47c9dd9859 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov <102754618+MikhailBurdukov@users.noreply.github.com> Date: Mon, 4 Nov 2024 21:40:59 +0300 Subject: [PATCH 765/816] Update tests/integration/test_named_collections/test.py Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- tests/integration/test_named_collections/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index bd04bb9e3c8..e2fa776a8f0 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -803,7 +803,7 @@ def test_keeper_storage_remove_on_cluster(cluster, ignore, expected_raise): def test_name_escaping(cluster, instance_name): node = cluster.instances[instance_name] - node.query("DROP NAMED COLLECTION IF EXISTS test;") + node.query("DROP NAMED COLLECTION IF EXISTS `test_!strange/symbols!`;") node.query("CREATE NAMED COLLECTION `test_!strange/symbols!` AS key1=1, key2=2") node.restart_clickhouse() From f9f1870a0e91f029849fa7897c74b9d3355f7f6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 4 Nov 2024 21:10:44 +0100 Subject: [PATCH 766/816] Fix upgrade check (24.11) --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 157054e5627..b95dc5f85ed 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -71,6 +71,7 @@ static std::initializer_list Date: Tue, 5 Nov 2024 01:58:23 +0000 Subject: [PATCH 767/816] attempt to fix irrelevant test --- tests/integration/test_quorum_inserts/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index eefc4882e8e..66f96d61b3e 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -366,7 +366,7 @@ def test_insert_quorum_with_ttl(started_cluster): zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") -def test_insert_quorum_with_keeper_loss_connection(): +def test_insert_quorum_with_keeper_loss_connection(started_cluster): zero.query( "DROP TABLE IF EXISTS test_insert_quorum_with_keeper_fail ON CLUSTER cluster" ) From a35cc85a68c9356f5697fa22e057bf74a28ee5bb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 5 Nov 2024 04:07:09 +0000 Subject: [PATCH 768/816] remove irrelevant changes --- tests/integration/test_quorum_inserts/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 66f96d61b3e..eefc4882e8e 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -366,7 +366,7 @@ def test_insert_quorum_with_ttl(started_cluster): zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") -def test_insert_quorum_with_keeper_loss_connection(started_cluster): +def test_insert_quorum_with_keeper_loss_connection(): zero.query( "DROP TABLE IF EXISTS test_insert_quorum_with_keeper_fail ON CLUSTER cluster" ) From 3491c0c0e83c5f76c5d5de5097ce513436b4d010 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 1 Nov 2024 09:42:02 +0100 Subject: [PATCH 769/816] CI: Remove deprecated release script --- tests/ci/mark_release_ready.py | 3 +- tests/ci/release.py | 693 --------------------------------- 2 files changed, 2 insertions(+), 694 deletions(-) delete mode 100755 tests/ci/release.py diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py index 7ffb3c9a89b..838961bd89f 100755 --- a/tests/ci/mark_release_ready.py +++ b/tests/ci/mark_release_ready.py @@ -9,9 +9,10 @@ from get_robot_token import get_best_robot_token from git_helper import commit as commit_arg from github_helper import GitHub from pr_info import PRInfo -from release import RELEASE_READY_STATUS from report import SUCCESS +RELEASE_READY_STATUS = "Ready for release" + def main(): parser = argparse.ArgumentParser( diff --git a/tests/ci/release.py b/tests/ci/release.py deleted file mode 100755 index ed9d60a5cad..00000000000 --- a/tests/ci/release.py +++ /dev/null @@ -1,693 +0,0 @@ -#!/usr/bin/env python3 - -""" -script to create releases for ClickHouse - -The `gh` CLI preferred over the PyGithub to have an easy way to rollback bad -release in command line by simple execution giving rollback commands - -On another hand, PyGithub is used for convenient getting commit's status from API - -To run this script on a freshly installed Ubuntu 22.04 system, it is enough to do the following commands: - -sudo apt install pip -pip install requests boto3 github PyGithub -sudo snap install gh -gh auth login -""" - - -import argparse -import json -import logging -import subprocess -from contextlib import contextmanager -from typing import Any, Final, Iterator, List, Optional, Tuple - -from ci_config import Labels -from git_helper import Git, commit, release_branch -from report import SUCCESS -from version_helper import ( - FILE_WITH_VERSION_PATH, - GENERATED_CONTRIBUTORS, - ClickHouseVersion, - VersionType, - get_abs_path, - get_version_from_repo, - update_cmake_version, - update_contributors, -) - -RELEASE_READY_STATUS = "Ready for release" - - -class Repo: - VALID = ("ssh", "https", "origin") - - def __init__(self, repo: str, protocol: str): - self._repo = repo - self._url = "" - self.url = protocol - - @property - def url(self) -> str: - return self._url - - @url.setter - def url(self, protocol: str) -> None: - if protocol == "ssh": - self._url = f"git@github.com:{self}.git" - elif protocol == "https": - self._url = f"https://github.com/{self}.git" - elif protocol == "origin": - self._url = protocol - else: - raise ValueError(f"protocol must be in {self.VALID}") - - def __str__(self): - return self._repo - - -class Release: - NEW = "new" # type: Final - PATCH = "patch" # type: Final - VALID_TYPE = (NEW, PATCH) # type: Final[Tuple[str, str]] - CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) - CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) - - def __init__( - self, - repo: Repo, - release_commit: str, - release_type: str, - dry_run: bool, - with_stderr: bool, - ): - self.repo = repo - self._release_commit = "" - self.release_commit = release_commit - self.dry_run = dry_run - self.with_stderr = with_stderr - assert release_type in self.VALID_TYPE - self.release_type = release_type - self._git = Git() - self._version = get_version_from_repo(git=self._git) - self.release_version = self.version - self._release_branch = "" - self._version_new_tag = None # type: Optional[ClickHouseVersion] - self._rollback_stack = [] # type: List[str] - - def run( - self, cmd: str, cwd: Optional[str] = None, dry_run: bool = False, **kwargs: Any - ) -> str: - cwd_text = "" - if cwd: - cwd_text = f" (CWD='{cwd}')" - if dry_run: - logging.info("Would run command%s:\n %s", cwd_text, cmd) - return "" - if not self.with_stderr: - kwargs["stderr"] = subprocess.DEVNULL - - logging.info("Running command%s:\n %s", cwd_text, cmd) - return self._git.run(cmd, cwd, **kwargs) - - def set_release_info(self): - # Fetch release commit and tags in case they don't exist locally - self.run( - f"git fetch {self.repo.url} {self.release_commit} --no-recurse-submodules" - ) - self.run(f"git fetch {self.repo.url} --tags --no-recurse-submodules") - - # Get the actual version for the commit before check - with self._checkout(self.release_commit, True): - self.release_branch = f"{self.version.major}.{self.version.minor}" - self.release_version = get_version_from_repo(git=self._git) - self.release_version.with_description(self.get_stable_release_type()) - - self.read_version() - - def read_version(self): - self._git.update() - self.version = get_version_from_repo(git=self._git) - - def get_stable_release_type(self) -> str: - if self.version.is_lts: - return VersionType.LTS - return VersionType.STABLE - - def check_commit_release_ready(self): - per_page = 100 - page = 1 - while True: - statuses = json.loads( - self.run( - f"gh api 'repos/{self.repo}/commits/{self.release_commit}" - f"/statuses?per_page={per_page}&page={page}'" - ) - ) - - if not statuses: - break - - for status in statuses: - if status["context"] == RELEASE_READY_STATUS: - if not status["state"] == SUCCESS: - raise ValueError( - f"the status {RELEASE_READY_STATUS} is {status['state']}" - ", not success" - ) - - return - - page += 1 - - raise KeyError( - f"the status {RELEASE_READY_STATUS} " - f"is not found for commit {self.release_commit}" - ) - - def check_prerequisites(self): - """ - Check tooling installed in the system, `git` is checked by Git() init - """ - try: - self.run("gh auth status") - except subprocess.SubprocessError: - logging.error( - "The github-cli either not installed or not setup, please follow " - "the instructions on https://github.com/cli/cli#installation and " - "https://cli.github.com/manual/" - ) - raise - - if self.release_type == self.PATCH: - self.check_commit_release_ready() - - def do( - self, check_dirty: bool, check_run_from_master: bool, check_branch: bool - ) -> None: - self.check_prerequisites() - - if check_dirty: - logging.info("Checking if repo is clean") - try: - self.run("git diff HEAD --exit-code") - except subprocess.CalledProcessError: - logging.fatal("Repo contains uncommitted changes") - raise - - if check_run_from_master and self._git.branch != "master": - raise RuntimeError("the script must be launched only from master") - - self.set_release_info() - - if check_branch: - self.check_branch() - - if self.release_type == self.NEW: - with self._checkout(self.release_commit, True): - # Checkout to the commit, it will provide the correct current version - with self.new_release(): - with self.create_release_branch(): - logging.info( - "Publishing release %s from commit %s is done", - self.release_version.describe, - self.release_commit, - ) - - elif self.release_type == self.PATCH: - with self._checkout(self.release_commit, True): - with self.patch_release(): - logging.info( - "Publishing release %s from commit %s is done", - self.release_version.describe, - self.release_commit, - ) - - if self.dry_run: - logging.info("Dry running, clean out possible changes") - rollback = self._rollback_stack.copy() - rollback.reverse() - for cmd in rollback: - self.run(cmd) - return - - self.log_post_workflows() - self.log_rollback() - - def check_no_tags_after(self): - tags_after_commit = self.run(f"git tag --contains={self.release_commit}") - if tags_after_commit: - raise RuntimeError( - f"Commit {self.release_commit} belongs to following tags:\n" - f"{tags_after_commit}\nChoose another commit" - ) - - def check_branch(self): - branch = self.release_branch - if self.release_type == self.NEW: - # Commit to spin up the release must belong to a main branch - branch = "master" - elif self.release_type != self.PATCH: - raise ( - ValueError(f"release_type {self.release_type} not in {self.VALID_TYPE}") - ) - - # Prefetch the branch to have it updated - if self._git.branch == branch: - self.run("git pull --no-recurse-submodules") - else: - self.run( - f"git fetch {self.repo.url} {branch}:{branch} --no-recurse-submodules" - ) - output = self.run(f"git branch --contains={self.release_commit} {branch}") - if branch not in output: - raise RuntimeError( - f"commit {self.release_commit} must belong to {branch} " - f"for {self.release_type} release" - ) - - def _update_cmake_contributors( - self, version: ClickHouseVersion, reset_tweak: bool = True - ) -> None: - if reset_tweak: - desc = version.description - version = version.reset_tweak() - version.with_description(desc) - update_cmake_version(version) - update_contributors(raise_error=True) - if self.dry_run: - logging.info( - "Dry running, resetting the following changes in the repo:\n%s", - self.run(f"git diff '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'"), - ) - self.run(f"git checkout '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}'") - - def _commit_cmake_contributors( - self, version: ClickHouseVersion, reset_tweak: bool = True - ) -> None: - if reset_tweak: - version = version.reset_tweak() - self.run( - f"git commit '{self.CMAKE_PATH}' '{self.CONTRIBUTORS_PATH}' " - f"-m 'Update autogenerated version to {version.string} and contributors'", - dry_run=self.dry_run, - ) - - @property - def bump_part(self) -> ClickHouseVersion.PART_TYPE: - if self.release_type == Release.NEW: - if self._version.minor >= 12: - return "major" - return "minor" - return "patch" - - @property - def has_rollback(self) -> bool: - return bool(self._rollback_stack) - - def log_rollback(self): - if self.has_rollback: - rollback = self._rollback_stack.copy() - rollback.reverse() - logging.info( - "To rollback the action run the following commands:\n %s", - "\n ".join(rollback), - ) - - def log_post_workflows(self): - logging.info( - "To verify all actions are running good visit the following links:\n %s", - "\n ".join( - f"https://github.com/{self.repo}/actions/workflows/{action}.yml" - for action in ("release", "tags_stable") - ), - ) - - @contextmanager - def create_release_branch(self): - self.check_no_tags_after() - # Create release branch - self.read_version() - assert self._version_new_tag is not None - with self._create_tag( - self._version_new_tag.describe, - self.release_commit, - f"Initial commit for release {self._version_new_tag.major}.{self._version_new_tag.minor}", - ): - with self._create_branch(self.release_branch, self.release_commit): - with self._checkout(self.release_branch, True): - with self._bump_release_branch(): - yield - - @contextmanager - def patch_release(self): - self.check_no_tags_after() - self.read_version() - version_type = self.get_stable_release_type() - self.version.with_description(version_type) - with self._create_gh_release(False): - self.version = self.version.update(self.bump_part) - self.version.with_description(version_type) - self._update_cmake_contributors(self.version) - # Checking out the commit of the branch and not the branch itself, - # then we are able to skip rollback - with self._checkout(f"{self.release_branch}^0", False): - current_commit = self.run("git rev-parse HEAD") - self._commit_cmake_contributors(self.version) - with self._push( - "HEAD", with_rollback_on_fail=False, remote_ref=self.release_branch - ): - # DO NOT PUT ANYTHING ELSE HERE - # The push must be the last action and mean the successful release - self._rollback_stack.append( - f"{self.dry_run_prefix}git push {self.repo.url} " - f"+{current_commit}:{self.release_branch}" - ) - yield - - @contextmanager - def new_release(self): - # Create branch for a version bump - self.read_version() - self.version = self.version.update(self.bump_part) - helper_branch = f"{self.version.major}.{self.version.minor}-prepare" - with self._create_branch(helper_branch, self.release_commit): - with self._checkout(helper_branch, True): - with self._bump_version_in_master(helper_branch): - yield - - @property - def version(self) -> ClickHouseVersion: - return self._version - - @version.setter - def version(self, version: ClickHouseVersion) -> None: - if not isinstance(version, ClickHouseVersion): - raise ValueError(f"version must be ClickHouseVersion, not {type(version)}") - self._version = version - - @property - def release_branch(self) -> str: - return self._release_branch - - @release_branch.setter - def release_branch(self, branch: str) -> None: - self._release_branch = release_branch(branch) - - @property - def release_commit(self) -> str: - return self._release_commit - - @release_commit.setter - def release_commit(self, release_commit: str) -> None: - self._release_commit = commit(release_commit) - - @property - def dry_run_prefix(self) -> str: - if self.dry_run: - return "# " - return "" - - @contextmanager - def _bump_release_branch(self): - # Update only git, original version stays the same - self._git.update() - new_version = self.version.copy() - version_type = self.get_stable_release_type() - pr_labels = f"--label {Labels.RELEASE}" - if version_type == VersionType.LTS: - pr_labels += f" --label {Labels.RELEASE_LTS}" - new_version.with_description(version_type) - self._update_cmake_contributors(new_version) - self._commit_cmake_contributors(new_version) - with self._push(self.release_branch): - with self._create_gh_label( - f"v{self.release_branch}-must-backport", "10dbed" - ): - with self._create_gh_label( - f"v{self.release_branch}-affected", "c2bfff" - ): - # The following command is rolled back by deleting branch - # in self._push - self.run( - f"gh pr create --repo {self.repo} --title " - f"'Release pull request for branch {self.release_branch}' " - f"--head {self.release_branch} {pr_labels} " - "--body 'This PullRequest is a part of ClickHouse release " - "cycle. It is used by CI system only. Do not perform any " - "changes with it.'", - dry_run=self.dry_run, - ) - # Here the release branch part is done. - # We don't create a release itself automatically to have a - # safe window to backport possible bug fixes. - yield - - @contextmanager - def _bump_version_in_master(self, helper_branch: str) -> Iterator[None]: - self.read_version() - self.version = self.version.update(self.bump_part) - self.version.with_description(VersionType.TESTING) - self._update_cmake_contributors(self.version) - self._commit_cmake_contributors(self.version) - # Create a version-new tag - self._version_new_tag = self.version.copy() - self._version_new_tag.tweak = 1 - self._version_new_tag.with_description(VersionType.NEW) - - with self._push(helper_branch): - body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") - # The following command is rolled back by deleting branch in self._push - self.run( - f"gh pr create --repo {self.repo} --title 'Update version after " - f"release' --head {helper_branch} --body-file '{body_file}' " - "--label 'do not test' --assignee @me", - dry_run=self.dry_run, - ) - # Here the new release part is done - yield - - @contextmanager - def _checkout(self, ref: str, with_checkout_back: bool = False) -> Iterator[None]: - self._git.update() - orig_ref = self._git.branch or self._git.sha - rollback_cmd = "" - if ref not in (self._git.branch, self._git.sha): - self.run(f"git checkout {ref}") - # checkout is not put into rollback_stack intentionally - rollback_cmd = f"git checkout {orig_ref}" - # always update version and git after checked out ref - self.read_version() - try: - yield - except (Exception, KeyboardInterrupt): - logging.warning("Rolling back checked out %s for %s", ref, orig_ref) - self.run(f"git reset --hard; git checkout -f {orig_ref}") - raise - # Normal flow when we need to checkout back - if with_checkout_back and rollback_cmd: - self.run(rollback_cmd) - - @contextmanager - def _create_branch(self, name: str, start_point: str = "") -> Iterator[None]: - self.run(f"git branch {name} {start_point}") - - rollback_cmd = f"git branch -D {name}" - self._rollback_stack.append(rollback_cmd) - try: - yield - except (Exception, KeyboardInterrupt): - logging.warning("Rolling back created branch %s", name) - self.run(rollback_cmd) - raise - - @contextmanager - def _create_gh_label(self, label: str, color_hex: str) -> Iterator[None]: - # API call, https://docs.github.com/en/rest/reference/issues#create-a-label - self.run( - f"gh api repos/{self.repo}/labels -f name={label} -f color={color_hex}", - dry_run=self.dry_run, - ) - rollback_cmd = ( - f"{self.dry_run_prefix}gh api repos/{self.repo}/labels/{label} -X DELETE" - ) - self._rollback_stack.append(rollback_cmd) - try: - yield - except (Exception, KeyboardInterrupt): - logging.warning("Rolling back label %s", label) - self.run(rollback_cmd) - raise - - @contextmanager - def _create_gh_release(self, as_prerelease: bool) -> Iterator[None]: - tag = self.release_version.describe - with self._create_tag(tag, self.release_commit): - # Preserve tag if version is changed - prerelease = "" - if as_prerelease: - prerelease = "--prerelease" - self.run( - f"gh release create {prerelease} --repo {self.repo} " - f"--title 'Release {tag}' '{tag}'", - dry_run=self.dry_run, - ) - rollback_cmd = ( - f"{self.dry_run_prefix}gh release delete --yes " - f"--repo {self.repo} '{tag}'" - ) - self._rollback_stack.append(rollback_cmd) - try: - yield - except (Exception, KeyboardInterrupt): - logging.warning("Rolling back release publishing") - self.run(rollback_cmd) - raise - - @contextmanager - def _create_tag( - self, tag: str, commit: str, tag_message: str = "" - ) -> Iterator[None]: - tag_message = tag_message or f"Release {tag}" - # Create tag even in dry-run - self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}") - rollback_cmd = f"git tag -d '{tag}'" - self._rollback_stack.append(rollback_cmd) - try: - with self._push(tag): - yield - except (Exception, KeyboardInterrupt): - logging.warning("Rolling back tag %s", tag) - self.run(rollback_cmd) - raise - - @contextmanager - def _push( - self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = "" - ) -> Iterator[None]: - if remote_ref == "": - remote_ref = ref - - self.run(f"git push {self.repo.url} {ref}:{remote_ref}", dry_run=self.dry_run) - if with_rollback_on_fail: - rollback_cmd = ( - f"{self.dry_run_prefix}git push -d {self.repo.url} {remote_ref}" - ) - self._rollback_stack.append(rollback_cmd) - - try: - yield - except (Exception, KeyboardInterrupt): - if with_rollback_on_fail: - logging.warning("Rolling back pushed ref %s", ref) - self.run(rollback_cmd) - - raise - - -def parse_args() -> argparse.Namespace: - parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter, - description="Script to release a new ClickHouse version, requires `git` and " - "`gh` (github-cli) commands " - "!!! LAUNCH IT ONLY FROM THE MASTER BRANCH !!!", - ) - - parser.add_argument( - "--commit", - required=True, - type=commit, - help="commit create a release", - ) - parser.add_argument( - "--repo", - default="ClickHouse/ClickHouse", - help="repository to create the release", - ) - parser.add_argument( - "--remote-protocol", - "-p", - default="ssh", - choices=Repo.VALID, - help="repo protocol for git commands remote, 'origin' is a special case and " - "uses 'origin' as a remote", - ) - parser.add_argument( - "--type", - required=True, - choices=Release.VALID_TYPE, - dest="release_type", - help="a release type to bump the major.minor.patch version part, " - "new branch is created only for the value 'new'", - ) - parser.add_argument("--with-release-branch", default=True, help=argparse.SUPPRESS) - parser.add_argument("--check-dirty", default=True, help=argparse.SUPPRESS) - parser.add_argument( - "--no-check-dirty", - dest="check_dirty", - action="store_false", - default=argparse.SUPPRESS, - help="(dangerous) if set, skip check repository for uncommitted changes", - ) - parser.add_argument("--check-run-from-master", default=True, help=argparse.SUPPRESS) - parser.add_argument( - "--no-run-from-master", - dest="check_run_from_master", - action="store_false", - default=argparse.SUPPRESS, - help="(for development) if set, the script could run from non-master branch", - ) - parser.add_argument("--check-branch", default=True, help=argparse.SUPPRESS) - parser.add_argument( - "--no-check-branch", - dest="check_branch", - action="store_false", - default=argparse.SUPPRESS, - help="(debug or development only, dangerous) if set, skip the branch check for " - "a run. By default, 'new' type work only for master, and 'patch' " - "works only for a release branches, that name " - "should be the same as '$MAJOR.$MINOR' version, e.g. 22.2", - ) - parser.add_argument( - "--dry-run", - action="store_true", - help="do not make any actual changes in the repo, just show what will be done", - ) - parser.add_argument( - "--with-stderr", - action="store_true", - help="if set, the stderr of all subprocess commands will be printed as well", - ) - - return parser.parse_args() - - -def main(): - logging.basicConfig(level=logging.INFO) - args = parse_args() - repo = Repo(args.repo, args.remote_protocol) - release = Release( - repo, args.commit, args.release_type, args.dry_run, args.with_stderr - ) - - try: - release.do(args.check_dirty, args.check_run_from_master, args.check_branch) - except: - if release.has_rollback: - logging.error( - "!!The release process finished with error, read the output carefully!!" - ) - logging.error( - "Probably, rollback finished with error. " - "If you don't see any of the following commands in the output, " - "execute them manually:" - ) - release.log_rollback() - raise - - -if __name__ == "__main__": - assert False, "Script Deprecated, ask ci team for help" - main() From 1abfa41b890d4cdcb09d06579b8e9b7f14d4f4f5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Nov 2024 11:18:11 +0100 Subject: [PATCH 770/816] Update CMakeLists.txt --- contrib/usearch-cmake/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index 69a986de192..fda061bf467 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -19,7 +19,8 @@ endif () add_library(ch_contrib::usearch ALIAS _usearch) - +# Cf. https://github.com/llvm/llvm-project/issues/107810 (though it is not 100% the same stack) +# # LLVM ERROR: Cannot select: 0x7996e7a73150: f32,ch = load<(load (s16) from %ir.22, !tbaa !54231), anyext from bf16> 0x79961cb737c0, 0x7996e7a1a500, undef:i64, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 # 0x7996e7a1a500: i64 = add 0x79961e770d00, Constant:i64<-16>, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 # 0x79961e770d00: i64,ch = CopyFromReg 0x79961cb737c0, Register:i64 %4, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 From 087a886bc9f312c4cc4fc6cba1d1ea5d1681c137 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Nov 2024 11:18:21 +0100 Subject: [PATCH 771/816] Update src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp Co-authored-by: Nikita Taranov --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 0b5ffa659dc..5a725922e14 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -118,8 +118,6 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( if (!result) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release())); swap(result.index); - - /// LOG_TRACE(getLogger("XXX"), "{}", simsimd_uses_dynamic_dispatch()); } void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const From 0cc8626279fefc6ceae0a806b4e326ea0a354476 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Nov 2024 11:31:27 +0000 Subject: [PATCH 772/816] Fix assert during insert into vector similarity index in presence of other skipping indexes --- .../MergeTreeIndexVectorSimilarity.cpp | 79 ++++++++++--------- .../02354_vector_search_bugs.reference | 1 + .../0_stateless/02354_vector_search_bugs.sql | 15 ++++ 3 files changed, 58 insertions(+), 37 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 5a725922e14..498d0131d5a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -347,53 +347,58 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (index_sample_block.columns() > 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); - const String & index_column_name = index_sample_block.getByPosition(0).name; - const ColumnPtr & index_column = block.getByName(index_column_name).column; - ColumnPtr column_cut = index_column->cut(*pos, rows_read); + for (size_t i = 0; i < index_sample_block.columns(); ++i) + { + const auto & index_column_with_type_and_name = index_sample_block.getByPosition(i); - const auto * column_array = typeid_cast(column_cut.get()); - if (!column_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float*) column"); + const auto & index_column_name = index_column_with_type_and_name.name; + const auto & index_column = block.getByName(index_column_name).column; + ColumnPtr column_cut = index_column->cut(*pos, rows_read); - if (column_array->empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); + const auto * column_array = typeid_cast(column_cut.get()); + if (!column_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float*) column"); - /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays - /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default - /// values which is also empty. - if (column_array->isDefaultAt(0)) - throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); + if (column_array->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); - const size_t rows = column_array->size(); + /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays + /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default + /// values which is also empty. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); - const auto & column_array_offsets = column_array->getOffsets(); - const size_t dimensions = column_array_offsets[0]; + const size_t rows = column_array->size(); - if (!index) - index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); + const auto & column_array_offsets = column_array->getOffsets(); + const size_t dimensions = column_array_offsets[0]; - /// Also check that previously inserted blocks have the same size as this block. - /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across - /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. - if (index->dimensions() != dimensions) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + if (!index) + index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); - /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp - if (index->size() + rows > std::numeric_limits::max()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. + if (index->dimensions() != dimensions) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); - DataTypePtr data_type = block.getDataTypes()[0]; - const auto * data_type_array = typeid_cast(data_type.get()); - if (!data_type_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); - const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); + /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp + if (index->size() + rows > std::numeric_limits::max()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); - if (WhichDataType(nested_type_index).isFloat32()) - updateImpl(column_array, column_array_offsets, index, dimensions, rows); - else if (WhichDataType(nested_type_index).isFloat64()) - updateImpl(column_array, column_array_offsets, index, dimensions, rows); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); + DataTypePtr data_type = index_column_with_type_and_name.type; + const auto * data_type_array = typeid_cast(data_type.get()); + if (!data_type_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); + const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); + + if (WhichDataType(nested_type_index).isFloat32()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else if (WhichDataType(nested_type_index).isFloat64()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); + } *pos += rows_read; diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.reference b/tests/queries/0_stateless/02354_vector_search_bugs.reference index 9b610cf543a..dec921cf586 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.reference +++ b/tests/queries/0_stateless/02354_vector_search_bugs.reference @@ -41,3 +41,4 @@ Expression (Projection) Parts: 1/1 Granules: 4/4 index_granularity_bytes = 0 is disallowed +Issue #71381: Vector similarity index and other skipping indexes used on the same table diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index d55bdb88a76..6bcb0f78e75 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -117,3 +117,18 @@ CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SE ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE } DROP TABLE tab; + +SELECT 'Issue #71381: Vector similarity index and other skipping indexes used on the same table'; + +CREATE TABLE tab( + val String, + vec Array(Float32), + INDEX ann_idx vec TYPE vector_similarity('hnsw', 'cosineDistance'), + INDEX set_idx val TYPE set(100) GRANULARITY 100 +) +ENGINE = MergeTree() +ORDER BY tuple(); + +INSERT INTO tab VALUES ('hello world', [0.0]); + +DROP TABLE tab; From bbe28d45bff0bd721685c812706f113e1412ed6b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Nov 2024 12:33:25 +0000 Subject: [PATCH 773/816] fix --- src/Parsers/ASTFunction.cpp | 5 ++- src/TableFunctions/TableFunctionMongoDB.cpp | 42 +++++++++---------- src/TableFunctions/TableFunctionMongoDB.h | 15 +++++++ .../TableFunctionMongoDBPocoLegacy.cpp | 15 ++----- .../03261_mongodb_argumetns_crash.sql | 1 + 5 files changed, 45 insertions(+), 33 deletions(-) create mode 100644 src/TableFunctions/TableFunctionMongoDB.h diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 53d44e2f325..11cfe2e584e 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -724,7 +724,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format { if (secret_arguments.are_named) { - assert_cast(argument.get())->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens); + if (const auto * func_ast = typeid_cast(argument.get())) + func_ast->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens); + else + argument->formatImpl(settings, state, nested_dont_need_parens); settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : ""); } if (!secret_arguments.replacement.empty()) diff --git a/src/TableFunctions/TableFunctionMongoDB.cpp b/src/TableFunctions/TableFunctionMongoDB.cpp index 966ce858875..9f91839fb33 100644 --- a/src/TableFunctions/TableFunctionMongoDB.cpp +++ b/src/TableFunctions/TableFunctionMongoDB.cpp @@ -15,7 +15,7 @@ #include #include #include - +#include namespace DB { @@ -85,17 +85,11 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt { if (const auto * ast_func = typeid_cast(args[i].get())) { - const auto * args_expr = assert_cast(ast_func->arguments.get()); - auto function_args = args_expr->children; - if (function_args.size() != 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); - - auto arg_name = function_args[0]->as()->name(); - + const auto & [arg_name, arg_value] = getKeyValueMongoDBArgument(ast_func); if (arg_name == "structure") - structure = checkAndGetLiteralArgument(function_args[1], "structure"); + structure = checkAndGetLiteralArgument(arg_value, arg_name); else if (arg_name == "options") - main_arguments.push_back(function_args[1]); + main_arguments.push_back(arg_value); } else if (i == 5) { @@ -117,19 +111,11 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt { if (const auto * ast_func = typeid_cast(args[i].get())) { - const auto * args_expr = assert_cast(ast_func->arguments.get()); - const auto & function_args = args_expr->children; - if (function_args.size() != 2 || ast_func->name != "equals" || function_args[0]->as()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); - - auto arg_name = function_args[0]->as()->name(); - + const auto & [arg_name, arg_value] = getKeyValueMongoDBArgument(ast_func); if (arg_name == "structure") - structure = checkAndGetLiteralArgument(function_args[1], "structure"); + structure = checkAndGetLiteralArgument(arg_value, arg_name); else if (arg_name == "options") - main_arguments.push_back(function_args[1]); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); + main_arguments.push_back(arg_value); } else if (i == 2) { @@ -149,6 +135,20 @@ void TableFunctionMongoDB::parseArguments(const ASTPtr & ast_function, ContextPt } +std::pair getKeyValueMongoDBArgument(const ASTFunction * ast_func) +{ + const auto * args_expr = assert_cast(ast_func->arguments.get()); + const auto & function_args = args_expr->children; + if (function_args.size() != 2 || ast_func->name != "equals" || !function_args[0]->as()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); + + const auto & arg_name = function_args[0]->as()->name(); + if (arg_name == "structure" || arg_name == "options") + return std::make_pair(arg_name, function_args[1]); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); +} + void registerTableFunctionMongoDB(TableFunctionFactory & factory) { factory.registerFunction( diff --git a/src/TableFunctions/TableFunctionMongoDB.h b/src/TableFunctions/TableFunctionMongoDB.h new file mode 100644 index 00000000000..2b75fda1675 --- /dev/null +++ b/src/TableFunctions/TableFunctionMongoDB.h @@ -0,0 +1,15 @@ + +#include + +#include +#include +#include + + +namespace DB +{ + +std::pair getKeyValueMongoDBArgument(const ASTFunction * ast_func); + +} + diff --git a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp b/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp index 70b28ddfaf0..4e27fd35e12 100644 --- a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp +++ b/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB @@ -97,19 +98,11 @@ void TableFunctionMongoDBPocoLegacy::parseArguments(const ASTPtr & ast_function, { if (const auto * ast_func = typeid_cast(args[i].get())) { - const auto * args_expr = assert_cast(ast_func->arguments.get()); - const auto & function_args = args_expr->children; - if (function_args.size() != 2 || ast_func->name != "equals" || function_args[0]->as()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); - - auto arg_name = function_args[0]->as()->name(); - + const auto & [arg_name, arg_value] = getKeyValueMongoDBArgument(ast_func); if (arg_name == "structure") - structure = checkAndGetLiteralArgument(function_args[1], "structure"); + structure = checkAndGetLiteralArgument(arg_value, "structure"); else if (arg_name == "options") - main_arguments.push_back(function_args[1]); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument, got {}", ast_func->formatForErrorMessage()); + main_arguments.push_back(arg_value); } else if (i == 5) { diff --git a/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql b/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql index 830d3995bd5..ca558ac6bc6 100644 --- a/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql +++ b/tests/queries/0_stateless/03261_mongodb_argumetns_crash.sql @@ -11,3 +11,4 @@ SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', SELECT * FROM mongodb('mongodb://some-cluster:27017/?retryWrites=false', 'test', 'my_collection', 'test_user', 'password', NULL, 'x Int32'); -- { serverError BAD_ARGUMENTS } SELECT * FROM mongodb(NULL, 'test', 'my_collection', 'test_user', 'password', 'x Int32'); -- { serverError BAD_ARGUMENTS } +CREATE TABLE IF NOT EXISTS store_version ( `_id` String ) ENGINE = MongoDB(`localhost:27017`, mongodb, storeinfo, adminUser, adminUser); -- { serverError NAMED_COLLECTION_DOESNT_EXIST } From d7977f0b916ccdcc240de8d413015532d492f668 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 5 Nov 2024 13:36:27 +0100 Subject: [PATCH 774/816] More correct assertion --- src/Interpreters/Cache/EvictionCandidates.cpp | 3 ++- src/Interpreters/Cache/FileSegment.cpp | 7 ++++--- src/Interpreters/Cache/FileSegment.h | 7 +++++-- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index 08776ad5aee..f5d5fdec6ba 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -83,7 +83,8 @@ void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & loc queue_iterator->invalidate(); chassert(candidate->releasable()); - candidate->file_segment->resetQueueIterator(); + candidate->file_segment->markDelayedRemovalAndResetQueueIterator(); + /// We need to set removed flag in file segment metadata, /// because in dynamic cache resize we first remove queue entries, /// then evict which also removes file segment metadata, diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 080b54feb06..307d9c8afe1 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -171,10 +171,11 @@ void FileSegment::setQueueIterator(Priority::IteratorPtr iterator) queue_iterator = iterator; } -void FileSegment::resetQueueIterator() +void FileSegment::markDelayedRemovalAndResetQueueIterator() { auto lk = lock(); - queue_iterator.reset(); + on_delayed_removal = true; + queue_iterator = {}; } size_t FileSegment::getCurrentWriteOffset() const @@ -861,7 +862,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) chassert(downloaded_size > 0); chassert(fs::file_size(getPath()) > 0); - chassert(queue_iterator); + chassert(queue_iterator || on_delayed_removal); check_iterator(queue_iterator); break; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 79adc342329..6946d70b764 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -177,7 +177,7 @@ public: void setQueueIterator(Priority::IteratorPtr iterator); - void resetQueueIterator(); + void markDelayedRemovalAndResetQueueIterator(); KeyMetadataPtr tryGetKeyMetadata() const; @@ -249,11 +249,12 @@ private: String tryGetPath() const; - Key file_key; + const Key file_key; Range segment_range; const FileSegmentKind segment_kind; /// Size of the segment is not known until it is downloaded and /// can be bigger than max_file_segment_size. + /// is_unbound == true for temporary data in cache. const bool is_unbound; const bool background_download_enabled; @@ -279,6 +280,8 @@ private: std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state + bool on_delayed_removal = false; + CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; }; From 5152984bb170e5c63144db3dd238099534353378 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Nov 2024 13:52:14 +0000 Subject: [PATCH 775/816] upd src/TableFunctions/TableFunctionMongoDB.h --- src/TableFunctions/TableFunctionMongoDB.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/TableFunctions/TableFunctionMongoDB.h b/src/TableFunctions/TableFunctionMongoDB.h index 2b75fda1675..2ab8ee9479f 100644 --- a/src/TableFunctions/TableFunctionMongoDB.h +++ b/src/TableFunctions/TableFunctionMongoDB.h @@ -1,3 +1,4 @@ +#pragma once #include From c16e1f021b7c24250ebf3bef1c764ba7c218de0d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 5 Nov 2024 14:57:35 +0000 Subject: [PATCH 776/816] fix memory usage in inserts with delayed streams --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 ++ .../MergeTree/IMergedBlockOutputStream.h | 5 +++++ .../MergeTreeDataPartWriterCompact.h | 2 ++ .../MergeTree/MergeTreeDataPartWriterWide.h | 2 ++ src/Storages/MergeTree/MergeTreeSink.cpp | 12 +++++++---- .../MergeTree/ReplicatedMergeTreeSink.cpp | 13 ++++++++---- .../03261_delayed_streams_memory.reference | 1 + .../03261_delayed_streams_memory.sql | 20 +++++++++++++++++++ 8 files changed, 49 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03261_delayed_streams_memory.reference create mode 100644 tests/queries/0_stateless/03261_delayed_streams_memory.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index b8ac14b1750..d1c76505d7c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -46,6 +46,8 @@ public: virtual void finish(bool sync) = 0; + virtual size_t getNumberOfOpenStreams() const = 0; + Columns releaseIndexColumns(); PlainMarksByName releaseCachedMarks(); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index a901b03c115..7dd6d720170 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -39,6 +39,11 @@ public: return writer->releaseCachedMarks(); } + size_t getNumberOfOpenStreams() const + { + return writer->getNumberOfOpenStreams(); + } + protected: /// Remove all columns marked expired in data_part. Also, clears checksums diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index b440a37222d..20c47fb8314 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -32,6 +32,8 @@ public: void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) override; void finish(bool sync) override; + size_t getNumberOfOpenStreams() const override { return 1; } + private: /// Finish serialization of the data. Flush rows in buffer to disk, compute checksums. void fillDataChecksums(MergeTreeDataPartChecksums & checksums); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 68f016a7421..b594b2d79bb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -43,6 +43,8 @@ public: void finish(bool sync) final; + size_t getNumberOfOpenStreams() const override { return column_streams.size(); } + private: /// Finish serialization of data: write final mark if required and compute checksums /// Also validate written data in debug mode diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 604112c26ea..99852309c77 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -94,7 +94,7 @@ void MergeTreeSink::consume(Chunk & chunk) DelayedPartitions partitions; const Settings & settings = context->getSettingsRef(); - size_t streams = 0; + size_t total_streams = 0; bool support_parallel_write = false; auto token_info = chunk.getChunkInfos().get(); @@ -153,16 +153,18 @@ void MergeTreeSink::consume(Chunk & chunk) max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. - streams += temp_part.streams.size(); + size_t current_streams = 0; + for (const auto & stream : temp_part.streams) + current_streams += stream.stream->getNumberOfOpenStreams(); - if (streams > max_insert_delayed_streams_for_parallel_write) + if (total_streams + current_streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(); - streams = 0; + total_streams = 0; support_parallel_write = false; partitions = DelayedPartitions{}; } @@ -174,6 +176,8 @@ void MergeTreeSink::consume(Chunk & chunk) .block_dedup_token = block_dedup_token, .part_counters = std::move(part_counters), }); + + total_streams += current_streams; } if (need_to_define_dedup_token) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f1b0e5ec385..f3ae6e77ac3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -341,7 +341,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) using DelayedPartitions = std::vector; DelayedPartitions partitions; - size_t streams = 0; + size_t total_streams = 0; bool support_parallel_write = false; for (auto & current_block : part_blocks) @@ -418,15 +418,18 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. - streams += temp_part.streams.size(); - if (streams > max_insert_delayed_streams_for_parallel_write) + size_t current_streams = 0; + for (const auto & stream : temp_part.streams) + current_streams += stream.stream->getNumberOfOpenStreams(); + + if (total_streams + current_streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique::DelayedChunk>(replicas_num); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(zookeeper); - streams = 0; + total_streams = 0; support_parallel_write = false; partitions = DelayedPartitions{}; } @@ -447,6 +450,8 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) std::move(unmerged_block), std::move(part_counters) /// profile_events_scope must be reset here. )); + + total_streams += current_streams; } if (need_to_define_dedup_token) diff --git a/tests/queries/0_stateless/03261_delayed_streams_memory.reference b/tests/queries/0_stateless/03261_delayed_streams_memory.reference new file mode 100644 index 00000000000..7326d960397 --- /dev/null +++ b/tests/queries/0_stateless/03261_delayed_streams_memory.reference @@ -0,0 +1 @@ +Ok diff --git a/tests/queries/0_stateless/03261_delayed_streams_memory.sql b/tests/queries/0_stateless/03261_delayed_streams_memory.sql new file mode 100644 index 00000000000..863644a0dff --- /dev/null +++ b/tests/queries/0_stateless/03261_delayed_streams_memory.sql @@ -0,0 +1,20 @@ +-- Tags: long, no-debug, no-asan, no-tsan, no-msan, no-ubsan, no-random-settings, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_100_columns; + +CREATE TABLE t_100_columns (id UInt64, c0 String, c1 String, c2 String, c3 String, c4 String, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 String, c14 String, c15 String, c16 String, c17 String, c18 String, c19 String, c20 String, c21 String, c22 String, c23 String, c24 String, c25 String, c26 String, c27 String, c28 String, c29 String, c30 String, c31 String, c32 String, c33 String, c34 String, c35 String, c36 String, c37 String, c38 String, c39 String, c40 String, c41 String, c42 String, c43 String, c44 String, c45 String, c46 String, c47 String, c48 String, c49 String, c50 String) +ENGINE = MergeTree +ORDER BY id PARTITION BY id % 50 +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1.0, max_compress_block_size = '1M', storage_policy = 's3_cache'; + +SET max_insert_delayed_streams_for_parallel_write = 55; + +INSERT INTO t_100_columns (id) SELECT number FROM numbers(100); + +SYSTEM FLUSH LOGS; + +SELECT if (memory_usage < 300000000, 'Ok', format('Fail: memory usage {}', formatReadableSize(memory_usage))) +FROM system.query_log +WHERE current_database = currentDatabase() AND query LIKE 'INSERT INTO t_100_columns%' AND type = 'QueryFinish'; + +DROP TABLE t_100_columns; From 6c63587f7747cc05e5df4aad259cee40c34ac7c6 Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Fri, 1 Nov 2024 13:27:09 +0100 Subject: [PATCH 777/816] More info in TOO_SLOW exception --- src/QueryPipeline/ExecutionSpeedLimits.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp index 05fd394db77..fc0e86781f0 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -86,10 +86,12 @@ void ExecutionSpeedLimits::throttle( if (timeout_overflow_mode == OverflowMode::THROW && estimated_execution_time_seconds > max_estimated_execution_time.totalSeconds()) throw Exception( ErrorCodes::TOO_SLOW, - "Estimated query execution time ({} seconds) is too long. Maximum: {}. Estimated rows to process: {}", + "Estimated query execution time ({:.5f} seconds) is too long. Maximum: {}. Estimated rows to process: {} ({} read in {:.5f} seconds).", estimated_execution_time_seconds, max_estimated_execution_time.totalSeconds(), - total_rows_to_read); + total_rows_to_read, + read_rows, + elapsed_seconds); } if (max_execution_rps && rows_per_second >= max_execution_rps) From 6ecc673f7d4a9890004a24c16d8c6b9b5a857d93 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 5 Nov 2024 16:02:40 +0000 Subject: [PATCH 778/816] Fix quorum inserts tests --- tests/integration/test_quorum_inserts/test.py | 114 +++++++++--------- 1 file changed, 54 insertions(+), 60 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index eefc4882e8e..de437fc3206 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -2,6 +2,7 @@ import concurrent import time import pytest +import uuid from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager @@ -46,10 +47,11 @@ def started_cluster(): def test_simple_add_replica(started_cluster): - zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster") + table_name = "test_simple_" + uuid.uuid4().hex + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( - "CREATE TABLE test_simple " + f"CREATE TABLE {table_name} " "(a Int8, d Date) " "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " "PARTITION BY d ORDER BY a" @@ -58,91 +60,81 @@ def test_simple_add_replica(started_cluster): zero.query(create_query) first.query(create_query) - first.query("SYSTEM STOP FETCHES test_simple") + first.query(f"SYSTEM STOP FETCHES {table_name}") zero.query( - "INSERT INTO test_simple VALUES (1, '2011-01-01')", + f"INSERT INTO {table_name} VALUES (1, '2011-01-01')", settings={"insert_quorum": 1}, ) - assert "1\t2011-01-01\n" == zero.query("SELECT * from test_simple") - assert "" == first.query("SELECT * from test_simple") + assert "1\t2011-01-01\n" == zero.query(f"SELECT * from {table_name}") + assert "" == first.query(f"SELECT * from {table_name}") - first.query("SYSTEM START FETCHES test_simple") + first.query(f"SYSTEM START FETCHES {table_name}") - first.query("SYSTEM SYNC REPLICA test_simple", timeout=20) + first.query(f"SYSTEM SYNC REPLICA {table_name}", timeout=20) - assert "1\t2011-01-01\n" == zero.query("SELECT * from test_simple") - assert "1\t2011-01-01\n" == first.query("SELECT * from test_simple") + assert "1\t2011-01-01\n" == zero.query(f"SELECT * from {table_name}") + assert "1\t2011-01-01\n" == first.query(f"SELECT * from {table_name}") second.query(create_query) - second.query("SYSTEM SYNC REPLICA test_simple", timeout=20) + second.query(f"SYSTEM SYNC REPLICA {table_name}", timeout=20) - assert "1\t2011-01-01\n" == zero.query("SELECT * from test_simple") - assert "1\t2011-01-01\n" == first.query("SELECT * from test_simple") - assert "1\t2011-01-01\n" == second.query("SELECT * from test_simple") + assert "1\t2011-01-01\n" == zero.query(f"SELECT * from {table_name}") + assert "1\t2011-01-01\n" == first.query(f"SELECT * from {table_name}") + assert "1\t2011-01-01\n" == second.query(f"SELECT * from {table_name}") - zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster") + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") def test_drop_replica_and_achieve_quorum(started_cluster): + table_name = "test_drop_replica_and_achieve_quorum_" + uuid.uuid4().hex zero.query( - "DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster" + f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster" ) - create_query = ( - "CREATE TABLE test_drop_replica_and_achieve_quorum " + f"CREATE TABLE {table_name} " "(a Int8, d Date) " "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " "PARTITION BY d ORDER BY a" ) - print("Create Replicated table with two replicas") zero.query(create_query) first.query(create_query) - print("Stop fetches on one replica. Since that, it will be isolated.") - first.query("SYSTEM STOP FETCHES test_drop_replica_and_achieve_quorum") - + first.query(f"SYSTEM STOP FETCHES {table_name}") print("Insert to other replica. This query will fail.") quorum_timeout = zero.query_and_get_error( - "INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')", + f"INSERT INTO {table_name}(a,d) VALUES (1, '2011-01-01')", settings={"insert_quorum_timeout": 5000}, ) assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." - assert TSV("1\t2011-01-01\n") == TSV( zero.query( - "SELECT * FROM test_drop_replica_and_achieve_quorum", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 0}, ) ) - assert TSV("") == TSV( zero.query( - "SELECT * FROM test_drop_replica_and_achieve_quorum", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 1}, ) ) - # TODO:(Mikhaylov) begin; maybe delete this lines. I want clickhouse to fetch parts and update quorum. print("START FETCHES first replica") - first.query("SYSTEM START FETCHES test_drop_replica_and_achieve_quorum") - + first.query(f"SYSTEM START FETCHES {table_name}") print("SYNC first replica") - first.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20) + first.query(f"SYSTEM SYNC REPLICA {table_name}", timeout=20) # TODO:(Mikhaylov) end - print("Add second replica") second.query(create_query) - print("SYNC second replica") - second.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20) - + second.query(f"SYSTEM SYNC REPLICA {table_name}", timeout=20) print("Quorum for previous insert achieved.") assert TSV("1\t2011-01-01\n") == TSV( second.query( - "SELECT * FROM test_drop_replica_and_achieve_quorum", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 1}, ) ) @@ -296,10 +288,11 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): def test_insert_quorum_with_ttl(started_cluster): - zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") + table_name = "test_insert_quorum_with_ttl_" + uuid.uuid4().hex + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( - "CREATE TABLE test_insert_quorum_with_ttl " + f"CREATE TABLE {table_name} " "(a Int8, d Date) " "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " "PARTITION BY d ORDER BY a " @@ -311,12 +304,12 @@ def test_insert_quorum_with_ttl(started_cluster): zero.query(create_query) first.query(create_query) - print("Stop fetches for test_insert_quorum_with_ttl at first replica.") - first.query("SYSTEM STOP FETCHES test_insert_quorum_with_ttl") + print(f"Stop fetches for {table_name} at first replica.") + first.query(f"SYSTEM STOP FETCHES {table_name}") print("Insert should fail since it can not reach the quorum.") quorum_timeout = zero.query_and_get_error( - "INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')", + f"INSERT INTO {table_name}(a,d) VALUES(1, '2011-01-01')", settings={"insert_quorum_timeout": 5000}, ) assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail." @@ -327,51 +320,52 @@ def test_insert_quorum_with_ttl(started_cluster): time.sleep(10) assert TSV("1\t2011-01-01\n") == TSV( zero.query( - "SELECT * FROM test_insert_quorum_with_ttl", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 0}, ) ) - print("Resume fetches for test_insert_quorum_with_ttl at first replica.") - first.query("SYSTEM START FETCHES test_insert_quorum_with_ttl") + print(f"Resume fetches for {table_name} at first replica.") + first.query(f"SYSTEM START FETCHES {table_name}") print("Sync first replica.") - first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl") + first.query(f"SYSTEM SYNC REPLICA {table_name}") zero.query( - "INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')", + f"INSERT INTO {table_name}(a,d) VALUES(1, '2011-01-01')", settings={"insert_quorum_timeout": 5000}, ) print("Inserts should resume.") - zero.query("INSERT INTO test_insert_quorum_with_ttl(a, d) VALUES(2, '2012-02-02')") + zero.query(f"INSERT INTO {table_name}(a, d) VALUES(2, '2012-02-02')") - first.query("OPTIMIZE TABLE test_insert_quorum_with_ttl") - first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl") - zero.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl") + first.query(f"OPTIMIZE TABLE {table_name}") + first.query(f"SYSTEM SYNC REPLICA {table_name}") + zero.query(f"SYSTEM SYNC REPLICA {table_name}") assert TSV("2\t2012-02-02\n") == TSV( first.query( - "SELECT * FROM test_insert_quorum_with_ttl", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 0}, ) ) assert TSV("2\t2012-02-02\n") == TSV( first.query( - "SELECT * FROM test_insert_quorum_with_ttl", + f"SELECT * FROM {table_name}", settings={"select_sequential_consistency": 1}, ) ) - zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") -def test_insert_quorum_with_keeper_loss_connection(): +def test_insert_quorum_with_keeper_loss_connection(started_cluster): + table_name = "test_insert_quorum_with_keeper_loss_" + uuid.uuid4().hex zero.query( - "DROP TABLE IF EXISTS test_insert_quorum_with_keeper_fail ON CLUSTER cluster" + f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster" ) create_query = ( - "CREATE TABLE test_insert_quorum_with_keeper_loss" + f"CREATE TABLE {table_name} " "(a Int8, d Date) " "Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " "ORDER BY a " @@ -380,7 +374,7 @@ def test_insert_quorum_with_keeper_loss_connection(): zero.query(create_query) first.query(create_query) - first.query("SYSTEM STOP FETCHES test_insert_quorum_with_keeper_loss") + first.query(f"SYSTEM STOP FETCHES {table_name}") zero.query("SYSTEM ENABLE FAILPOINT replicated_merge_tree_commit_zk_fail_after_op") zero.query("SYSTEM ENABLE FAILPOINT replicated_merge_tree_insert_retry_pause") @@ -388,7 +382,7 @@ def test_insert_quorum_with_keeper_loss_connection(): with concurrent.futures.ThreadPoolExecutor(max_workers=5) as executor: insert_future = executor.submit( lambda: zero.query( - "INSERT INTO test_insert_quorum_with_keeper_loss(a,d) VALUES(1, '2011-01-01')", + f"INSERT INTO {table_name}(a,d) VALUES(1, '2011-01-01')", settings={"insert_quorum_timeout": 150000}, ) ) @@ -401,7 +395,7 @@ def test_insert_quorum_with_keeper_loss_connection(): while True: if ( zk.exists( - "/clickhouse/tables/test_insert_quorum_with_keeper_loss/replicas/zero/is_active" + f"/clickhouse/tables/{table_name}/replicas/zero/is_active" ) is None ): @@ -418,7 +412,7 @@ def test_insert_quorum_with_keeper_loss_connection(): "SYSTEM WAIT FAILPOINT finish_set_quorum_failed_parts", timeout=300 ) ) - first.query("SYSTEM START FETCHES test_insert_quorum_with_keeper_loss") + first.query(f"SYSTEM START FETCHES {table_name}") concurrent.futures.wait([quorum_fail_future]) From 3eedc74c5943f23ed4e360533e6e3bb5a6238109 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 5 Nov 2024 16:25:58 +0000 Subject: [PATCH 779/816] Reformatted because of style check --- tests/integration/test_quorum_inserts/test.py | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index de437fc3206..824cb371595 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -88,10 +88,8 @@ def test_simple_add_replica(started_cluster): def test_drop_replica_and_achieve_quorum(started_cluster): - table_name = "test_drop_replica_and_achieve_quorum_" + uuid.uuid4().hex - zero.query( - f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster" - ) + table_name = "test_drop_replica_and_achieve_quorum_" + uuid.uuid4().hex + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " "(a Int8, d Date) " @@ -361,9 +359,7 @@ def test_insert_quorum_with_ttl(started_cluster): def test_insert_quorum_with_keeper_loss_connection(started_cluster): table_name = "test_insert_quorum_with_keeper_loss_" + uuid.uuid4().hex - zero.query( - f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster" - ) + zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " "(a Int8, d Date) " @@ -394,9 +390,7 @@ def test_insert_quorum_with_keeper_loss_connection(started_cluster): zk = cluster.get_kazoo_client("zoo1") while True: if ( - zk.exists( - f"/clickhouse/tables/{table_name}/replicas/zero/is_active" - ) + zk.exists(f"/clickhouse/tables/{table_name}/replicas/zero/is_active") is None ): break From 27153bfc27d45a9fddddf070bb82c7f1e164b455 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 5 Nov 2024 16:58:21 +0000 Subject: [PATCH 780/816] Resolve issues --- tests/integration/test_quorum_inserts/test.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 824cb371595..7adc51121b4 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -1,8 +1,8 @@ import concurrent import time +import uuid import pytest -import uuid from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager @@ -48,7 +48,6 @@ def started_cluster(): def test_simple_add_replica(started_cluster): table_name = "test_simple_" + uuid.uuid4().hex - zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " @@ -89,7 +88,6 @@ def test_simple_add_replica(started_cluster): def test_drop_replica_and_achieve_quorum(started_cluster): table_name = "test_drop_replica_and_achieve_quorum_" + uuid.uuid4().hex - zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " "(a Int8, d Date) " @@ -287,7 +285,6 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): def test_insert_quorum_with_ttl(started_cluster): table_name = "test_insert_quorum_with_ttl_" + uuid.uuid4().hex - zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " @@ -359,7 +356,6 @@ def test_insert_quorum_with_ttl(started_cluster): def test_insert_quorum_with_keeper_loss_connection(started_cluster): table_name = "test_insert_quorum_with_keeper_loss_" + uuid.uuid4().hex - zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( f"CREATE TABLE {table_name} " "(a Int8, d Date) " From 0687f7a83f1a64abd586c5046dbc5ddda427e00a Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 5 Nov 2024 17:09:03 +0000 Subject: [PATCH 781/816] Resolve issue --- tests/integration/test_quorum_inserts/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 7adc51121b4..a646319c5f9 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -143,7 +143,7 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): "test_quorum_insert_with_drop_partition_new_data" if add_new_data else "test_quorum_insert_with_drop_partition" - ) + ) + uuid.uuid4().hex zero.query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER cluster") create_query = ( @@ -206,12 +206,12 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): "test_insert_quorum_with_move_partition_source_new_data" if add_new_data else "test_insert_quorum_with_move_partition_source" - ) + ) + uuid.uuid4().hex destination_table_name = ( "test_insert_quorum_with_move_partition_destination_new_data" if add_new_data else "test_insert_quorum_with_move_partition_destination" - ) + ) + uuid.uuid4().hex zero.query(f"DROP TABLE IF EXISTS {source_table_name} ON CLUSTER cluster") zero.query(f"DROP TABLE IF EXISTS {destination_table_name} ON CLUSTER cluster") From 76683d021d96309bd3a19d2afde36f9ba802814f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Nov 2024 17:22:08 +0000 Subject: [PATCH 782/816] Fix constants in WHERE expression which could apparently contain Join. --- src/Interpreters/ExpressionAnalyzer.cpp | 8 +++++-- ...3258_old_analyzer_const_expr_bug.reference | 0 .../03258_old_analyzer_const_expr_bug.sql | 23 +++++++++++++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.reference create mode 100644 tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 4e5cf7d2549..a89e8ca9b3c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1981,7 +1981,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - before_prewhere_sample = prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample); + ExpressionActions( + prewhere_dag_and_flags->dag.clone(), + ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -2013,7 +2015,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where_sample = before_where->dag.updateHeader(before_where_sample); + ExpressionActions( + before_where->dag.clone(), + ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); diff --git a/tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.reference b/tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.sql b/tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.sql new file mode 100644 index 00000000000..913de3b849c --- /dev/null +++ b/tests/queries/0_stateless/03258_old_analyzer_const_expr_bug.sql @@ -0,0 +1,23 @@ +WITH + multiIf('-1' = '-1', 10080, '-1' = '7', 60, '-1' = '1', 5, 1440) AS interval_start, -- noqa + multiIf('-1' = '-1', CEIL((today() - toDate('2017-06-22')) / 7)::UInt16, '-1' = '7', 168, '-1' = '1', 288, 90) AS days_run, -- noqa:L045 + block_time as (SELECT arrayJoin( + arrayMap( + i -> toDateTime(toStartOfInterval(now(), INTERVAL interval_start MINUTE) - interval_start * 60 * i, 'UTC'), + range(days_run) + ) + )), + +sales AS ( + SELECT + toDateTime(toStartOfInterval(now(), INTERVAL interval_start MINUTE), 'UTC') AS block_time + FROM + numbers(1) + GROUP BY + block_time + ORDER BY + block_time) + +SELECT + block_time +FROM sales where block_time >= (SELECT MIN(block_time) FROM sales) format Null; From 349010012e7f29ad38b159e99dce7f297f076f63 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Tue, 5 Nov 2024 09:41:01 -0800 Subject: [PATCH 783/816] [Docs] Add cloud not supported badge for EmbeddedRocksDB engine --- .../engines/table-engines/integrations/embedded-rocksdb.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 1958250ed73..41c4e8fc4a9 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -4,9 +4,13 @@ sidebar_position: 50 sidebar_label: EmbeddedRocksDB --- +import CloudNotSupportedBadge from '@theme/badges/CloudNotSupportedBadge'; + # EmbeddedRocksDB Engine -This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). + + +This engine allows integrating ClickHouse with [RocksDB](http://rocksdb.org/). ## Creating a Table {#creating-a-table} From 27efa296849e1aaa649adb51ef280410169d8018 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Tue, 5 Nov 2024 18:04:59 +0000 Subject: [PATCH 784/816] update docs --- .../statements/select/order-by.md | 61 ++++++++++++++++++- 1 file changed, 60 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 512a58d7cd9..25d2e7123fd 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -291,7 +291,7 @@ All missed values of `expr` column will be filled sequentially and other columns To fill multiple columns, add `WITH FILL` modifier with optional parameters after each field name in `ORDER BY` section. ``` sql -ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] +ORDER BY expr [WITH FILL] [FROM const_expr] [TO const_expr] [STEP const_numeric_expr] [STALENESS const_numeric_expr], ... exprN [WITH FILL] [FROM expr] [TO expr] [STEP numeric_expr] [STALENESS numeric_expr] [INTERPOLATE [(col [AS expr], ... colN [AS exprN])]] ``` @@ -300,6 +300,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. +When `STALENESS const_numeric_expr` is defined, the query will generate rows until the difference from the previous row in the original data exceeds `const_numeric_expr`. `INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeat previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: @@ -497,6 +498,64 @@ Result: └────────────┴────────────┴──────────┘ ``` +Example of a query without `STALENESS`: + +``` sql +SELECT number as key, 5 * number value, 'original' AS source +FROM numbers(16) WHERE key % 5 == 0 +ORDER BY key WITH FILL; +``` + +Result: + +``` text + ┌─key─┬─value─┬─source───┐ + 1. │ 0 │ 0 │ original │ + 2. │ 1 │ 0 │ │ + 3. │ 2 │ 0 │ │ + 4. │ 3 │ 0 │ │ + 5. │ 4 │ 0 │ │ + 6. │ 5 │ 25 │ original │ + 7. │ 6 │ 0 │ │ + 8. │ 7 │ 0 │ │ + 9. │ 8 │ 0 │ │ +10. │ 9 │ 0 │ │ +11. │ 10 │ 50 │ original │ +12. │ 11 │ 0 │ │ +13. │ 12 │ 0 │ │ +14. │ 13 │ 0 │ │ +15. │ 14 │ 0 │ │ +16. │ 15 │ 75 │ original │ + └─────┴───────┴──────────┘ +``` + +Same query after applying `STALENESS 3`: + +``` sql +SELECT number as key, 5 * number value, 'original' AS source +FROM numbers(16) WHERE key % 5 == 0 +ORDER BY key WITH FILL STALENESS 3; +``` + +Result: + +``` text + ┌─key─┬─value─┬─source───┐ + 1. │ 0 │ 0 │ original │ + 2. │ 1 │ 0 │ │ + 3. │ 2 │ 0 │ │ + 4. │ 5 │ 25 │ original │ + 5. │ 6 │ 0 │ │ + 6. │ 7 │ 0 │ │ + 7. │ 10 │ 50 │ original │ + 8. │ 11 │ 0 │ │ + 9. │ 12 │ 0 │ │ +10. │ 15 │ 75 │ original │ +11. │ 16 │ 0 │ │ +12. │ 17 │ 0 │ │ + └─────┴───────┴──────────┘ +``` + Example of a query without `INTERPOLATE`: ``` sql From 9ec0dda6eeb52c482b4e1e5929b2e03f61672659 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Nov 2024 20:40:32 +0100 Subject: [PATCH 785/816] Prevent crash in SortCursor with 0 columns --- src/Core/SortCursor.h | 19 +++++++++++---- .../IMergingAlgorithmWithDelayedChunk.cpp | 9 +++++-- .../IMergingAlgorithmWithSharedChunks.cpp | 5 ++-- .../Algorithms/MergingSortedAlgorithm.cpp | 4 ++-- .../Transforms/MergeJoinTransform.cpp | 2 +- .../Transforms/SortingTransform.cpp | 2 +- .../03261_sort_cursor_crash.reference | 4 ++++ .../0_stateless/03261_sort_cursor_crash.sql | 24 +++++++++++++++++++ 8 files changed, 57 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/03261_sort_cursor_crash.reference create mode 100644 tests/queries/0_stateless/03261_sort_cursor_crash.sql diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index 3d568be199c..6eb009fa259 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -35,6 +35,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + /** Cursor allows to compare rows in different blocks (and parts). * Cursor moves inside single block. * It is used in priority queue. @@ -83,21 +88,27 @@ struct SortCursorImpl SortCursorImpl( const Block & header, const Columns & columns, + size_t num_rows, const SortDescription & desc_, size_t order_ = 0, IColumn::Permutation * perm = nullptr) : desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size()) { - reset(columns, header, perm); + reset(columns, header, num_rows, perm); } bool empty() const { return rows == 0; } /// Set the cursor to the beginning of the new block. - void reset(const Block & block, IColumn::Permutation * perm = nullptr) { reset(block.getColumns(), block, perm); } + void reset(const Block & block, IColumn::Permutation * perm = nullptr) + { + if (block.getColumns().empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column list in block"); + reset(block.getColumns(), block, block.getColumns()[0]->size(), perm); + } /// Set the cursor to the beginning of the new block. - void reset(const Columns & columns, const Block & block, IColumn::Permutation * perm = nullptr) + void reset(const Columns & columns, const Block & block, UInt64 num_rows, IColumn::Permutation * perm = nullptr) { all_columns.clear(); sort_columns.clear(); @@ -125,7 +136,7 @@ struct SortCursorImpl } pos = 0; - rows = all_columns[0]->size(); + rows = num_rows; permutation = perm; } diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp index cbad6813fbc..5e271e12943 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp @@ -24,7 +24,12 @@ void IMergingAlgorithmWithDelayedChunk::initializeQueue(Inputs inputs) continue; cursors[source_num] = SortCursorImpl( - header, current_inputs[source_num].chunk.getColumns(), description, source_num, current_inputs[source_num].permutation); + header, + current_inputs[source_num].chunk.getColumns(), + current_inputs[source_num].chunk.getNumRows(), + description, + source_num, + current_inputs[source_num].permutation); inputs_origin_merge_tree_part_level[source_num] = getPartLevelFromChunk(current_inputs[source_num].chunk); } @@ -41,7 +46,7 @@ void IMergingAlgorithmWithDelayedChunk::updateCursor(Input & input, size_t sourc last_chunk_sort_columns = std::move(cursors[source_num].sort_columns); current_input.swap(input); - cursors[source_num].reset(current_input.chunk.getColumns(), header, current_input.permutation); + cursors[source_num].reset(current_input.chunk.getColumns(), header, current_input.chunk.getNumRows(), current_input.permutation); inputs_origin_merge_tree_part_level[source_num] = getPartLevelFromChunk(current_input.chunk); diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp index 47b7ddf38dc..f99f021286e 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp @@ -31,7 +31,8 @@ void IMergingAlgorithmWithSharedChunks::initialize(Inputs inputs) source.skip_last_row = inputs[source_num].skip_last_row; source.chunk = chunk_allocator.alloc(inputs[source_num].chunk); - cursors[source_num] = SortCursorImpl(header, source.chunk->getColumns(), description, source_num, inputs[source_num].permutation); + cursors[source_num] = SortCursorImpl( + header, source.chunk->getColumns(), source.chunk->getNumRows(), description, source_num, inputs[source_num].permutation); source.chunk->all_columns = cursors[source_num].all_columns; source.chunk->sort_columns = cursors[source_num].sort_columns; @@ -49,7 +50,7 @@ void IMergingAlgorithmWithSharedChunks::consume(Input & input, size_t source_num auto & source = sources[source_num]; source.skip_last_row = input.skip_last_row; source.chunk = chunk_allocator.alloc(input.chunk); - cursors[source_num].reset(source.chunk->getColumns(), header, input.permutation); + cursors[source_num].reset(source.chunk->getColumns(), header, source.chunk->getNumRows(), input.permutation); source.chunk->all_columns = cursors[source_num].all_columns; source.chunk->sort_columns = cursors[source_num].sort_columns; diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 3a9cf7ee141..28c6cb473e5 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -59,7 +59,7 @@ void MergingSortedAlgorithm::initialize(Inputs inputs) if (!chunk) continue; - cursors[source_num] = SortCursorImpl(header, chunk.getColumns(), description, source_num); + cursors[source_num] = SortCursorImpl(header, chunk.getColumns(), chunk.getNumRows(), description, source_num); } if (sorting_queue_strategy == SortingQueueStrategy::Default) @@ -84,7 +84,7 @@ void MergingSortedAlgorithm::consume(Input & input, size_t source_num) { removeConstAndSparse(input); current_inputs[source_num].swap(input); - cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), header); + cursors[source_num].reset(current_inputs[source_num].chunk.getColumns(), header, current_inputs[source_num].chunk.getNumRows()); if (sorting_queue_strategy == SortingQueueStrategy::Default) { diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 1675e5d0386..77a437d4b97 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -394,7 +394,7 @@ void FullMergeJoinCursor::setChunk(Chunk && chunk) convertToFullIfSparse(chunk); current_chunk = std::move(chunk); - cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), desc); + cursor = SortCursorImpl(sample_block, current_chunk.getColumns(), current_chunk.getNumRows(), desc); } bool FullMergeJoinCursor::fullyCompleted() const diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 6e65093e9e2..6a11354e2bf 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -42,7 +42,7 @@ MergeSorter::MergeSorter(const Block & header, Chunks chunks_, SortDescription & /// Convert to full column, because some cursors expect non-contant columns convertToFullIfConst(chunk); - cursors.emplace_back(header, chunk.getColumns(), description, chunk_index); + cursors.emplace_back(header, chunk.getColumns(), chunk.getNumRows(), description, chunk_index); has_collation |= cursors.back().has_collation; nonempty_chunks.emplace_back(std::move(chunk)); diff --git a/tests/queries/0_stateless/03261_sort_cursor_crash.reference b/tests/queries/0_stateless/03261_sort_cursor_crash.reference new file mode 100644 index 00000000000..7299f2f5a5f --- /dev/null +++ b/tests/queries/0_stateless/03261_sort_cursor_crash.reference @@ -0,0 +1,4 @@ +42 +43 +44 +45 diff --git a/tests/queries/0_stateless/03261_sort_cursor_crash.sql b/tests/queries/0_stateless/03261_sort_cursor_crash.sql new file mode 100644 index 00000000000..b659f3d4a92 --- /dev/null +++ b/tests/queries/0_stateless/03261_sort_cursor_crash.sql @@ -0,0 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/70779 +-- Crash in SortCursorImpl with the old analyzer, which produces a block with 0 columns and 1 row +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; + +CREATE TABLE t0 (c0 Int) ENGINE = AggregatingMergeTree() ORDER BY tuple(); +INSERT INTO TABLE t0 (c0) VALUES (1); +SELECT 42 FROM t0 FINAL PREWHERE t0.c0 = 1; +DROP TABLE t0; + +CREATE TABLE t0 (c0 Int) ENGINE = SummingMergeTree() ORDER BY tuple(); +INSERT INTO TABLE t0 (c0) VALUES (1); +SELECT 43 FROM t0 FINAL PREWHERE t0.c0 = 1; +DROP TABLE t0; + +CREATE TABLE t0 (c0 Int) ENGINE = ReplacingMergeTree() ORDER BY tuple(); +INSERT INTO TABLE t0 (c0) VALUES (1); +SELECT 44 FROM t0 FINAL PREWHERE t0.c0 = 1; +DROP TABLE t0; + +CREATE TABLE t1 (a0 UInt8, c0 Int32, c1 UInt8) ENGINE = AggregatingMergeTree() ORDER BY tuple(); +INSERT INTO TABLE t1 (a0, c0, c1) VALUES (1, 1, 1); +SELECT 45 FROM t1 FINAL PREWHERE t1.c0 = t1.c1; +DROP TABLE t1; From 9931b61d6fc0989facbc430d353e611d70d44b5c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 5 Nov 2024 20:56:04 +0100 Subject: [PATCH 786/816] fix test --- ...03255_parallel_replicas_join_algo_and_analyzer_4.reference | 4 ++-- .../03255_parallel_replicas_join_algo_and_analyzer_4.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference index 52c4e872f84..d846b26b72b 100644 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.reference @@ -84,7 +84,7 @@ SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP 500020000 500030000 500040000 -SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_4551627371769371400_3093038500622465792` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_x_y_` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC 4999950000 4999950000 SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t` AS `__table1` GROUP BY `__table1`.`item_id` @@ -113,4 +113,4 @@ SELECT `__table1`.`item_id` AS `item_id` FROM `default`.`t1` AS `__table1` GROUP 500020000 500030000 500040000 -SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_4551627371769371400_3093038500622465792` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC +SELECT sum(`__table1`.`item_id`) AS `sum(item_id)` FROM (SELECT `__table2`.`item_id` AS `item_id`, `__table2`.`price_sold` AS `price_sold` FROM `default`.`t` AS `__table2`) AS `__table1` GLOBAL ALL LEFT JOIN `_data_x_y_` AS `__table3` ON `__table1`.`item_id` = `__table3`.`item_id` GROUP BY `__table1`.`price_sold` ORDER BY `__table1`.`price_sold` ASC diff --git a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh index 18a2fbd317b..19866f26949 100755 --- a/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh +++ b/tests/queries/0_stateless/03255_parallel_replicas_join_algo_and_analyzer_4.sh @@ -88,7 +88,7 @@ for parallel_replicas_prefer_local_join in 1 0; do --SELECT '----- enable_parallel_replicas=$enable_parallel_replicas prefer_local_plan=$prefer_local_plan parallel_replicas_prefer_local_join=$parallel_replicas_prefer_local_join -----'; ${query}; - SELECT replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1') + SELECT replaceRegexpAll(replaceRegexpAll(explain, '.*Query: (.*) Replicas:.*', '\\1'), '(.*)_data_[\d]+_[\d]+(.*)', '\1_data_x_y_\2') FROM ( EXPLAIN actions=1 ${query} From 24c5ef9a052b464671cfb78e887b11237281f53b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 5 Nov 2024 23:08:15 +0100 Subject: [PATCH 787/816] Expose base setting for merge selector --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 ++ src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 62ad9d4a52a..6b9638b11d2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -71,6 +71,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsUInt64 parts_to_throw_insert; extern const MergeTreeSettingsMergeSelectorAlgorithm merge_selector_algorithm; extern const MergeTreeSettingsBool merge_selector_enable_heuristic_to_remove_small_parts_at_right; + extern const MergeTreeSettingsFloat merge_selector_base; } namespace ErrorCodes @@ -542,6 +543,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( simple_merge_settings.window_size = (*data_settings)[MergeTreeSetting::merge_selector_window_size]; simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once]; simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSetting::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; + simple_merge_settings.base = (*data_settings)[MergeTreeSetting::merge_selector_base]; if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only]) simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds]; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 883191d59ab..33910d1048d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -101,6 +101,7 @@ namespace ErrorCodes DECLARE(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \ DECLARE(Bool, merge_selector_enable_heuristic_to_remove_small_parts_at_right, true, "Enable heuristic for selecting parts for merge which removes parts from right side of range, if their size is less than specified ratio (0.01) of sum_size. Works for Simple and StochasticSimple merge selectors", 0) \ + DECLARE(Float, merge_selector_base, 5.0, "Affects write amplification of assigned merges (expert level setting, don't change if you don't understand what it is doing). Works for Simple and StochasticSimple merge selectors", 0) \ \ /** Inserts settings. */ \ DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 0c1aa03cb172ca666b7054863626d563e1de21e7 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Wed, 6 Nov 2024 00:05:55 -0800 Subject: [PATCH 788/816] [Docs] Update note about Prometheus integration and ClickHouse Cloud --- docs/en/interfaces/prometheus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/prometheus.md b/docs/en/interfaces/prometheus.md index 8e7023cc51f..11f503b54d7 100644 --- a/docs/en/interfaces/prometheus.md +++ b/docs/en/interfaces/prometheus.md @@ -9,7 +9,7 @@ sidebar_label: Prometheus protocols ## Exposing metrics {#expose} :::note -ClickHouse Cloud does not currently support connecting to Prometheus. To be notified when this feature is supported, please contact support@clickhouse.com. +If you are using ClickHouse Cloud, you can expose metrics to Prometheus using the [Prometheus Integration](/en/integrations/prometheus). ::: ClickHouse can expose its own metrics for scraping from Prometheus: From 4f8099d7aa6d1dff2ad79fc020810fe36a3cfd3b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 08:51:44 +0000 Subject: [PATCH 789/816] Simplify the code --- .../MergeTreeIndexVectorSimilarity.cpp | 81 +++++++++---------- .../0_stateless/02354_vector_search_bugs.sql | 2 +- 2 files changed, 40 insertions(+), 43 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index 498d0131d5a..e55010ac9ec 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -345,60 +345,57 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception(ErrorCodes::INCORRECT_DATA, "Index granularity is too big: more than {} rows per index granule.", std::numeric_limits::max()); if (index_sample_block.columns() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected that index is build over a single column"); - for (size_t i = 0; i < index_sample_block.columns(); ++i) - { - const auto & index_column_with_type_and_name = index_sample_block.getByPosition(i); + const auto & index_column_with_type_and_name = index_sample_block.getByPosition(0); - const auto & index_column_name = index_column_with_type_and_name.name; - const auto & index_column = block.getByName(index_column_name).column; - ColumnPtr column_cut = index_column->cut(*pos, rows_read); + const auto & index_column_name = index_column_with_type_and_name.name; + const auto & index_column = block.getByName(index_column_name).column; + ColumnPtr column_cut = index_column->cut(*pos, rows_read); - const auto * column_array = typeid_cast(column_cut.get()); - if (!column_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float*) column"); + const auto * column_array = typeid_cast(column_cut.get()); + if (!column_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array(Float*) column"); - if (column_array->empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); + if (column_array->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Array is unexpectedly empty"); - /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays - /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default - /// values which is also empty. - if (column_array->isDefaultAt(0)) - throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); + /// The vector similarity algorithm naturally assumes that the indexed vectors have dimension >= 1. This condition is violated if empty arrays + /// are INSERTed into an vector-similarity-indexed column or if no value was specified at all in which case the arrays take on their default + /// values which is also empty. + if (column_array->isDefaultAt(0)) + throw Exception(ErrorCodes::INCORRECT_DATA, "The arrays in column '{}' must not be empty. Did you try to INSERT default values?", index_column_name); - const size_t rows = column_array->size(); + const size_t rows = column_array->size(); - const auto & column_array_offsets = column_array->getOffsets(); - const size_t dimensions = column_array_offsets[0]; + const auto & column_array_offsets = column_array->getOffsets(); + const size_t dimensions = column_array_offsets[0]; - if (!index) - index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); + if (!index) + index = std::make_shared(dimensions, metric_kind, scalar_kind, usearch_hnsw_params); - /// Also check that previously inserted blocks have the same size as this block. - /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across - /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. - if (index->dimensions() != dimensions) - throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); + /// Also check that previously inserted blocks have the same size as this block. + /// Note that this guarantees consistency of dimension only within parts. We are unable to detect inconsistent dimensions across + /// parts - for this, a little help from the user is needed, e.g. CONSTRAINT cnstr CHECK length(array) = 42. + if (index->dimensions() != dimensions) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column with vector similarity index must have equal length"); - /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp - if (index->size() + rows > std::numeric_limits::max()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); + /// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp + if (index->size() + rows > std::numeric_limits::max()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); - DataTypePtr data_type = index_column_with_type_and_name.type; - const auto * data_type_array = typeid_cast(data_type.get()); - if (!data_type_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); - const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); + DataTypePtr data_type = index_column_with_type_and_name.type; + const auto * data_type_array = typeid_cast(data_type.get()); + if (!data_type_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); + const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); - if (WhichDataType(nested_type_index).isFloat32()) - updateImpl(column_array, column_array_offsets, index, dimensions, rows); - else if (WhichDataType(nested_type_index).isFloat64()) - updateImpl(column_array, column_array_offsets, index, dimensions, rows); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); - } + if (WhichDataType(nested_type_index).isFloat32()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else if (WhichDataType(nested_type_index).isFloat64()) + updateImpl(column_array, column_array_offsets, index, dimensions, rows); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); *pos += rows_read; diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql index 6bcb0f78e75..276d4eb5b59 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ b/tests/queries/0_stateless/02354_vector_search_bugs.sql @@ -124,7 +124,7 @@ CREATE TABLE tab( val String, vec Array(Float32), INDEX ann_idx vec TYPE vector_similarity('hnsw', 'cosineDistance'), - INDEX set_idx val TYPE set(100) GRANULARITY 100 + INDEX set_idx val TYPE set(100) ) ENGINE = MergeTree() ORDER BY tuple(); From 918ad5c4d54c27b6c14e1221ae56a40dd937e2cc Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 6 Nov 2024 09:42:35 +0000 Subject: [PATCH 790/816] fix_test_drop_complex_columns: tests passed --- .../test.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py b/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py index 6d2bb0a3b70..9937c0ed4ea 100644 --- a/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py +++ b/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py @@ -68,9 +68,19 @@ CREATE TABLE test_s3(c1 Int8, c2 Date) ENGINE = ReplicatedMergeTree('/test/table def test_drop_complex_columns(started_cluster): + node1 = cluster.instances["node1"] + node1.query( + """ +CREATE TABLE warming_up( +id Int8 +) ENGINE = MergeTree +order by (id) SETTINGS storage_policy = 's3';""" + ) + + # Now we are sure that s3 storage is up and running start_objects = get_objects_in_data_path() print("Objects before", start_objects) - node1 = cluster.instances["node1"] + node1.query( """ CREATE TABLE test_s3_complex_types( @@ -104,3 +114,4 @@ vertical_merge_algorithm_min_columns_to_activate=1;""" end_objects = get_objects_in_data_path() print("Objects after drop", end_objects) assert start_objects == end_objects + node1.query("DROP TABLE warming_up SYNC") From b38dc1d8ca791c6fc686ae9d8efedeb77e354de2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 6 Nov 2024 11:05:43 +0100 Subject: [PATCH 791/816] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index ae3c9c58fc5..f7b7ffc5aea 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1438,8 +1438,6 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) "cached file `{}` does not fit in cache anymore (size: {})", size_limit, offset_it->path().string(), size); - chassert(false); /// TODO: remove before merge. - fs::remove(offset_it->path()); } } From f0bb69f12667108659b5ed9803f4b290c7faafee Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 11:46:49 +0000 Subject: [PATCH 792/816] Simplify more --- src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index e55010ac9ec..f95b840e223 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -347,9 +347,8 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (index_sample_block.columns() > 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected that index is build over a single column"); - const auto & index_column_with_type_and_name = index_sample_block.getByPosition(0); + const auto & index_column_name = index_sample_block.getByPosition(0).name; - const auto & index_column_name = index_column_with_type_and_name.name; const auto & index_column = block.getByName(index_column_name).column; ColumnPtr column_cut = index_column->cut(*pos, rows_read); @@ -384,8 +383,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (index->size() + rows > std::numeric_limits::max()) throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); - DataTypePtr data_type = index_column_with_type_and_name.type; - const auto * data_type_array = typeid_cast(data_type.get()); + const auto * data_type_array = typeid_cast(block.getByName(index_column_name).type.get()); if (!data_type_array) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); From 7c6472a09034715bbeb8374667203076c3458e82 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Wed, 6 Nov 2024 13:34:39 +0100 Subject: [PATCH 793/816] Fix documentation for system.grants.is_partial_revoke --- docs/en/operations/system-tables/grants.md | 4 ++-- src/Storages/System/StorageSystemGrants.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/system-tables/grants.md b/docs/en/operations/system-tables/grants.md index 262a53a87a5..debc3146008 100644 --- a/docs/en/operations/system-tables/grants.md +++ b/docs/en/operations/system-tables/grants.md @@ -19,7 +19,7 @@ Columns: - `column` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Name of a column to which access is granted. - `is_partial_revoke` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Logical value. It shows whether some privileges have been revoked. Possible values: -- `0` — The row describes a partial revoke. -- `1` — The row describes a grant. +- `0` — The row describes a grant. +- `1` — The row describes a partial revoke. - `grant_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Permission is granted `WITH GRANT OPTION`, see [GRANT](../../sql-reference/statements/grant.md#granting-privilege-syntax). diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index 5de1f8cef55..aa010e44388 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -30,8 +30,8 @@ ColumnsDescription StorageSystemGrants::getColumnsDescription() {"column", std::make_shared(std::make_shared()), "Name of a column to which access is granted."}, {"is_partial_revoke", std::make_shared(), "Logical value. It shows whether some privileges have been revoked. Possible values: " - "0 — The row describes a partial revoke, " - "1 — The row describes a grant." + "0 — The row describes a grant, " + "1 — The row describes a partial revoke." }, {"grant_option", std::make_shared(), "Permission is granted WITH GRANT OPTION."}, }; From 9ee22533a067fc235aea65ff7b89c801b112b918 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 6 Nov 2024 13:46:30 +0100 Subject: [PATCH 794/816] Move bitShift function changelog entries to backward incompatible Move bitShift function changelog entries to backward incompatible --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 90285582b4e..dacee73440f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -488,6 +488,7 @@ * Remove `is_deterministic` field from the `system.functions` table. [#66630](https://github.com/ClickHouse/ClickHouse/pull/66630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Function `tuple` will now try to construct named tuples in query (controlled by `enable_named_columns_in_function_tuple`). Introduce function `tupleNames` to extract names from tuples. [#54881](https://github.com/ClickHouse/ClickHouse/pull/54881) ([Amos Bird](https://github.com/amosbird)). * Change how deduplication for Materialized Views works. Fixed a lot of cases like: - on destination table: data is split for 2 or more blocks and that blocks is considered as duplicate when that block is inserted in parallel. - on MV destination table: the equal blocks are deduplicated, that happens when MV often produces equal data as a result for different input data due to performing aggregation. - on MV destination table: the equal blocks which comes from different MV are deduplicated. [#61601](https://github.com/ClickHouse/ClickHouse/pull/61601) ([Sema Checherinda](https://github.com/CheSema)). +* Functions `bitShiftLeft` and `bitShitfRight` return an error for out of bounds shift positions [#65838](https://github.com/ClickHouse/ClickHouse/pull/65838) ([Pablo Marcos](https://github.com/pamarcos)). #### New Feature * Add `ASOF JOIN` support for `full_sorting_join` algorithm. [#55051](https://github.com/ClickHouse/ClickHouse/pull/55051) ([vdimir](https://github.com/vdimir)). @@ -599,7 +600,6 @@ * Functions `bitTest`, `bitTestAll`, and `bitTestAny` now return an error if the specified bit index is out-of-bounds [#65818](https://github.com/ClickHouse/ClickHouse/pull/65818) ([Pablo Marcos](https://github.com/pamarcos)). * Setting `join_any_take_last_row` is supported in any query with hash join. [#65820](https://github.com/ClickHouse/ClickHouse/pull/65820) ([vdimir](https://github.com/vdimir)). * Better handling of join conditions involving `IS NULL` checks (for example `ON (a = b AND (a IS NOT NULL) AND (b IS NOT NULL) ) OR ( (a IS NULL) AND (b IS NULL) )` is rewritten to `ON a <=> b`), fix incorrect optimization when condition other then `IS NULL` are present. [#65835](https://github.com/ClickHouse/ClickHouse/pull/65835) ([vdimir](https://github.com/vdimir)). -* Functions `bitShiftLeft` and `bitShitfRight` return an error for out of bounds shift positions [#65838](https://github.com/ClickHouse/ClickHouse/pull/65838) ([Pablo Marcos](https://github.com/pamarcos)). * Fix growing memory usage in S3Queue. [#65839](https://github.com/ClickHouse/ClickHouse/pull/65839) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix tie handling in `arrayAUC` to match sklearn. [#65840](https://github.com/ClickHouse/ClickHouse/pull/65840) ([gabrielmcg44](https://github.com/gabrielmcg44)). * Fix possible issues with MySQL server protocol TLS connections. [#65917](https://github.com/ClickHouse/ClickHouse/pull/65917) ([Azat Khuzhin](https://github.com/azat)). From 533009b914761e317025b256b31474f44a9b4734 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 6 Nov 2024 08:57:32 -0400 Subject: [PATCH 795/816] Update AlterCommands.cpp --- src/Storages/AlterCommands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ab4403b3a94..c14775057a5 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1496,7 +1496,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const if (command.to_remove == AlterCommand::RemoveProperty::CODEC && column_from_table.codec == nullptr) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Column {} doesn't have TTL, cannot remove it", + "Column {} doesn't have CODEC, cannot remove it", backQuote(column_name)); if (command.to_remove == AlterCommand::RemoveProperty::COMMENT && column_from_table.comment.empty()) throw Exception( From e5b6a3c1fe9773953e01f7de161bc0c36a75b454 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Nov 2024 14:33:25 +0100 Subject: [PATCH 796/816] Update 03261_tuple_map_object_to_json_cast.sql --- .../queries/0_stateless/03261_tuple_map_object_to_json_cast.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql index 91d3f504f92..2e5cecaf502 100644 --- a/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql +++ b/tests/queries/0_stateless/03261_tuple_map_object_to_json_cast.sql @@ -5,6 +5,7 @@ set allow_experimental_object_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; set enable_named_columns_in_function_tuple = 1; +set enable_analyzer = 1; select 'Map to JSON'; select map('a', number::UInt32, 'b', toDate(number), 'c', range(number), 'd', [map('e', number::UInt32)])::JSON as json, JSONAllPathsWithTypes(json) from numbers(5); From d270885bfa52548dbf342b5ddacf8803a354d2a8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Nov 2024 21:37:47 +0800 Subject: [PATCH 797/816] Allow specifying cmdline flags in integration test --- tests/integration/helpers/cluster.py | 26 +++++++++++++++++--------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6751f205fb8..e2237363131 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1653,6 +1653,7 @@ class ClickHouseCluster: copy_common_configs=True, config_root_name="clickhouse", extra_configs=[], + extra_args="", randomize_settings=True, ) -> "ClickHouseInstance": """Add an instance to the cluster. @@ -1740,6 +1741,7 @@ class ClickHouseCluster: with_postgres_cluster=with_postgres_cluster, with_postgresql_java_client=with_postgresql_java_client, clickhouse_start_command=clickhouse_start_command, + clickhouse_start_extra_args=extra_args, main_config_name=main_config_name, users_config_name=users_config_name, copy_common_configs=copy_common_configs, @@ -3368,6 +3370,7 @@ class ClickHouseInstance: with_postgres_cluster, with_postgresql_java_client, clickhouse_start_command=CLICKHOUSE_START_COMMAND, + clickhouse_start_extra_args="", main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, @@ -3463,11 +3466,18 @@ class ClickHouseInstance: self.users_config_name = users_config_name self.copy_common_configs = copy_common_configs - self.clickhouse_start_command = clickhouse_start_command.replace( + clickhouse_start_command_with_conf = clickhouse_start_command.replace( "{main_config_file}", self.main_config_name ) - self.clickhouse_stay_alive_command = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( - clickhouse_start_command + + self.clickhouse_start_command = "{} -- {}".format( + clickhouse_start_command_with_conf, clickhouse_start_extra_args + ) + self.clickhouse_start_command_in_daemon = "{} --daemon -- {}".format( + clickhouse_start_command_with_conf, clickhouse_start_extra_args + ) + self.clickhouse_stay_alive_command = "bash -c \"trap 'pkill tail' INT TERM; {}; coproc tail -f /dev/null; wait $$!\"".format( + self.clickhouse_start_command_in_daemon ) self.path = p.join(self.cluster.instances_dir, name) @@ -3910,7 +3920,7 @@ class ClickHouseInstance: if pid is None: logging.debug("No clickhouse process running. Start new one.") self.exec_in_container( - ["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], + ["bash", "-c", self.clickhouse_start_command_in_daemon], user=str(os.getuid()), ) if expected_to_fail: @@ -4230,7 +4240,7 @@ class ClickHouseInstance: user="root", ) self.exec_in_container( - ["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], + ["bash", "-c", self.clickhouse_start_command_in_daemon], user=str(os.getuid()), ) @@ -4311,7 +4321,7 @@ class ClickHouseInstance: ] ) self.exec_in_container( - ["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], + ["bash", "-c", self.clickhouse_start_command_in_daemon], user=str(os.getuid()), ) @@ -4704,9 +4714,7 @@ class ClickHouseInstance: entrypoint_cmd = self.clickhouse_start_command if self.stay_alive: - entrypoint_cmd = self.clickhouse_stay_alive_command.replace( - "{main_config_file}", self.main_config_name - ) + entrypoint_cmd = self.clickhouse_stay_alive_command else: entrypoint_cmd = ( "[" From 71a0e7f07f41c0388b98849717240e845c53dc67 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 13:34:05 +0000 Subject: [PATCH 798/816] Split tests --- ...> 02354_vector_search_bug_52282.reference} | 0 .../02354_vector_search_bug_52282.sql | 13 ++ ...> 02354_vector_search_bug_69085.reference} | 9 -- .../02354_vector_search_bug_69085.sql | 52 +++++++ .../02354_vector_search_bug_71381.reference | 0 .../02354_vector_search_bug_71381.sql | 20 +++ ...h_bug_adaptive_index_granularity.reference | 0 ..._search_bug_adaptive_index_granularity.sql | 20 +++ ...search_bug_different_array_sizes.reference | 0 ...ector_search_bug_different_array_sizes.sql | 24 ++++ ...ctor_search_bug_multiple_indexes.reference | 0 ...354_vector_search_bug_multiple_indexes.sql | 14 ++ ...vector_search_bug_multiple_marks.reference | 2 + ...02354_vector_search_bug_multiple_marks.sql | 25 ++++ .../0_stateless/02354_vector_search_bugs.sql | 134 ------------------ .../02354_vector_search_multiple_indexes.sql | 1 + 16 files changed, 171 insertions(+), 143 deletions(-) rename tests/queries/0_stateless/{02354_vector_search_multiple_indexes.reference => 02354_vector_search_bug_52282.reference} (100%) create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_52282.sql rename tests/queries/0_stateless/{02354_vector_search_bugs.reference => 02354_vector_search_bug_69085.reference} (68%) create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_69085.sql create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_71381.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_71381.sql create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.sql create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.sql create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.sql create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.reference create mode 100644 tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.sql delete mode 100644 tests/queries/0_stateless/02354_vector_search_bugs.sql diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_bug_52282.reference similarity index 100% rename from tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference rename to tests/queries/0_stateless/02354_vector_search_bug_52282.reference diff --git a/tests/queries/0_stateless/02354_vector_search_bug_52282.sql b/tests/queries/0_stateless/02354_vector_search_bug_52282.sql new file mode 100644 index 00000000000..b8066ce278a --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_52282.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; + +-- Issue #52258: Vector similarity indexes must reject empty Arrays or Arrays with default values + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id; +INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } +INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.reference b/tests/queries/0_stateless/02354_vector_search_bug_69085.reference similarity index 68% rename from tests/queries/0_stateless/02354_vector_search_bugs.reference rename to tests/queries/0_stateless/02354_vector_search_bug_69085.reference index dec921cf586..3b4e2d9ef17 100644 --- a/tests/queries/0_stateless/02354_vector_search_bugs.reference +++ b/tests/queries/0_stateless/02354_vector_search_bug_69085.reference @@ -1,10 +1,3 @@ -Rejects INSERTs of Arrays with different sizes -Issue #52258: Empty Arrays or Arrays with default values are rejected -It is possible to create parts with different Array vector sizes but there will be an error at query time -Correctness of index with > 1 mark -1 [1,0] 0 -9000 [9000,0] 0 -Issue #69085: Reference vector computed by a subquery Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) @@ -40,5 +33,3 @@ Expression (Projection) Condition: true Parts: 1/1 Granules: 4/4 -index_granularity_bytes = 0 is disallowed -Issue #71381: Vector similarity index and other skipping indexes used on the same table diff --git a/tests/queries/0_stateless/02354_vector_search_bug_69085.sql b/tests/queries/0_stateless/02354_vector_search_bug_69085.sql new file mode 100644 index 00000000000..4dbcdf66e36 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_69085.sql @@ -0,0 +1,52 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; +SET enable_analyzer = 0; + +-- Issue #69085: Reference vector for vector search is computed by a subquery + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; +INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); + +-- works +EXPLAIN indexes = 1 +WITH [0., 2.] AS reference_vec +SELECT + id, + vec, + cosineDistance(vec, reference_vec) AS distance +FROM tab +ORDER BY distance +LIMIT 1; + +-- does not work +EXPLAIN indexes = 1 +WITH ( + SELECT vec + FROM tab + LIMIT 1 +) AS reference_vec +SELECT + id, + vec, + cosineDistance(vec, reference_vec) AS distance +FROM tab +ORDER BY distance +LIMIT 1; + +-- does not work as well +EXPLAIN indexes = 1 +WITH ( + SELECT [0., 2.] +) AS reference_vec +SELECT + id, + vec, + cosineDistance(vec, reference_vec) AS distance +FROM tab +ORDER BY distance +LIMIT 1; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bug_71381.reference b/tests/queries/0_stateless/02354_vector_search_bug_71381.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_bug_71381.sql b/tests/queries/0_stateless/02354_vector_search_bug_71381.sql new file mode 100644 index 00000000000..9e3246700b8 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_71381.sql @@ -0,0 +1,20 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; + +-- Issue #71381: Usage of vector similarity index and further skipping indexes on the same table + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab( + val String, + vec Array(Float32), + INDEX ann_idx vec TYPE vector_similarity('hnsw', 'cosineDistance'), + INDEX set_idx val TYPE set(100) +) +ENGINE = MergeTree() +ORDER BY tuple(); + +INSERT INTO tab VALUES ('hello world', [0.0]); + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.reference b/tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.sql b/tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.sql new file mode 100644 index 00000000000..208b5b7a874 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_adaptive_index_granularity.sql @@ -0,0 +1,20 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests that vector similarity indexes cannot be created with index_granularity_bytes = 0 + +SET allow_experimental_vector_similarity_index = 1; + +DROP TABLE IF EXISTS tab; + +-- If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs. +-- SET allow_experimental_vector_similarity_index = 1; +-- CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; +-- INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000); +-- WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100; +-- As a workaround, force enabled adaptive index granularity for now (it is the default anyways). +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -- { serverError INVALID_SETTING_VALUE } + +CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; +ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE } + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.reference b/tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.sql b/tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.sql new file mode 100644 index 00000000000..41b9d7869e4 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_different_array_sizes.sql @@ -0,0 +1,24 @@ +-- Tags: no-fasttest, no-ordinary-database + +SET allow_experimental_vector_similarity_index = 1; +SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; + +-- Vector similarity indexes reject INSERTs of Arrays with different sizes +INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } + +-- It is possible to create parts with different Array vector sizes but there will be an error at query time +SYSTEM STOP MERGES tab; +INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); +INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); + +WITH [0.0, 2.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 3; -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.sql b/tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.sql new file mode 100644 index 00000000000..f1cfc041233 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_multiple_indexes.sql @@ -0,0 +1,14 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests that multiple vector similarity indexes can be created on the same column (even if that makes no sense) + +SET allow_experimental_vector_similarity_index = 1; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); + +ALTER TABLE tab ADD INDEX idx(vec) TYPE minmax; +ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); +ALTER TABLE tab ADD INDEX vec_idx2(vec) TYPE vector_similarity('hnsw', 'L2Distance'); -- silly but creating the same index also works for non-vector indexes ... + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.reference b/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.reference new file mode 100644 index 00000000000..117bf2cead8 --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.reference @@ -0,0 +1,2 @@ +1 [1,0] 0 +9000 [9000,0] 0 diff --git a/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.sql b/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.sql new file mode 100644 index 00000000000..fb99dd2361c --- /dev/null +++ b/tests/queries/0_stateless/02354_vector_search_bug_multiple_marks.sql @@ -0,0 +1,25 @@ +-- Tags: no-fasttest, no-ordinary-database + +-- Tests correctness of vector similarity index with > 1 mark + +SET allow_experimental_vector_similarity_index = 1; +SET enable_analyzer = 0; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; +INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); + +WITH [1.0, 0.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 1; + +WITH [9000.0, 0.0] AS reference_vec +SELECT id, vec, L2Distance(vec, reference_vec) +FROM tab +ORDER BY L2Distance(vec, reference_vec) +LIMIT 1; + +DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_bugs.sql b/tests/queries/0_stateless/02354_vector_search_bugs.sql deleted file mode 100644 index 276d4eb5b59..00000000000 --- a/tests/queries/0_stateless/02354_vector_search_bugs.sql +++ /dev/null @@ -1,134 +0,0 @@ --- Tags: no-fasttest, no-ordinary-database - --- Tests various bugs and special cases for vector indexes. - -SET allow_experimental_vector_similarity_index = 1; -SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof - -DROP TABLE IF EXISTS tab; - -SELECT 'Rejects INSERTs of Arrays with different sizes'; - -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA } -DROP TABLE tab; - -SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected'; - -CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id; -INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA } -INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA } -DROP TABLE tab; - -SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time'; - -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -SYSTEM STOP MERGES tab; -INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]); -INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]); - -WITH [0.0, 2.0] AS reference_vec -SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab -ORDER BY L2Distance(vec, reference_vec) -LIMIT 3; -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - -DROP TABLE tab; - -SELECT 'Correctness of index with > 1 mark'; - -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192; -INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000); - -WITH [1.0, 0.0] AS reference_vec -SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab -ORDER BY L2Distance(vec, reference_vec) -LIMIT 1; - -WITH [9000.0, 0.0] AS reference_vec -SELECT id, vec, L2Distance(vec, reference_vec) -FROM tab -ORDER BY L2Distance(vec, reference_vec) -LIMIT 1; - -DROP TABLE tab; - -SELECT 'Issue #69085: Reference vector computed by a subquery'; - -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3; -INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]); - --- works -EXPLAIN indexes = 1 -WITH [0., 2.] AS reference_vec -SELECT - id, - vec, - cosineDistance(vec, reference_vec) AS distance -FROM tab -ORDER BY distance -LIMIT 1 -SETTINGS enable_analyzer = 0; - --- does not work -EXPLAIN indexes = 1 -WITH ( - SELECT vec - FROM tab - LIMIT 1 -) AS reference_vec -SELECT - id, - vec, - cosineDistance(vec, reference_vec) AS distance -FROM tab -ORDER BY distance -LIMIT 1 -SETTINGS enable_analyzer = 0; - --- does not work as well -EXPLAIN indexes = 1 -WITH ( - SELECT [0., 2.] -) AS reference_vec -SELECT - id, - vec, - cosineDistance(vec, reference_vec) AS distance -FROM tab -ORDER BY distance -LIMIT 1 -SETTINGS enable_analyzer = 0; - -DROP TABLE tab; - -SELECT 'index_granularity_bytes = 0 is disallowed'; - --- If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs. --- SET allow_experimental_vector_similarity_index = 1; --- CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; --- INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000); --- WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100; --- As a workaround, force enabled adaptive index granularity for now (it is the default anyways). -CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -- { serverError INVALID_SETTING_VALUE } - -CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE } - -DROP TABLE tab; - -SELECT 'Issue #71381: Vector similarity index and other skipping indexes used on the same table'; - -CREATE TABLE tab( - val String, - vec Array(Float32), - INDEX ann_idx vec TYPE vector_similarity('hnsw', 'cosineDistance'), - INDEX set_idx val TYPE set(100) -) -ENGINE = MergeTree() -ORDER BY tuple(); - -INSERT INTO tab VALUES ('hello world', [0.0]); - -DROP TABLE tab; diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql index f1cfc041233..aedba286a9f 100644 --- a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql +++ b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql @@ -5,6 +5,7 @@ SET allow_experimental_vector_similarity_index = 1; DROP TABLE IF EXISTS tab; + CREATE TABLE tab (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); ALTER TABLE tab ADD INDEX idx(vec) TYPE minmax; From 4e3bde24605e1401749703bfe2eb28d7298f6630 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Nov 2024 14:52:59 +0100 Subject: [PATCH 799/816] Add ProfileEvents for merge selector timings --- src/Common/ProfileEvents.cpp | 6 ++++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 30 +++++++++++++++++-- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0774d36462d..7b9f670d340 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -746,6 +746,12 @@ The server successfully detected this situation and will download merged part fr M(ReadTaskRequestsSentElapsedMicroseconds, "Time spent in callbacks requested from the remote server back to the initiator server to choose the read task (for s3Cluster table function and similar). Measured on the remote server side.", ValueType::Microseconds) \ M(MergeTreeReadTaskRequestsSentElapsedMicroseconds, "Time spent in callbacks requested from the remote server back to the initiator server to choose the read task (for MergeTree tables). Measured on the remote server side.", ValueType::Microseconds) \ M(MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds, "Time spent in sending the announcement from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.", ValueType::Microseconds) \ + M(MergerMutatorsGetPartsForMergeElapsedMicroseconds, "Time spent to take data parts snapshot to build ranges from them.", ValueType::Microseconds) \ + M(MergerMutatorPrepareRangesForMergeElapsedMicroseconds, "Time spent to prepare parts ranges which can be merged according to merge predicate.", ValueType::Microseconds) \ + M(MergerMutatorSelectPartsForMergeElapsedMicroseconds, "Time spent to select parts from ranges which can be merged.", ValueType::Microseconds) \ + M(MergerMutatorRangesForMergeCount, "Amount of candidate ranges for merge", ValueType::Number) \ + M(MergerMutatorPartsInRangesForMergeCount, "Amount of candidate parts for merge", ValueType::Number) \ + M(MergerMutatorSelectRangePartsCount, "Amount of parts in selected range for merge", ValueType::Number) \ \ M(ConnectionPoolIsFullMicroseconds, "Total time spent waiting for a slot in connection pool.", ValueType::Microseconds) \ M(AsyncLoaderWaitMicroseconds, "Total time a query was waiting for async loader jobs.", ValueType::Microseconds) \ diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6b9638b11d2..3d935f8b70d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -48,6 +48,17 @@ namespace CurrentMetrics { extern const Metric BackgroundMergesAndMutationsPoolTask; } +namespace ProfileEvents +{ + + extern const Event MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds; + extern const Event MergerMutatorsGetPartsForMergeElapsedMicroseconds; + extern const Event MergerMutatorPrepareRangesForMergeElapsedMicroseconds; + extern const Event MergerMutatorSelectPartsForMergeElapsedMicroseconds; + extern const Event MergerMutatorRangesForMergeCount; + extern const Event MergerMutatorPartsInRangesForMergeCount; + extern const Event MergerMutatorSelectRangePartsCount; +} namespace DB { @@ -215,6 +226,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart { PartitionIdsHint res; MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn); + if (data_parts.empty()) return res; @@ -272,6 +284,8 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectMergeFrom( const MergeTreeTransactionPtr & txn, const PartitionIdsHint * partitions_hint) const { + + Stopwatch get_data_parts_for_merge_timer; auto res = getDataPartsToSelectMergeFrom(txn); if (!partitions_hint) return res; @@ -280,6 +294,8 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectM { return !partitions_hint->contains(part->info.partition_id); }); + + ProfileEvents::increment(ProfileEvents::MergerMutatorsGetPartsForMergeElapsedMicroseconds, get_data_parts_for_merge_timer.elapsedMicroseconds()); return res; } @@ -357,6 +373,7 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo const MergeTreeTransactionPtr & txn, PreformattedMessage & out_disable_reason) const { + Stopwatch ranges_for_merge_timer; MergeSelectingInfo res; res.current_time = std::time(nullptr); @@ -457,6 +474,10 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo prev_part = ∂ } + ProfileEvents::increment(ProfileEvents::MergerMutatorPartsInRangesForMergeCount, res.parts_selected_precondition); + ProfileEvents::increment(ProfileEvents::MergerMutatorRangesForMergeCount, res.parts_ranges.size()); + ProfileEvents::increment(ProfileEvents::MergerMutatorPrepareRangesForMergeElapsedMicroseconds, ranges_for_merge_timer.elapsedMicroseconds()); + return res; } @@ -471,6 +492,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( PreformattedMessage & out_disable_reason, bool dry_run) { + Stopwatch select_parts_from_ranges_timer; const auto data_settings = data.getSettings(); IMergeSelector::PartsRange parts_to_merge; @@ -570,7 +592,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( if (parts_to_merge.empty()) { - out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors)"); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectPartsForMergeElapsedMicroseconds, select_parts_from_ranges_timer.elapsedMicroseconds()); + out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors) in {}", select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); return SelectPartsDecision::CANNOT_SELECT; } } @@ -583,8 +606,11 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( parts.push_back(part); } - LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); + LOG_DEBUG(log, "Selected {} parts from {} to {} in {}ms", parts.size(), parts.front()->name, parts.back()->name, select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectRangePartsCount, parts.size()); + future_part->assign(std::move(parts)); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectPartsForMergeElapsedMicroseconds, select_parts_from_ranges_timer.elapsedMicroseconds()); return SelectPartsDecision::SELECTED; } From afb92f04e62b446fb5c8b0417c658f206ce2a55d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 6 Nov 2024 14:56:30 +0100 Subject: [PATCH 800/816] Added ms --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3d935f8b70d..4d0fb7f9eeb 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -593,7 +593,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( if (parts_to_merge.empty()) { ProfileEvents::increment(ProfileEvents::MergerMutatorSelectPartsForMergeElapsedMicroseconds, select_parts_from_ranges_timer.elapsedMicroseconds()); - out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors) in {}", select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); + out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors) in {}ms", select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); return SelectPartsDecision::CANNOT_SELECT; } } From f4c0254254b7cfe1f603dc57350a226c9d5dd993 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 6 Nov 2024 14:52:55 +0000 Subject: [PATCH 801/816] fix_test_drop_complex_columns: flaky check for test_drop_after_fetch --- .../test_replicated_s3_zero_copy_drop_partition/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py b/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py index 9937c0ed4ea..7623a24c0ef 100644 --- a/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py +++ b/tests/integration/test_replicated_s3_zero_copy_drop_partition/test.py @@ -65,6 +65,8 @@ CREATE TABLE test_s3(c1 Int8, c2 Date) ENGINE = ReplicatedMergeTree('/test/table objects_after = get_objects_in_data_path() assert objects_before == objects_after + node1.query("DROP TABLE test_local SYNC") + node1.query("DROP TABLE test_s3 SYNC") def test_drop_complex_columns(started_cluster): From 33bd082149ca207b55915cd78c8c19cdc6aacdc9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Nov 2024 16:00:25 +0100 Subject: [PATCH 802/816] Followup --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3d935f8b70d..40c4db3a69d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -51,7 +51,6 @@ namespace CurrentMetrics namespace ProfileEvents { - extern const Event MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds; extern const Event MergerMutatorsGetPartsForMergeElapsedMicroseconds; extern const Event MergerMutatorPrepareRangesForMergeElapsedMicroseconds; extern const Event MergerMutatorSelectPartsForMergeElapsedMicroseconds; From 15337692e68961c247dd809f3b13e89a8acc74b7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 15:10:10 +0000 Subject: [PATCH 803/816] Minor: Remove "experimental" mention of analyzer --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 081e07ca2ce..7e8d0aabce0 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -4239,7 +4239,7 @@ Rewrite aggregate functions with if expression as argument when logically equiva For example, `avg(if(cond, col, null))` can be rewritten to `avgOrNullIf(cond, col)`. It may improve performance. :::note -Supported only with experimental analyzer (`enable_analyzer = 1`). +Supported only with the analyzer (`enable_analyzer = 1`). ::: )", 0) \ DECLARE(Bool, optimize_rewrite_array_exists_to_has, false, R"( From 12ab488453796a46f1f37d91cf60c6a6007e0134 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Nov 2024 16:20:57 +0100 Subject: [PATCH 804/816] Revert "Selection of hash join inner table" --- src/Core/Joins.h | 11 - src/Core/Settings.cpp | 3 - src/Core/Settings.h | 1 - src/Core/SettingsEnums.cpp | 4 - src/Core/SettingsEnums.h | 2 +- src/Interpreters/ConcurrentHashJoin.h | 11 - src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 16 +- src/Interpreters/HashJoin/HashJoin.h | 5 +- .../HashJoin/HashJoinMethodsImpl.h | 18 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Interpreters/TableJoin.cpp | 56 +---- src/Interpreters/TableJoin.h | 19 +- src/Interpreters/TreeRewriter.cpp | 5 +- src/Parsers/CreateQueryUUIDs.cpp | 2 +- src/Planner/CollectColumnIdentifiers.cpp | 1 - src/Planner/PlannerJoinTree.cpp | 152 +++++-------- src/Processors/QueryPlan/JoinStep.cpp | 103 +-------- src/Processors/QueryPlan/JoinStep.h | 17 +- .../QueryPlan/Optimizations/Optimizations.h | 1 - .../QueryPlan/Optimizations/optimizeJoin.cpp | 102 --------- .../QueryPlan/Optimizations/optimizeTree.cpp | 3 - .../QueryPlan/ReadFromMemoryStorageStep.h | 2 - .../Transforms/ColumnPermuteTransform.cpp | 49 ----- .../Transforms/ColumnPermuteTransform.h | 30 --- .../Transforms/JoiningTransform.cpp | 1 - tests/clickhouse-test | 4 - tests/integration/helpers/cluster.py | 13 +- tests/integration/helpers/random_settings.py | 2 - .../test_peak_memory_usage/test.py | 2 +- .../0_stateless/00826_cross_to_inner_join.sql | 13 +- .../00847_multiple_join_same_column.sql | 14 +- .../01015_empty_in_inner_right_join.sql.j2 | 2 - .../01107_join_right_table_totals.reference | 7 - .../01107_join_right_table_totals.sql | 10 +- .../01763_filter_push_down_bugs.reference | 2 +- .../01881_join_on_conditions_hash.sql.j2 | 10 +- .../0_stateless/02000_join_on_const.reference | 18 +- .../0_stateless/02000_join_on_const.sql | 16 +- .../02001_join_on_const_bs_long.sql.j2 | 4 +- ...oin_with_nullable_lowcardinality_crash.sql | 5 +- .../0_stateless/02282_array_distance.sql | 12 +- .../02381_join_dup_columns_in_plan.reference | 1 + .../0_stateless/02461_join_lc_issue_42380.sql | 3 +- ...emove_redundant_sorting_analyzer.reference | 4 +- ...move_redundant_distinct_analyzer.reference | 18 +- .../02514_analyzer_drop_join_on.reference | 55 +++-- .../02514_analyzer_drop_join_on.sql | 1 - ...oin_with_totals_and_subquery_bug.reference | 2 +- .../02835_join_step_explain.reference | 32 +-- .../0_stateless/02835_join_step_explain.sql | 2 - .../02962_join_using_bug_57894.reference | 1 - .../02962_join_using_bug_57894.sql | 2 - ...filter_push_down_equivalent_sets.reference | 206 ++++++++---------- ..._join_filter_push_down_equivalent_sets.sql | 40 +--- .../03038_recursive_cte_postgres_4.reference | 4 +- .../03038_recursive_cte_postgres_4.sql | 4 +- .../0_stateless/03094_one_thousand_joins.sql | 1 - ...convert_outer_join_to_inner_join.reference | 36 +-- ...03130_convert_outer_join_to_inner_join.sql | 13 +- ...ter_push_down_equivalent_columns.reference | 3 +- .../03236_squashing_high_memory.sql | 1 - 62 files changed, 314 insertions(+), 869 deletions(-) delete mode 100644 src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp delete mode 100644 src/Processors/Transforms/ColumnPermuteTransform.cpp delete mode 100644 src/Processors/Transforms/ColumnPermuteTransform.h diff --git a/src/Core/Joins.h b/src/Core/Joins.h index dd6d86fc902..0964bf86e6b 100644 --- a/src/Core/Joins.h +++ b/src/Core/Joins.h @@ -119,15 +119,4 @@ enum class JoinTableSide : uint8_t const char * toString(JoinTableSide join_table_side); -/// Setting to choose which table to use as the inner table in hash join -enum class JoinInnerTableSelectionMode : uint8_t -{ - /// Use left table - Left, - /// Use right table - Right, - /// Use the table with the smallest number of rows - Auto, -}; - } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 081e07ca2ce..ada6b674c87 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1912,9 +1912,6 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ - DECLARE(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, R"( -Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'. -)", 0) \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. )", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1cc58deb94a..ac3b1fe651e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -66,7 +66,6 @@ class WriteBuffer; M(CLASS_NAME, IntervalOutputFormat) \ M(CLASS_NAME, JoinAlgorithm) \ M(CLASS_NAME, JoinStrictness) \ - M(CLASS_NAME, JoinInnerTableSelectionMode) \ M(CLASS_NAME, LightweightMutationProjectionMode) \ M(CLASS_NAME, LoadBalancing) \ M(CLASS_NAME, LocalFSReadMethod) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 89e9cb295c3..cef63039277 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -55,10 +55,6 @@ IMPLEMENT_SETTING_MULTI_ENUM(JoinAlgorithm, ErrorCodes::UNKNOWN_JOIN, {"full_sorting_merge", JoinAlgorithm::FULL_SORTING_MERGE}, {"grace_hash", JoinAlgorithm::GRACE_HASH}}) -IMPLEMENT_SETTING_ENUM(JoinInnerTableSelectionMode, ErrorCodes::BAD_ARGUMENTS, - {{"left", JoinInnerTableSelectionMode::Left}, - {"right", JoinInnerTableSelectionMode::Right}, - {"auto", JoinInnerTableSelectionMode::Auto}}) IMPLEMENT_SETTING_ENUM(TotalsMode, ErrorCodes::UNKNOWN_TOTALS_MODE, {{"before_having", TotalsMode::BEFORE_HAVING}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 35bdb8a7f65..607011b505b 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -128,8 +128,8 @@ constexpr auto getEnumValues(); DECLARE_SETTING_ENUM(LoadBalancing) DECLARE_SETTING_ENUM(JoinStrictness) + DECLARE_SETTING_MULTI_ENUM(JoinAlgorithm) -DECLARE_SETTING_ENUM(JoinInnerTableSelectionMode) /// Which rows should be included in TOTALS. diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index b377727a134..a911edaccc3 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -60,17 +60,6 @@ public: IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; - - bool isCloneSupported() const override - { - return !getTotals() && getTotalRowCount() == 0; - } - - std::shared_ptr clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override - { - return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); - } - private: struct InternalHashJoin { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index faa9114c618..3f1e0d59287 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,7 @@ public: bool isCloneSupported() const override { - return !getTotals(); + return true; } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index dad8a487745..3e7f3deea8b 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -383,16 +383,6 @@ size_t HashJoin::getTotalByteCount() const return res; } -bool HashJoin::isUsedByAnotherAlgorithm() const -{ - return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH); -} - -bool HashJoin::canRemoveColumnsFromLeftBlock() const -{ - return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm(); -} - void HashJoin::initRightBlockStructure(Block & saved_block_sample) { if (isCrossOrComma(kind)) @@ -404,7 +394,8 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = isUsedByAnotherAlgorithm() || + bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || + table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression(); @@ -1237,10 +1228,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, { if (!JoinCommon::hasNonJoinedBlocks(*table_join)) return {}; - size_t left_columns_count = left_sample_block.columns(); - if (canRemoveColumnsFromLeftBlock()) - left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); if (!flag_per_row) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 8a27961354a..4c1ebbcdc66 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -127,7 +127,7 @@ public: bool isCloneSupported() const override { - return !getTotals() && getTotalRowCount() == 0; + return true; } std::shared_ptr clone(const std::shared_ptr & table_join_, @@ -464,9 +464,6 @@ private: bool empty() const; - bool isUsedByAnotherAlgorithm() const; - bool canRemoveColumnsFromLeftBlock() const; - void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 7e8a2658b9c..45a766e2df6 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -56,6 +56,7 @@ Block HashJoinMethods::joinBlockImpl( const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } + size_t existing_columns = block.columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -98,22 +99,6 @@ Block HashJoinMethods::joinBlockImpl( added_columns.buildJoinGetOutput(); else added_columns.buildOutput(); - - const auto & table_join = join.table_join; - std::set block_columns_to_erase; - if (join.canRemoveColumnsFromLeftBlock()) - { - std::unordered_set left_output_columns; - for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) - left_output_columns.insert(out_column.name); - for (size_t i = 0; i < block.columns(); ++i) - { - if (!left_output_columns.contains(block.getByPosition(i).name)) - block_columns_to_erase.insert(i); - } - } - size_t existing_columns = block.columns(); - for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); @@ -175,7 +160,6 @@ Block HashJoinMethods::joinBlockImpl( block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } } - block.erase(block_columns_to_erase); return remaining_block; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8ddf51fa25e..3918c1c37ea 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1888,9 +1888,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 555aaff2e06..2532dddba3c 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -41,7 +41,6 @@ namespace DB namespace Setting { extern const SettingsBool allow_experimental_join_right_table_sorting; - extern const SettingsBool allow_experimental_analyzer; extern const SettingsUInt64 cross_join_min_bytes_to_compress; extern const SettingsUInt64 cross_join_min_rows_to_compress; extern const SettingsUInt64 default_max_bytes_in_join; @@ -144,7 +143,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_memory_usage(settings[Setting::max_memory_usage]) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) - , enable_analyzer(settings[Setting::allow_experimental_analyzer]) { } @@ -163,8 +161,6 @@ void TableJoin::resetCollected() clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); - columns_from_left_table.clear(); - result_columns_from_left_table.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); @@ -207,20 +203,6 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return count; } -void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) -{ - columns_from_left_table = std::move(left_output_columns); - columns_from_joined_table = std::move(right_output_columns); -} - - -const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side) -{ - if (side == JoinTableSide::Left) - return result_columns_from_left_table; - return columns_added_by_join; -} - void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix) { NameSet joined_columns; @@ -369,18 +351,9 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const return forceNullableRight() && JoinCommon::canBecomeNullable(column_type); } -void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side) -{ - if (side == JoinTableSide::Left) - result_columns_from_left_table.push_back(joined_column); - else - columns_added_by_join.push_back(joined_column); - -} - void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - setUsedColumn(joined_column, JoinTableSide::Right); + columns_added_by_join.emplace_back(joined_column); } NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const @@ -1022,32 +995,5 @@ size_t TableJoin::getMaxMemoryUsage() const return max_memory_usage; } -void TableJoin::swapSides() -{ - assertEnableEnalyzer(); - - std::swap(key_asts_left, key_asts_right); - std::swap(left_type_map, right_type_map); - for (auto & clause : clauses) - { - std::swap(clause.key_names_left, clause.key_names_right); - std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right); - std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name); - } - - std::swap(columns_from_left_table, columns_from_joined_table); - std::swap(result_columns_from_left_table, columns_added_by_join); - - if (table_join.kind == JoinKind::Left) - table_join.kind = JoinKind::Right; - else if (table_join.kind == JoinKind::Right) - table_join.kind = JoinKind::Left; -} - -void TableJoin::assertEnableEnalyzer() const -{ - if (!enable_analyzer) - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled"); -} } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index e0e1926fb12..e1bae55a4ed 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -167,9 +167,6 @@ private: ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals; - NamesAndTypesList columns_from_left_table; - NamesAndTypesList result_columns_from_left_table; - /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; /// Columns will be added to block by JOIN. @@ -205,8 +202,6 @@ private: bool is_join_with_constant = false; - bool enable_analyzer = false; - Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -271,8 +266,6 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } - bool enableEnalyzer() const { return enable_analyzer; } - void assertEnableEnalyzer() const; ActionsDAG createJoinedBlockActions(ContextPtr context) const; @@ -289,7 +282,6 @@ public: } bool allowParallelHashJoin() const; - void swapSides(); bool joinUseNulls() const { return join_use_nulls; } @@ -380,9 +372,6 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); - - void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side); - void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) { columns_added_by_join = columns_added_by_join_value; @@ -408,17 +397,11 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_) + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) { columns_from_joined_table = std::move(columns_from_joined_table_value); deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); - result_columns_from_left_table = columns_from_left_table_; - columns_from_left_table = columns_from_left_table_; } - - void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns); - const NamesAndTypesList & getOutputColumns(JoinTableSide side); - const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 28e11166762..ea08fd92339 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1353,15 +1353,12 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { - auto columns_from_left_table = tables_with_columns[0].columns; const auto & right_table = tables_with_columns[1]; auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end()); - result.analyzed_join->setColumnsFromJoinedTable( - std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table); + result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index 70848440a0e..c788cc7a025 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// Thus it's not safe for example to replace /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with - /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b" + /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b" /// This replacement is safe only for CREATE queries when inner target tables don't exist yet. if (!query.attach) { diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index dd5bdd4d141..95f1c7d53d8 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -2,7 +2,6 @@ #include #include -#include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a1ce455f266..5c153f6db39 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,7 +104,6 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; - extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection; } namespace ErrorCodes @@ -1242,55 +1241,6 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } -std::optional createStepToDropColumns( - const Block & header, - const ColumnIdentifierSet & outer_scope_columns, - const PlannerContextPtr & planner_context) -{ - ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; - std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; - std::optional first_skipped_column_node_index; - - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); - size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); - - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - - for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) - { - const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; - - if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) - || !global_planner_context->hasColumnIdentifier(output->result_name)) - continue; - - if (!outer_scope_columns.contains(output->result_name)) - { - if (!first_skipped_column_node_index) - first_skipped_column_node_index = i; - continue; - } - - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); - drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); - } - - if (!first_skipped_column_node_index) - return {}; - - /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. - * - * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; - */ - if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); - - drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); - - return drop_unused_columns_after_join_actions_dag; -} - JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, @@ -1563,37 +1513,21 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } const Block & left_header = left_plan.getCurrentHeader(); + auto left_table_names = left_header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + for (auto & column_from_joined_table : columns_from_joined_table) + { + /// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + outer_scope_columns.contains(column_from_joined_table.name)) + table_join->addJoinedColumn(column_from_joined_table); + } + const Block & right_header = right_plan.getCurrentHeader(); - - auto columns_from_left_table = left_header.getNamesAndTypesList(); - auto columns_from_right_table = right_header.getNamesAndTypesList(); - - table_join->setInputColumns(columns_from_left_table, columns_from_right_table); - - for (auto & column_from_joined_table : columns_from_left_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); - } - - for (auto & column_from_joined_table : columns_from_right_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); - } - - if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) - { - if (!columns_from_left_table.empty()) - table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left); - else if (!columns_from_right_table.empty()) - table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right); - } - auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); auto result_plan = QueryPlan(); @@ -1681,26 +1615,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } auto join_pipeline_type = join_algorithm->pipelineType(); - - ColumnIdentifierSet outer_scope_columns_nonempty; - if (outer_scope_columns.empty()) - { - if (left_header.columns() > 1) - outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); - else if (right_header.columns() > 1) - outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); - } - auto join_step = std::make_unique( left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), std::move(join_algorithm), settings[Setting::max_block_size], settings[Setting::max_threads], - outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, - false /*optimize_read_in_order*/, - true /*optimize_skip_unused_shards*/); - join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; + false /*optimize_read_in_order*/); join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); @@ -1711,18 +1632,47 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - const auto & header_after_join = result_plan.getCurrentHeader(); - if (header_after_join.columns() > outer_scope_columns.size()) + ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; + std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; + std::optional first_skipped_column_node_index; + + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); + size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); + + for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) { - auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context); - if (drop_unused_columns_after_join_actions_dag) + const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; + + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) + || !global_planner_context->hasColumnIdentifier(output->result_name)) + continue; + + if (!outer_scope_columns.contains(output->result_name)) { - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + if (!first_skipped_column_node_index) + first_skipped_column_node_index = i; + continue; } + + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); + drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); } + /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. + * + * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; + */ + if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); + + drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); + + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 7ade437822e..018b52a5c68 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB { @@ -37,37 +36,6 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } -std::vector getPermutationForBlock( - const Block & block, - const Block & lhs_block, - const Block & rhs_block, - const NameSet & name_filter) -{ - std::vector permutation; - permutation.reserve(block.columns()); - Block::NameMap name_map = block.getNamesToIndexesMap(); - - bool is_trivial = true; - for (const auto & other_block : {lhs_block, rhs_block}) - { - for (const auto & col : other_block) - { - if (!name_filter.contains(col.name)) - continue; - if (auto it = name_map.find(col.name); it != name_map.end()) - { - is_trivial = is_trivial && it->second == permutation.size(); - permutation.push_back(it->second); - } - } - } - - if (is_trivial && permutation.size() == block.columns()) - return {}; - - return permutation; -} - } JoinStep::JoinStep( @@ -76,15 +44,8 @@ JoinStep::JoinStep( JoinPtr join_, size_t max_block_size_, size_t max_streams_, - NameSet required_output_, - bool keep_left_read_in_order_, - bool use_new_analyzer_) - : join(std::move(join_)) - , max_block_size(max_block_size_) - , max_streams(max_streams_) - , required_output(std::move(required_output_)) - , keep_left_read_in_order(keep_left_read_in_order_) - , use_new_analyzer(use_new_analyzer_) + bool keep_left_read_in_order_) + : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) { updateInputHeaders({left_header_, right_header_}); } @@ -94,43 +55,23 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); - Block lhs_header = pipelines[0]->getHeader(); - Block rhs_header = pipelines[1]->getHeader(); - - if (swap_streams) - std::swap(pipelines[0], pipelines[1]); - if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors); joined_pipeline->resize(max_streams); return joined_pipeline; } - auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( + return QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), std::move(pipelines[1]), join, - join_algorithm_header, + *output_header, max_block_size, max_streams, keep_left_read_in_order, &processors); - - if (!use_new_analyzer) - return pipeline; - - auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output); - if (!column_permutation.empty()) - { - pipeline->addSimpleTransform([&column_permutation](const Block & header) - { - return std::make_shared(header, column_permutation); - }); - } - - return pipeline; } bool JoinStep::allowPushDownToRight() const @@ -149,49 +90,17 @@ void JoinStep::describeActions(FormatSettings & settings) const for (const auto & [name, value] : describeJoinActions(join)) settings.out << prefix << name << ": " << value << '\n'; - if (swap_streams) - settings.out << prefix << "Swapped: true\n"; } void JoinStep::describeActions(JSONBuilder::JSONMap & map) const { for (const auto & [name, value] : describeJoinActions(join)) map.add(name, value); - if (swap_streams) - map.add("Swapped", true); -} - -void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) -{ - join_algorithm_header.clear(); - swap_streams = swap_streams_; - join = std::move(join_); - updateOutputHeader(); } void JoinStep::updateOutputHeader() { - if (join_algorithm_header) - return; - - const auto & header = swap_streams ? input_headers[1] : input_headers[0]; - - Block result_header = JoiningTransform::transformHeader(header, join); - join_algorithm_header = result_header; - - if (!use_new_analyzer) - { - if (swap_streams) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer"); - output_header = result_header; - return; - } - - auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output); - if (!column_permutation.empty()) - result_header = ColumnPermuteTransform::permute(result_header, column_permutation); - - output_header = result_header; + output_header = JoiningTransform::transformHeader(input_headers.front(), join); } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 1eca42c62cf..2793784d633 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { @@ -20,9 +19,7 @@ public: JoinPtr join_, size_t max_block_size_, size_t max_streams_, - NameSet required_output_, - bool keep_left_read_in_order_, - bool use_new_analyzer_); + bool keep_left_read_in_order_); String getName() const override { return "Join"; } @@ -34,26 +31,16 @@ public: void describeActions(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } - void setJoin(JoinPtr join_, bool swap_streams_ = false); + void setJoin(JoinPtr join_) { join = std::move(join_); } bool allowPushDownToRight() const; - JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right; - private: void updateOutputHeader() override; - /// Header that expected to be returned from IJoin - Block join_algorithm_header; - JoinPtr join; size_t max_block_size; size_t max_streams; - - const NameSet required_output; - std::set columns_to_remove; bool keep_left_read_in_order; - bool use_new_analyzer = false; - bool swap_streams = false; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index c1c4d1e1635..751d5182dc3 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -113,7 +113,6 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &); void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); /// A separate tree traverse to apply sorting properties after *InOrder optimizations. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp deleted file mode 100644 index c0b31864eac..00000000000 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ /dev/null @@ -1,102 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - -namespace DB::QueryPlanOptimizations -{ - -static std::optional estimateReadRowsCount(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (const auto * reading = typeid_cast(step)) - { - if (auto analyzed_result = reading->getAnalyzedResult()) - return analyzed_result->selected_rows; - if (auto analyzed_result = reading->selectRangesToRead()) - return analyzed_result->selected_rows; - return {}; - } - - if (const auto * reading = typeid_cast(step)) - return reading->getStorage()->totalRows(Settings{}); - - if (node.children.size() != 1) - return {}; - - if (typeid_cast(step) || typeid_cast(step)) - return estimateReadRowsCount(*node.children.front()); - - return {}; -} - -void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) -{ - auto * join_step = typeid_cast(node.step.get()); - if (!join_step || node.children.size() != 2) - return; - - const auto & join = join_step->getJoin(); - if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported()) - return; - - const auto & table_join = join->getTableJoin(); - - /// Algorithms other than HashJoin may not support OUTER JOINs - if (table_join.kind() != JoinKind::Inner && !typeid_cast(join.get())) - return; - - /// fixme: USING clause handled specially in join algorithm, so swap breaks it - /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test - if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) - return; - - bool need_swap = false; - if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto) - { - auto lhs_extimation = estimateReadRowsCount(*node.children[0]); - auto rhs_extimation = estimateReadRowsCount(*node.children[1]); - LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}", - lhs_extimation.transform(toString).value_or("unknown"), - rhs_extimation.transform(toString).value_or("unknown")); - - if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) - need_swap = true; - } - else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left) - { - need_swap = true; - } - - if (!need_swap) - return; - - const auto & headers = join_step->getInputHeaders(); - if (headers.size() != 2) - return; - - const auto & left_stream_input_header = headers.front(); - const auto & right_stream_input_header = headers.back(); - - auto updated_table_join = std::make_shared(table_join); - updated_table_join->swapSides(); - auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); - join_step->setJoin(std::move(updated_join), /* swap_streams= */ true); -} - -} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index c034ca79181..03418c752d4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -227,9 +227,6 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); - if (frame.next_child == 0) - optimizeJoin(*frame.node, nodes); - /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index a9c2d2df2c4..238c1a3aad0 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -35,8 +35,6 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - const StoragePtr & getStorage() const { return storage; } - private: static constexpr auto name = "ReadFromMemoryStorage"; diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp deleted file mode 100644 index f371689814c..00000000000 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ /dev/null @@ -1,49 +0,0 @@ -#include - -namespace DB -{ - -namespace -{ - -template -void applyPermutation(std::vector & data, const std::vector & permutation) -{ - std::vector res; - res.reserve(permutation.size()); - for (size_t i : permutation) - res.push_back(data[i]); - data = std::move(res); -} - -void permuteChunk(Chunk & chunk, const std::vector & permutation) -{ - size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - applyPermutation(columns, permutation); - chunk.setColumns(std::move(columns), num_rows); -} - -} - -Block ColumnPermuteTransform::permute(const Block & block, const std::vector & permutation) -{ - auto columns = block.getColumnsWithTypeAndName(); - applyPermutation(columns, permutation); - return Block(columns); -} - -ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) - : ISimpleTransform(header_, permute(header_, permutation_), false) - , permutation(permutation_) -{ -} - - -void ColumnPermuteTransform::transform(Chunk & chunk) -{ - permuteChunk(chunk, permutation); -} - - -} diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h deleted file mode 100644 index 25f3a8d0825..00000000000 --- a/src/Processors/Transforms/ColumnPermuteTransform.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class ColumnPermuteTransform : public ISimpleTransform -{ -public: - ColumnPermuteTransform(const Block & header_, const std::vector & permutation_); - - String getName() const override { return "ColumnPermuteTransform"; } - - void transform(Chunk & chunk) override; - - static Block permute(const Block & block, const std::vector & permutation); - -private: - Names column_names; - std::vector permutation; -}; - - -} diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 187f4bf6728..f2fb6327129 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -19,7 +19,6 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); - materializeBlockInplace(header); LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f4c3b368632..9c035b7cc35 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -789,7 +789,6 @@ def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) -# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { "max_insert_threads": lambda: ( @@ -920,9 +919,6 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), - "query_plan_join_inner_table_selection": lambda: random.choice( - ["left", "auto", "right"] - ), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6751f205fb8..7c531cdd493 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -67,7 +67,6 @@ DEFAULT_ENV_NAME = ".env" DEFAULT_BASE_CONFIG_DIR = os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" ) -DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest") SANITIZER_SIGN = "==================" @@ -504,6 +503,7 @@ class ClickHouseCluster: "CLICKHOUSE_TESTS_DOCKERD_HOST" ) self.docker_api_version = os.environ.get("DOCKER_API_VERSION") + self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") self.base_cmd = ["docker", "compose"] if custom_dockerd_host: @@ -1079,7 +1079,7 @@ class ClickHouseCluster: env_variables["keeper_binary"] = binary_path env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix - env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG + env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag env_variables["user"] = str(os.getuid()) env_variables["keeper_fs"] = "bind" for i in range(1, 4): @@ -1675,7 +1675,7 @@ class ClickHouseCluster: ) if tag is None: - tag = DOCKER_BASE_TAG + tag = self.docker_base_tag if not env_variables: env_variables = {} self.use_keeper = use_keeper @@ -4538,12 +4538,7 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if ( - self.randomize_settings - and self.image == "clickhouse/integration-test" - and self.tag == DOCKER_BASE_TAG - and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR - ): + if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index 32cde54d0e7..b2319561fd7 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -5,8 +5,6 @@ def randomize_settings(): yield "max_joined_block_size_rows", random.randint(8000, 100000) if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) - if random.random() < 0.5: - yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"]) def write_random_settings_config(destination): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 69057573173..51268dcf386 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_inner_table_selection = 'right'", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index 5ab7a2d0626..e9f9e13e2d3 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,15 +41,14 @@ SELECT '--- is null or ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) -ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); SELECT '--- cross ---'; diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index bbb4eb12466..c7f0c6383c2 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) left join y on (y.a = s.a and y.b = s.b) order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a as t_a from t left join s on s.a = t_a order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, s.a as s_a from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; drop table t; drop table s; diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index cdbb0542ffb..cdb9d253b9b 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,7 +1,5 @@ SET joined_subquery_requires_alias = 0; -SET query_plan_join_inner_table_selection = 'auto'; - {% for join_algorithm in ['partial_merge', 'hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.reference b/tests/queries/0_stateless/01107_join_right_table_totals.reference index aa569ff9331..daf503b776d 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.reference +++ b/tests/queries/0_stateless/01107_join_right_table_totals.reference @@ -18,35 +18,28 @@ 0 0 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 0 0 -- 1 foo 1 1 300 0 foo 1 0 300 -- 1 100 1970-01-01 1 100 1970-01-01 1 100 1970-01-01 1 200 1970-01-02 1 200 1970-01-02 1 100 1970-01-01 diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.sql b/tests/queries/0_stateless/01107_join_right_table_totals.sql index 7e549282489..ad8954d5d70 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.sql +++ b/tests/queries/0_stateless/01107_join_right_table_totals.sql @@ -64,47 +64,39 @@ USING (id); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l RIGHT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r -ON l.item_id = r.item_id -ORDER BY ALL; +ON l.item_id = r.item_id; DROP TABLE t; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 229ac6eae09..19018a610b7 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter (WHERE) + Filter ((WHERE + DROP unused columns after JOIN)) Join (JOIN FillRightFirst) Expression ReadFromMergeTree (default.t1) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c13722f431a..c2d85cefb18 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; {% endfor -%} diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index f8e46a2b976..3bd1633ce32 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -0 3 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 2 2 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +0 3 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 @@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 1 ('',0) SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 0 ('b',256) -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) 0 ('b',256) SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) 2 4 2 Nullable(UInt64) UInt8 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 33638edafa5..da70973ed87 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -- { echoOff } diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 83548e087bd..1726bcb7062 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int) ENGINE = TinyLog; -CREATE TABLE t2 (id Int) ENGINE = TinyLog; +CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql index c3c84ebaded..abc2ee41402 100644 --- a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -12,9 +12,8 @@ CREATE TABLE without_nullable insert into with_nullable values(0,'f'),(0,'usa'); insert into without_nullable values(0,'usa'),(0,'us2a'); -select if(t0.country is null ,t2.country,t0.country) "country" -from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country -ORDER BY 1 DESC; +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; drop table with_nullable; drop table without_nullable; diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 85abc8fa381..2cca853fd67 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -48,8 +48,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2 v1, vec2 v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -62,8 +61,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2f v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -76,8 +74,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2d v1, vec2d v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); SELECT v1.id, @@ -89,8 +86,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2d v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 90aab0a0eb2..365725f8ffe 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -148,6 +148,7 @@ Header: key String value String Join Header: __table1.key String + __table3.key String __table3.value String Sorting Header: __table1.key String diff --git a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql index 8b5c6846bd0..f0ecbf64e58 100644 --- a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql +++ b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql @@ -9,5 +9,4 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B INSERT INTO t1__fuzz_13 VALUES (1); INSERT INTO t2__fuzz_47 VALUES (1); -SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2 -ORDER BY ALL; +SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index c9bf36f88ea..3c68d14fdf2 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + Projection)) + Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers @@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + Projection)) + Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index baa2be9dfdb..867ae394c1f 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -79,7 +79,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) Distinct (Preliminary DISTINCT) - Expression (Projection) + Expression ((Projection + DROP unused columns after JOIN)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + Project names)) Distinct (DISTINCT) @@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -280,7 +280,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -315,7 +315,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -386,7 +386,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -457,7 +457,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index bbfdf1ad5f4..2c62e278050 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -8,21 +8,24 @@ Header: count() UInt64 Aggregating Header: __table1.a2 String count() UInt64 - Expression (Before GROUP BY) + Expression ((Before GROUP BY + DROP unused columns after JOIN)) Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - Expression (JOIN actions) + __table3.c1 UInt64 + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String + __table2.b1 UInt64 __table3.c1 UInt64 - Expression (JOIN actions) + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a2 String + Header: __table1.a1 UInt64 + __table1.a2 String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 @@ -45,32 +48,39 @@ Header: count() UInt64 EXPLAIN PLAN header = 1 SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) ; -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String + __table1.k UInt64 __table4.d2 String - Join (JOIN FillRightFirst) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression (Change column names to column identifiers) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 - ReadFromMemoryStorage - Header: a2 String - k UInt64 + Join (JOIN FillRightFirst) + Header: __table1.a2 String + __table1.k UInt64 + Expression (Change column names to column identifiers) + Header: __table1.a2 String + __table1.k UInt64 + ReadFromMemoryStorage + Header: a2 String + k UInt64 + Expression (Change column names to column identifiers) + Header: __table2.k UInt64 + ReadFromMemoryStorage + Header: k UInt64 Expression (Change column names to column identifiers) - Header: __table2.k UInt64 + Header: __table3.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression (Change column names to column identifiers) - Header: __table3.k UInt64 - ReadFromMemoryStorage - Header: k UInt64 Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 @@ -96,24 +106,27 @@ Header: bx String Header: __table1.a2 String __table2.bx String __table4.c2 String + __table4.c1 UInt64 Expression Header: __table1.a2 String __table2.bx String - __table4.c1 UInt64 __table4.c2 String + __table4.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table4.c1 UInt64 + __table2.b1 UInt64 __table4.c2 String - Expression (JOIN actions) + __table4.c1 UInt64 + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String - __table2.b1 UInt64 __table2.bx String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table2.b1 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String __table2.bx String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 __table1.a2 String diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index b10bf38e495..df84e2f50b2 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,7 +16,6 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; -SET query_plan_join_inner_table_selection = 'right'; -- { echoOn } diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 116c78a15e4..86e7e2a6a49 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -5,7 +5,7 @@ 1 1 -0 +1 \N 100000000000000000000 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index bdbc019d4f8..06f4a9cfc99 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -1,22 +1,22 @@ -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value_1 String : 3 + INPUT : 2 -> __table2.value_1 String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.id UInt64 __table2.value_1 String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -50,25 +50,29 @@ Positions: 4 0 1 2 Parts: 1 Granules: 1 -- -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 2 -> __table1.value_2 UInt64 : 2 INPUT : 3 -> __table2.value_1 String : 3 - ALIAS __table1.id :: 0 -> id UInt64 : 4 + INPUT :: 4 -> __table2.value_2 UInt64 : 4 + INPUT : 5 -> __table2.id UInt64 : 5 + ALIAS __table1.id :: 0 -> id UInt64 : 6 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.id UInt64 + __table1.value_2 UInt64 __table2.value_1 String + __table2.value_2 UInt64 + __table2.id UInt64 Type: INNER Strictness: ASOF Algorithm: HashJoin diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index b803ddbd911..1cdd3684a0b 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,8 +19,6 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); -SET query_plan_join_inner_table_selection = 'right'; - EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.reference b/tests/queries/0_stateless/02962_join_using_bug_57894.reference index fc6fe462205..454655081df 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.reference +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.reference @@ -31,7 +31,6 @@ 8 9 \N ---- analyzer --- 0 1 2 diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index e29347beb5e..96190241da5 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -21,8 +21,6 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; -SELECT '--- analyzer ---'; - SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index b7718d926c6..80f4e309505 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -2,9 +2,7 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -12,18 +10,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -71,9 +69,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right'; -; +WHERE rhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -81,18 +77,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -140,9 +136,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5 AND rhs.id = 6; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -150,18 +144,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -212,9 +206,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -222,18 +214,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -281,9 +273,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -291,31 +281,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -357,9 +347,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -367,31 +355,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -433,9 +421,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -443,18 +429,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -502,9 +488,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -512,31 +496,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin @@ -578,9 +562,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -588,31 +570,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin @@ -654,9 +636,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5 AND rhs.id = 6; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -664,34 +644,34 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6 - FUNCTION equals(__table2.id : 2, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 + FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index d6dcc34c796..e1a13d1ce71 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -22,9 +22,7 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; SELECT '--'; @@ -35,9 +33,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right'; -; +WHERE rhs.id = 5; SELECT '--'; @@ -48,9 +44,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5 AND rhs.id = 6; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6; @@ -59,9 +53,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; SELECT '--'; @@ -72,9 +64,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; SELECT '--'; @@ -85,9 +75,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; SELECT '--'; @@ -98,9 +86,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; SELECT '--'; @@ -111,9 +97,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5; SELECT '--'; @@ -124,9 +108,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE rhs.id = 5; SELECT '--'; @@ -137,9 +119,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +WHERE lhs.id = 5 AND rhs.id = 6; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index 7df38e855f6..cf070eebc38 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -52,9 +52,7 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph -SETTINGS query_plan_join_inner_table_selection = 'right' -; +SELECT * FROM search_graph; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] 2 3 arc 2 -> 3 false [(2,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index d33ca7b078e..7dad74893b9 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -55,9 +55,7 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph -SETTINGS query_plan_join_inner_table_selection = 'right' -; +SELECT * FROM search_graph; -- ordering by the path column has same effect as SEARCH DEPTH FIRST WITH RECURSIVE search_graph AS ( diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 69c4fb42a6b..6ae4e4d4d3c 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,7 +3,6 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES -SET query_plan_join_inner_table_selection = 'auto'; -- 'left' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index 5fde4f80c5d..d35bdeff98b 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -5,18 +5,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -75,18 +75,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -145,18 +145,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index ddefc322b4f..b3d1827d98f 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -22,10 +22,7 @@ SETTINGS index_granularity = 16 INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); - -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0; SELECT '--'; @@ -33,9 +30,7 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; SELECT '--'; @@ -43,9 +38,7 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; SELECT '--'; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference index 1c82e76cc65..7058d36aaf9 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference @@ -65,7 +65,8 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice'; Expression ((Project names + (Projection + ))) Header: name String Join (JOIN FillRightFirst) - Header: __table2.name String + Header: __table1.name String + __table2.name String Filter (( + Change column names to column identifiers)) Header: __table1.name String ReadFromMergeTree (default.users) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index eeb3ae85e84..f6e5dbdef03 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,7 +11,6 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; -SET query_plan_join_inner_table_selection = 'right'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, From 4e30cf7e333312968bebe57dc0f6dd381cbccff5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Nov 2024 16:30:16 +0100 Subject: [PATCH 805/816] Cleanup SettingsChangesHistory for revert --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ed87fde8b7e..64964f294bd 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -73,7 +73,6 @@ static std::initializer_list Date: Wed, 6 Nov 2024 10:50:45 +0100 Subject: [PATCH 806/816] Upgrade clickhouse-server and keeper base images --- docker/keeper/Dockerfile | 10 +++++++--- docker/server/Dockerfile.ubuntu | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index bc76bdbb619..4ecc087afb4 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -1,7 +1,7 @@ # The Dockerfile.ubuntu exists for the tests/ci/docker_server.py script # If the image is built from Dockerfile.alpine, then the `-alpine` suffix is added automatically, # so the only purpose of Dockerfile.ubuntu is to push `latest`, `head` and so on w/o suffixes -FROM ubuntu:20.04 AS glibc-donor +FROM ubuntu:22.04 AS glibc-donor ARG TARGETARCH RUN arch=${TARGETARCH:-amd64} \ @@ -9,7 +9,11 @@ RUN arch=${TARGETARCH:-amd64} \ amd64) rarch=x86_64 ;; \ arm64) rarch=aarch64 ;; \ esac \ - && ln -s "${rarch}-linux-gnu" /lib/linux-gnu + && ln -s "${rarch}-linux-gnu" /lib/linux-gnu \ + && case $arch in \ + amd64) ln /lib/linux-gnu/ld-linux-x86-64.so.2 /lib/linux-gnu/ld-2.35.so ;; \ + arm64) ln /lib/linux-gnu/ld-linux-aarch64.so.1 /lib/linux-gnu/ld-2.35.so ;; \ + esac FROM alpine @@ -20,7 +24,7 @@ ENV LANG=en_US.UTF-8 \ TZ=UTC \ CLICKHOUSE_CONFIG=/etc/clickhouse-server/config.xml -COPY --from=glibc-donor /lib/linux-gnu/libc.so.6 /lib/linux-gnu/libdl.so.2 /lib/linux-gnu/libm.so.6 /lib/linux-gnu/libpthread.so.0 /lib/linux-gnu/librt.so.1 /lib/linux-gnu/libnss_dns.so.2 /lib/linux-gnu/libnss_files.so.2 /lib/linux-gnu/libresolv.so.2 /lib/linux-gnu/ld-2.31.so /lib/ +COPY --from=glibc-donor /lib/linux-gnu/libc.so.6 /lib/linux-gnu/libdl.so.2 /lib/linux-gnu/libm.so.6 /lib/linux-gnu/libpthread.so.0 /lib/linux-gnu/librt.so.1 /lib/linux-gnu/libnss_dns.so.2 /lib/linux-gnu/libnss_files.so.2 /lib/linux-gnu/libresolv.so.2 /lib/linux-gnu/ld-2.35.so /lib/ COPY --from=glibc-donor /etc/nsswitch.conf /etc/ COPY entrypoint.sh /entrypoint.sh diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 506a627b11c..0d5c983f5e6 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -1,4 +1,4 @@ -FROM ubuntu:20.04 +FROM ubuntu:22.04 # see https://github.com/moby/moby/issues/4032#issuecomment-192327844 # It could be removed after we move on a version 23:04+ From 2903227143360795fc4912322de9963ec7f8c3ef Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Nov 2024 10:58:21 +0100 Subject: [PATCH 807/816] Remove strange wrong named dockerfile --- .../clickhouse-statelest-test-runner.Dockerfile | 16 ---------------- 1 file changed, 16 deletions(-) delete mode 100644 docker/test/stateless/clickhouse-statelest-test-runner.Dockerfile diff --git a/docker/test/stateless/clickhouse-statelest-test-runner.Dockerfile b/docker/test/stateless/clickhouse-statelest-test-runner.Dockerfile deleted file mode 100644 index a9802f6f1da..00000000000 --- a/docker/test/stateless/clickhouse-statelest-test-runner.Dockerfile +++ /dev/null @@ -1,16 +0,0 @@ -# Since right now we can't set volumes to the docker during build, we split building container in stages: -# 1. build base container -# 2. run base conatiner with mounted volumes -# 3. commit container as image -FROM ubuntu:20.04 as clickhouse-test-runner-base - -# A volume where directory with clickhouse packages to be mounted, -# for later installing. -VOLUME /packages - -CMD apt-get update ;\ - DEBIAN_FRONTEND=noninteractive \ - apt install -y /packages/clickhouse-common-static_*.deb \ - /packages/clickhouse-client_*.deb \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* From 7b1de3fcf792aeae2cc2b197e841afcda9092654 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Nov 2024 11:12:26 +0100 Subject: [PATCH 808/816] We use `aarch64` everywhere in code, so the vars should reflect it --- tests/ci/ci_config.py | 54 ++++++++++++++++----------------- tests/ci/ci_definitions.py | 30 +++++++++--------- tests/ci/compatibility_check.py | 2 +- tests/ci/test_ci_config.py | 8 ++--- tests/ci/test_ci_options.py | 4 +-- 5 files changed, 49 insertions(+), 49 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 6d23b594b24..67cdbbdcf6d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -51,11 +51,11 @@ class CI: TAG_CONFIGS = { Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), - Tags.CI_SET_ARM: LabelConfig( + Tags.CI_SET_AARCH64: LabelConfig( run_jobs=[ JobNames.STYLE_CHECK, BuildNames.PACKAGE_AARCH64, - JobNames.INTEGRATION_TEST_ARM, + JobNames.INTEGRATION_TEST_AARCH64, ] ), Tags.CI_SET_REQUIRED: LabelConfig( @@ -95,16 +95,16 @@ class CI: static_binary_name="aarch64", additional_pkgs=True, ), - runner_type=Runners.BUILDER_ARM, + runner_type=Runners.BUILDER_AARCH64, ), - BuildNames.PACKAGE_ARM_ASAN: CommonJobConfigs.BUILD.with_properties( + BuildNames.PACKAGE_AARCH64_ASAN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( - name=BuildNames.PACKAGE_ARM_ASAN, + name=BuildNames.PACKAGE_AARCH64_ASAN, compiler="clang-18-aarch64", sanitizer="address", package_type="deb", ), - runner_type=Runners.BUILDER_ARM, + runner_type=Runners.BUILDER_AARCH64, ), BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( @@ -276,16 +276,16 @@ class CI: JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] ), - JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties( + JobNames.INSTALL_TEST_AARCH64: CommonJobConfigs.INSTALL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, ), JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN] ), - JobNames.STATEFUL_TEST_ARM_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ARM_ASAN], - runner_type=Runners.FUNC_TESTER_ARM, + JobNames.STATEFUL_TEST_AARCH64_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64_ASAN], + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.STATEFUL_TEST_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN] @@ -307,7 +307,7 @@ class CI: ), JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] @@ -335,10 +335,10 @@ class CI: JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 ), - JobNames.STATELESS_TEST_ARM_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ARM_ASAN], + JobNames.STATELESS_TEST_AARCH64_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64_ASAN], num_batches=2, - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], num_batches=4 @@ -360,7 +360,7 @@ class CI: ), JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=2 @@ -432,10 +432,10 @@ class CI: num_batches=6, timeout=9000, # the job timed out with default value (7200) ), - JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( + JobNames.INTEGRATION_TEST_AARCH64: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=6, - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -453,10 +453,10 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE], required_on_release_branch=True, ), - JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( + JobNames.COMPATIBILITY_TEST_AARCH64: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], required_on_release_branch=True, - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, ), JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties( required_builds=[BuildNames.BINARY_RELEASE], @@ -499,22 +499,22 @@ class CI: required_builds=[BuildNames.BINARY_RELEASE], run_by_labels=[Labels.JEPSEN_TEST], run_command="jepsen_check.py keeper", - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, ), JobNames.JEPSEN_SERVER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], run_by_labels=[Labels.JEPSEN_TEST], run_command="jepsen_check.py server", - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, ), JobNames.PERFORMANCE_TEST_AMD64: CommonJobConfigs.PERF_TESTS.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), - JobNames.PERFORMANCE_TEST_ARM64: CommonJobConfigs.PERF_TESTS.with_properties( + JobNames.PERFORMANCE_TEST_AARCH64: CommonJobConfigs.PERF_TESTS.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=4, run_by_labels=[Labels.PR_PERFORMANCE], - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -532,9 +532,9 @@ class CI: JobNames.CLICKBENCH_TEST: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.CLICKBENCH_TEST.with_properties( + JobNames.CLICKBENCH_TEST_AARCH64: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], - runner_type=Runners.FUNC_TESTER_ARM, + runner_type=Runners.FUNC_TESTER_AARCH64, ), JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], @@ -572,7 +572,7 @@ class CI: ), JobNames.STYLE_CHECK: JobConfig( run_always=True, - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, ), JobNames.BUGFIX_VALIDATE: JobConfig( run_by_labels=[Labels.PR_BUGFIX, Labels.PR_CRITICAL_BUGFIX], diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index dd86dc320c2..fb3e55fdbe3 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -58,11 +58,11 @@ class Runners(metaclass=WithIter): """ BUILDER = "builder" - BUILDER_ARM = "builder-aarch64" + BUILDER_AARCH64 = "builder-aarch64" STYLE_CHECKER = "style-checker" - STYLE_CHECKER_ARM = "style-checker-aarch64" + STYLE_CHECKER_AARCH64 = "style-checker-aarch64" FUNC_TESTER = "func-tester" - FUNC_TESTER_ARM = "func-tester-aarch64" + FUNC_TESTER_AARCH64 = "func-tester-aarch64" FUZZER_UNIT_TESTER = "fuzzer-unit-tester" @@ -78,7 +78,7 @@ class Tags(metaclass=WithIter): # to upload all binaries from build jobs UPLOAD_ALL_ARTIFACTS = "upload_all" CI_SET_SYNC = "ci_set_sync" - CI_SET_ARM = "ci_set_arm" + CI_SET_AARCH64 = "ci_set_aarch64" CI_SET_REQUIRED = "ci_set_required" CI_SET_BUILDS = "ci_set_builds" @@ -106,7 +106,7 @@ class BuildNames(metaclass=WithIter): PACKAGE_MSAN = "package_msan" PACKAGE_DEBUG = "package_debug" PACKAGE_AARCH64 = "package_aarch64" - PACKAGE_ARM_ASAN = "package_aarch64_asan" + PACKAGE_AARCH64_ASAN = "package_aarch64_asan" PACKAGE_RELEASE_COVERAGE = "package_release_coverage" BINARY_RELEASE = "binary_release" BINARY_TIDY = "binary_tidy" @@ -134,14 +134,14 @@ class JobNames(metaclass=WithIter): DOCKER_SERVER = "Docker server image" DOCKER_KEEPER = "Docker keeper image" INSTALL_TEST_AMD = "Install packages (release)" - INSTALL_TEST_ARM = "Install packages (aarch64)" + INSTALL_TEST_AARCH64 = "Install packages (aarch64)" STATELESS_TEST_DEBUG = "Stateless tests (debug)" STATELESS_TEST_RELEASE = "Stateless tests (release)" STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" STATELESS_TEST_ASAN = "Stateless tests (asan)" - STATELESS_TEST_ARM_ASAN = "Stateless tests (aarch64, asan)" + STATELESS_TEST_AARCH64_ASAN = "Stateless tests (aarch64, asan)" STATELESS_TEST_TSAN = "Stateless tests (tsan)" STATELESS_TEST_MSAN = "Stateless tests (msan)" STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" @@ -158,7 +158,7 @@ class JobNames(metaclass=WithIter): STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" STATEFUL_TEST_ASAN = "Stateful tests (asan)" - STATEFUL_TEST_ARM_ASAN = "Stateful tests (aarch64, asan)" + STATEFUL_TEST_AARCH64_ASAN = "Stateful tests (aarch64, asan)" STATEFUL_TEST_TSAN = "Stateful tests (tsan)" STATEFUL_TEST_MSAN = "Stateful tests (msan)" STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" @@ -181,7 +181,7 @@ class JobNames(metaclass=WithIter): INTEGRATION_TEST_ASAN = "Integration tests (asan)" INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)" INTEGRATION_TEST_TSAN = "Integration tests (tsan)" - INTEGRATION_TEST_ARM = "Integration tests (aarch64)" + INTEGRATION_TEST_AARCH64 = "Integration tests (aarch64)" INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" UPGRADE_TEST_DEBUG = "Upgrade check (debug)" @@ -205,7 +205,7 @@ class JobNames(metaclass=WithIter): JEPSEN_SERVER = "ClickHouse Server Jepsen" PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" - PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" + PERFORMANCE_TEST_AARCH64 = "Performance Comparison (aarch64)" # SQL_LOGIC_TEST = "Sqllogic test (release)" @@ -214,10 +214,10 @@ class JobNames(metaclass=WithIter): SQLTEST = "SQLTest" COMPATIBILITY_TEST = "Compatibility check (release)" - COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" + COMPATIBILITY_TEST_AARCH64 = "Compatibility check (aarch64)" CLICKBENCH_TEST = "ClickBench (release)" - CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" + CLICKBENCH_TEST_AARCH64 = "ClickBench (aarch64)" LIBFUZZER_TEST = "libFuzzer tests" @@ -387,7 +387,7 @@ class CommonJobConfigs: "./tests/ci/upload_result_helper.py", ], ), - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER_AARCH64, disable_await=True, ) COMPATIBILITY_TEST = JobConfig( @@ -634,8 +634,8 @@ REQUIRED_CHECKS = [ JobNames.STATEFUL_TEST_RELEASE, JobNames.STATELESS_TEST_RELEASE, JobNames.STATELESS_TEST_ASAN, - JobNames.STATELESS_TEST_ARM_ASAN, - JobNames.STATEFUL_TEST_ARM_ASAN, + JobNames.STATELESS_TEST_AARCH64_ASAN, + JobNames.STATEFUL_TEST_AARCH64_ASAN, JobNames.STATELESS_TEST_FLAKY_ASAN, JobNames.STATEFUL_TEST_ASAN, JobNames.STYLE_CHECK, diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index bb0c717160e..38fb2eceb28 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -131,7 +131,7 @@ def main(): check_name = args.check_name or os.getenv("CHECK_NAME") assert check_name check_glibc = True - # currently hardcoded to x86, don't enable for ARM + # currently hardcoded to x86, don't enable for AARCH64 check_distributions = ( "aarch64" not in check_name.lower() and "arm64" not in check_name.lower() ) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 0e396b827ea..03f28983262 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -36,11 +36,11 @@ class TestCIConfig(unittest.TestCase): elif "binary_" in job.lower() or "package_" in job.lower(): if job.lower() in ( CI.BuildNames.PACKAGE_AARCH64, - CI.BuildNames.PACKAGE_ARM_ASAN, + CI.BuildNames.PACKAGE_AARCH64_ASAN, ): self.assertTrue( - CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_ARM,), - f"Job [{job}] must have [{CI.Runners.BUILDER_ARM}] runner", + CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_AARCH64,), + f"Job [{job}] must have [{CI.Runners.BUILDER_AARCH64}] runner", ) else: self.assertTrue( @@ -96,7 +96,7 @@ class TestCIConfig(unittest.TestCase): else: self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) if "asan" in job and "aarch" in job: - expected_builds = [CI.BuildNames.PACKAGE_ARM_ASAN] + expected_builds = [CI.BuildNames.PACKAGE_AARCH64_ASAN] elif "asan" in job: expected_builds = [CI.BuildNames.PACKAGE_ASAN] elif "msan" in job: diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 536e18758f8..e1b780387e7 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -10,7 +10,7 @@ from ci_settings import CiSettings _TEST_BODY_1 = """ #### Run only: - [ ] Some Set -- [x] Integration tests (arm64) +- [x] Integration tests (aarch64) - [x] Integration tests - [x] Integration tests - [ ] Integration tests @@ -150,7 +150,7 @@ class TestCIOptions(unittest.TestCase): self.assertFalse(ci_options.no_ci_cache) self.assertTrue(ci_options.no_merge_commit) self.assertTrue(ci_options.woolen_wolfdog) - self.assertEqual(ci_options.ci_sets, ["ci_set_arm"]) + self.assertEqual(ci_options.ci_sets, ["ci_set_aarch64"]) self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"]) self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"]) From 52dfad190dc2bb938f68464d42f69bd80ea1b422 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Nov 2024 15:46:58 +0000 Subject: [PATCH 809/816] Automatic style fix --- tests/ci/test_ci_config.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 03f28983262..65418310c31 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -39,7 +39,8 @@ class TestCIConfig(unittest.TestCase): CI.BuildNames.PACKAGE_AARCH64_ASAN, ): self.assertTrue( - CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_AARCH64,), + CI.JOB_CONFIGS[job].runner_type + in (CI.Runners.BUILDER_AARCH64,), f"Job [{job}] must have [{CI.Runners.BUILDER_AARCH64}] runner", ) else: From 8bb656ddec205c9836db55c8a459a6b9c2cbf3d1 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 6 Nov 2024 15:55:41 +0000 Subject: [PATCH 810/816] Add context manager for partition manager --- tests/integration/test_quorum_inserts/test.py | 81 ++++++++++--------- 1 file changed, 43 insertions(+), 38 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index a646319c5f9..5e4a960acdf 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -379,50 +379,55 @@ def test_insert_quorum_with_keeper_loss_connection(started_cluster): ) ) - pm = PartitionManager() - pm.drop_instance_zk_connections(zero) + with PartitionManager() as pm: + pm.drop_instance_zk_connections(zero) - retries = 0 - zk = cluster.get_kazoo_client("zoo1") - while True: - if ( - zk.exists(f"/clickhouse/tables/{table_name}/replicas/zero/is_active") - is None - ): - break - print("replica is still active") - time.sleep(1) - retries += 1 - if retries == 120: - raise Exception("Can not wait cluster replica inactive") + retries = 0 + zk = cluster.get_kazoo_client("zoo1") + while True: + if ( + zk.exists( + f"/clickhouse/tables/{table_name}/replicas/zero/is_active" + ) + is None + ): + break + print("replica is still active") + time.sleep(1) + retries += 1 + if retries == 120: + raise Exception("Can not wait cluster replica inactive") - first.query("SYSTEM ENABLE FAILPOINT finish_set_quorum_failed_parts") - quorum_fail_future = executor.submit( - lambda: first.query( - "SYSTEM WAIT FAILPOINT finish_set_quorum_failed_parts", timeout=300 + first.query("SYSTEM ENABLE FAILPOINT finish_set_quorum_failed_parts") + quorum_fail_future = executor.submit( + lambda: first.query( + "SYSTEM WAIT FAILPOINT finish_set_quorum_failed_parts", timeout=300 + ) ) - ) - first.query(f"SYSTEM START FETCHES {table_name}") + first.query(f"SYSTEM START FETCHES {table_name}") - concurrent.futures.wait([quorum_fail_future]) + concurrent.futures.wait([quorum_fail_future]) - assert quorum_fail_future.exception() is None + assert quorum_fail_future.exception() is None - zero.query("SYSTEM ENABLE FAILPOINT finish_clean_quorum_failed_parts") - clean_quorum_fail_parts_future = executor.submit( - lambda: first.query( - "SYSTEM WAIT FAILPOINT finish_clean_quorum_failed_parts", timeout=300 + zero.query("SYSTEM ENABLE FAILPOINT finish_clean_quorum_failed_parts") + clean_quorum_fail_parts_future = executor.submit( + lambda: first.query( + "SYSTEM WAIT FAILPOINT finish_clean_quorum_failed_parts", + timeout=300, + ) ) - ) - pm.restore_instance_zk_connections(zero) - concurrent.futures.wait([clean_quorum_fail_parts_future]) + pm.restore_instance_zk_connections(zero) + concurrent.futures.wait([clean_quorum_fail_parts_future]) - assert clean_quorum_fail_parts_future.exception() is None + assert clean_quorum_fail_parts_future.exception() is None - zero.query("SYSTEM DISABLE FAILPOINT replicated_merge_tree_insert_retry_pause") - concurrent.futures.wait([insert_future]) - assert insert_future.exception() is not None - assert not zero.contains_in_log("LOGICAL_ERROR") - assert zero.contains_in_log( - "fails to commit and will not retry or clean garbage" - ) + zero.query( + "SYSTEM DISABLE FAILPOINT replicated_merge_tree_insert_retry_pause" + ) + concurrent.futures.wait([insert_future]) + assert insert_future.exception() is not None + assert not zero.contains_in_log("LOGICAL_ERROR") + assert zero.contains_in_log( + "fails to commit and will not retry or clean garbage" + ) From e8a8a4f62eabf854ebabff367d500bcc52456e83 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Nov 2024 17:31:57 +0100 Subject: [PATCH 811/816] Add test to check that accessing system.functions does not populate query_log used_functions --- ...nctions_should_not_fill_query_log_functions.reference | 1 + ...tem_functions_should_not_fill_query_log_functions.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.reference create mode 100644 tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.sql diff --git a/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.reference b/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.reference new file mode 100644 index 00000000000..021c06382c8 --- /dev/null +++ b/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.reference @@ -0,0 +1 @@ +[] ['equals'] [] diff --git a/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.sql b/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.sql new file mode 100644 index 00000000000..7e6f384c0a8 --- /dev/null +++ b/tests/queries/0_stateless/03262_system_functions_should_not_fill_query_log_functions.sql @@ -0,0 +1,9 @@ +SELECT * FROM system.functions WHERE name = 'bitShiftLeft' format Null; +SYSTEM FLUSH LOGS; +SELECT used_aggregate_functions, used_functions, used_table_functions +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND current_database = currentDatabase() + AND query LIKE '%bitShiftLeft%'; From ea3f9e582184b024bf0cb83c637bed20de5f3cda Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 17:48:04 +0000 Subject: [PATCH 812/816] Add missing reference file --- .../0_stateless/02354_vector_search_multiple_indexes.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference new file mode 100644 index 00000000000..e69de29bb2d From de21dde4cfac2c2fcb7257d018afda9e99c19a11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 6 Nov 2024 19:26:39 +0100 Subject: [PATCH 813/816] Avoid crash when using UDF in a constraint --- .../UserDefinedSQLFunctionVisitor.cpp | 99 +++---------------- src/Parsers/ASTColumnDeclaration.cpp | 10 ++ src/Parsers/ASTColumnDeclaration.h | 3 + .../03262_udf_in_constraint.reference | 2 + .../0_stateless/03262_udf_in_constraint.sh | 17 ++++ 5 files changed, 45 insertions(+), 86 deletions(-) create mode 100644 tests/queries/0_stateless/03262_udf_in_constraint.reference create mode 100755 tests/queries/0_stateless/03262_udf_in_constraint.sh diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index ebd65471449..a04b8d7b998 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -24,92 +24,7 @@ namespace ErrorCodes void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) { - if (!ast) - { - chassert(false); - return; - } - - /// FIXME: this helper should use updatePointerToChild(), but - /// forEachPointerToChild() is not implemented for ASTColumnDeclaration - /// (and also some members should be adjusted for this). - const auto visit_child_with_shared_ptr = [&](ASTPtr & child) - { - if (!child) - return; - - auto * old_value = child.get(); - visit(child); - - // child did not change - if (old_value == child.get()) - return; - - // child changed, we need to modify it in the list of children of the parent also - for (auto & current_child : ast->children) - { - if (current_child.get() == old_value) - current_child = child; - } - }; - - if (auto * col_decl = ast->as()) - { - visit_child_with_shared_ptr(col_decl->default_expression); - visit_child_with_shared_ptr(col_decl->ttl); - return; - } - - if (auto * storage = ast->as()) - { - const auto visit_child = [&](IAST * & child) - { - if (!child) - return; - - if (const auto * function = child->template as()) - { - std::unordered_set udf_in_replace_process; - auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process); - if (replace_result) - ast->setOrReplace(child, replace_result); - } - - visit(child); - }; - - visit_child(storage->partition_by); - visit_child(storage->primary_key); - visit_child(storage->order_by); - visit_child(storage->sample_by); - visit_child(storage->ttl_table); - - return; - } - - if (auto * alter = ast->as()) - { - /// It is OK to use updatePointerToChild() because ASTAlterCommand implements forEachPointerToChild() - const auto visit_child_update_parent = [&](ASTPtr & child) - { - if (!child) - return; - - auto * old_ptr = child.get(); - visit(child); - auto * new_ptr = child.get(); - - /// Some AST classes have naked pointers to children elements as members. - /// We have to replace them if the child was replaced. - if (new_ptr != old_ptr) - ast->updatePointerToChild(old_ptr, new_ptr); - }; - - for (auto & children : alter->children) - visit_child_update_parent(children); - - return; - } + chassert(ast); if (const auto * function = ast->template as()) { @@ -120,7 +35,19 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) } for (auto & child : ast->children) + { + if (!child) + return; + + auto * old_ptr = child.get(); visit(child); + auto * new_ptr = child.get(); + + /// Some AST classes have naked pointers to children elements as members. + /// We have to replace them if the child was replaced. + if (new_ptr != old_ptr) + ast->updatePointerToChild(old_ptr, new_ptr); + } } void UserDefinedSQLFunctionVisitor::visit(IAST * ast) diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index e7c3fdbb548..1c7d72bafcc 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -128,4 +128,14 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo } } +void ASTColumnDeclaration::forEachPointerToChild(std::function f) +{ + f(reinterpret_cast(&default_expression)); + f(reinterpret_cast(&comment)); + f(reinterpret_cast(&codec)); + f(reinterpret_cast(&statistics_desc)); + f(reinterpret_cast(&ttl)); + f(reinterpret_cast(&collation)); + f(reinterpret_cast(&settings)); +} } diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 914916d5074..0c5076f0201 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -29,6 +29,9 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override; + +protected: + void forEachPointerToChild(std::function f) override; }; } diff --git a/tests/queries/0_stateless/03262_udf_in_constraint.reference b/tests/queries/0_stateless/03262_udf_in_constraint.reference new file mode 100644 index 00000000000..29d403b85a8 --- /dev/null +++ b/tests/queries/0_stateless/03262_udf_in_constraint.reference @@ -0,0 +1,2 @@ +CREATE TABLE default.t0\n(\n `c0` Int32,\n CONSTRAINT c1 CHECK c0 > 5\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +10 diff --git a/tests/queries/0_stateless/03262_udf_in_constraint.sh b/tests/queries/0_stateless/03262_udf_in_constraint.sh new file mode 100755 index 00000000000..3c36e7caeb4 --- /dev/null +++ b/tests/queries/0_stateless/03262_udf_in_constraint.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + CREATE FUNCTION ${CLICKHOUSE_DATABASE}_function AS (x) -> x > 5; + CREATE TABLE t0 (c0 Int, CONSTRAINT c1 CHECK ${CLICKHOUSE_DATABASE}_function(c0)) ENGINE = MergeTree() ORDER BY tuple(); + SHOW CREATE TABLE t0; + INSERT INTO t0(c0) VALUES (10); + INSERT INTO t0(c0) VALUES (3); -- {serverError VIOLATED_CONSTRAINT} + SELECT * FROM t0; + + DROP TABLE t0; + DROP FUNCTION ${CLICKHOUSE_DATABASE}_function; +" From c55840794195689299ccb1b9f838fdb3d1a7edfa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Nov 2024 19:53:01 +0000 Subject: [PATCH 814/816] Remove duplicate test (same as 02354_vector_search_bugs_multiple_indexes.sql) --- ...02354_vector_search_multiple_indexes.reference | 0 .../02354_vector_search_multiple_indexes.sql | 15 --------------- 2 files changed, 15 deletions(-) delete mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference delete mode 100644 tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql b/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql deleted file mode 100644 index aedba286a9f..00000000000 --- a/tests/queries/0_stateless/02354_vector_search_multiple_indexes.sql +++ /dev/null @@ -1,15 +0,0 @@ --- Tags: no-fasttest, no-ordinary-database - --- Tests that multiple vector similarity indexes can be created on the same column (even if that makes no sense) - -SET allow_experimental_vector_similarity_index = 1; - -DROP TABLE IF EXISTS tab; - -CREATE TABLE tab (id Int32, vec Array(Float32), PRIMARY KEY id, INDEX vec_idx(vec) TYPE vector_similarity('hnsw', 'L2Distance')); - -ALTER TABLE tab ADD INDEX idx(vec) TYPE minmax; -ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -ALTER TABLE tab ADD INDEX vec_idx2(vec) TYPE vector_similarity('hnsw', 'L2Distance'); -- silly but creating the same index also works for non-vector indexes ... - -DROP TABLE tab; From 26f0ba2c4ceb4b6d52f159943de63d4f2ca10520 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Nov 2024 21:23:06 +0100 Subject: [PATCH 815/816] Update compatibility section for clickhouse-server docker image --- docker/server/README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/server/README.md b/docker/server/README.md index 65239126790..1dc636414ac 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,6 +20,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A) and additionally the Load-Acquire RCpc register. The register is optional in version ARMv8.2-A and mandatory in [ARMv8.3-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.3-A). Supported in Graviton >=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A). +- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run [--privileged | --security-opt seccomp=unconfined]` instead, however that has security implications. ## How to use this image From 157f745136094eb2eaeae72f17d103928194fd52 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 6 Nov 2024 22:09:12 +0100 Subject: [PATCH 816/816] Write a simple troubleshooting for an old docker and clickhouse-server --- docs/en/operations/_troubleshooting.md | 28 ++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/docs/en/operations/_troubleshooting.md b/docs/en/operations/_troubleshooting.md index 77389782675..f0ee1ca1d29 100644 --- a/docs/en/operations/_troubleshooting.md +++ b/docs/en/operations/_troubleshooting.md @@ -65,6 +65,34 @@ sudo rm -f /etc/yum.repos.d/clickhouse.repo After that follow the [install guide](../getting-started/install.md#from-rpm-packages) +### You Can't Run Docker Container + +You are running a simple `docker run clickhouse/clickhouse-server` and it crashes with a stack trace similar to following: + +``` +$ docker run -it clickhouse/clickhouse-server +........ +2024.11.06 21:04:48.912036 [ 1 ] {} SentryWriter: Sending crash reports is disabled +Poco::Exception. Code: 1000, e.code() = 0, System exception: cannot start thread, Stack trace (when copying this message, always include the lines below): + +0. Poco::ThreadImpl::startImpl(Poco::SharedPtr>) @ 0x00000000157c7b34 +1. Poco::Thread::start(Poco::Runnable&) @ 0x00000000157c8a0e +2. BaseDaemon::initializeTerminationAndSignalProcessing() @ 0x000000000d267a14 +3. BaseDaemon::initialize(Poco::Util::Application&) @ 0x000000000d2652cb +4. DB::Server::initialize(Poco::Util::Application&) @ 0x000000000d128b38 +5. Poco::Util::Application::run() @ 0x000000001581cfda +6. DB::Server::run() @ 0x000000000d1288f0 +7. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000015825e27 +8. mainEntryClickHouseServer(int, char**) @ 0x000000000d125b38 +9. main @ 0x0000000007ea4eee +10. ? @ 0x00007f67ff946d90 +11. ? @ 0x00007f67ff946e40 +12. _start @ 0x00000000062e802e + (version 24.10.1.2812 (official build)) +``` + +The reason is an old docker daemon with version lower than `20.10.10`. A way to fix it either upgrading it, or running `docker run [--privileged | --security-opt seccomp=unconfined]`. The latter has security implications. + ## Connecting to the Server {#troubleshooting-accepts-no-connections} Possible issues: