From 3e754532d5200a183d6f5aae9c91447b3b6faa77 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 5 Jun 2024 16:15:25 +0200 Subject: [PATCH 001/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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/281] 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 bb2716251b46ebacaf5b3767f5d12d1b2870a68c Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 31 Aug 2024 19:44:31 +0000 Subject: [PATCH 019/281] 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 020/281] 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 021/281] 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 022/281] 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 023/281] 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 024/281] 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 025/281] 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 026/281] 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 027/281] 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 2ef36b36acb1926b70b1d4b64c7d3d83783e483c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:06:44 +0000 Subject: [PATCH 028/281] 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 029/281] 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 030/281] 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 031/281] 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 9edc66d458d2e9376ed52582dc25ab7934ea9085 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 6 Sep 2024 19:22:59 +0000 Subject: [PATCH 032/281] 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 14542d6779652c7c0b78efca3fa74fb6ae4a66f6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:26:54 +0000 Subject: [PATCH 033/281] 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 034/281] 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 035/281] 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 036/281] 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 037/281] 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 038/281] 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 039/281] 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 040/281] 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 041/281] 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 042/281] 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 043/281] 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 ec4e0ed1b2c2b355dddd07d18736d8e993a9d620 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 17 Sep 2024 11:18:19 +0000 Subject: [PATCH 044/281] 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 045/281] 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 b83fd18c9cebd4a7af4287f9c1e11b4f5410f21d Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 18:12:43 +0000 Subject: [PATCH 046/281] 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 047/281] 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 048/281] .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 049/281] 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 050/281] 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 64359a54fd82491a41dba78cdf6259569c6c5e6b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 14:35:48 +0000 Subject: [PATCH 051/281] 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 052/281] 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 053/281] 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 054/281] 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 055/281] 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 056/281] 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 057/281] 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 058/281] 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 059/281] 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 060/281] 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 061/281] 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 062/281] 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 53896e1a1f9366fd9c2e38b0c3045c8606a4aaaf Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 25 Sep 2024 13:21:10 +0000 Subject: [PATCH 063/281] 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 064/281] 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 2c073ec53eb70d6a10d56c60713b22a6acca10e6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 27 Sep 2024 22:22:03 -0300 Subject: [PATCH 065/281] 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 066/281] 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 067/281] 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 ffe61c3007ec216a9b8010f3238c351834b1e068 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 30 Sep 2024 11:18:28 +0000 Subject: [PATCH 068/281] 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 2117a29eb17ea81c653755396599ffaa06477cac Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 11:09:27 -0300 Subject: [PATCH 069/281] 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 070/281] 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 417a0a8017649502de262414043de00dce6413c4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Sep 2024 13:47:08 -0300 Subject: [PATCH 071/281] 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 072/281] 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 073/281] 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 074/281] 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 075/281] 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 076/281] 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 b25b711a7135fa82bd3bd41a2547d316bb5c9ff1 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 10:52:16 +0000 Subject: [PATCH 077/281] 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 078/281] 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 326ae4cac36a0868d0683cadc78b89b9e20b22f1 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 1 Oct 2024 11:21:54 -0300 Subject: [PATCH 079/281] 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 809f0ee0a2b8f928cb655b98e654a1304717abb6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 16:36:32 +0000 Subject: [PATCH 080/281] 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 7722a5e4fa78763a7d88f69b826b7103d0f306a3 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 2 Oct 2024 17:54:43 +0000 Subject: [PATCH 081/281] 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 0d8ff26706c1879debbb4cfa029fbaa9239cd004 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 00:18:42 +0000 Subject: [PATCH 082/281] 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 083/281] 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 d6b3c106db864f9ae1b471c84c67a8032862d52f Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 07:51:47 +0000 Subject: [PATCH 084/281] 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 1fee0534d654dc201b4e2479cdb044cd15887105 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 4 Oct 2024 18:19:35 -0300 Subject: [PATCH 085/281] 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 086/281] 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 087/281] 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 088/281] 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 089/281] 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 090/281] 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 091/281] 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 092/281] 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 093/281] 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 094/281] 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 095/281] 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 096/281] 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 097/281] 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 dca5c250fe0a70e8d0bb88714fd42cb7b1e85168 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 18:19:05 +0000 Subject: [PATCH 098/281] 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 099/281] 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 100/281] 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 101/281] 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 102/281] 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 103/281] 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 104/281] 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 105/281] 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 106/281] 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 384ba4217dcb8bc526e50dea7cbf88b9ed7e734d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Oct 2024 05:15:29 +0000 Subject: [PATCH 107/281] 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 46faeaafd2bc93a60b3ad9671a476e9e66830675 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 10 Oct 2024 11:17:03 -0300 Subject: [PATCH 108/281] 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 164cc1211a9d322e4f8842b3fb21b6bfa3755d36 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 11 Oct 2024 14:11:54 -0300 Subject: [PATCH 109/281] 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 110/281] 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 111/281] 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 112/281] 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 113/281] 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 114/281] 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 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 115/281] 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 116/281] 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 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 117/281] 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 8a854bd5ac225f2cbb1bef441b1d8c930fd1356f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:10:05 +0000 Subject: [PATCH 118/281] 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 119/281] 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 120/281] 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 121/281] 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 122/281] 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 123/281] 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 124/281] 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 125/281] 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 126/281] 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 127/281] 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 128/281] 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 129/281] 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 @@ - + - - - 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 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 267/281] 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 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 268/281] 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 96f073f49a182daad247e77d1686505048e63b88 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 30 Oct 2024 14:53:25 +0100 Subject: [PATCH 269/281] 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 2760f283a1c54351c3103e4a9ce6556b06009149 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 30 Oct 2024 16:18:35 +0100 Subject: [PATCH 270/281] 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 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 271/281] 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 272/281] 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 124736756f6b60f915c47e0844214f98590c8574 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 16:18:43 -0300 Subject: [PATCH 273/281] 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 274/281] 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 8d622000b05c7bb54d7e4587a0568bdba327d059 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:26:13 -0300 Subject: [PATCH 275/281] 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 276/281] 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 26da759cf20459c38bc504b2d9a790c38c66c5a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Oct 2024 07:51:20 +0100 Subject: [PATCH 277/281] 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 278/281] 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 279/281] 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 280/281] 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 281/281] 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)).