From 3e754532d5200a183d6f5aae9c91447b3b6faa77 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 5 Jun 2024 16:15:25 +0200 Subject: [PATCH 0001/1218] 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 0002/1218] 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 0003/1218] 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 0004/1218] 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 0005/1218] 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 0006/1218] 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 0007/1218] 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 9b36d0b61c261f80b9174256e7e279a06a59e9f5 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 26 Jun 2024 09:08:33 +0000 Subject: [PATCH 0008/1218] Add system.query_log_metric table --- programs/server/config.xml | 12 +++++ src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 3 +- src/Interpreters/Context.cpp | 10 ++++ src/Interpreters/Context.h | 2 + src/Interpreters/ErrorLog.cpp | 1 - src/Interpreters/ErrorLog.h | 1 - src/Interpreters/MetricLog.cpp | 1 - src/Interpreters/MetricLog.h | 1 - src/Interpreters/QueryLogMetric.cpp | 78 +++++++++++++++++++++++++++++ src/Interpreters/QueryLogMetric.h | 40 +++++++++++++++ src/Interpreters/SystemLog.cpp | 4 ++ src/Interpreters/SystemLog.h | 2 + 13 files changed, 151 insertions(+), 5 deletions(-) create mode 100644 src/Interpreters/QueryLogMetric.cpp create mode 100644 src/Interpreters/QueryLogMetric.h diff --git a/programs/server/config.xml b/programs/server/config.xml index 94825a55f67..64c04287607 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1182,6 +1182,18 @@ false + + + system + query_log_metric
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ + system query_log_metric
diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 90b278b1f50..095b9601d23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -509,7 +509,7 @@ class IColumn; M(Bool, log_query_threads, false, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \ M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \ - M(UInt64, query_log_metric_interval, 0, "Periodic interval in milliseconds to collect query metrics.", 0) \ + M(UInt64, query_log_metric_interval, 0, "Periodic interval in milliseconds to collect query log metrics.", 0) \ M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(String, send_logs_source_regexp, "", "Send server text logs with specified regexp to match log source name. Empty means all sources.", 0) \ M(Bool, enable_optimize_predicate_expression, true, "If it is set to true, optimize predicates to subqueries.", 0) \ diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 611fad6fa82..e5e91d4bf53 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -543,7 +543,13 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback) { progress_callback = [callback](const Progress & progress) { - // TODO: PMO to update counters only for the query log metric interval + // Performance counters need to be updated from the same thread the query is being executed + // on because most info is taken using getrusage with RUSAGE_THREAD. Ideally, we would only + // update the counters once we're close to the interval at which the query log metric data + // needs to be collected. However, since the progress callback is called not very + // frequently, we'd rather update them as needed. Using the + // updatePerformanceCountersIfNeeded instead of just updatePerformanceCounters we make sure + // that we update them with a sufficiently frequent interval. auto context = CurrentThread::getQueryContext(); if (auto query_log_metric = context->getQueryLogMetric()) CurrentThread::updatePerformanceCountersIfNeeded(); From 65a61e1c95e7517a6a8f965dc3484fa037d8d1ab Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 15 Jul 2024 14:41:07 +0000 Subject: [PATCH 0025/1218] Update documentation --- docs/en/operations/settings/settings.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b85b249dd21..2b18ecf57f1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1823,8 +1823,9 @@ Default value: 0 (no restriction). ## query_log_metric_interval (#query_log_metric_interval) The interval in milliseconds at which the [query_log_metric](../../operations/system-tables/query_log_metric.md) for individual queries is collected. +If set to 0, it will take the `collect_interval_milliseconds` from the [query_log_metric setting](../../operations/server-configuration-parameters/settings.md#query_log_metric). -Default value: 1000 +Default value: 0 ## insert_quorum {#insert_quorum} From 7fdf3a0c2d78ba4c02141e92819bdcf9e7356e07 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 15 Jul 2024 14:43:59 +0000 Subject: [PATCH 0026/1218] Normalize tag name to meet ClickHouse convention instead of boost --- src/Interpreters/QueryLogMetric.cpp | 8 ++++---- src/Interpreters/QueryLogMetric.h | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/QueryLogMetric.cpp b/src/Interpreters/QueryLogMetric.cpp index 556be89c4c3..22a33eba522 100644 --- a/src/Interpreters/QueryLogMetric.cpp +++ b/src/Interpreters/QueryLogMetric.cpp @@ -103,7 +103,7 @@ void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_t queries.emplace(std::move(status)); // Wake up the sleeping thread only if the collection for this query needs to wake up sooner - const auto & queries_by_next_collect_time = queries.get(); + const auto & queries_by_next_collect_time = queries.get(); if (query_id == queries_by_next_collect_time.begin()->query_id) { std::unique_lock cv_lock(queries_cv_mutex); @@ -114,7 +114,7 @@ void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_t void QueryLogMetric::finishQuery(const String & query_id) { std::lock_guard lock(queries_mutex); - auto & queries_by_name = queries.get(); + auto & queries_by_name = queries.get(); queries_by_name.erase(query_id); } @@ -131,7 +131,7 @@ void QueryLogMetric::threadFunction() const auto current_time = std::chrono::system_clock::now(); if (!queries.empty()) { - auto & queries_by_next_collect_time = queries.get(); + auto & queries_by_next_collect_time = queries.get(); stepFunction(current_time); desired_timepoint = queries_by_next_collect_time.begin()->next_collect_time; } @@ -184,7 +184,7 @@ void QueryLogMetric::stepFunction(TimePoint current_time) { static const auto & process_list = context->getProcessList(); - auto & queries_by_next_collect_time = queries.get(); + auto & queries_by_next_collect_time = queries.get(); for (const auto & query_status : queries_by_next_collect_time) { // The queries are already sorted by next_collect_time, so once we find a query with a next_collect_time diff --git a/src/Interpreters/QueryLogMetric.h b/src/Interpreters/QueryLogMetric.h index 4c966563e68..d4ef95d98a8 100644 --- a/src/Interpreters/QueryLogMetric.h +++ b/src/Interpreters/QueryLogMetric.h @@ -56,14 +56,14 @@ class QueryLogMetric : public PeriodicLog using PeriodicLog::PeriodicLog; public: - struct by_query_id{}; - struct by_next_collect_time{}; + struct ByQueryId{}; + struct ByNextCollectTime{}; using QuerySet = boost::multi_index_container< QueryLogMetricStatus, boost::multi_index::indexed_by< - boost::multi_index::hashed_unique, boost::multi_index::member>, - boost::multi_index::ordered_non_unique, boost::multi_index::member>>>; + boost::multi_index::hashed_unique, boost::multi_index::member>, + boost::multi_index::ordered_non_unique, boost::multi_index::member>>>; // Both startQuery and finishQuery are called from the thread that executes the query void startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds); From a06f2aca76af4e9d4e790463e1234e184c176bbd Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 15 Jul 2024 15:03:39 +0000 Subject: [PATCH 0027/1218] Fix previous commit --- src/Interpreters/QueryLogMetric.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/QueryLogMetric.cpp b/src/Interpreters/QueryLogMetric.cpp index 22a33eba522..07c168d9e38 100644 --- a/src/Interpreters/QueryLogMetric.cpp +++ b/src/Interpreters/QueryLogMetric.cpp @@ -103,7 +103,7 @@ void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_t queries.emplace(std::move(status)); // Wake up the sleeping thread only if the collection for this query needs to wake up sooner - const auto & queries_by_next_collect_time = queries.get(); + const auto & queries_by_next_collect_time = queries.get(); if (query_id == queries_by_next_collect_time.begin()->query_id) { std::unique_lock cv_lock(queries_cv_mutex); @@ -131,7 +131,7 @@ void QueryLogMetric::threadFunction() const auto current_time = std::chrono::system_clock::now(); if (!queries.empty()) { - auto & queries_by_next_collect_time = queries.get(); + auto & queries_by_next_collect_time = queries.get(); stepFunction(current_time); desired_timepoint = queries_by_next_collect_time.begin()->next_collect_time; } @@ -184,7 +184,7 @@ void QueryLogMetric::stepFunction(TimePoint current_time) { static const auto & process_list = context->getProcessList(); - auto & queries_by_next_collect_time = queries.get(); + auto & queries_by_next_collect_time = queries.get(); for (const auto & query_status : queries_by_next_collect_time) { // The queries are already sorted by next_collect_time, so once we find a query with a next_collect_time From e1efd5b54b8f589115e1ac9f4e8433474d68a1b9 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 16 Jul 2024 12:11:43 +0000 Subject: [PATCH 0028/1218] Fix segfault whenever there's no context --- src/QueryPipeline/QueryPipeline.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index e5e91d4bf53..6a1321bd64e 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -551,8 +551,11 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback) // updatePerformanceCountersIfNeeded instead of just updatePerformanceCounters we make sure // that we update them with a sufficiently frequent interval. auto context = CurrentThread::getQueryContext(); - if (auto query_log_metric = context->getQueryLogMetric()) - CurrentThread::updatePerformanceCountersIfNeeded(); + if (context) + { + if (auto query_log_metric = context->getQueryLogMetric()) + CurrentThread::updatePerformanceCountersIfNeeded(); + } if (callback) callback(progress); From ca77e7252f04c241ff4c1d57673f2b9d1dad08f1 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 16 Jul 2024 12:40:05 +0000 Subject: [PATCH 0029/1218] Add query_log_metric to config samples --- programs/server/config.yaml.example | 7 +++++++ tests/integration/test_backup_restore_new/test.py | 1 + .../configs/config.d/query_log_metric.xml | 8 ++++++++ tests/integration/test_config_xml_full/test.py | 1 + .../configs/config.d/query_log_metric.yaml | 6 ++++++ .../configs/config.d/query_log_metric.xml | 8 ++++++++ tests/integration/test_config_xml_yaml_mix/test.py | 1 + .../configs/config.d/query_log_metric.yaml | 6 ++++++ tests/integration/test_config_yaml_full/test.py | 1 + .../configs/config.d/query_log_metric.xml | 8 ++++++++ .../integration/test_config_yaml_main/configs/config.yaml | 5 +++++ tests/integration/test_config_yaml_main/test.py | 1 + tests/integration/test_system_flush_logs/test.py | 1 + tests/integration/test_system_logs_recreate/test.py | 1 + 14 files changed, 55 insertions(+) create mode 100644 tests/integration/test_config_xml_full/configs/config.d/query_log_metric.xml create mode 100644 tests/integration/test_config_xml_main/configs/config.d/query_log_metric.yaml create mode 100644 tests/integration/test_config_xml_yaml_mix/configs/config.d/query_log_metric.xml create mode 100644 tests/integration/test_config_yaml_full/configs/config.d/query_log_metric.yaml create mode 100644 tests/integration/test_config_yaml_main/configs/config.d/query_log_metric.xml diff --git a/programs/server/config.yaml.example b/programs/server/config.yaml.example index 5d5499f876c..0d4800e0bbc 100644 --- a/programs/server/config.yaml.example +++ b/programs/server/config.yaml.example @@ -743,6 +743,13 @@ error_log: flush_interval_milliseconds: 7500 collect_interval_milliseconds: 1000 +# Query log metric contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. +query_log_metric: + database: system + table: query_log_metric + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 + # Asynchronous metric log contains values of metrics from # system.asynchronous_metrics. asynchronous_metric_log: diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index d8662fad011..c8d2ca20b7c 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1486,6 +1486,7 @@ def test_backup_all(exclude_system_log_tables): "asynchronous_insert_log", "backup_log", "error_log", + "query_log_metric", ] exclude_from_backup += ["system." + table_name for table_name in log_tables] diff --git a/tests/integration/test_config_xml_full/configs/config.d/query_log_metric.xml b/tests/integration/test_config_xml_full/configs/config.d/query_log_metric.xml new file mode 100644 index 00000000000..6462316d6ac --- /dev/null +++ b/tests/integration/test_config_xml_full/configs/config.d/query_log_metric.xml @@ -0,0 +1,8 @@ + + + system + query_log_metric
+ 7500 + 1000 +
+
diff --git a/tests/integration/test_config_xml_full/test.py b/tests/integration/test_config_xml_full/test.py index 8513792f3b3..ea8ae4f28a7 100644 --- a/tests/integration/test_config_xml_full/test.py +++ b/tests/integration/test_config_xml_full/test.py @@ -18,6 +18,7 @@ def test_xml_full_conf(): "configs/config.d/more_clusters.xml", "configs/config.d/part_log.xml", "configs/config.d/path.xml", + "configs/config.d/query_log_metric.xml", "configs/config.d/query_masking_rules.xml", "configs/config.d/tcp_with_proxy.xml", "configs/config.d/text_log.xml", diff --git a/tests/integration/test_config_xml_main/configs/config.d/query_log_metric.yaml b/tests/integration/test_config_xml_main/configs/config.d/query_log_metric.yaml new file mode 100644 index 00000000000..55729f0be24 --- /dev/null +++ b/tests/integration/test_config_xml_main/configs/config.d/query_log_metric.yaml @@ -0,0 +1,6 @@ +query_log_metric: + database: system + table: query_log_metric + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 + diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_log_metric.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_log_metric.xml new file mode 100644 index 00000000000..6462316d6ac --- /dev/null +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.d/query_log_metric.xml @@ -0,0 +1,8 @@ + + + system + query_log_metric
+ 7500 + 1000 +
+
diff --git a/tests/integration/test_config_xml_yaml_mix/test.py b/tests/integration/test_config_xml_yaml_mix/test.py index df1eaa9ded7..e59cc0cf987 100644 --- a/tests/integration/test_config_xml_yaml_mix/test.py +++ b/tests/integration/test_config_xml_yaml_mix/test.py @@ -20,6 +20,7 @@ def test_extra_yaml_mix(): "configs/config.d/more_clusters.yaml", "configs/config.d/part_log.xml", "configs/config.d/path.yaml", + "configs/config.d/query_log_metric.xml", "configs/config.d/query_masking_rules.xml", "configs/config.d/tcp_with_proxy.yaml", "configs/config.d/test_cluster_with_incorrect_pw.xml", diff --git a/tests/integration/test_config_yaml_full/configs/config.d/query_log_metric.yaml b/tests/integration/test_config_yaml_full/configs/config.d/query_log_metric.yaml new file mode 100644 index 00000000000..55729f0be24 --- /dev/null +++ b/tests/integration/test_config_yaml_full/configs/config.d/query_log_metric.yaml @@ -0,0 +1,6 @@ +query_log_metric: + database: system + table: query_log_metric + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 + diff --git a/tests/integration/test_config_yaml_full/test.py b/tests/integration/test_config_yaml_full/test.py index 986199fd228..52acdc0cb29 100644 --- a/tests/integration/test_config_yaml_full/test.py +++ b/tests/integration/test_config_yaml_full/test.py @@ -19,6 +19,7 @@ def test_yaml_full_conf(): "configs/config.d/more_clusters.yaml", "configs/config.d/part_log.yaml", "configs/config.d/path.yaml", + "configs/config.d/query_log_metric.xml", "configs/config.d/query_masking_rules.yaml", "configs/config.d/tcp_with_proxy.yaml", "configs/config.d/test_cluster_with_incorrect_pw.yaml", diff --git a/tests/integration/test_config_yaml_main/configs/config.d/query_log_metric.xml b/tests/integration/test_config_yaml_main/configs/config.d/query_log_metric.xml new file mode 100644 index 00000000000..6462316d6ac --- /dev/null +++ b/tests/integration/test_config_yaml_main/configs/config.d/query_log_metric.xml @@ -0,0 +1,8 @@ + + + system + query_log_metric
+ 7500 + 1000 +
+
diff --git a/tests/integration/test_config_yaml_main/configs/config.yaml b/tests/integration/test_config_yaml_main/configs/config.yaml index 6e62b13a0ee..d7afaa005cf 100644 --- a/tests/integration/test_config_yaml_main/configs/config.yaml +++ b/tests/integration/test_config_yaml_main/configs/config.yaml @@ -95,6 +95,11 @@ error_log: table: error_log flush_interval_milliseconds: 7500 collect_interval_milliseconds: 1000 +query_log_metric: + database: system + table: query_log_metric + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 asynchronous_metric_log: database: system table: asynchronous_metric_log diff --git a/tests/integration/test_config_yaml_main/test.py b/tests/integration/test_config_yaml_main/test.py index fb1d62b8cc7..638da427558 100644 --- a/tests/integration/test_config_yaml_main/test.py +++ b/tests/integration/test_config_yaml_main/test.py @@ -19,6 +19,7 @@ def test_yaml_main_conf(): "configs/config.d/more_clusters.xml", "configs/config.d/part_log.xml", "configs/config.d/path.xml", + "configs/config.d/query_log_metric.xml", "configs/config.d/query_masking_rules.xml", "configs/config.d/tcp_with_proxy.xml", "configs/config.d/test_cluster_with_incorrect_pw.xml", diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 2022f9d4a89..4418eb0fe67 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -22,6 +22,7 @@ system_logs = [ ("system.trace_log", 1), ("system.metric_log", 1), ("system.error_log", 1), + ("system.query_log_metric", 1), ] diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index 1bdb1fe3261..add9a81b228 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -31,6 +31,7 @@ def test_system_logs_recreate(): "trace_log", "metric_log", "error_log", + "query_log_metric", ] node.query("SYSTEM FLUSH LOGS") From 5450f6e15f54f29fe131bdcc7937cc9d4c73590c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 16 Jul 2024 18:48:25 +0000 Subject: [PATCH 0030/1218] Add new setting to SettingsChangesHistory --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9b72209103..4665178237d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,8 @@ static std::initializer_list Date: Wed, 17 Jul 2024 19:47:05 +0200 Subject: [PATCH 0031/1218] 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 ca769373adc7f98d55f69e634c2dbf2068d592ca Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Jul 2024 09:18:29 +0000 Subject: [PATCH 0032/1218] Relax the test condition Let's use a slower interval than default 1000ms and relax the stddev condition to make it less flaky. --- tests/queries/0_stateless/03203_system_query_log_metric.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03203_system_query_log_metric.sh b/tests/queries/0_stateless/03203_system_query_log_metric.sh index c5a47feecd3..1c4db749773 100755 --- a/tests/queries/0_stateless/03203_system_query_log_metric.sh +++ b/tests/queries/0_stateless/03203_system_query_log_metric.sh @@ -8,9 +8,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) readonly query_prefix=$CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT --query-id="${query_prefix}_1000" -q "SELECT sleep(3) + sleep(2) FORMAT Null" & +$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1234" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=1234 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_123" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=123 FORMAT Null" & $CLICKHOUSE_CLIENT --query-id="${query_prefix}_47" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=47 FORMAT Null" & -$CLICKHOUSE_CLIENT --query-id="${query_prefix}_11" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=11 FORMAT Null" & wait @@ -30,11 +30,11 @@ function check_log() ORDER BY event_time_microseconds OFFSET 1 ) - SELECT count() BETWEEN least(5000 / $interval - 2, 5000 / $interval * 0.9) AND (5000 / $interval - 1) * 1.1, avg(diff) BETWEEN $interval * 0.9 AND $interval * 1.1, stddevPopStable(diff) BETWEEN 0 AND $interval * 0.2 FROM diff + SELECT count() BETWEEN least(5000 / $interval - 2, 5000 / $interval * 0.9) AND (5000 / $interval - 1) * 1.1, avg(diff) BETWEEN $interval * 0.9 AND $interval * 1.1, stddevPopStable(diff) BETWEEN 0 AND $interval * 0.5 FROM diff """ } check_log 1000 +check_log 1234 check_log 123 check_log 47 -check_log 11 From 4253f8787e76ca304042c07d8afb4fe9d3b8ccaa Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Jul 2024 10:29:03 +0000 Subject: [PATCH 0033/1218] Fix doc --- .../server-configuration-parameters/settings.md | 10 +++++----- docs/en/operations/system-tables/query_log_metric.md | 2 +- src/Interpreters/QueryLogMetric.cpp | 4 ++-- src/Interpreters/executeQuery.cpp | 1 - 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 902fe2ddaef..ddd3d3afa0d 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2208,26 +2208,26 @@ To manually turn on metrics history collection [`system.query_log_metric`](../.. ``` xml - + system - query_metric_log
+ query_log_metric
7500 1000 1048576 8192 524288 false - +
``` **Disabling** -To disable `query_metric_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_metric_log.xml` with the following content: +To disable `query_log_metric` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_log_metric.xml` with the following content: ``` xml - + ``` diff --git a/docs/en/operations/system-tables/query_log_metric.md b/docs/en/operations/system-tables/query_log_metric.md index 6489a46c1f0..167e50a7780 100644 --- a/docs/en/operations/system-tables/query_log_metric.md +++ b/docs/en/operations/system-tables/query_log_metric.md @@ -1,7 +1,7 @@ --- slug: /en/operations/system-tables/query_log_metric --- -# metric_log +# query_log_metric Contains history of memory and metric values from table `system.events` for individual queries, periodically flushed to disk. diff --git a/src/Interpreters/QueryLogMetric.cpp b/src/Interpreters/QueryLogMetric.cpp index 07c168d9e38..29ef34d70ec 100644 --- a/src/Interpreters/QueryLogMetric.cpp +++ b/src/Interpreters/QueryLogMetric.cpp @@ -114,8 +114,8 @@ void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_t void QueryLogMetric::finishQuery(const String & query_id) { std::lock_guard lock(queries_mutex); - auto & queries_by_name = queries.get(); - queries_by_name.erase(query_id); + auto & queries_by_id = queries.get(); + queries_by_id.erase(query_id); } void QueryLogMetric::threadFunction() diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 301cead9a75..226652a7284 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1338,7 +1338,6 @@ static std::tuple executeQueryImpl( query_database, query_table, async_insert); - /// Also make possible for caller to log successful query finish and exception during execution. auto finish_callback = [elem, context, From 316d191acc948038207f6565edd7ef884e6720cf Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 18 Jul 2024 10:44:01 +0000 Subject: [PATCH 0034/1218] Fix silly mistake collecting ProfileEvents --- src/Interpreters/QueryLogMetric.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/QueryLogMetric.cpp b/src/Interpreters/QueryLogMetric.cpp index 29ef34d70ec..fdfd971fa46 100644 --- a/src/Interpreters/QueryLogMetric.cpp +++ b/src/Interpreters/QueryLogMetric.cpp @@ -170,9 +170,9 @@ QueryLogMetricElement QueryLogMetric::createLogMetricElement(const String & quer for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { - const auto & value = (*profile_counters)[i]; - elem.profile_events[i] = new_query_status.last_profile_events[i] - value; - new_query_status.last_profile_events[i] = value; + const auto & new_value = (*profile_counters)[i]; + elem.profile_events[i] = new_value - new_query_status.last_profile_events[i]; + new_query_status.last_profile_events[i] = new_value; } queries.modify(query_status_it, [&](QueryLogMetricStatus & query_status) { query_status = std::move(new_query_status); }); From 639383b7a145a8c03fbc987354e5d4b9678fd5cc Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Jul 2024 13:21:33 +0000 Subject: [PATCH 0035/1218] Fix configs for query_log_metric --- .../configs/no_system_log.xml | 1 + .../test_backup_restore_new/test.py | 2 +- .../test_config_xml_full/configs/config.xml | 13 +++++++++++++ .../test_config_xml_main/configs/config.xml | 19 ++++++++++++++++--- .../integration/test_config_xml_main/test.py | 1 + .../configs/config.xml | 19 ++++++++++++++++--- .../test_config_yaml_full/configs/config.yaml | 5 +++++ .../integration/test_config_yaml_full/test.py | 2 +- .../configs/conf.xml | 1 + .../test_system_flush_logs/test.py | 2 +- .../test_system_logs_recreate/test.py | 2 +- 11 files changed, 57 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_MemoryTracking/configs/no_system_log.xml b/tests/integration/test_MemoryTracking/configs/no_system_log.xml index 7d80c7fbf78..ff26cec3c03 100644 --- a/tests/integration/test_MemoryTracking/configs/no_system_log.xml +++ b/tests/integration/test_MemoryTracking/configs/no_system_log.xml @@ -3,6 +3,7 @@ + diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index c8d2ca20b7c..41334eb2ed5 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1468,6 +1468,7 @@ def test_backup_all(exclude_system_log_tables): # See the list of log tables in src/Interpreters/SystemLog.cpp log_tables = [ "query_log", + "query_log_metric", "query_thread_log", "part_log", "trace_log", @@ -1486,7 +1487,6 @@ def test_backup_all(exclude_system_log_tables): "asynchronous_insert_log", "backup_log", "error_log", - "query_log_metric", ] exclude_from_backup += ["system." + table_name for table_name in log_tables] diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index 61aa0a5c724..f34250c1f86 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -764,6 +764,19 @@ 1000 + + + system + query_log_metric
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ + + system + query_log_metric
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ system asynchronous_metric_log
diff --git a/tests/integration/test_config_xml_main/test.py b/tests/integration/test_config_xml_main/test.py index 4d74edfa01e..85234315469 100644 --- a/tests/integration/test_config_xml_main/test.py +++ b/tests/integration/test_config_xml_main/test.py @@ -18,6 +18,7 @@ def test_xml_main_conf(): "configs/config.d/more_clusters.yaml", "configs/config.d/part_log.yaml", "configs/config.d/path.yaml", + "configs/config.d/query_metric_log.yaml", "configs/config.d/query_masking_rules.yaml", "configs/config.d/tcp_with_proxy.yaml", "configs/config.d/test_cluster_with_incorrect_pw.yaml", diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.xml index 13e51581ba4..3b5009e7e49 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.xml @@ -26,7 +26,7 @@ false - + /etc/clickhouse-server/server.crt /etc/clickhouse-server/server.key /etc/clickhouse-server/dhparam.pem @@ -101,9 +101,9 @@ system query_log
- + toYYYYMM(event_date) - + 7500
@@ -136,6 +136,19 @@ 1000 + + + system + query_log_metric
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ system asynchronous_metric_log
diff --git a/tests/integration/test_config_yaml_full/configs/config.yaml b/tests/integration/test_config_yaml_full/configs/config.yaml index 3bc8ccdf601..42f58ab84b7 100644 --- a/tests/integration/test_config_yaml_full/configs/config.yaml +++ b/tests/integration/test_config_yaml_full/configs/config.yaml @@ -95,6 +95,11 @@ error_log: table: error_log flush_interval_milliseconds: 7500 collect_interval_milliseconds: 1000 +query_log_metric: + database: system + table: query_log_metric + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 asynchronous_metric_log: database: system table: asynchronous_metric_log diff --git a/tests/integration/test_config_yaml_full/test.py b/tests/integration/test_config_yaml_full/test.py index 52acdc0cb29..1233e964401 100644 --- a/tests/integration/test_config_yaml_full/test.py +++ b/tests/integration/test_config_yaml_full/test.py @@ -19,7 +19,7 @@ def test_yaml_full_conf(): "configs/config.d/more_clusters.yaml", "configs/config.d/part_log.yaml", "configs/config.d/path.yaml", - "configs/config.d/query_log_metric.xml", + "configs/config.d/query_log_metric.yaml", "configs/config.d/query_masking_rules.yaml", "configs/config.d/tcp_with_proxy.yaml", "configs/config.d/test_cluster_with_incorrect_pw.yaml", diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml index 4dceb11e2cd..82f3bdd90ce 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/configs/conf.xml @@ -4,6 +4,7 @@ + diff --git a/tests/integration/test_system_flush_logs/test.py b/tests/integration/test_system_flush_logs/test.py index 4418eb0fe67..4e3ab293985 100644 --- a/tests/integration/test_system_flush_logs/test.py +++ b/tests/integration/test_system_flush_logs/test.py @@ -17,12 +17,12 @@ system_logs = [ ("system.text_log", 0), # enabled by default ("system.query_log", 1), + ("system.query_log_metric", 1), ("system.query_thread_log", 1), ("system.part_log", 1), ("system.trace_log", 1), ("system.metric_log", 1), ("system.error_log", 1), - ("system.query_log_metric", 1), ] diff --git a/tests/integration/test_system_logs_recreate/test.py b/tests/integration/test_system_logs_recreate/test.py index add9a81b228..711e866244b 100644 --- a/tests/integration/test_system_logs_recreate/test.py +++ b/tests/integration/test_system_logs_recreate/test.py @@ -26,12 +26,12 @@ def test_system_logs_recreate(): system_logs = [ # enabled by default "query_log", + "query_log_metric", "query_thread_log", "part_log", "trace_log", "metric_log", "error_log", - "query_log_metric", ] node.query("SYSTEM FLUSH LOGS") From a61a50946011a12dd7ddf8bd711e17f2684b97e1 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Jul 2024 13:24:37 +0000 Subject: [PATCH 0036/1218] Remove redundant metric_log, error_log and query_log_metric conf They're already included as separate files. There's no need to have them in the main config. --- .../test_config_xml_full/configs/config.xml | 29 ------------------- .../test_config_xml_main/configs/config.xml | 27 ----------------- .../configs/config.xml | 27 ----------------- .../test_config_yaml_full/configs/config.yaml | 15 ---------- .../test_config_yaml_main/configs/config.yaml | 15 ---------- 5 files changed, 113 deletions(-) diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index f34250c1f86..80b6a702032 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -748,35 +748,6 @@ --> - - - system - metric_log
- 7500 - 1000 -
- - - - system - error_log
- 7500 - 1000 -
- - - - system - query_log_metric
- 7500 - 1048576 - 8192 - 524288 - 1000 - false -
- - - system - query_log_metric
- 7500 - 1048576 - 8192 - 524288 - 1000 - false -
- system asynchronous_metric_log
diff --git a/tests/integration/test_config_xml_yaml_mix/configs/config.xml b/tests/integration/test_config_xml_yaml_mix/configs/config.xml index 3b5009e7e49..f1e5137ac00 100644 --- a/tests/integration/test_config_xml_yaml_mix/configs/config.xml +++ b/tests/integration/test_config_xml_yaml_mix/configs/config.xml @@ -122,33 +122,6 @@ 7500
- - system - metric_log
- 7500 - 1000 -
- - - system - error_log
- 7500 - 1000 -
- - - - system - query_log_metric
- 7500 - 1048576 - 8192 - 524288 - 1000 - false -
- system asynchronous_metric_log
diff --git a/tests/integration/test_config_yaml_full/configs/config.yaml b/tests/integration/test_config_yaml_full/configs/config.yaml index 42f58ab84b7..894f1a37467 100644 --- a/tests/integration/test_config_yaml_full/configs/config.yaml +++ b/tests/integration/test_config_yaml_full/configs/config.yaml @@ -85,21 +85,6 @@ query_thread_log: table: query_thread_log partition_by: toYYYYMM(event_date) flush_interval_milliseconds: 7500 -metric_log: - database: system - table: metric_log - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 -error_log: - database: system - table: error_log - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 -query_log_metric: - database: system - table: query_log_metric - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 asynchronous_metric_log: database: system table: asynchronous_metric_log diff --git a/tests/integration/test_config_yaml_main/configs/config.yaml b/tests/integration/test_config_yaml_main/configs/config.yaml index d7afaa005cf..e8483f95bb0 100644 --- a/tests/integration/test_config_yaml_main/configs/config.yaml +++ b/tests/integration/test_config_yaml_main/configs/config.yaml @@ -85,21 +85,6 @@ query_thread_log: table: query_thread_log partition_by: toYYYYMM(event_date) flush_interval_milliseconds: 7500 -metric_log: - database: system - table: metric_log - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 -error_log: - database: system - table: error_log - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 -query_log_metric: - database: system - table: query_log_metric - flush_interval_milliseconds: 7500 - collect_interval_milliseconds: 1000 asynchronous_metric_log: database: system table: asynchronous_metric_log From b157d40c922af8ce34dd75937f27da874de7b0c1 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Jul 2024 13:33:40 +0000 Subject: [PATCH 0037/1218] Clarify comment --- src/QueryPipeline/QueryPipeline.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 6a1321bd64e..cb14a20b225 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -549,7 +549,7 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback) // needs to be collected. However, since the progress callback is called not very // frequently, we'd rather update them as needed. Using the // updatePerformanceCountersIfNeeded instead of just updatePerformanceCounters we make sure - // that we update them with a sufficiently frequent interval. + // that we don't update them too frequently. auto context = CurrentThread::getQueryContext(); if (context) { From d735019ccddc5f3a77da5ee634bd186f3d64ef9c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Jul 2024 13:38:19 +0000 Subject: [PATCH 0038/1218] Add missing query_log_metric config --- .../test_config_xml_main/configs/config.d/error_log.yaml | 2 +- .../integration/test_memory_limit/configs/async_metrics_no.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_config_xml_main/configs/config.d/error_log.yaml b/tests/integration/test_config_xml_main/configs/config.d/error_log.yaml index f115989d203..0ba80776dda 100644 --- a/tests/integration/test_config_xml_main/configs/config.d/error_log.yaml +++ b/tests/integration/test_config_xml_main/configs/config.d/error_log.yaml @@ -1,6 +1,6 @@ error_log: database: system - table: error_log + table: error_log flush_interval_milliseconds: 7500 collect_interval_milliseconds: 1000 diff --git a/tests/integration/test_memory_limit/configs/async_metrics_no.xml b/tests/integration/test_memory_limit/configs/async_metrics_no.xml index 96cae3bf387..30837ceb049 100644 --- a/tests/integration/test_memory_limit/configs/async_metrics_no.xml +++ b/tests/integration/test_memory_limit/configs/async_metrics_no.xml @@ -5,6 +5,7 @@ + From a561899da005fa60b06d62301894063b93398e82 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 19 Jul 2024 13:58:50 +0000 Subject: [PATCH 0039/1218] Rename system.query_log_metric to system.query_metric_log Seems I had at the very beginning a brainfart and started using query_log_metric instead of query_metric_log and it went on and on :facepalm --- .../settings.md | 14 +++++----- docs/en/operations/settings/settings.md | 6 ++--- ...uery_log_metric.md => query_metric_log.md} | 12 ++++----- programs/server/config.xml | 8 +++--- programs/server/config.yaml.example | 6 ++--- src/Common/SystemLogBase.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/Context.cpp | 4 +-- src/Interpreters/Context.h | 4 +-- src/Interpreters/PeriodicLog.cpp | 2 +- src/Interpreters/PeriodicLog.h | 2 +- ...{QueryLogMetric.cpp => QueryMetricLog.cpp} | 26 +++++++++---------- .../{QueryLogMetric.h => QueryMetricLog.h} | 22 ++++++++-------- src/Interpreters/SystemLog.cpp | 12 ++++----- src/Interpreters/SystemLog.h | 4 +-- src/Interpreters/executeQuery.cpp | 22 ++++++++-------- src/QueryPipeline/QueryPipeline.cpp | 4 +-- .../configs/no_system_log.xml | 2 +- .../test_backup_restore_new/test.py | 2 +- .../configs/config.d/query_metric_log.xml} | 6 ++--- .../integration/test_config_xml_full/test.py | 2 +- ..._log_metric.yaml => query_metric_log.yaml} | 4 +-- .../integration/test_config_xml_main/test.py | 2 +- .../configs/config.d/query_metric_log.xml} | 6 ++--- .../test_config_xml_yaml_mix/test.py | 2 +- ..._log_metric.yaml => query_metric_log.yaml} | 4 +-- .../integration/test_config_yaml_full/test.py | 2 +- .../configs/config.d/query_metric_log.xml} | 6 ++--- .../integration/test_config_yaml_main/test.py | 2 +- .../configs/conf.xml | 2 +- .../configs/async_metrics_no.xml | 2 +- .../test_system_flush_logs/test.py | 2 +- .../test_system_logs_recreate/test.py | 2 +- ...> 03203_system_query_metric_log.reference} | 0 ...ic.sh => 03203_system_query_metric_log.sh} | 8 +++--- 36 files changed, 105 insertions(+), 105 deletions(-) rename docs/en/operations/system-tables/{query_log_metric.md => query_metric_log.md} (83%) rename src/Interpreters/{QueryLogMetric.cpp => QueryMetricLog.cpp} (90%) rename src/Interpreters/{QueryLogMetric.h => QueryMetricLog.h} (72%) rename tests/integration/{test_config_xml_yaml_mix/configs/config.d/query_log_metric.xml => test_config_xml_full/configs/config.d/query_metric_log.xml} (70%) rename tests/integration/test_config_xml_main/configs/config.d/{query_log_metric.yaml => query_metric_log.yaml} (68%) rename tests/integration/{test_config_yaml_main/configs/config.d/query_log_metric.xml => test_config_xml_yaml_mix/configs/config.d/query_metric_log.xml} (70%) rename tests/integration/test_config_yaml_full/configs/config.d/{query_log_metric.yaml => query_metric_log.yaml} (68%) rename tests/integration/{test_config_xml_full/configs/config.d/query_log_metric.xml => test_config_yaml_main/configs/config.d/query_metric_log.xml} (70%) rename tests/queries/0_stateless/{03203_system_query_log_metric.reference => 03203_system_query_metric_log.reference} (100%) rename tests/queries/0_stateless/{03203_system_query_log_metric.sh => 03203_system_query_metric_log.sh} (85%) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ddd3d3afa0d..7eca519d907 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2198,36 +2198,36 @@ If the table does not exist, ClickHouse will create it. If the structure of the ``` -# query_log_metric {#query_log_metric} +# query_metric_log {#query_metric_log} It is disabled by default. **Enabling** -To manually turn on metrics history collection [`system.query_log_metric`](../../operations/system-tables/query_log_metric.md), create `/etc/clickhouse-server/config.d/query_log_metric.xml` with the following content: +To manually turn on metrics history collection [`system.query_metric_log`](../../operations/system-tables/query_metric_log.md), create `/etc/clickhouse-server/config.d/query_metric_log.xml` with the following content: ``` xml - + system - query_log_metric
+ query_metric_log
7500 1000 1048576 8192 524288 false -
+
``` **Disabling** -To disable `query_log_metric` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_log_metric.xml` with the following content: +To disable `query_metric_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_metric_log.xml` with the following content: ``` xml - + ``` diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b18ecf57f1..cceb7e528d0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1820,10 +1820,10 @@ Possible values: Default value: 0 (no restriction). -## query_log_metric_interval (#query_log_metric_interval) +## query_metric_log_interval (#query_metric_log_interval) -The interval in milliseconds at which the [query_log_metric](../../operations/system-tables/query_log_metric.md) for individual queries is collected. -If set to 0, it will take the `collect_interval_milliseconds` from the [query_log_metric setting](../../operations/server-configuration-parameters/settings.md#query_log_metric). +The interval in milliseconds at which the [query_metric_log](../../operations/system-tables/query_metric_log.md) for individual queries is collected. +If set to 0, it will take the `collect_interval_milliseconds` from the [query_metric_log setting](../../operations/server-configuration-parameters/settings.md#query_metric_log). Default value: 0 diff --git a/docs/en/operations/system-tables/query_log_metric.md b/docs/en/operations/system-tables/query_metric_log.md similarity index 83% rename from docs/en/operations/system-tables/query_log_metric.md rename to docs/en/operations/system-tables/query_metric_log.md index 167e50a7780..01f063e597f 100644 --- a/docs/en/operations/system-tables/query_log_metric.md +++ b/docs/en/operations/system-tables/query_metric_log.md @@ -1,11 +1,11 @@ --- -slug: /en/operations/system-tables/query_log_metric +slug: /en/operations/system-tables/query_metric_log --- -# query_log_metric +# query_metric_log Contains history of memory and metric values from table `system.events` for individual queries, periodically flushed to disk. -Once a query starts, data is collected at periodic intervals of `query_log_metric_interval` milliseconds (which is set to 1000 +Once a query starts, data is collected at periodic intervals of `query_metric_log_interval` milliseconds (which is set to 1000 by default) and when the query finishes. Columns: @@ -18,7 +18,7 @@ Columns: **Example** ``` sql -SELECT * FROM system.query_log_metric LIMIT 1 FORMAT Vertical; +SELECT * FROM system.query_metric_log LIMIT 1 FORMAT Vertical; ``` ``` text @@ -40,8 +40,8 @@ ProfileEvent_FailedSelectQuery: 0 **See also** -- [query_log_metric setting](../../operations/server-configuration-parameters/settings.md#query_log_metric) — Enabling and disabling the setting. -- [query_log_metric_interval](../../operations/settings/settings.md#query_log_metric_interval) +- [query_metric_log setting](../../operations/server-configuration-parameters/settings.md#query_metric_log) — Enabling and disabling the setting. +- [query_metric_log_interval](../../operations/settings/settings.md#query_metric_log_interval) - [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md) — Contains periodically calculated metrics. - [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. - [system.metrics](../../operations/system-tables/metrics.md) — Contains instantly calculated metrics. diff --git a/programs/server/config.xml b/programs/server/config.xml index 0f3b97488cd..bf53f09c274 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1182,18 +1182,18 @@ false - - + system - query_log_metric
+ query_metric_log
7500 1048576 8192 524288 1000 false -
+ + + + 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 0081/1218] 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 0082/1218] 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 0083/1218] 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 0084/1218] 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 0085/1218] 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 0086/1218] 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 0087/1218] 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 922ea9f5a2d67715596369b44766ba4f42f920c7 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 2 Sep 2024 08:57:25 +0000 Subject: [PATCH 0088/1218] Revert "Defensive programming to avoid leaking resources" This reverts commit 181bc7f34c1c84c37b06967f7406fc366319a355. --- src/Interpreters/QueryMetricLog.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 25e7d06a425..072907b0bed 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -116,12 +116,7 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t /// The query info should always be found because whenever a query ends, finishQuery is /// called and the query is removed from the list if (!query_info) - { - std::lock_guard lock(queries_mutex); - /// Removing the query info from the list automatically deactivates the task - queries.erase(query_id); throw Exception(ErrorCodes::LOGICAL_ERROR, "Query info not found: {}", query_id); - } auto elem = createLogMetricElement(query_id, *query_info, current_time); add(std::move(elem)); From 09f47936d1e8b5c4a2a57ebce22165d7011a65ff Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 2 Sep 2024 10:55:14 +0000 Subject: [PATCH 0089/1218] Move BackgroundSchedulePool::TaskHolder's ownership to QueryStatus This way we ensure that the background task is encapsulated within QueryStatus, ensuring we will never ever leak resources once the query finishes. Code-wise the previous implementation was ok, but this is making it more bullet-proof for future changes. --- src/Interpreters/ProcessList.cpp | 37 +++++++++++++++++++++++++---- src/Interpreters/ProcessList.h | 8 ++++++- src/Interpreters/QueryMetricLog.cpp | 16 ++----------- src/Interpreters/QueryMetricLog.h | 3 --- 4 files changed, 42 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 4f8b886d37f..28a5b20f9b6 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -685,9 +686,9 @@ ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_ev return per_query_infos; } -QueryStatusInfoPtr ProcessList::getQueryInfo(const String & query_id, bool get_thread_list, bool get_profile_events, bool get_settings) const +QueryStatusPtr ProcessList::getProcessListElement(const String & query_id) const { - std::optional process_found; + QueryStatusPtr process_found; { auto lock = safeLock(); for (const auto & process : processes) @@ -700,12 +701,40 @@ QueryStatusInfoPtr ProcessList::getQueryInfo(const String & query_id, bool get_t } } - if (process_found) - return std::make_shared(process_found.value()->getInfo(get_thread_list, get_profile_events, get_settings)); + return process_found; +} + +QueryStatusInfoPtr ProcessList::getQueryInfo(const String & query_id, bool get_thread_list, bool get_profile_events, bool get_settings) const +{ + auto process = getProcessListElement(query_id); + if (process) + return std::make_shared(process->getInfo(get_thread_list, get_profile_events, get_settings)); return nullptr; } +void ProcessList::createQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds, const BackgroundSchedulePool::TaskFunc & function) const +{ + auto process = getProcessListElement(query_id); + if (!process) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} not found in process list", query_id); + + process->query_metric_log_task = std::make_unique(process->getContext()->getSchedulePool().createTask("QueryMetricLog", function)); + (*process->query_metric_log_task)->scheduleAfter(interval_milliseconds); +} + +void ProcessList::scheduleQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds) const +{ + auto process = getProcessListElement(query_id); + if (!process) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} not found in process list", query_id); + + if (!process->query_metric_log_task) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} doesn't have any query metric log task", query_id); + + (*process->query_metric_log_task)->scheduleAfter(interval_milliseconds); +} + ProcessListForUser::ProcessListForUser(ProcessList * global_process_list) : ProcessListForUser(nullptr, global_process_list) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 7f96b37a157..429fa2591a3 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -22,7 +23,6 @@ #include #include -#include #include #include #include @@ -167,6 +167,7 @@ protected: /// increments/decrements metric in constructor/destructor. CurrentMetrics::Increment num_queries_increment; + std::unique_ptr query_metric_log_task; public: QueryStatus( ContextPtr context_, @@ -399,6 +400,8 @@ protected: /// Call under lock. Finds process with specified current_user and current_query_id. QueryStatusPtr tryGetProcessListElement(const String & current_query_id, const String & current_user); + QueryStatusPtr getProcessListElement(const String & query_id) const; + /// limit for insert. 0 means no limit. Otherwise, when limit exceeded, an exception is thrown. size_t max_insert_queries_amount = 0; @@ -501,6 +504,9 @@ public: CancellationCode sendCancelToQuery(QueryStatusPtr elem, bool kill = false); void killAllQueries(); + + void createQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds, const BackgroundSchedulePool::TaskFunc & function) const; + void scheduleQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds) const; }; } diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 072907b0bed..e630f147777 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -86,17 +86,9 @@ void QueryMetricLogElement::appendToBlock(MutableColumns & columns) const void QueryMetricLog::shutdown() { - stopCollect(); Base::shutdown(); } -void QueryMetricLog::stopCollect() -{ - std::lock_guard lock(queries_mutex); - for (auto & [query_id, status] : queries) - status.task->deactivate(); -} - void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds) { QueryMetricLogStatus status; @@ -109,7 +101,7 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t auto context = getContext(); const auto & process_list = context->getProcessList(); - status.task = context->getSchedulePool().createTask("QueryMetricLog", [this, &process_list, query_id] { + process_list.createQueryMetricLogTask(query_id, interval_milliseconds, [this, &process_list, query_id] { auto current_time = std::chrono::system_clock::now(); const auto query_info = process_list.getQueryInfo(query_id, false, true, false); @@ -122,8 +114,6 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t add(std::move(elem)); }); - status.task->scheduleAfter(interval_milliseconds); - std::lock_guard lock(queries_mutex); queries.emplace(query_id, std::move(status)); } @@ -138,8 +128,6 @@ void QueryMetricLog::finishQuery(const String & query_id, QueryStatusInfoPtr que if (it == queries.end()) return; - it->second.task->deactivate(); - if (query_info) { auto elem = createLogMetricElement(query_id, *query_info, std::chrono::system_clock::now()); @@ -174,7 +162,7 @@ QueryMetricLogElement QueryMetricLog::createLogMetricElement(const String & quer query_status.next_collect_time += std::chrono::milliseconds(query_status.interval_milliseconds); const auto wait_time = std::chrono::duration_cast(query_status.next_collect_time - std::chrono::system_clock::now()).count(); - query_status.task->scheduleAfter(wait_time); + getContext()->getProcessList().scheduleQueryMetricLogTask(query_id, wait_time); return elem; } diff --git a/src/Interpreters/QueryMetricLog.h b/src/Interpreters/QueryMetricLog.h index 3a8cb9c1513..3ccb55a53e0 100644 --- a/src/Interpreters/QueryMetricLog.h +++ b/src/Interpreters/QueryMetricLog.h @@ -39,7 +39,6 @@ struct QueryMetricLogStatus UInt64 interval_milliseconds; std::chrono::system_clock::time_point next_collect_time; std::vector last_profile_events = std::vector(ProfileEvents::end()); - BackgroundSchedulePool::TaskHolder task; }; class QueryMetricLog : public SystemLog @@ -51,8 +50,6 @@ class QueryMetricLog : public SystemLog public: void shutdown() final; - void stopCollect(); - // Both startQuery and finishQuery are called from the thread that executes the query void startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds); void finishQuery(const String & query_id, QueryStatusInfoPtr query_info = nullptr); From 0cc2a0847cb221cdbc3b316758fa98189b5b9bbf Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 2 Sep 2024 14:07:18 +0000 Subject: [PATCH 0090/1218] Relax the logical errors because they could happen on very fast queries --- src/Interpreters/ProcessList.cpp | 11 +++++------ src/Interpreters/QueryMetricLog.cpp | 5 +++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 28a5b20f9b6..9c940771f23 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -716,8 +716,10 @@ QueryStatusInfoPtr ProcessList::getQueryInfo(const String & query_id, bool get_t void ProcessList::createQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds, const BackgroundSchedulePool::TaskFunc & function) const { auto process = getProcessListElement(query_id); + /// Some extra quick queries might have already finished + /// e.g. SHOW PROCESSLIST FORMAT Null if (!process) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} not found in process list", query_id); + return; process->query_metric_log_task = std::make_unique(process->getContext()->getSchedulePool().createTask("QueryMetricLog", function)); (*process->query_metric_log_task)->scheduleAfter(interval_milliseconds); @@ -726,11 +728,8 @@ void ProcessList::createQueryMetricLogTask(const String & query_id, UInt64 inter void ProcessList::scheduleQueryMetricLogTask(const String & query_id, UInt64 interval_milliseconds) const { auto process = getProcessListElement(query_id); - if (!process) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} not found in process list", query_id); - - if (!process->query_metric_log_task) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query {} doesn't have any query metric log task", query_id); + if (!process || !process->query_metric_log_task) + return; (*process->query_metric_log_task)->scheduleAfter(interval_milliseconds); } diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index e630f147777..0a28171139b 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -105,8 +105,9 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t auto current_time = std::chrono::system_clock::now(); const auto query_info = process_list.getQueryInfo(query_id, false, true, false); - /// The query info should always be found because whenever a query ends, finishQuery is - /// called and the query is removed from the list + /// The query info should always be found because this task is owned by the QueryStatus, + /// so whenever a query actually finishes the task is destroyed, deactivated and thus this + /// lambda should never run anymore. if (!query_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query info not found: {}", query_id); From d932d0ae4f459a3a7ed3d1942034aa85b72f7ee5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 2 Sep 2024 14:16:00 +0000 Subject: [PATCH 0091/1218] fix performance of parsing row formats --- src/Columns/ColumnArray.cpp | 6 ++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnDynamic.h | 5 +++ src/Columns/ColumnMap.cpp | 5 +++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 6 ++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnObject.cpp | 34 +++++++++++++++---- src/Columns/ColumnObject.h | 1 + src/Columns/ColumnSparse.cpp | 6 ++++ src/Columns/ColumnSparse.h | 1 + src/Columns/ColumnTuple.cpp | 10 ++++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 10 ++++++ src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 7 ++-- .../Executors/StreamingFormatExecutor.cpp | 13 +++---- .../Executors/StreamingFormatExecutor.h | 2 -- src/Processors/Formats/IRowInputFormat.cpp | 5 ++- 19 files changed, 97 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 4f3d0f0ec4b..0c6d7c4e5c6 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -374,6 +374,12 @@ ColumnCheckpointPtr ColumnArray::getCheckpoint() const return std::make_shared(size(), getData().getCheckpoint()); } +void ColumnArray::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + getData().updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnArray::rollback(const ColumnCheckpoint & checkpoint) { getOffsets().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index fd42d502c16..ec14b096055 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -162,6 +162,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 1445aee4176..6f8335044a7 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -309,6 +309,11 @@ public: return variant_column_ptr->getCheckpoint(); } + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override + { + variant_column_ptr->updateCheckpoint(checkpoint); + } + void rollback(const ColumnCheckpoint & checkpoint) override { variant_column_ptr->rollback(checkpoint); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 0561e8f398f..7ebbed930d8 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -317,6 +317,11 @@ ColumnCheckpointPtr ColumnMap::getCheckpoint() const return nested->getCheckpoint(); } +void ColumnMap::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + nested->updateCheckpoint(checkpoint); +} + void ColumnMap::rollback(const ColumnCheckpoint & checkpoint) { nested->rollback(checkpoint); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index d534cfa7295..575114f8d3a 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -103,6 +103,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 560d37721ad..61feca60e42 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -310,6 +310,12 @@ ColumnCheckpointPtr ColumnNullable::getCheckpoint() const return std::make_shared(size(), nested_column->getCheckpoint()); } +void ColumnNullable::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + nested_column->updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnNullable::rollback(const ColumnCheckpoint & checkpoint) { getNullMapData().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 39b326a1c44..32ce66c5965 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -144,6 +144,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 6be6e9d833e..4c33f042954 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -686,22 +686,44 @@ ColumnCheckpointPtr ColumnObject::getCheckpoint() const return std::make_shared(size(), get_checkpoints(typed_paths), get_checkpoints(dynamic_paths_ptrs), shared_data->getCheckpoint()); } +void ColumnObject::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & object_checkpoint = assert_cast(checkpoint); + + auto update_checkpoints = [&](const auto & columns_map, auto & checkpoints_map) + { + for (const auto & [name, column] : columns_map) + { + auto & nested = checkpoints_map[name]; + if (!nested) + nested = column->getCheckpoint(); + else + column->updateCheckpoint(*nested); + } + }; + + checkpoint.size = size(); + update_checkpoints(typed_paths, object_checkpoint.typed_paths); + update_checkpoints(dynamic_paths, object_checkpoint.dynamic_paths); + shared_data->updateCheckpoint(*object_checkpoint.shared_data); +} + void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) { const auto & object_checkpoint = assert_cast(checkpoint); for (auto & [name, column] : typed_paths) { - const auto & nested_checkpoint = object_checkpoint.typed_paths.at(name); - chassert(nested_checkpoint); - column->rollback(*nested_checkpoint); + const auto & nested = object_checkpoint.typed_paths.at(name); + chassert(nested); + column->rollback(*nested); } for (auto & [name, column] : dynamic_paths_ptrs) { - const auto & nested_checkpoint = object_checkpoint.dynamic_paths.at(name); - chassert(nested_checkpoint); - column->rollback(*nested_checkpoint); + const auto & nested = object_checkpoint.dynamic_paths.at(name); + chassert(nested); + column->rollback(*nested); } shared_data->rollback(*object_checkpoint.shared_data); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 84fe2dcafad..e444db099b0 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -160,6 +160,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0c91174007c..a0e47e65fc6 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -313,6 +313,12 @@ ColumnCheckpointPtr ColumnSparse::getCheckpoint() const return std::make_shared(size(), values->getCheckpoint()); } +void ColumnSparse::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + values->updateCheckpoint(*assert_cast(checkpoint).nested); +} + void ColumnSparse::rollback(const ColumnCheckpoint & checkpoint) { _size = checkpoint.size; diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index dabf38a252d..619dce63c1e 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -150,6 +150,7 @@ public: ColumnPtr compress() const override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 65f3285bcfc..51617359318 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -265,6 +265,16 @@ ColumnCheckpointPtr ColumnTuple::getCheckpoint() const return std::make_shared(size(), std::move(checkpoints)); } +void ColumnTuple::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & checkpoints = assert_cast(checkpoint).nested; + chassert(checkpoints.size() == columns.size()); + + checkpoint.size = size(); + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->updateCheckpoint(*checkpoints[i]); +} + void ColumnTuple::rollback(const ColumnCheckpoint & checkpoint) { column_length = checkpoint.size; diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 74c4dd1ffd3..c73f90f13d9 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -119,6 +119,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index a8cb779ad16..b03313fd6d0 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -750,6 +750,16 @@ ColumnCheckpointPtr ColumnVariant::getCheckpoint() const return std::make_shared(size(), std::move(checkpoints)); } +void ColumnVariant::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & checkpoints = assert_cast(checkpoint).nested; + chassert(checkpoints.size() == variants.size()); + + checkpoint.size = size(); + for (size_t i = 0; i < variants.size(); ++i) + variants[i]->updateCheckpoint(*checkpoints[i]); +} + void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) { getOffsets().resize_assume_reserved(checkpoint.size); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index c7182467611..332c36d1153 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -249,6 +249,7 @@ public: size_t allocatedBytes() const override; void protect() override; ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override; void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 53f31d2b96d..95becba3fdb 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -60,7 +60,7 @@ struct ColumnCheckpoint virtual ~ColumnCheckpoint() = default; }; -using ColumnCheckpointPtr = std::shared_ptr; +using ColumnCheckpointPtr = std::shared_ptr; using ColumnCheckpoints = std::vector; struct ColumnCheckpointWithNested : public ColumnCheckpoint @@ -546,9 +546,12 @@ public: /// Returns checkpoint of current state of column. virtual ColumnCheckpointPtr getCheckpoint() const { return std::make_shared(size()); } + /// Updates the checkpoint with current state. It is used to avoid extra allocations in 'getCheckpoint'. + virtual void updateCheckpoint(ColumnCheckpoint & checkpoint) const { checkpoint.size = size(); } + /// Rollbacks column to the checkpoint. /// Unlike 'popBack' this method should work correctly even if column has invalid state. - /// Sizes of columns in checkpoint must be less or equal than current. + /// Sizes of columns in checkpoint must be less or equal than current size. virtual void rollback(const ColumnCheckpoint & checkpoint) { popBack(size() - checkpoint.size); } /// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them. diff --git a/src/Processors/Executors/StreamingFormatExecutor.cpp b/src/Processors/Executors/StreamingFormatExecutor.cpp index 1491198f0cb..2d4b87e9f4d 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.cpp +++ b/src/Processors/Executors/StreamingFormatExecutor.cpp @@ -25,6 +25,9 @@ StreamingFormatExecutor::StreamingFormatExecutor( , checkpoints(result_columns.size()) { connect(format->getPort(), port); + + for (size_t i = 0; i < result_columns.size(); ++i) + checkpoints[i] = result_columns[i]->getCheckpoint(); } MutableColumns StreamingFormatExecutor::getResultColumns() @@ -54,7 +57,8 @@ size_t StreamingFormatExecutor::execute(ReadBuffer & buffer) size_t StreamingFormatExecutor::execute() { - setCheckpoints(); + for (size_t i = 0; i < result_columns.size(); ++i) + result_columns[i]->updateCheckpoint(*checkpoints[i]); try { @@ -117,11 +121,4 @@ size_t StreamingFormatExecutor::insertChunk(Chunk chunk) return chunk_rows; } -void StreamingFormatExecutor::setCheckpoints() -{ - for (size_t i = 0; i < result_columns.size(); ++i) - checkpoints[i] = result_columns[i]->getCheckpoint(); -} - - } diff --git a/src/Processors/Executors/StreamingFormatExecutor.h b/src/Processors/Executors/StreamingFormatExecutor.h index f0c443edd13..3db5a92ae98 100644 --- a/src/Processors/Executors/StreamingFormatExecutor.h +++ b/src/Processors/Executors/StreamingFormatExecutor.h @@ -43,8 +43,6 @@ public: void setQueryParameters(const NameToNameMap & parameters); private: - void setCheckpoints(); - const Block header; const InputFormatPtr format; const ErrorCallback on_error; diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 2a0695764b2..0d65fc3b5fa 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -104,7 +104,10 @@ Chunk IRowInputFormat::read() size_t num_columns = header.columns(); MutableColumns columns = header.cloneEmptyColumns(); + ColumnCheckpoints checkpoints(columns.size()); + for (size_t column_idx = 0; column_idx < columns.size(); ++column_idx) + checkpoints[column_idx] = columns[column_idx]->getCheckpoint(); block_missing_values.clear(); @@ -132,7 +135,7 @@ Chunk IRowInputFormat::read() try { for (size_t column_idx = 0; column_idx < columns.size(); ++column_idx) - checkpoints[column_idx] = columns[column_idx]->getCheckpoint(); + columns[column_idx]->updateCheckpoint(*checkpoints[column_idx]); info.read_columns.clear(); continue_reading = readRow(columns, info); From ebf916175e79e36f860f69257f6849426520649b Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 2 Sep 2024 14:35:10 +0000 Subject: [PATCH 0092/1218] Remove unnecessary update of performance counters They're already updated enough in the ReadProgressCallback --- src/QueryPipeline/QueryPipeline.cpp | 20 +------------------- 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 600d8d15d72..ba41ecb3192 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -571,25 +571,7 @@ Block QueryPipeline::getHeader() const void QueryPipeline::setProgressCallback(const ProgressCallback & callback) { - progress_callback = [callback](const Progress & progress) - { - // Performance counters need to be updated from the same thread the query is being executed - // on because most info is taken using getrusage with RUSAGE_THREAD. Ideally, we would only - // update the counters once we're close to the interval at which the query metric log data - // needs to be collected. However, since the progress callback is called not very - // frequently, we'd rather update them as needed. Using the - // updatePerformanceCountersIfNeeded instead of just updatePerformanceCounters we make sure - // that we don't update them too frequently. - auto context = CurrentThread::getQueryContext(); - if (context) - { - if (auto query_metric_log = context->getQueryMetricLog()) - CurrentThread::updatePerformanceCountersIfNeeded(); - } - - if (callback) - callback(progress); - }; + progress_callback = callback; } void QueryPipeline::setProcessListElement(QueryStatusPtr elem) From a909352a3f701191adc5b625899d6a1eaf176dad Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 3 Sep 2024 08:53:00 +0000 Subject: [PATCH 0093/1218] Relax another logical error that could happen --- src/Interpreters/QueryMetricLog.cpp | 11 +++++++---- src/Interpreters/QueryMetricLog.h | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index 0a28171139b..f7c65af8808 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -112,7 +112,8 @@ void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_t throw Exception(ErrorCodes::LOGICAL_ERROR, "Query info not found: {}", query_id); auto elem = createLogMetricElement(query_id, *query_info, current_time); - add(std::move(elem)); + if (elem) + add(std::move(elem.value())); }); std::lock_guard lock(queries_mutex); @@ -132,19 +133,21 @@ void QueryMetricLog::finishQuery(const String & query_id, QueryStatusInfoPtr que if (query_info) { auto elem = createLogMetricElement(query_id, *query_info, std::chrono::system_clock::now()); - add(std::move(elem)); + if (elem) + add(std::move(elem.value())); } queries.erase(it); } -QueryMetricLogElement QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time) +std::optional QueryMetricLog::createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time) { std::lock_guard lock(queries_mutex); auto query_status_it = queries.find(query_id); + /// The query might have finished while the scheduled task is running. if (query_status_it == queries.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query not found: {}", query_id); + return {}; QueryMetricLogElement elem; elem.event_time = timeInSeconds(current_time); diff --git a/src/Interpreters/QueryMetricLog.h b/src/Interpreters/QueryMetricLog.h index 3ccb55a53e0..95f91069a75 100644 --- a/src/Interpreters/QueryMetricLog.h +++ b/src/Interpreters/QueryMetricLog.h @@ -55,7 +55,7 @@ public: void finishQuery(const String & query_id, QueryStatusInfoPtr query_info = nullptr); private: - QueryMetricLogElement createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time); + std::optional createLogMetricElement(const String & query_id, const QueryStatusInfo & query_info, TimePoint current_time); std::recursive_mutex queries_mutex; std::unordered_map queries; From 2ef36b36acb1926b70b1d4b64c7d3d83783e483c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:06:44 +0000 Subject: [PATCH 0094/1218] 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 0095/1218] 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 0096/1218] 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 0097/1218] add system.workloads table with test --- .../Workload/IWorkloadEntityStorage.h | 3 ++ .../Workload/WorkloadEntityStorageBase.cpp | 24 ++++++++++ .../Workload/WorkloadEntityStorageBase.h | 1 + .../System/StorageSystemWorkloads.cpp | 48 +++++++++++++++++++ src/Storages/System/StorageSystemWorkloads.h | 29 +++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + .../03232_workload_create_and_drop.reference | 5 ++ .../03232_workload_create_and_drop.sql | 11 +++++ 8 files changed, 123 insertions(+) create mode 100644 src/Storages/System/StorageSystemWorkloads.cpp create mode 100644 src/Storages/System/StorageSystemWorkloads.h create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.sql diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index 113cefe3f46..cff09a2259d 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -47,6 +47,9 @@ public: /// Get all entity names. virtual std::vector getAllEntityNames() const = 0; + /// Get all entity names of specified type. + virtual std::vector getAllEntityNames(WorkloadEntityType entity_type) const = 0; + /// Get all entities. virtual std::vector> getAllEntities() const = 0; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 33e6227b998..e3bf6d4af7f 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -37,6 +37,16 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const Conte return ptr; } +WorkloadEntityType getEntityType(const ASTPtr & ptr) +{ + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Workload; + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Resource; + chassert(false); + return WorkloadEntityType::MAX; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -85,6 +95,20 @@ std::vector WorkloadEntityStorageBase::getAllEntityNames() const return entity_names; } +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + for (const auto & [name, entity] : entities) + { + if (getEntityType(entity) == entity_type) + entity_names.emplace_back(name); + } + + return entity_names; +} + bool WorkloadEntityStorageBase::empty() const { std::lock_guard lock(mutex); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 9e9e8170a8e..8ec92675ddb 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -24,6 +24,7 @@ public: bool has(const String & entity_name) const override; std::vector getAllEntityNames() const override; + std::vector getAllEntityNames(WorkloadEntityType entity_type) const override; std::vector> getAllEntities() const override; diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp new file mode 100644 index 00000000000..dad2750d8c0 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemWorkloads::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the workload."}, + {"parent", std::make_shared(), "The name of the parent workload."}, + {"create_query", std::make_shared(), "CREATE query of the workload."}, + }; +} + +void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & workload_names = storage.getAllEntityNames(WorkloadEntityType::Workload); + for (const auto & workload_name : workload_names) + { + auto ast = storage.get(workload_name); + auto & workload = typeid_cast(*ast); + res_columns[0]->insert(workload_name); + res_columns[1]->insert(workload.getWorkloadParent()); + res_columns[2]->insert(queryToString(ast)); + } +} + +void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for workloads and resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for workloads and resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemWorkloads.h b/src/Storages/System/StorageSystemWorkloads.h new file mode 100644 index 00000000000..9d4770a02b8 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `workloads` system table, which allows you to get a list of all workloads +class StorageSystemWorkloads final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemWorkloads"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 816ba5095b1..728e83135a3 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -229,6 +230,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "s3queue", "Contains in-memory state of S3Queue metadata and currently processed rows per file."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); + attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); if (has_zookeeper) { diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference new file mode 100644 index 00000000000..4bac2ef71f2 --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -0,0 +1,5 @@ +all CREATE WORKLOAD `all` +development all CREATE WORKLOAD development IN `all` +production all CREATE WORKLOAD production IN `all` +all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql new file mode 100644 index 00000000000..38a7dad7cbc --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +CREATE OR REPLACE WORKLOAD all; +SELECT name, parent, create_query FROM system.workloads; +CREATE WORKLOAD IF NOT EXISTS production IN all; +CREATE WORKLOAD development IN all; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD IF EXISTS production; +DROP WORKLOAD development; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD all; From 4cd8272186416c94a371be79d6845f3d010b52b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 18:21:53 +0000 Subject: [PATCH 0098/1218] fix rollback of ColumnDynamic --- src/Columns/ColumnDynamic.cpp | 84 ++++++++++++++++++++++ src/Columns/ColumnDynamic.h | 12 ++-- src/Columns/ColumnObject.cpp | 28 +++++--- src/Columns/tests/gtest_column_dynamic.cpp | 69 ++++++++++++++++++ 4 files changed, 174 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 9b55879a4f0..8e345c1fc2f 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -979,6 +979,90 @@ ColumnPtr ColumnDynamic::compress() const }); } +void ColumnDynamic::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + auto & nested = assert_cast(checkpoint).nested; + const auto & variants = variant_column_ptr->getVariants(); + size_t old_size = nested.size(); + + for (size_t i = 0; i < old_size; ++i) + { + variants[i]->updateCheckpoint(*nested[i]); + } + + /// If column has new variants since last checkpoint create checkpoints for them. + if (old_size < variants.size()) + { + nested.resize(variants.size()); + for (size_t i = old_size; i < variants.size(); ++i) + nested[i] = variants[i]->getCheckpoint(); + } + + checkpoint.size = size(); +} + + +DataTypePtr ColumnDynamic::popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n) +{ + const auto & type_variant = assert_cast(*info.variant_type); + + std::unordered_map discriminator_to_name; + std::unordered_map name_to_data_type; + + for (const auto & [name, discriminator] : info.variant_name_to_discriminator) + discriminator_to_name.emplace(discriminator, name); + + for (const auto & type : type_variant.getVariants()) + name_to_data_type.emplace(type->getName(), type); + + /// Remove last n variants according to global discriminators. + /// This code relies on invariant that new variants are always added to the end in ColumnVariant. + for (auto it = local_to_global_discriminators.rbegin(); it < local_to_global_discriminators.rbegin() + n; ++it) + discriminator_to_name.erase(*it); + + DataTypes new_variants; + for (const auto & [d, name] : discriminator_to_name) + new_variants.push_back(name_to_data_type.at(name)); + + return std::make_shared(std::move(new_variants)); +} + +void ColumnDynamic::rollback(const ColumnCheckpoint & checkpoint) +{ + const auto & nested = assert_cast(checkpoint).nested; + chassert(nested.size() <= variant_column_ptr->getNumVariants()); + + /// The structure hasn't changed, so we can use generic rollback of Variant column + if (nested.size() == variant_column_ptr->getNumVariants()) + { + variant_column_ptr->rollback(checkpoint); + return; + } + + auto new_subcolumns = variant_column_ptr->getVariants(); + auto new_discriminators_map = variant_column_ptr->getLocalToGlobalDiscriminatorsMapping(); + auto new_discriminators_column = variant_column_ptr->getLocalDiscriminatorsPtr(); + auto new_offses_column = variant_column_ptr->getOffsetsPtr(); + + /// Remove new variants that were added since last checkpoint. + auto new_variant_type = popBackVariants(variant_info, new_discriminators_map, variant_column_ptr->getNumVariants() - nested.size()); + createVariantInfo(new_variant_type); + variant_mappings_cache.clear(); + + new_subcolumns.resize(nested.size()); + new_discriminators_map.resize(nested.size()); + + /// Manually rollback internals of Variant column + new_discriminators_column->assumeMutable()->popBack(new_discriminators_column->size() - checkpoint.size); + new_offses_column->assumeMutable()->popBack(new_offses_column->size() - checkpoint.size); + + for (size_t i = 0; i < nested.size(); ++i) + new_subcolumns[i]->rollback(*nested[i]); + + variant_column = ColumnVariant::create(new_discriminators_column, new_offses_column, Columns(new_subcolumns.begin(), new_subcolumns.end()), new_discriminators_map); + variant_column_ptr = variant_column_ptr = assert_cast(variant_column.get()); +} + void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { if (source_columns.empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 6f8335044a7..6a050f6e5b1 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -309,15 +309,9 @@ public: return variant_column_ptr->getCheckpoint(); } - void updateCheckpoint(ColumnCheckpoint & checkpoint) const override - { - variant_column_ptr->updateCheckpoint(checkpoint); - } + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; - void rollback(const ColumnCheckpoint & checkpoint) override - { - variant_column_ptr->rollback(checkpoint); - } + void rollback(const ColumnCheckpoint & checkpoint) override; void forEachSubcolumn(MutableColumnCallback callback) override { @@ -456,6 +450,8 @@ private: void updateVariantInfoAndExpandVariantColumn(const DataTypePtr & new_variant_type); + static DataTypePtr popBackVariants(const VariantInfo & info, const std::vector & local_to_global_discriminators, size_t n); + WrappedPtr variant_column; /// Store and use pointer to ColumnVariant to avoid virtual calls. /// ColumnDynamic is widely used inside ColumnObject for each path and diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 4c33f042954..c1b31731147 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -712,20 +712,26 @@ void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) { const auto & object_checkpoint = assert_cast(checkpoint); - for (auto & [name, column] : typed_paths) + auto rollback_columns = [&](auto & columns_map, const auto & checkpoints_map) { - const auto & nested = object_checkpoint.typed_paths.at(name); - chassert(nested); - column->rollback(*nested); - } + NameSet names_to_remove; - for (auto & [name, column] : dynamic_paths_ptrs) - { - const auto & nested = object_checkpoint.dynamic_paths.at(name); - chassert(nested); - column->rollback(*nested); - } + /// Rollback subcolumns and remove paths that were not in checkpoint. + for (auto & [name, column] : columns_map) + { + auto it = checkpoints_map.find(name); + if (it == checkpoints_map.end()) + names_to_remove.insert(name); + else + column->rollback(*it->second); + } + for (const auto & name : names_to_remove) + columns_map.erase(name); + }; + + rollback_columns(typed_paths, object_checkpoint.typed_paths); + rollback_columns(dynamic_paths, object_checkpoint.dynamic_paths); shared_data->rollback(*object_checkpoint.shared_data); } diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index de76261229d..f956f60b378 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -920,3 +920,72 @@ TEST(ColumnDynamic, compare) ASSERT_EQ(column_from->compareAt(3, 2, *column_from, -1), -1); ASSERT_EQ(column_from->compareAt(3, 4, *column_from, -1), -1); } + +TEST(ColumnDynamic, rollback) +{ + auto check_variant = [](const ColumnVariant & column_variant, std::vector sizes) + { + ASSERT_EQ(column_variant.getNumVariants(), sizes.size()); + size_t num_rows = 0; + + for (size_t i = 0; i < sizes.size(); ++i) + { + ASSERT_EQ(column_variant.getVariants()[i]->size(), sizes[i]); + num_rows += sizes[i]; + } + + ASSERT_EQ(num_rows, column_variant.size()); + }; + + auto check_checkpoint = [&](const ColumnCheckpoint & cp, std::vector sizes) + { + const auto & nested = assert_cast(cp).nested; + ASSERT_EQ(nested.size(), sizes.size()); + size_t num_rows = 0; + + for (size_t i = 0; i < sizes.size(); ++i) + { + ASSERT_EQ(nested[i]->size, sizes[i]); + num_rows += sizes[i]; + } + + ASSERT_EQ(num_rows, cp.size); + }; + + std::vector>> checkpoints; + + auto column = ColumnDynamic::create(2); + auto checkpoint = column->getCheckpoint(); + + column->insert(Field(42)); + + column->updateCheckpoint(*checkpoint); + checkpoints.emplace_back(checkpoint, std::vector{0, 1}); + + column->insert(Field("str1")); + column->rollback(*checkpoint); + + check_checkpoint(*checkpoint, checkpoints.back().second); + check_variant(column->getVariantColumn(), checkpoints.back().second); + + column->insert("str1"); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{0, 1, 1}); + + column->insert("str2"); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{0, 1, 2}); + + column->insert(Array({1, 2})); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{1, 1, 2}); + + column->insert(Field(42.42)); + checkpoints.emplace_back(column->getCheckpoint(), std::vector{2, 1, 2}); + + for (const auto & [cp, sizes] : checkpoints) + { + auto column_copy = column->clone(); + column_copy->rollback(*cp); + + check_checkpoint(*cp, sizes); + check_variant(assert_cast(*column_copy).getVariantColumn(), sizes); + } +} From ae1a8393b0ae0ceefc3627683d7ae6a0cb42659f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 22:16:22 +0000 Subject: [PATCH 0099/1218] add test for ColumnObject --- src/Columns/tests/gtest_column_object.cpp | 63 +++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/src/Columns/tests/gtest_column_object.cpp b/src/Columns/tests/gtest_column_object.cpp index f6a1da64ba3..a20bd26fabd 100644 --- a/src/Columns/tests/gtest_column_object.cpp +++ b/src/Columns/tests/gtest_column_object.cpp @@ -5,6 +5,7 @@ #include #include +#include "Core/Field.h" #include using namespace DB; @@ -349,3 +350,65 @@ TEST(ColumnObject, SkipSerializedInArena) pos = col2->skipSerializedInArena(pos); ASSERT_EQ(pos, end); } + +TEST(ColumnObject, rollback) +{ + auto type = DataTypeFactory::instance().get("JSON(max_dynamic_types=10, max_dynamic_paths=2, a.a UInt32, a.b UInt32)"); + auto col = type->createColumn(); + auto & col_object = assert_cast(*col); + const auto & typed_paths = col_object.getTypedPaths(); + const auto & dynamic_paths = col_object.getDynamicPaths(); + const auto & shared_data = col_object.getSharedDataColumn(); + + auto assert_sizes = [&](size_t size) + { + for (const auto & [name, column] : typed_paths) + ASSERT_EQ(column->size(), size); + + for (const auto & [name, column] : dynamic_paths) + ASSERT_EQ(column->size(), size); + + ASSERT_EQ(shared_data.size(), size); + }; + + auto checkpoint = col_object.getCheckpoint(); + + col_object.insert(Object{{"a.a", Field{1u}}}); + col_object.updateCheckpoint(*checkpoint); + + col_object.insert(Object{{"a.b", Field{2u}}}); + col_object.insert(Object{{"a.a", Field{3u}}}); + + col_object.rollback(*checkpoint); + + assert_sizes(1); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 0); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*typed_paths.at("a.b"))[0], Field{0u}); + + col_object.insert(Object{{"a.c", Field{"ccc"}}}); + + checkpoint = col_object.getCheckpoint(); + + col_object.insert(Object{{"a.d", Field{"ddd"}}}); + col_object.insert(Object{{"a.e", Field{"eee"}}}); + + assert_sizes(4); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 2); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field{"ccc"}); + ASSERT_EQ((*dynamic_paths.at("a.d"))[2], Field{"ddd"}); + + col_object.rollback(*checkpoint); + + assert_sizes(2); + ASSERT_EQ(typed_paths.size(), 2); + ASSERT_EQ(dynamic_paths.size(), 1); + + ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); + ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field{"ccc"}); +} From cef9eb80d9004b55887e3a88063fba8300b721f9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 3 Sep 2024 22:47:59 +0000 Subject: [PATCH 0100/1218] better checkpoints for ColumnString --- src/Columns/ColumnString.cpp | 17 +++++++++++++++++ src/Columns/ColumnString.h | 4 ++++ 2 files changed, 21 insertions(+) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 00cf3bd9c30..269c20397b4 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -240,6 +240,23 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const return permuteImpl(*this, perm, limit); } +ColumnCheckpointPtr ColumnString::getCheckpoint() const +{ + auto nested = std::make_shared(chars.size()); + return std::make_shared(size(), std::move(nested)); +} + +void ColumnString::updateCheckpoint(ColumnCheckpoint & checkpoint) const +{ + checkpoint.size = size(); + assert_cast(checkpoint).nested->size = chars.size(); +} + +void ColumnString::rollback(const ColumnCheckpoint & checkpoint) +{ + offsets.resize_assume_reserved(checkpoint.size); + chars.resize_assume_reserved(assert_cast(checkpoint).nested->size); +} void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index ec0563b3f00..c2371412437 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -194,6 +194,10 @@ public: offsets.resize_assume_reserved(offsets.size() - n); } + ColumnCheckpointPtr getCheckpoint() const override; + void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; + void rollback(const ColumnCheckpoint & checkpoint) override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; From a1cdf9b94196d15a4f8b9d17905ffcd552faca93 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Fri, 6 Sep 2024 08:13:52 +0000 Subject: [PATCH 0101/1218] Fix segmentation fault in case profile events is not collected --- src/Interpreters/QueryMetricLog.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/QueryMetricLog.cpp b/src/Interpreters/QueryMetricLog.cpp index f7c65af8808..ac2dada90f3 100644 --- a/src/Interpreters/QueryMetricLog.cpp +++ b/src/Interpreters/QueryMetricLog.cpp @@ -157,11 +157,18 @@ std::optional QueryMetricLog::createLogMetricElement(cons elem.peak_memory_usage = query_info.peak_memory_usage > 0 ? query_info.peak_memory_usage : 0; auto & query_status = query_status_it->second; - for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) + if (query_info.profile_counters) { - const auto & new_value = (*(query_info.profile_counters))[i]; - elem.profile_events[i] = new_value - query_status.last_profile_events[i]; - query_status.last_profile_events[i] = new_value; + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) + { + const auto & new_value = (*(query_info.profile_counters))[i]; + elem.profile_events[i] = new_value - query_status.last_profile_events[i]; + query_status.last_profile_events[i] = new_value; + } + } + else + { + elem.profile_events = query_status.last_profile_events; } query_status.next_collect_time += std::chrono::milliseconds(query_status.interval_milliseconds); From 9edc66d458d2e9376ed52582dc25ab7934ea9085 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 6 Sep 2024 19:22:59 +0000 Subject: [PATCH 0102/1218] simplify scheduler constraints --- src/Common/Scheduler/ISchedulerConstraint.h | 25 +------------- .../Scheduler/Nodes/SemaphoreConstraint.h | 9 +---- .../Scheduler/Nodes/ThrottlerConstraint.h | 11 ++----- src/Common/Scheduler/ResourceRequest.cpp | 33 +++++++++++++++++-- src/Common/Scheduler/ResourceRequest.h | 18 +++++++--- 5 files changed, 49 insertions(+), 47 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index a976206de74..754f6dd404f 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -15,8 +15,7 @@ namespace DB * When constraint is again satisfied, scheduleActivation() is called from finishRequest(). * * Derived class behaviour requirements: - * - dequeueRequest() must fill `request->constraint` iff it is nullptr; - * - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`. + * - dequeueRequest() must call `request->addConstraint()`. */ class ISchedulerConstraint : public ISchedulerNode { @@ -29,30 +28,8 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; - void setParent(ISchedulerNode * parent_) override - { - ISchedulerNode::setParent(parent_); - - // Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint - for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent) - { - if (auto * constraint = dynamic_cast(node)) - { - parent_constraint = constraint; - break; - } - } - } - /// For introspection of current state (true = satisfied, false = violated) virtual bool isSatisfied() = 0; - -protected: - // Reference to nearest parent that is also derived from ISchedulerConstraint. - // Request can traverse through multiple constraints while being dequeue from hierarchy, - // while finishing request should traverse the same chain in reverse order. - // NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest() - ISchedulerConstraint * parent_constraint = nullptr; }; } diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 92c6af9db18..46b048ce34c 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -69,10 +69,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + request->addConstraint(this); // Update state on request arrival std::unique_lock lock(mutex); @@ -87,10 +84,6 @@ public: void finishRequest(ResourceRequest * request) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // Update state on request departure std::unique_lock lock(mutex); bool was_active = active(); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 56866336f50..4e2faa6b233 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -79,10 +79,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + // We don't do `request->addConstraint(this)` because `finishRequest()` is no-op updateBucket(request->cost); @@ -93,12 +90,8 @@ public: return {request, active()}; } - void finishRequest(ResourceRequest * request) override + void finishRequest(ResourceRequest *) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // NOTE: Token-bucket constraint does not require any action when consumption ends } diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 26e8084cdfa..91394108f5d 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -1,13 +1,42 @@ #include #include +#include + +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + void ResourceRequest::finish() { - if (constraint) - constraint->finishRequest(this); + // Iterate over constraints in reverse order + for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints)) + { + if (constraint) + constraint->finishRequest(this); + } +} + +void ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +{ + for (auto & constraint : constraints) + { + if (!constraint) + { + constraint = new_constraint; + return; + } + } + // TODO(serxa): is it possible to validate it during enqueue of resource request to avoid LOGICAL_ERRORs in the scheduler thread? possible but will not cover case of moving queue with requests inside to invalid position + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Max number of simultaneous workload constraints exceeded ({}). Remove extra constraints before using this workload.", + ResourceMaxConstraints); } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d64f624cec5..635353b569b 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -15,6 +16,10 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); +// TODO(serxa): validate hierarchy to avoid too many constrants +/// Max number of constraints for a request to pass though (depth of constaints chain) +constexpr size_t ResourceMaxConstraints = 8; + /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -49,9 +54,10 @@ public: /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; - /// Scheduler node to be notified on consumption finish - /// Auto-filled during request enqueue/dequeue - ISchedulerConstraint * constraint; + /// Scheduler nodes to be notified on consumption finish + /// Auto-filled during request dequeue + /// Vector is not used to avoid allocations in the scheduler thread + std::array constraints; explicit ResourceRequest(ResourceCost cost_ = 1) { @@ -62,7 +68,8 @@ public: void reset(ResourceCost cost_) { cost = cost_; - constraint = nullptr; + for (auto & constraint : constraints) + constraint = nullptr; // Note that list_base_hook should be reset independently (by intrusive list) } @@ -79,6 +86,9 @@ public: /// ResourceRequest should not be destructed or reset before calling to `finish()`. /// WARNING: this function MUST not be called if request was canceled. void finish(); + + /// Is called from the scheduler thread to fill `constraints` chain + void addConstraint(ISchedulerConstraint * new_constraint); }; } From 8e2f98a032378588e932e929fa1a46680846f367 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 7 Sep 2024 15:47:39 +0000 Subject: [PATCH 0103/1218] Make a clean start with v1.21.2 --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 71b06c22760..878cf51ff05 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 71b06c2276009ae649c7703019f3b4605f66fd3d +Subproject commit 878cf51ff0516da8e50235e770f52c75e8dc11d8 From 35f27bf36db43d67121584bcf7bfc407c05ae2c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 7 Sep 2024 15:59:48 +0000 Subject: [PATCH 0104/1218] Bump krb5 to v1.21.3 --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 878cf51ff05..c5b4b994c18 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 878cf51ff0516da8e50235e770f52c75e8dc11d8 +Subproject commit c5b4b994c18db86933255907a97eee5993fd18fe From 14542d6779652c7c0b78efca3fa74fb6ae4a66f6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:26:54 +0000 Subject: [PATCH 0105/1218] 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 0106/1218] 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 bf7ec7d639dbf8e097d423c63c5a2710d599b634 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Aug 2024 16:00:54 +0200 Subject: [PATCH 0107/1218] Revert "Revert "Fix unexpected behavior with `FORMAT` and `SETTINGS` parsing"" --- programs/client/Client.cpp | 3 + programs/server/Server.cpp | 2 +- src/Access/AccessControl.cpp | 8 ++- src/Access/AccessControl.h | 5 +- src/Access/SettingsConstraints.cpp | 8 +-- src/Client/ClientBase.cpp | 50 +++---------- src/Interpreters/InterpreterSetQuery.cpp | 34 ++++----- src/Interpreters/InterpreterSetQuery.h | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 71 ++++++++++++------- ...QueryWithOutputSettingsPushDownVisitor.cpp | 56 --------------- .../QueryWithOutputSettingsPushDownVisitor.h | 39 ---------- .../00857_global_joinsavel_table_alias.sql | 1 - .../01401_FORMAT_SETTINGS.reference | 4 +- .../0_stateless/01401_FORMAT_SETTINGS.sh | 2 +- .../03003_compatibility_setting_bad_value.sql | 3 +- .../03172_format_settings_clauses.reference | 14 ++++ .../03172_format_settings_clauses.sql | 30 ++++++++ 17 files changed, 138 insertions(+), 194 deletions(-) delete mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp delete mode 100644 src/Parsers/QueryWithOutputSettingsPushDownVisitor.h create mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.reference create mode 100644 tests/queries/0_stateless/03172_format_settings_clauses.sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 25c94c56aa6..39edaf3497e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1164,6 +1164,9 @@ void Client::processOptions(const OptionsDescription & options_description, /// (There is no need to copy the context because clickhouse-client has no background tasks so it won't use that context in parallel.) client_context = global_context; initClientContext(); + + /// Allow to pass-through unknown settings to the server. + client_context->getAccessControl().allowAllSettings(); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fb5717ba33f..55fbadbf835 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1923,7 +1923,7 @@ try auto & access_control = global_context->getAccessControl(); try { - access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); + access_control.setupFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); } catch (...) { diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 95a467bbbe5..d4f8c7bc859 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -280,7 +280,7 @@ void AccessControl::shutdown() } -void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, +void AccessControl::setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_) { if (config_.has("custom_settings_prefixes")) @@ -868,4 +868,10 @@ const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const return *external_authenticators; } + +void AccessControl::allowAllSettings() +{ + custom_settings_prefixes->registerPrefixes({""}); +} + } diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index bfaf256ad48..7d8ee1232d0 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -57,7 +57,7 @@ public: void shutdown() override; /// Initializes access storage (user directories). - void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, + void setupFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_); /// Parses access entities from a configuration loaded from users.xml. @@ -238,6 +238,9 @@ public: /// Gets manager of notifications. AccessChangesNotifier & getChangesNotifier(); + /// Allow all setting names - this can be used in clients to pass-through unknown settings to the server. + void allowAllSettings(); + private: class ContextAccessCache; class CustomSettingsPrefixes; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index a274f6b54f2..7506e365035 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -219,8 +219,8 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang }); } -template -bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) +template +bool getNewValueToCheck(const SettingsT & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) { Field current_value; bool has_current_value = current_settings.tryGet(change.name, current_value); @@ -230,12 +230,12 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel return false; if (throw_on_failure) - new_value = T::castValueUtil(change.name, change.value); + new_value = SettingsT::castValueUtil(change.name, change.value); else { try { - new_value = T::castValueUtil(change.name, change.value); + new_value = SettingsT::castValueUtil(change.name, change.value); } catch (...) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 717a9bbe95a..0c27f29ecf9 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -59,6 +59,7 @@ #include #include #include +#include #include #include #include @@ -1609,14 +1610,14 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des auto metadata = storage->getInMemoryMetadataPtr(); QueryPlan plan; storage->read( - plan, - sample.getNames(), - storage->getStorageSnapshot(metadata, client_context), - query_info, - client_context, - {}, - client_context->getSettingsRef().max_block_size, - getNumberOfPhysicalCPUCores()); + plan, + sample.getNames(), + storage->getStorageSnapshot(metadata, client_context), + query_info, + client_context, + {}, + client_context->getSettingsRef().max_block_size, + getNumberOfPhysicalCPUCores()); auto builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(client_context), @@ -1914,42 +1915,13 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin if (old_settings) client_context->setSettings(*old_settings); }); - - auto apply_query_settings = [&](const IAST & settings_ast) - { - if (!old_settings) - old_settings.emplace(client_context->getSettingsRef()); - client_context->applySettingsChanges(settings_ast.as()->changes); - client_context->resetSettingsToDefaultValue(settings_ast.as()->default_settings); - }; - - const auto * insert = parsed_query->as(); - if (const auto * select = parsed_query->as(); select && select->settings()) - apply_query_settings(*select->settings()); - else if (const auto * select_with_union = parsed_query->as()) - { - const ASTs & children = select_with_union->list_of_selects->children; - if (!children.empty()) - { - // On the client it is enough to apply settings only for the - // last SELECT, since the only thing that is important to apply - // on the client is format settings. - const auto * last_select = children.back()->as(); - if (last_select && last_select->settings()) - { - apply_query_settings(*last_select->settings()); - } - } - } - else if (const auto * query_with_output = parsed_query->as(); query_with_output && query_with_output->settings_ast) - apply_query_settings(*query_with_output->settings_ast); - else if (insert && insert->settings_ast) - apply_query_settings(*insert->settings_ast); + InterpreterSetQuery::applySettingsFromQuery(parsed_query, client_context); if (!connection->checkConnected(connection_parameters.timeouts)) connect(); ASTPtr input_function; + const auto * insert = parsed_query->as(); if (insert && insert->select) insert->tryFindInputFunction(input_function); diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 7e68fc5c4c1..2ae35c4313b 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -9,6 +9,7 @@ #include #include + namespace DB { @@ -45,9 +46,7 @@ static void applySettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & sel // It is flattened later, when we process UNION ALL/DISTINCT. const auto * last_select = children.back()->as(); if (last_select && last_select->settings()) - { - InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(); - } + InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext(/* ignore_setting_constraints= */ false); } void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context_) @@ -55,10 +54,20 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta if (!ast) return; + /// First apply the outermost settings. Then they could be overridden by deeper settings. + if (const auto * query_with_output = dynamic_cast(ast.get())) + { + if (query_with_output->settings_ast) + InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); + + if (const auto * create_query = ast->as(); create_query && create_query->select) + applySettingsFromSelectWithUnion(create_query->select->as(), context_); + } + if (const auto * select_query = ast->as()) { if (auto new_settings = select_query->settings()) - InterpreterSetQuery(new_settings, context_).executeForCurrentContext(); + InterpreterSetQuery(new_settings, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); } else if (const auto * select_with_union_query = ast->as()) { @@ -67,28 +76,15 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta else if (const auto * explain_query = ast->as()) { if (explain_query->settings_ast) - InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(); + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } - else if (const auto * query_with_output = dynamic_cast(ast.get())) - { - if (query_with_output->settings_ast) - InterpreterSetQuery(query_with_output->settings_ast, context_).executeForCurrentContext(); - - if (const auto * create_query = ast->as()) - { - if (create_query->select) - { - applySettingsFromSelectWithUnion(create_query->select->as(), context_); - } - } - } else if (auto * insert_query = ast->as()) { context_->setInsertFormat(insert_query->format); if (insert_query->settings_ast) - InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(); + InterpreterSetQuery(insert_query->settings_ast, context_).executeForCurrentContext(/* ignore_setting_constraints= */ false); } } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index 2438762f347..f50105c39f4 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(bool ignore_setting_constraints = false); + void executeForCurrentContext(bool ignore_setting_constraints); bool supportsTransactions() const override { return true; } diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index cb0c10cd1c9..ac8f7d560e0 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -152,37 +151,55 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } + /// These two sections are allowed in an arbitrary order. ParserKeyword s_format(Keyword::FORMAT); - - if (s_format.ignore(pos, expected)) - { - ParserIdentifier format_p; - - if (!format_p.parse(pos, query_with_output.format, expected)) - return false; - setIdentifierSpecial(query_with_output.format); - - query_with_output.children.push_back(query_with_output.format); - } - - // SETTINGS key1 = value1, key2 = value2, ... ParserKeyword s_settings(Keyword::SETTINGS); - if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) - { - ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) - return false; - query_with_output.children.push_back(query_with_output.settings_ast); - // SETTINGS after FORMAT is not parsed by the SELECT parser (ParserSelectQuery) - // Pass them manually, to apply in InterpreterSelectQuery::initSettings() - if (query->as()) + /** Why: let's take the following example: + * SELECT 1 UNION ALL SELECT 2 FORMAT TSV + * Each subquery can be put in parentheses and have its own settings: + * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV + * And the whole query can have settings: + * (SELECT 1 SETTINGS a=b) UNION ALL (SELECT 2 SETTINGS c=d) FORMAT TSV SETTINGS e=f + * A single query with output is parsed in the same way as the UNION ALL chain: + * SELECT 1 SETTINGS a=b FORMAT TSV SETTINGS e=f + * So while these forms have a slightly different meaning, they both exist: + * SELECT 1 SETTINGS a=b FORMAT TSV + * SELECT 1 FORMAT TSV SETTINGS e=f + * And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. + * But while this work: + * (SELECT 1) UNION ALL (SELECT 2) FORMAT TSV SETTINGS d=f + * This does not work automatically, unless we explicitly allow different orders: + * (SELECT 1) UNION ALL (SELECT 2) SETTINGS d=f FORMAT TSV + * Inevitably, we also allow this: + * SELECT 1 SETTINGS a=b SETTINGS d=f FORMAT TSV + * ^^^^^^^^^^^^^^^^^^^^^ + * Because this part is consumed into ASTSelectWithUnionQuery + * and the rest into ASTQueryWithOutput. + */ + + for (size_t i = 0; i < 2; ++i) + { + if (!query_with_output.format && s_format.ignore(pos, expected)) { - auto settings = query_with_output.settings_ast->clone(); - assert_cast(settings.get())->print_in_format = false; - QueryWithOutputSettingsPushDownVisitor::Data data{settings}; - QueryWithOutputSettingsPushDownVisitor(data).visit(query); + ParserIdentifier format_p; + + if (!format_p.parse(pos, query_with_output.format, expected)) + return false; + setIdentifierSpecial(query_with_output.format); + + query_with_output.children.push_back(query_with_output.format); } + else if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) + { + // SETTINGS key1 = value1, key2 = value2, ... + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) + return false; + query_with_output.children.push_back(query_with_output.settings_ast); + } + else + break; } node = std::move(query); diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp deleted file mode 100644 index 8cf0d0063ae..00000000000 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ /dev/null @@ -1,56 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -bool QueryWithOutputSettingsPushDownMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) -{ - if (node->as()) - return true; - if (node->as()) - return true; - if (child->as()) - return true; - return false; -} - -void QueryWithOutputSettingsPushDownMatcher::visit(ASTPtr & ast, Data & data) -{ - if (auto * select_query = ast->as()) - visit(*select_query, ast, data); -} - -void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query, ASTPtr &, Data & data) -{ - ASTPtr select_settings_ast = select_query.settings(); - if (!select_settings_ast) - { - select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, data.settings_ast->clone()); - return; - } - - SettingsChanges & select_settings = select_settings_ast->as().changes; - SettingsChanges & settings = data.settings_ast->as().changes; - - for (auto & setting : settings) - { - auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) - { - return select_setting.name == setting.name; - }); - if (it == select_settings.end()) - select_settings.push_back(setting); - else - it->value = setting.value; - } -} - -} diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h deleted file mode 100644 index fde8a07b555..00000000000 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ASTSelectQuery; -struct SettingChange; -class SettingsChanges; - -/// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: -/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput) -/// -/// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 -> -/// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1 -/// -/// Otherwise settings after FORMAT will not be applied. -class QueryWithOutputSettingsPushDownMatcher -{ -public: - using Visitor = InDepthNodeVisitor; - - struct Data - { - const ASTPtr & settings_ast; - }; - - static bool needChildVisit(ASTPtr & node, const ASTPtr & child); - static void visit(ASTPtr & ast, Data & data); - -private: - static void visit(ASTSelectQuery &, ASTPtr &, Data &); -}; - -using QueryWithOutputSettingsPushDownVisitor = QueryWithOutputSettingsPushDownMatcher::Visitor; - -} diff --git a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql index 2044a9b8d22..092b071cb48 100644 --- a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql +++ b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql @@ -1,4 +1,3 @@ - DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS other_table; diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference index 22405bf1866..a8b99666654 100644 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.reference @@ -1,7 +1,7 @@ 1 1 1 -1 -1 +2 +1 2 2 diff --git a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh index b70c28422c9..173cc949500 100755 --- a/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh +++ b/tests/queries/0_stateless/01401_FORMAT_SETTINGS.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) FORMAT CSV SETTINGS max_block_size = 1' # push down append ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_compress_block_size = 1 FORMAT CSV SETTINGS max_block_size = 1' -# overwrite on push down (since these settings goes latest) +# not overwrite on push down ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 2 FORMAT CSV SETTINGS max_block_size = 1' # on push-down ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT DISTINCT blockSize() FROM numbers(2) SETTINGS max_block_size = 1 FORMAT CSV' diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql index 48e98798c51..3a09eec7452 100644 --- a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -1,2 +1 @@ -select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS} - +select 42 settings compatibility=NULL; -- {clientError BAD_GET} diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.reference b/tests/queries/0_stateless/03172_format_settings_clauses.reference new file mode 100644 index 00000000000..8a98b137f4b --- /dev/null +++ b/tests/queries/0_stateless/03172_format_settings_clauses.reference @@ -0,0 +1,14 @@ +1 +2 +1 +2 +1 +2 +1 +1 +3 +3 +3 +3 +3 +1 diff --git a/tests/queries/0_stateless/03172_format_settings_clauses.sql b/tests/queries/0_stateless/03172_format_settings_clauses.sql new file mode 100644 index 00000000000..0d1aa4dcfbb --- /dev/null +++ b/tests/queries/0_stateless/03172_format_settings_clauses.sql @@ -0,0 +1,30 @@ +SET max_block_size = 10, max_threads = 1; + +-- Take the following example: +SELECT 1 UNION ALL SELECT 2 FORMAT TSV; + +-- Each subquery can be put in parentheses and have its own settings: +(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV; + +-- And the whole query can have settings: +(SELECT getSetting('max_block_size') SETTINGS max_block_size = 1) UNION ALL (SELECT getSetting('max_block_size') SETTINGS max_block_size = 2) FORMAT TSV SETTINGS max_block_size = 3; + +-- A single query with output is parsed in the same way as the UNION ALL chain: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV SETTINGS max_block_size = 3; + +-- So while these forms have a slightly different meaning, they both exist: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 FORMAT TSV; +SELECT getSetting('max_block_size') FORMAT TSV SETTINGS max_block_size = 3; + +-- And due to this effect, the users expect that the FORMAT and SETTINGS may go in an arbitrary order. +-- But while this work: +(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) FORMAT TSV SETTINGS max_block_size = 3; + +-- This does not work automatically, unless we explicitly allow different orders: +(SELECT getSetting('max_block_size')) UNION ALL (SELECT getSetting('max_block_size')) SETTINGS max_block_size = 3 FORMAT TSV; + +-- Inevitably, we allow this: +SELECT getSetting('max_block_size') SETTINGS max_block_size = 1 SETTINGS max_block_size = 3 FORMAT TSV; +/*^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^*/ +-- Because this part is consumed into ASTSelectWithUnionQuery +-- and the rest into ASTQueryWithOutput. From 4f70f48272444a07514c42268862a952dae29e49 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 11 Sep 2024 19:29:53 +0000 Subject: [PATCH 0108/1218] 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 0109/1218] 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 0110/1218] 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 0111/1218] 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 0112/1218] 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 0113/1218] 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 0114/1218] 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 0115/1218] 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 0116/1218] 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 a2f9329e18c07ffbdcf63492e8176129e06e6316 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 16 Sep 2024 13:56:43 +0800 Subject: [PATCH 0117/1218] support quantileExactWeightedInterpolated --- .../AggregateFunctionQuantile.h | 3 + ...AggregateFunctionQuantileExactWeighted.cpp | 237 ++++++++++++++++-- 2 files changed, 214 insertions(+), 26 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index 423fd4bc569..aa6755f237d 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -312,6 +312,9 @@ struct NameQuantilesExactInclusive { static constexpr auto name = "quantilesExac struct NameQuantileExactWeighted { static constexpr auto name = "quantileExactWeighted"; }; struct NameQuantilesExactWeighted { static constexpr auto name = "quantilesExactWeighted"; }; +struct NameQuantileExactWeightedInterpolated { static constexpr auto name = "quantileExactWeightedInterpolated"; }; +struct NameQuantilesExactWeightedInterpolated { static constexpr auto name = "quantilesExactWeightedInterpolated"; }; + struct NameQuantileInterpolatedWeighted { static constexpr auto name = "quantileInterpolatedWeighted"; }; struct NameQuantilesInterpolatedWeighted { static constexpr auto name = "quantilesInterpolatedWeighted"; }; diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp index 469abdf45a2..85acac8cb50 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactWeighted.cpp @@ -29,7 +29,7 @@ namespace * It uses O(distinct(N)) memory. Can be naturally applied for values with weight. * In case of many identical values, it can be more efficient than QuantileExact even when weight is not used. */ -template +template struct QuantileExactWeighted { struct Int128Hash @@ -46,6 +46,7 @@ struct QuantileExactWeighted /// When creating, the hash table must be small. using Map = HashMapWithStackMemory; + using Pair = typename Map::value_type; Map map; @@ -85,6 +86,42 @@ struct QuantileExactWeighted /// Get the value of the `level` quantile. The level must be between 0 and 1. Value get(Float64 level) const + { + if constexpr (interpolated) + return getInterpolatedImpl(level); + else + return getImpl(level); + } + + /// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address. + /// indices - an array of index levels such that the corresponding elements will go in ascending order. + void getMany(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const + { + if constexpr (interpolated) + getManyInterpolatedImpl(levels, indices, num_levels, result); + else + getManyImpl(levels, indices, num_levels, result); + } + + Float64 getFloat(Float64 level) const + { + if constexpr (interpolated) + return getFloatInterpolatedImpl(level); + else + return getFloatImpl(level); + } + + void getManyFloat(const Float64 * levels, const size_t * indices, size_t num_levels, Float64 * result) const + { + if constexpr (interpolated) + getManyFloatInterpolatedImpl(levels, indices, num_levels, result); + else + getManyFloatImpl(levels, indices, num_levels, result); + } + +private: + /// get implementation without interpolation + Value getImpl(Float64 level) const { size_t size = map.size(); @@ -92,7 +129,6 @@ struct QuantileExactWeighted return std::numeric_limits::quiet_NaN(); /// Copy the data to a temporary array to get the element you need in order. - using Pair = typename Map::value_type; std::unique_ptr array_holder(new Pair[size]); Pair * array = array_holder.get(); @@ -135,9 +171,8 @@ struct QuantileExactWeighted return it->first; } - /// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address. - /// indices - an array of index levels such that the corresponding elements will go in ascending order. - void getMany(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const + /// getMany implementation without interpolation + void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const { size_t size = map.size(); @@ -149,7 +184,6 @@ struct QuantileExactWeighted } /// Copy the data to a temporary array to get the element you need in order. - using Pair = typename Map::value_type; std::unique_ptr array_holder(new Pair[size]); Pair * array = array_holder.get(); @@ -197,23 +231,167 @@ struct QuantileExactWeighted } } - /// The same, but in the case of an empty state, NaN is returned. - Float64 getFloat(Float64) const + /// getFloat implementation without interpolation + Float64 getFloatImpl(Float64) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat is not implemented for QuantileExact"); } - void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const + /// getManyFloat implementation without interpolation + void getManyFloatImpl(const Float64 *, const size_t *, size_t, Float64 *) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for QuantileExact"); } + + /// get implementation with interpolation + Value getInterpolatedImpl(Float64 level) const + { + size_t size = map.size(); + if (0 == size) + return std::numeric_limits::quiet_NaN(); + + Float64 res = getFloatInterpolatedImpl(level); + if constexpr (is_decimal) + return Value(static_cast(res)); + else + return static_cast(res); + } + + /// getMany implementation with interpolation + void getManyInterpolatedImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const + { + size_t size = map.size(); + if (0 == size) + { + for (size_t i = 0; i < num_levels; ++i) + result[i] = Value(); + return; + } + + std::unique_ptr res_holder(new Float64[num_levels]); + Float64 * res = res_holder.get(); + getManyFloatInterpolatedImpl(levels, indices, num_levels, res); + for (size_t i = 0; i < num_levels; ++i) + { + if constexpr (is_decimal) + result[i] = Value(static_cast(res[i])); + else + result[i] = Value(res[i]); + } + } + + /// getFloat implementation with interpolation + Float64 getFloatInterpolatedImpl(Float64 level) const + { + size_t size = map.size(); + + if (0 == size) + return std::numeric_limits::quiet_NaN(); + + /// Copy the data to a temporary array to get the element you need in order. + std::unique_ptr array_holder(new Pair[size]); + Pair * array = array_holder.get(); + + size_t i = 0; + for (const auto & pair : map) + { + array[i] = pair.getValue(); + ++i; + } + + ::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; }); + std::partial_sum(array, array + size, array, [](const Pair & acc, const Pair & p) { return Pair(p.first, acc.second + p.second); }); + Weight max_position = array[size - 1].second - 1; + Float64 position = max_position * level; + return quantileInterpolated(array, size, position); + } + + /// getManyFloat implementation with interpolation + void getManyFloatInterpolatedImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Float64 * result) const + { + size_t size = map.size(); + if (0 == size) + { + for (size_t i = 0; i < num_levels; ++i) + result[i] = std::numeric_limits::quiet_NaN(); + return; + } + + /// Copy the data to a temporary array to get the element you need in order. + std::unique_ptr array_holder(new Pair[size]); + Pair * array = array_holder.get(); + + size_t i = 0; + for (const auto & pair : map) + { + array[i] = pair.getValue(); + ++i; + } + + ::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; }); + std::partial_sum(array, array + size, array, [](Pair acc, Pair & p) { return Pair(p.first, acc.second + p.second); }); + Weight max_position = array[size - 1].second - 1; + + for (size_t j = 0; j < num_levels; ++j) + { + Float64 position = max_position * levels[indices[j]]; + result[indices[j]] = quantileInterpolated(array, size, position); + } + } + + /// Calculate quantile, using linear interpolation between two closest values + Float64 NO_SANITIZE_UNDEFINED quantileInterpolated(const Pair * array, size_t size, Float64 position) const + { + /* + for (size_t i = 0; i < size; ++i) + std::cout << "array[" << i << "]: " << toString(Field(array[i].first)) << ", " << array[i].second << std::endl; + std::cout << "position: " << position << std::endl; + */ + size_t lower = static_cast(std::floor(position)); + size_t higher = static_cast(std::ceil(position)); + // std::cout << "lower: " << lower << ", higher: " << higher << std::endl; + + const auto * lower_it = std::lower_bound(array, array + size, lower + 1, [](const Pair & a, size_t b) { return a.second < b; }); + const auto * higher_it = std::lower_bound(array, array + size, higher + 1, [](const Pair & a, size_t b) { return a.second < b; }); + if (lower_it == array + size) + lower_it = array + size - 1; + if (higher_it == array + size) + higher_it = array + size - 1; + // std::cout << "lower_index:" << lower_it - array << ", higher_index:" << higher_it - array << std::endl; + + UnderlyingType lower_key = lower_it->first; + UnderlyingType higher_key = higher_it->first; + + if (lower == higher) + return static_cast(lower_key); + if (lower_key == higher_key) + return static_cast(lower_key); + + return (static_cast(higher) - position) * lower_key + (position - static_cast(lower)) * higher_key; + } }; -template using FuncQuantileExactWeighted = AggregateFunctionQuantile, NameQuantileExactWeighted, true, void, false, false>; -template using FuncQuantilesExactWeighted = AggregateFunctionQuantile, NameQuantilesExactWeighted, true, void, true, false>; +template +using FuncQuantileExactWeighted = AggregateFunctionQuantile< + Value, + QuantileExactWeighted, + NameQuantileExactWeighted, + true, + std::conditional_t, + false, + false>; +template +using FuncQuantilesExactWeighted = AggregateFunctionQuantile< + Value, + QuantileExactWeighted, + NameQuantilesExactWeighted, + true, + std::conditional_t, + true, + false>; -template