From 03ccf05d14bcd6bcdd5346a0aa73ee3fd96f743a Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 3 Sep 2024 16:10:46 +0300 Subject: [PATCH 01/44] Allow custom settings in database engine --- src/Databases/DatabaseFactory.cpp | 21 +++++++---------- src/Databases/DatabaseFactory.h | 23 ++++++++++++++++--- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabaseS3.cpp | 2 +- .../MySQL/DatabaseMaterializedMySQL.cpp | 10 ++++++-- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 6 ++++- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- 12 files changed, 49 insertions(+), 27 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 05a5e057c55..358cdccf8c5 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -66,28 +66,23 @@ void validate(const ASTCreateQuery & create_query) { auto * storage = create_query.storage; - /// Check engine may have arguments - static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; - const String & engine_name = storage->engine->name; - bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); + const EngineFeatures & engine_features = database_engines.at(engine_name).features; - if (storage->engine->arguments && !engine_may_have_arguments) + /// Check engine may have arguments + if (storage->engine->arguments && !engine_features.supports_arguments) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name); /// Check engine may have settings - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; bool has_unexpected_element = storage->engine->parameters || storage->partition_by || storage->primary_key || storage->order_by || storage->sample_by; - if (has_unexpected_element || (!may_have_settings && storage->settings)) + if (has_unexpected_element || (!engine_features.supports_settings && storage->settings)) throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST, "Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name); /// Check engine with table overrides - static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; - if (create_query.table_overrides && !engines_with_table_overrides.contains(engine_name)) + if (create_query.table_overrides && !engine_features.supports_table_overrides) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name); } @@ -121,9 +116,9 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m return impl; } -void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn) +void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features) { - if (!database_engines.emplace(name, std::move(creator_fn)).second) + if (!database_engines.emplace(name, Creator{std::move(creator_fn), features}).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "DatabaseFactory: the database engine name '{}' is not unique", name); } @@ -154,7 +149,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String .context = context}; // creator_fn creates and returns a DatabasePtr with the supplied arguments - auto creator_fn = database_engines.at(engine_name); + auto creator_fn = database_engines.at(engine_name).creator_fn; return creator_fn(arguments); } diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 494c9e0076e..36275820cbe 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -43,13 +43,30 @@ public: ContextPtr & context; }; - DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + struct EngineFeatures + { + bool supports_arguments = false; + bool supports_settings = false; + bool supports_table_overrides = false; + }; using CreatorFn = std::function; - using DatabaseEngines = std::unordered_map; + struct Creator + { + CreatorFn creator_fn; + EngineFeatures features; + }; - void registerDatabase(const std::string & name, CreatorFn creator_fn); + DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + + using DatabaseEngines = std::unordered_map; + + void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ + supports_arguments = false, + supports_parameters = false, + supports_table_overrides = false, + }); const DatabaseEngines & getDatabaseEngines() const { return database_engines; } diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 31701e665a1..4b50e79da4a 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -257,6 +257,6 @@ void registerDatabaseFilesystem(DatabaseFactory & factory) return std::make_shared(args.database_name, init_path, args.context); }; - factory.registerDatabase("Filesystem", create_fn); + factory.registerDatabase("Filesystem", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 7fa67a5678e..ceca2666e49 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -253,7 +253,7 @@ void registerDatabaseHDFS(DatabaseFactory & factory) return std::make_shared(args.database_name, source_url, args.context); }; - factory.registerDatabase("HDFS", create_fn); + factory.registerDatabase("HDFS", create_fn, {.supports_arguments = true}); } } // DB diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 2ccdd8510a8..0a4b02c4917 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -398,6 +398,6 @@ void registerDatabaseLazy(DatabaseFactory & factory) cache_expiration_time_seconds, args.context); }; - factory.registerDatabase("Lazy", create_fn); + factory.registerDatabase("Lazy", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 8e3378bcc12..53064703c2c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1949,6 +1949,6 @@ void registerDatabaseReplicated(DatabaseFactory & factory) replica_name, std::move(database_replicated_settings), args.context); }; - factory.registerDatabase("Replicated", create_fn); + factory.registerDatabase("Replicated", create_fn, {.supports_arguments = true, .supports_settings = true}); } } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 2b2d978a846..d80cc6d0953 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -326,7 +326,7 @@ void registerDatabaseS3(DatabaseFactory & factory) return std::make_shared(args.database_name, config, args.context); }; - factory.registerDatabase("S3", create_fn); + factory.registerDatabase("S3", create_fn, {.supports_arguments = true}); } } #endif diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 2f5477a6b9d..50c7a5bf588 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -290,8 +290,14 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) binlog_client, std::move(materialize_mode_settings)); }; - factory.registerDatabase("MaterializeMySQL", create_fn); - factory.registerDatabase("MaterializedMySQL", create_fn); + + DatabaseFactory::Features features{ + .supports_arguments = true, + .supports_settings = true, + .supports_table_overrides = true, + } + factory.registerDatabase("MaterializeMySQL", create_fn, features); + factory.registerDatabase("MaterializedMySQL", create_fn, features); } } diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 7aa29018f4d..3b72f2aeae5 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -584,7 +584,7 @@ void registerDatabaseMySQL(DatabaseFactory & factory) throw Exception(ErrorCodes::CANNOT_CREATE_DATABASE, "Cannot create MySQL database, because {}", exception_message); } }; - factory.registerDatabase("MySQL", create_fn); + factory.registerDatabase("MySQL", create_fn, {.supports_arguments = true, .supports_settings = true}); } } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 6b0548b85c7..ed62398e594 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -546,7 +546,11 @@ void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory) args.database_name, configuration.database, connection_info, std::move(postgresql_replica_settings)); }; - factory.registerDatabase("MaterializedPostgreSQL", create_fn); + factory.registerDatabase("MaterializedPostgreSQL", create_fn, { + .supports_arguments = true, + .supports_settings = true, + .supports_table_overrides = true, + }); } } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 032fc33ea16..0eafd1c3b5b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -558,7 +558,7 @@ void registerDatabasePostgreSQL(DatabaseFactory & factory) pool, use_table_cache); }; - factory.registerDatabase("PostgreSQL", create_fn); + factory.registerDatabase("PostgreSQL", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 471730fce29..5af9eb1920e 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -220,7 +220,7 @@ void registerDatabaseSQLite(DatabaseFactory & factory) return std::make_shared(args.context, engine_define, args.create_query.attach, database_path); }; - factory.registerDatabase("SQLite", create_fn); + factory.registerDatabase("SQLite", create_fn, {.supports_arguments = true}); } } From e874c6e1de7a8b1bc92fd1e60dcdbc53caac3ded Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 3 Sep 2024 18:58:39 +0300 Subject: [PATCH 02/44] Fix typo --- src/Databases/DatabaseFactory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 36275820cbe..09ebd2bb2b2 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -64,7 +64,7 @@ public: void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ supports_arguments = false, - supports_parameters = false, + supports_settings = false, supports_table_overrides = false, }); From 83854cf293bf52376aa9aa5359fa606f4ccd713e Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 3 Sep 2024 19:13:05 +0300 Subject: [PATCH 03/44] Make method of DatabaseFactory --- src/Databases/DatabaseFactory.cpp | 5 +---- src/Databases/DatabaseFactory.h | 10 +++++++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 358cdccf8c5..d97474bd245 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -59,10 +59,7 @@ void cckMetadataPathForOrdinary(const ASTCreateQuery & create, const String & me } -/// validate validates the database engine that's specified in the create query for -/// engine arguments, settings and table overrides. -void validate(const ASTCreateQuery & create_query) - +void DatabaseFactory::validate(const ASTCreateQuery & create_query) const { auto * storage = create_query.storage; diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 09ebd2bb2b2..ede4394e435 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -63,9 +63,9 @@ public: using DatabaseEngines = std::unordered_map; void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ - supports_arguments = false, - supports_settings = false, - supports_table_overrides = false, + .supports_arguments = false, + .supports_settings = false, + .supports_table_overrides = false, }); const DatabaseEngines & getDatabaseEngines() const { return database_engines; } @@ -82,6 +82,10 @@ private: DatabaseEngines database_engines; DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + + /// validate validates the database engine that's specified in the create query for + /// engine arguments, settings and table overrides. + void validate(const ASTCreateQuery & create_query) const; }; } From 21bd47f09e1823b8812b71b34dd29b93ab871e63 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Wed, 11 Sep 2024 17:17:15 +0000 Subject: [PATCH 04/44] Add settings min_free_disk_bytes_to_throw_insert and min_free_disk_ratio_to_throw_insert and update documentation. --- .../settings/merge-tree-settings.md | 20 +++++++++++++++++++ src/Core/Settings.h | 2 ++ src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ 3 files changed, 24 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index a13aacc76e6..376c1c66ad5 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -156,6 +156,26 @@ Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. +## min_free_disk_bytes_to_throw_insert {#min_free_disk_bytes_to_throw_insert} + +The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes - `keep_free_space_bytes` is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting does not take into account the amount of data that will be written by the `INSERT` operation. + +Possible values: + +- Any positive integer. + +Default value: 0 bytes. + +## min_free_disk_ratio_to_throw_insert {#min_free_disk_ratio_to_throw_insert} + +The minimum free to total disk space ratio to perform an `INSERT`. The free space is calculated by subtracting `keep_free_space_bytes` from the total available space in disk. + +Possible values: + +- Float, 0.0 - 1.0 + +Default value: 0.0 + ## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 493752fc3fe..86522e6c378 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,6 +343,8 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 0769b60dc6b..02ba56f6e9a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -99,6 +99,8 @@ struct Settings; M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ From 7d36f3b7646b56b5e7abc54df6e3a2c305023db0 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 12 Sep 2024 09:53:07 +0000 Subject: [PATCH 05/44] Implemented checks for new settings. --- .../MergeTree/MergeTreeDataWriter.cpp | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f29d715e791..fa280e6080a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -688,8 +689,25 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataPartType part_type; /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); + + // If not temporary insert try to reserve respecting min free disk bytes + size_t reserve_extra = 0; + + if (!is_temp) + { + const auto context = CurrentThread::getQueryContext(); + const auto * settings = context ? &context->getSettingsRef() : nullptr; + + const UInt64 min_bytes = settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = settings->min_free_disk_ratio_to_throw_insert; + + const auto total_disk_space = parent_part->getDataPartStorage().calculateTotalSizeOnDisk(); + const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_space); + reserve_extra = std::min(min_bytes, min_bytes_from_ratio); + } + // just check if there is enough space on parent volume - MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); + MergeTreeData::reserveSpace(expected_size + reserve_extra, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; auto new_data_part = parent_part->getProjectionPartBuilder(part_name, is_temp).withPartType(part_type).build(); From 99ede620bec26b9b3922abc3225794a1afb3dda7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 12 Sep 2024 15:24:25 +0200 Subject: [PATCH 06/44] Add `kill_ci_runner` to ci_utils, will allow restarts --- tests/ci/ci_utils.py | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 8b60f61b006..86fa1c008c9 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,4 +1,5 @@ import json +import logging import os import re import subprocess @@ -6,10 +7,12 @@ import sys import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Sequence +from typing import Any, Dict, Iterator, List, Optional, Sequence, Tuple, Union import requests +logger = logging.getLogger(__name__) + class Envs: GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") @@ -36,6 +39,34 @@ def cd(path: Union[Path, str]) -> Iterator[None]: os.chdir(oldpwd) +def kill_ci_runner(message: str) -> None: + """The function to kill the current process with all parents when it's possible. + Works only when run with the set `CI` environment""" + if not os.getenv("CI", ""): # cycle import env_helper + logger.info("Running outside the CI, won't kill the runner") + return + print(f"::error::{message}") + + def get_ppid_name(pid: int) -> Tuple[int, str]: + # Avoid using psutil, it's not in stdlib + stats = Path(f"/proc/{pid}/stat").read_text(encoding="utf-8").split() + return int(stats[3]), stats[1] + + pid = os.getpid() + pids = {} # type: Dict[str, str] + while pid: + ppid, name = get_ppid_name(pid) + pids[str(pid)] = name + pid = ppid + logger.error( + "Sleeping 5 seconds and killing all possible processes from following:\n %s", + "\n ".join(f"{p}: {n}" for p, n in pids.items()), + ) + time.sleep(5) + # The current process will be killed too + subprocess.run(f"kill -9 {' '.join(pids.keys())}", check=False, shell=True) + + class GH: class ActionsNames: RunConfig = "RunConfig" From 8d5babf65fe78449e1bf3e3979f3fb7d09d18708 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 12 Sep 2024 15:26:21 +0200 Subject: [PATCH 07/44] Kill the runner process if integration tests fail to pre-pull --- tests/ci/integration_tests_runner.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 6405492cd9c..21ca1649dc9 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -19,11 +19,12 @@ from collections import defaultdict from itertools import chain from typing import Any, Dict, Optional +from ci_utils import kill_ci_runner from env_helper import IS_CI from integration_test_images import IMAGES -from tee_popen import TeePopen from report import JOB_TIMEOUT_TEST_NAME from stopwatch import Stopwatch +from tee_popen import TeePopen MAX_RETRY = 1 NUM_WORKERS = 5 @@ -332,7 +333,9 @@ class ClickhouseIntegrationTestsRunner: except subprocess.CalledProcessError as err: logging.info("docker-compose pull failed: %s", str(err)) continue - logging.error("Pulling images failed for 5 attempts. Will fail the worker.") + message = "Pulling images failed for 5 attempts. Will fail the worker." + logging.error(message) + kill_ci_runner(message) # We pass specific retcode to to ci/integration_test_check.py to skip status reporting and restart job sys.exit(13) From 562c23eac6b73199529aad5d85a7b81aca853376 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 12 Sep 2024 13:28:49 +0000 Subject: [PATCH 08/44] Add new settings to settings change history. --- src/Core/SettingsChangesHistory.cpp | 2 ++ src/Storages/MergeTree/MergeTreeDataWriter.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index fae8c25f5ed..ed36d69fba1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,6 +80,8 @@ static std::initializer_listgetDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; From beffb92411ad726e2b2302934f0c37b9223359c4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 14:51:59 +0000 Subject: [PATCH 09/44] Keep original order of conditions during move to prewhere --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index f0c26c302e1..76a02bbd2c4 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -361,11 +361,23 @@ std::optional MergeTreeWhereOptimizer:: UInt64 total_size_of_moved_conditions = 0; UInt64 total_number_of_moved_columns = 0; + /// Remember positions of conditions in where_conditions list + /// to keep original order of conditions in prewhere_conditions while moving. + std::unordered_map condition_positions; + size_t position= 0; + for (const auto & condition : where_conditions) + condition_positions[&condition] = position++; + /// Move condition and all other conditions depend on the same set of columns. auto move_condition = [&](Conditions::iterator cond_it) { LOG_TRACE(log, "Condition {} moved to PREWHERE", cond_it->node.getColumnName()); - prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it); + /// Keep the original order of conditions in prewhere_conditions. + position = condition_positions[&(*cond_it)]; + auto prewhere_it = prewhere_conditions.begin(); + while (condition_positions[&(*prewhere_it)] < position && prewhere_it != prewhere_conditions.end()) + ++prewhere_it; + prewhere_conditions.splice(prewhere_it, where_conditions, cond_it); total_size_of_moved_conditions += cond_it->columns_size; total_number_of_moved_columns += cond_it->table_columns.size(); @@ -375,7 +387,12 @@ std::optional MergeTreeWhereOptimizer:: if (jt->viable && jt->columns_size == cond_it->columns_size && jt->table_columns == cond_it->table_columns) { LOG_TRACE(log, "Condition {} moved to PREWHERE", jt->node.getColumnName()); - prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++); + /// Keep the original order of conditions in prewhere_conditions. + position = condition_positions[&(*jt)]; + prewhere_it = prewhere_conditions.begin(); + while (condition_positions[&(*prewhere_it)] < position && prewhere_it != prewhere_conditions.end()) + ++prewhere_it; + prewhere_conditions.splice(prewhere_it, where_conditions, jt++); } else { From 401a3d09317d4cbe401d64cdf50c056a5c08e63a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 15:10:29 +0000 Subject: [PATCH 10/44] Add test --- .../0_stateless/03231_prewhere_conditions_order.reference | 1 + .../queries/0_stateless/03231_prewhere_conditions_order.sql | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03231_prewhere_conditions_order.reference create mode 100644 tests/queries/0_stateless/03231_prewhere_conditions_order.sql diff --git a/tests/queries/0_stateless/03231_prewhere_conditions_order.reference b/tests/queries/0_stateless/03231_prewhere_conditions_order.reference new file mode 100644 index 00000000000..bb14c5f88f2 --- /dev/null +++ b/tests/queries/0_stateless/03231_prewhere_conditions_order.reference @@ -0,0 +1 @@ +1 [0,1] [0,1] diff --git a/tests/queries/0_stateless/03231_prewhere_conditions_order.sql b/tests/queries/0_stateless/03231_prewhere_conditions_order.sql new file mode 100644 index 00000000000..acaba12684c --- /dev/null +++ b/tests/queries/0_stateless/03231_prewhere_conditions_order.sql @@ -0,0 +1,6 @@ +drop table if exists test; +create table test (x UInt32, arr1 Array(UInt32), arr2 Array(UInt32)) engine=MergeTree order by x; +insert into test values (1, [0, 1], [0, 1]), (2, [0], [0, 1]); +select * from test where x == 1 and arrayExists((x1, x2) -> (x1 == x2), arr1, arr2); +drop table test; + From 2e82e06330502b7c64a9bf5812f66a6147e79539 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 16:59:25 +0000 Subject: [PATCH 11/44] Update tests --- tests/queries/0_stateless/00000_test.reference | 2 ++ tests/queries/0_stateless/00000_test.sh | 12 ++++++++++++ tests/queries/0_stateless/00000_test.sql | 14 ++++++++++++++ .../02156_storage_merge_prewhere.reference | 14 +++++++------- .../02842_move_pk_to_end_of_prewhere.reference | 10 +++++----- ...tics_delayed_materialization_in_merge.reference | 4 ++-- .../0_stateless/02864_statistics_usage.reference | 8 ++++---- 7 files changed, 46 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/00000_test.reference create mode 100755 tests/queries/0_stateless/00000_test.sh create mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/tests/queries/0_stateless/00000_test.reference b/tests/queries/0_stateless/00000_test.reference new file mode 100644 index 00000000000..676c099a485 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.reference @@ -0,0 +1,2 @@ +42 +select 42 diff --git a/tests/queries/0_stateless/00000_test.sh b/tests/queries/0_stateless/00000_test.sh new file mode 100755 index 00000000000..ae20733a597 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query_id="my_id_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "select 42"; +$CLICKHOUSE_CLIENT -q "system flush logs"; +$CLICKHOUSE_CLIENT -q "select query from system.query_log where query_id='my_id_$CLICKHOUSE_TEST_UNIQUE_NAME' and type='QueryFinish'" + diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql new file mode 100644 index 00000000000..266af42f3db --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sql @@ -0,0 +1,14 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_dynamic_type = 1; +set merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 1; + +drop table if exists test; + +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, use_adaptive_write_buffer_for_dynamic_subcolumns=1, min_bytes_for_full_part_storage=100000000000; + +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000) settings min_insert_block_size_rows=50000; + +select count() from test where dynamicType(d) == 'UInt64'; + +drop table test; diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 876cee60baa..8aa1f0b59d3 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,25 +1,25 @@ Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 Filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 diff --git a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference index b91a4dd2f68..254e59d479a 100644 --- a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference +++ b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference @@ -1,15 +1,15 @@ Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), less(c, 20), equals(b, \'3\'), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), equals(b, \'3\'), less(c, 20), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), less(c, 20), greater(c, 0), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), less(c, 20), greater(c, 0), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), equals(b, \'3\'), less(c, 20)) (removed) + Prewhere filter column: and(equals(b, \'3\'), less(c, 20), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), equals(b, \'3\'), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), equals(b, \'3\'), like(d, \'%es%\')) (removed) 1 diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference index eb5e685597c..c4ef127ebc0 100644 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -5,8 +5,8 @@ After insert After merge Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After truncate, insert, and materialize Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference index a9f669b88c1..fd4181a59c3 100644 --- a/tests/queries/0_stateless/02864_statistics_usage.reference +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -1,7 +1,7 @@ After insert Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After drop statistic Prewhere info Prewhere filter @@ -9,12 +9,12 @@ After drop statistic After add and materialize statistic Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After merge Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After rename Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) + Prewhere filter column: and(less(c, 10_UInt8), less(a, 10_UInt8)) (removed) From 9c1f4f4545dae17824a7d772cb9d1fab11c4f0db Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 17:21:28 +0000 Subject: [PATCH 12/44] Remove bad files --- tests/queries/0_stateless/00000_test.reference | 2 -- tests/queries/0_stateless/00000_test.sh | 12 ------------ tests/queries/0_stateless/00000_test.sql | 14 -------------- 3 files changed, 28 deletions(-) delete mode 100644 tests/queries/0_stateless/00000_test.reference delete mode 100755 tests/queries/0_stateless/00000_test.sh delete mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/tests/queries/0_stateless/00000_test.reference b/tests/queries/0_stateless/00000_test.reference deleted file mode 100644 index 676c099a485..00000000000 --- a/tests/queries/0_stateless/00000_test.reference +++ /dev/null @@ -1,2 +0,0 @@ -42 -select 42 diff --git a/tests/queries/0_stateless/00000_test.sh b/tests/queries/0_stateless/00000_test.sh deleted file mode 100755 index ae20733a597..00000000000 --- a/tests/queries/0_stateless/00000_test.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -$CLICKHOUSE_CLIENT --query_id="my_id_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "select 42"; -$CLICKHOUSE_CLIENT -q "system flush logs"; -$CLICKHOUSE_CLIENT -q "select query from system.query_log where query_id='my_id_$CLICKHOUSE_TEST_UNIQUE_NAME' and type='QueryFinish'" - diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql deleted file mode 100644 index 266af42f3db..00000000000 --- a/tests/queries/0_stateless/00000_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- Tags: long, no-tsan, no-msan, no-ubsan, no-asan - -set allow_experimental_dynamic_type = 1; -set merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 1; - -drop table if exists test; - -create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, use_adaptive_write_buffer_for_dynamic_subcolumns=1, min_bytes_for_full_part_storage=100000000000; - -insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000) settings min_insert_block_size_rows=50000; - -select count() from test where dynamicType(d) == 'UInt64'; - -drop table test; From 8a89d7b2b956372e3c7a7e94ee599f1dfcae8aed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:46:38 +0800 Subject: [PATCH 13/44] allow empty inputs for arrayZip or arrayZipUnaligned --- src/Functions/array/arrayZip.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 6e1cc0f7788..36ab56006b3 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -38,13 +38,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.empty()) - throw Exception( - ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} needs at least one argument; passed {}.", - getName(), - arguments.size()); - DataTypes arguments_types; for (size_t index = 0; index < arguments.size(); ++index) { @@ -68,9 +61,16 @@ public: } ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { size_t num_arguments = arguments.size(); + if (num_arguments == 0) + { + auto res_col = result_type->createColumn(); + res_col->insertDefault(); + return ColumnConst::create(std::move(res_col), input_rows_count); + } + Columns holders(num_arguments); Columns tuple_columns(num_arguments); From f9335a2fd5e4746a8b3fa9bb704640e1a4d3e61c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:50:50 +0800 Subject: [PATCH 14/44] update uts --- tests/queries/0_stateless/01045_array_zip.reference | 1 + tests/queries/0_stateless/01045_array_zip.sql | 2 +- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 1 + tests/queries/0_stateless/03230_array_zip_unaligned.sql | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 955ed98033e..85b53dd0c9f 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,2 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] +[()] diff --git a/tests/queries/0_stateless/01045_array_zip.sql b/tests/queries/0_stateless/01045_array_zip.sql index 0bf77747123..801df5a3230 100644 --- a/tests/queries/0_stateless/01045_array_zip.sql +++ b/tests/queries/0_stateless/01045_array_zip.sql @@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']); SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZip(); SELECT arrayZip('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7067f8788e5..7327d25eab8 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,5 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] +[()] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.sql b/tests/queries/0_stateless/03230_array_zip_unaligned.sql index 90b7aa47bfd..08d77737e54 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.sql +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.sql @@ -2,7 +2,7 @@ SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f']) as x, toTypeName(x); SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZipUnaligned(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZipUnaligned(); SELECT arrayZipUnaligned('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 11d2963497ad62d729b9f445f97083a490c30a43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 11:56:47 +0800 Subject: [PATCH 15/44] fix style --- src/Functions/array/arrayZip.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 36ab56006b3..2f8c9a3af02 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; -extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } From 53e1975833a833951b110b21462a6a501cea48a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:15:47 +0800 Subject: [PATCH 16/44] fix 01045_array_zip --- tests/queries/0_stateless/01045_array_zip.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 85b53dd0c9f..154afa7eb89 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,3 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] From 71553022e0c36b7740445d1806f132613a394066 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:16:13 +0800 Subject: [PATCH 17/44] fix 03230_array_zip_unaligned --- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7327d25eab8..d373cf47a9c 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,6 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] From 2812953a8ac38cddab356d2e24856ea8c8eb7ab1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 13 Sep 2024 13:37:42 +0000 Subject: [PATCH 18/44] Try to fix tests --- tests/queries/1_stateful/00091_prewhere_two_conditions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql index cd88743160c..649c63f2ec9 100644 --- a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql +++ b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -7,7 +7,7 @@ SET optimize_move_to_prewhere = 1; SET enable_multiple_prewhere_read_steps = 1; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; +SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00' AND URL != ''; SELECT uniq(*) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; WITH toTimeZone(EventTime, 'Asia/Dubai') AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; From ddc506a677253b206922b1faa9b72f36a866d6f2 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 13:21:35 +0000 Subject: [PATCH 19/44] Corrected implementation for check of new settings and fix lint of settings change history. --- src/Core/SettingsChangesHistory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 48 +++++++++++-------- 2 files changed, 29 insertions(+), 23 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 03d0ef8ea76..490198c2376 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -82,9 +82,9 @@ static std::initializer_list #include #include -#include #include #include #include @@ -61,6 +60,7 @@ namespace ErrorCodes extern const int ABORTED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_PARTS; + extern const int NOT_ENOUGH_SPACE; } namespace @@ -554,6 +554,31 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); + const auto & data_settings = data.getSettings(); + + { + const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + + const auto disk = data_part_volume->getDisk(); + const UInt64 total_disk_bytes = *disk->getTotalSpace(); + const UInt64 free_disk_bytes = *disk->getUnreservedSpace(); + + const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); + const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); + + if (needed_free_bytes > free_disk_bytes) + { + throw Exception( + ErrorCodes::NOT_ENOUGH_SPACE, + "Could not perform insert: less than {} free bytes in disk space. " + "Configure this limit with user settings {} or {}", + needed_free_bytes, + "min_free_disk_bytes_to_throw_insert", + "min_free_disk_ratio_to_throw_insert"); + } + } + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -565,8 +590,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - const auto & data_settings = data.getSettings(); - SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -690,25 +713,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); - // If not temporary insert try to reserve respecting min free disk bytes - size_t reserve_extra = 0; - - if (!is_temp) - { - const auto context = CurrentThread::getQueryContext(); - const auto * settings = context ? &context->getSettingsRef() : nullptr; - - const UInt64 min_bytes = settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = settings->min_free_disk_ratio_to_throw_insert; - - const auto total_disk_space = parent_part->getDataPartStorage().calculateTotalSizeOnDisk(); - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_space); - reserve_extra = std::min(min_bytes, min_bytes_from_ratio); - } - // just check if there is enough space on parent volume - // down the line in reserving space there is concurrency safety so no need to worry about 'over-reserving' - MergeTreeData::reserveSpace(expected_size + reserve_extra, parent_part->getDataPartStorage()); + MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; auto new_data_part = parent_part->getProjectionPartBuilder(part_name, is_temp).withPartType(part_type).build(); From 2650a2062899f5f232176ee56814cce66c800139 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:21:17 +0200 Subject: [PATCH 20/44] Make dedup logic O(n*log(n)) instead of O(n^2) --- .../ParallelReplicasReadingCoordinator.cpp | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 8abf735b49f..c9fb09cd0ba 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,8 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info == part.info; }); + auto the_same_it = all_parts_to_read.find(Part{.description = part}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,12 +881,28 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa if (state_initialized) continue; - auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); + /// Look for the first part >= current + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); - /// It is covering part or we have covering - skip it - if (covering_or_the_same_it != all_parts_to_read.end()) - continue; + if (covering_it != all_parts_to_read.end()) + { + /// Checks if other part covers this one or this one covers the other + auto is_covered_or_covering = [&part] (const Part & other) + { + return other.description.info.contains(part.info) || part.info.contains(other.description.info); + }; + + if (is_covered_or_covering(*covering_it)) + continue; + + /// Also look at the previous part, it could be covering the current one + if (covering_it != all_parts_to_read.begin()) + { + --covering_it; + if (is_covered_or_covering(*covering_it)) + continue; + } + } new_rows_to_read += part.rows; From e13247b67ee66d510af988cf0799a7286dab4ea4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:50:43 +0200 Subject: [PATCH 21/44] Fix clang-18 build --- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index c9fb09cd0ba..ddbed5db7dc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,7 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = all_parts_to_read.find(Part{.description = part}); + auto the_same_it = all_parts_to_read.find(Part{.description = part, .replicas = {}}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,14 +882,14 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa continue; /// Look for the first part >= current - auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part, .replicas = {}}); if (covering_it != all_parts_to_read.end()) { /// Checks if other part covers this one or this one covers the other auto is_covered_or_covering = [&part] (const Part & other) { - return other.description.info.contains(part.info) || part.info.contains(other.description.info); + return other.description.info.contains(part.info) || part.info.contains(other.description.info); }; if (is_covered_or_covering(*covering_it)) From 5cc12ca9eed1eca78a89bacdf8b824105a089aa9 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 17:16:16 +0000 Subject: [PATCH 22/44] Added integration testing for newly implemented settings. --- .../MergeTree/MergeTreeDataWriter.cpp | 5 +- .../__init__.py | 0 .../config.d/storage_configuration.xml | 19 +++++++ .../test.py | 51 +++++++++++++++++++ 4 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 66422dd621e..b606bff7faa 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -562,7 +562,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( const auto disk = data_part_volume->getDisk(); const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getUnreservedSpace(); + const UInt64 free_disk_bytes = *disk->getAvailableSpace(); const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); @@ -571,9 +571,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( { throw Exception( ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space. " + "Could not perform insert: less than {} free bytes in disk space ({}). " "Configure this limit with user settings {} or {}", needed_free_bytes, + free_disk_bytes, "min_free_disk_bytes_to_throw_insert", "min_free_disk_ratio_to_throw_insert"); } diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..d4031ff656c --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml @@ -0,0 +1,19 @@ + + + + + local + /disk1/ + + + + + +
+ disk1 +
+
+
+
+
+
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py new file mode 100644 index 00000000000..d8533ba90bc --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -0,0 +1,51 @@ +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/storage_configuration.xml"], + tmpfs=["/disk1:size=100M"], + macros={"shard": 0, "replica": 1}, +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_insert_stops_when_disk_full(start_cluster): + min_free_bytes = 3 * 1024 * 1024 # 3 MiB + + node.query(f""" + CREATE TABLE test_table ( + id UInt32, + data String + ) ENGINE = MergeTree() + ORDER BY id + SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} + """) + + count = 0 + + # Insert data to fill up disk + try: + for _ in range(100000): + node.query("INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)") + count += 1 + except QueryRuntimeException as e: + assert "Could not perform insert" in str(e) + assert "free bytes in disk space" in str(e) + + free_space = int(node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip()) + assert free_space <= min_free_bytes, f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" + + rows = int(node.query("SELECT count() from test_table").strip()) + assert rows == count + + node.query("DROP TABLE test_table") \ No newline at end of file From 56f3030b1795a4c3afddfec600cb22afda5a204f Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 17:32:33 +0000 Subject: [PATCH 23/44] Black formatting python test. --- .../test.py | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index d8533ba90bc..9b8943705fd 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -11,6 +11,7 @@ node = cluster.add_instance( macros={"shard": 0, "replica": 1}, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -19,33 +20,42 @@ def start_cluster(): finally: cluster.shutdown() + def test_insert_stops_when_disk_full(start_cluster): min_free_bytes = 3 * 1024 * 1024 # 3 MiB - node.query(f""" + node.query( + f""" CREATE TABLE test_table ( id UInt32, data String ) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} - """) + """ + ) count = 0 # Insert data to fill up disk try: for _ in range(100000): - node.query("INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)") + node.query( + "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" + ) count += 1 except QueryRuntimeException as e: assert "Could not perform insert" in str(e) assert "free bytes in disk space" in str(e) - free_space = int(node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip()) - assert free_space <= min_free_bytes, f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" + free_space = int( + node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() + ) + assert ( + free_space <= min_free_bytes + ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" rows = int(node.query("SELECT count() from test_table").strip()) assert rows == count - node.query("DROP TABLE test_table") \ No newline at end of file + node.query("DROP TABLE test_table") From dbb1d043fec40cb3ef403645483c3aad622b4bee Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 05:46:08 +0000 Subject: [PATCH 24/44] unification of FunctionSecretArgumentsFinder --- .../FunctionSecretArgumentsFinderTreeNode.h | 408 ++---------- src/Parsers/FunctionSecretArgumentsFinder.h | 512 ++++++++++++++- .../FunctionSecretArgumentsFinderAST.h | 583 +++--------------- 3 files changed, 642 insertions(+), 861 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 439ddffe5e5..3c0e5974d16 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -3,370 +3,84 @@ #include #include #include -#include #include -#include -#include -#include - -#include namespace DB { +class FunctionTreeNode : public AbstractFunction +{ +public: + class ArgumentTreeNode : public Argument + { + public: + explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->getValue().getType() != Field::Types::String) + return false; + if (res) + *res = literal->getValue().safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->getIdentifier().getFullName(); + return true; + } + } + + return false; + } + private: + const IQueryTreeNode * argument = nullptr; + }; + + class ArgumentsTreeNode : public Arguments + { + public: + explicit ArgumentsTreeNode(const QueryTreeNodes * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override { return std::make_unique(arguments->at(n).get()); } + private: + const QueryTreeNodes * arguments = nullptr; + }; + + explicit FunctionTreeNode(const FunctionNode & function_) : function(&function_) + { + if (const auto & nodes = function->getArguments().getNodes(); !nodes.empty()) + arguments = std::make_unique(&nodes); + } + String name() const override { return function->getFunctionName(); } +private: + const FunctionNode * function = nullptr; +}; + /// Finds arguments of a specified function which should not be displayed for most users for security reasons. /// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderTreeNode +class FunctionSecretArgumentsFinderTreeNode : public FunctionSecretArgumentsFinder { public: - explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) : function(function_), arguments(function.getArguments()) + explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) { - if (arguments.getNodes().empty()) + if (!function->hasArguments()) return; - findFunctionSecretArguments(); + findOrdinaryFunctionSecretArguments(); } - struct Result - { - /// Result constructed by default means no arguments will be hidden. - size_t start = static_cast(-1); - size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). - /// In all known cases secret arguments are consecutive - bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. - /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` - std::vector nested_maps; - - bool hasSecrets() const - { - return count != 0 || !nested_maps.empty(); - } - }; - FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const FunctionNode & function; - const ListNode & arguments; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments.getNodes().size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findFunctionSecretArguments() - { - const auto & name = function.getFunctionName(); - - if ((name == "mysql") || (name == "postgresql") || (name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((name == "s3") || (name == "cosn") || (name == "oss") || - (name == "deltaLake") || (name == "hudi") || (name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((name == "remote") || (name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((name == "encrypt") || (name == "decrypt") || - (name == "aes_encrypt_mysql") || (name == "aes_decrypt_mysql") || - (name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - const auto & nodes = arguments.getNodes(); - size_t count = nodes.size(); - while (count > 0) - { - const FunctionNode * f = nodes.at(count - 1)->as(); - if (!f) - break; - if (f->getFunctionName() == "headers") - result.nested_maps.push_back(f->getFunctionName()); - else if (f->getFunctionName() != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments.getNodes().size()) - return false; - - return tryGetStringFromArgument(arguments.getNodes()[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const QueryTreeNodePtr argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument->as()) - { - if (literal->getValue().getType() != Field::Types::String) - return false; - if (res) - *res = literal->getValue().safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument->as()) - { - if (res) - *res = id->getIdentifier().getFullName(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments.getNodes().size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = arguments.getNodes()[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->getFunctionName())) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments.getNodes().empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments.getNodes().size() - 1; - } - - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments.getNodes().size() <= arg_idx) - return false; - - const auto * identifier = arguments.getNodes()[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments.getNodes().size(); ++i) - { - const auto & argument = arguments.getNodes()[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->getFunctionName() != "equals")) - continue; - - const auto * expr_list = equals_func->getArguments().as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->getNodes(); - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 002ad94f6ea..77facd715f8 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,10 +1,42 @@ #pragma once -#include +#include +#include +#include +#include + namespace DB { +class AbstractFunction +{ + friend class FunctionSecretArgumentsFinder; +public: + class Argument + { + public: + virtual ~Argument() = default; + virtual std::unique_ptr getFunction() const = 0; + virtual bool isIdentifier() const = 0; + virtual bool tryGetString(String * res, bool allow_identifier) const = 0; + }; + class Arguments + { + public: + virtual ~Arguments() = default; + virtual size_t size() const = 0; + virtual std::unique_ptr at(size_t n) const = 0; + }; + + virtual ~AbstractFunction() = default; + virtual String name() const = 0; + bool hasArguments() const { return !!arguments; } + +protected: + std::unique_ptr arguments; +}; + class FunctionSecretArgumentsFinder { public: @@ -23,6 +55,484 @@ public: return count != 0 || !nested_maps.empty(); } }; + + explicit FunctionSecretArgumentsFinder(std::unique_ptr && function_) : function(std::move(function_)) {} + + FunctionSecretArgumentsFinder::Result getResult() const { return result; } + +protected: + const std::unique_ptr function; + Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= function->arguments->size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findOrdinaryFunctionSecretArguments() + { + if ((function->name() == "mysql") || (function->name() == "postgresql") || (function->name() == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if (function->name() == "azureBlobStorage") + { + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "azureBlobStorageCluster") + { + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((function->name() == "remote") || (function->name() == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((function->name() == "encrypt") || (function->name() == "decrypt") || + (function->name() == "aes_encrypt_mysql") || (function->name() == "aes_decrypt_mysql") || + (function->name() == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (function->name() == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = function->arguments->size(); + while (count > 0) + { + const auto f = function->arguments->at(count - 1)->getFunction(); + if (!f) + break; + if (f->name() == "headers") + result.nested_maps.push_back(f->name()); + else if (f->name() != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) + { + /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 1); + return; + } + else if (is_cluster_function && isNamedCollectionName(1)) + { + /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 2); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + size_t count = function->arguments->size(); + if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature + if (url_arg_idx + 4 < count) + markSecretArgument(url_arg_idx + 4); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= function->arguments->size()) + return false; + + return tryGetStringFromArgument(*function->arguments->at(arg_idx), res, allow_identifier); + } + + static bool tryGetStringFromArgument(const AbstractFunction::Argument & argument, String * res, bool allow_identifier = true) + { + return argument.tryGetString(res, allow_identifier); + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (function->arguments->size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + auto table_function = function->arguments->at(arg_num)->getFunction(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name())) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (function->arguments->size() == 0) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = function->arguments->size() - 1; + } + + void findTableEngineSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + findExternalDistributedTableEngineSecretArguments(); + } + else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + findS3TableEngineSecretArguments(); + } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } + } + + void findExternalDistributedTableEngineSecretArguments() + { + if (isNamedCollectionName(1)) + { + /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 2); + } + else + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + markSecretArgument(5); + } + } + + void findS3TableEngineSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'secret_access_key') + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (count == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (2 < count) + markSecretArgument(2); + } + + void findDatabaseEngineSecretArguments() + { + const String & engine_name = function->name(); + if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || + (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL")) + { + /// MySQL('host:port', 'database', 'user', 'password') + /// PostgreSQL('host:port', 'database', 'user', 'password') + findMySQLDatabaseSecretArguments(); + } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } + } + + void findMySQLDatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// MySQL(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// MySQL('host:port', 'database', 'user', 'password') + markSecretArgument(3); + } + } + + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + + void findBackupNameSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + markSecretArgument(2); + } + } + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (function->arguments->size() <= arg_idx) + return false; + + return function->arguments->at(arg_idx)->isIdentifier(); + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < function->arguments->size(); ++i) + { + const auto & argument = function->arguments->at(i); + const auto equals_func = argument->getFunction(); + if (!equals_func || (equals_func->name() != "equals")) + continue; + + if (!equals_func->arguments || equals_func->arguments->size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(*equals_func->arguments->at(0), &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 94da30922cc..d2d7a63aff1 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -1,35 +1,91 @@ #pragma once #include -#include #include #include #include -#include - -#include namespace DB { - -/// Finds arguments of a specified function which should not be displayed for most users for security reasons. -/// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderAST +class FunctionAST : public AbstractFunction { public: - explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) : function(function_) + class ArgumentAST : public Argument + { + public: + explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + private: + const IAST * argument = nullptr; + }; + class ArgumentsAST : public Arguments { - if (!function.arguments) + public: + explicit ArgumentsAST(const ASTs * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override + { + return std::make_unique(arguments->at(n).get()); + } + private: + const ASTs * arguments = nullptr; + }; + + explicit FunctionAST(const ASTFunction & function_) : function(&function_) + { + if (!function->arguments) return; - const auto * expr_list = function.arguments->as(); + const auto * expr_list = function->arguments->as(); if (!expr_list) return; - arguments = &expr_list->children; - switch (function.kind) + arguments = std::make_unique(&expr_list->children); + } + + String name() const override { return function->name; } +private: + const ASTFunction * function = nullptr; +}; + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinderAST : public FunctionSecretArgumentsFinder +{ +public: + explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) + { + if (!function->hasArguments()) + return; + + switch (function_.kind) { case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; case ASTFunction::Kind::WINDOW_FUNCTION: break; @@ -43,506 +99,7 @@ public: } FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const ASTFunction & function; - const ASTs * arguments = nullptr; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments->size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findOrdinaryFunctionSecretArguments() - { - if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || - (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if (function.name == "azureBlobStorage") - { - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "azureBlobStorageCluster") - { - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((function.name == "remote") || (function.name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((function.name == "encrypt") || (function.name == "decrypt") || - (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || - (function.name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (function.name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - size_t count = arguments->size(); - while (count > 0) - { - const ASTFunction * f = arguments->at(count - 1)->as(); - if (!f) - break; - if (f->name == "headers") - result.nested_maps.push_back(f->name); - else if (f->name != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) - { - /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 1); - return; - } - else if (is_cluster_function && isNamedCollectionName(1)) - { - /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 2); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - size_t count = arguments->size(); - if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature - if (url_arg_idx + 4 < count) - markSecretArgument(url_arg_idx + 4); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments->size()) - return false; - - return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument.as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument.as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments->size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = (*arguments)[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments->empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments->size() - 1; - } - - void findTableEngineSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - findExternalDistributedTableEngineSecretArguments(); - } - else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || - (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - findS3TableEngineSecretArguments(); - } - else if (engine_name == "URL") - { - findURLSecretArguments(); - } - } - - void findExternalDistributedTableEngineSecretArguments() - { - if (isNamedCollectionName(1)) - { - /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 2); - } - else - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - markSecretArgument(5); - } - } - - void findS3TableEngineSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'secret_access_key') - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((3 <= count) && (count <= 4)) - { - String second_arg; - if (tryGetStringFromArgument(1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (count == 3) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: S3('url', 'format', ...) - } - } - } - - /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (2 < count) - markSecretArgument(2); - } - - void findDatabaseEngineSecretArguments() - { - const String & engine_name = function.name; - if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || - (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL")) - { - /// MySQL('host:port', 'database', 'user', 'password') - /// PostgreSQL('host:port', 'database', 'user', 'password') - findMySQLDatabaseSecretArguments(); - } - else if (engine_name == "S3") - { - /// S3('url', 'access_key_id', 'secret_access_key') - findS3DatabaseSecretArguments(); - } - } - - void findMySQLDatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// MySQL(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// MySQL('host:port', 'database', 'user', 'password') - markSecretArgument(3); - } - } - - void findS3DatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'password', ...) - findSecretNamedArgument("secret_access_key", 1); - } - else - { - /// S3('url', 'access_key_id', 'secret_access_key') - markSecretArgument(2); - } - } - - void findBackupNameSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - markSecretArgument(2); - } - } - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments->size() <= arg_idx) - return false; - - const auto * identifier = (*arguments)[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments->size(); ++i) - { - const auto & argument = (*arguments)[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->name != "equals")) - continue; - - const auto * expr_list = equals_func->arguments->as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->children; - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(*equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; + } From 56cfa74a14f36e89c7ae9b7ab5f27f4b042cab48 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 13:32:52 +0000 Subject: [PATCH 25/44] fix --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 7 ++++++- src/Parsers/FunctionSecretArgumentsFinderAST.h | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 3c0e5974d16..8e8f56760e4 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index d2d7a63aff1..9430053d6a5 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { From 6f63a7b213170b5b561b904d44c4cafa62a88dbb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 16:46:48 +0000 Subject: [PATCH 26/44] fix tidy --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 2 +- src/Parsers/FunctionSecretArgumentsFinderAST.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 8e8f56760e4..c598adfd98e 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 9430053d6a5..b89007c619d 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { From 038f56cb5e4fec023cc56a9cc1688e6985857230 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Sat, 14 Sep 2024 21:04:12 +0000 Subject: [PATCH 27/44] Only make checks to stop inserts if settings are being used. --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b606bff7faa..e766ae01dfc 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -555,11 +555,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); const auto & data_settings = data.getSettings(); + const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + if (min_bytes > 0 || min_ratio > 0.0) { - const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; - const auto disk = data_part_volume->getDisk(); const UInt64 total_disk_bytes = *disk->getTotalSpace(); const UInt64 free_disk_bytes = *disk->getAvailableSpace(); From 03737ddcab8e2355e6f6dd17348e0123272466da Mon Sep 17 00:00:00 2001 From: marco-vb Date: Sat, 14 Sep 2024 22:24:17 +0000 Subject: [PATCH 28/44] Reduced disk size on test for faster execution. --- .../test_stop_insert_when_disk_close_to_full/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index 9b8943705fd..328de674de1 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/disk1:size=100M"], + tmpfs=["/disk1:size=7M"], macros={"shard": 0, "replica": 1}, ) From d223c4547faf36ba909e57c8a7d1094f33d44ea3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 08:35:05 -0400 Subject: [PATCH 29/44] fix after master merge --- src/Parsers/FunctionSecretArgumentsFinder.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 77facd715f8..434d027c9fc 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -89,7 +89,8 @@ protected: findMySQLFunctionSecretArguments(); } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || - (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || + (function.name == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 19e219758290e80261700bbe55adb48a607a0309 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 10:38:28 -0400 Subject: [PATCH 30/44] fix --- src/Parsers/FunctionSecretArgumentsFinder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 434d027c9fc..7836a863920 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -90,7 +90,7 @@ protected: } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || - (function.name == "gcs")) + (function->name() == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 5aaff37b36fd6b8b4f7aaf3f14a622a1816eb5b1 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 17 Sep 2024 09:16:52 +0000 Subject: [PATCH 31/44] sync changes to InterpreterDropQuery::executeToDatabaseImpl from the private fork --- src/Interpreters/InterpreterDropQuery.cpp | 165 +++++++++++----------- 1 file changed, 82 insertions(+), 83 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ef560ec3405..5161fd19d87 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -380,100 +380,99 @@ BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait) { + if (query.kind != ASTDropQuery::Kind::Detach && query.kind != ASTDropQuery::Kind::Drop && query.kind != ASTDropQuery::Kind::Truncate) + return {}; + const auto & database_name = query.getDatabase(); auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); database = tryGetDatabase(database_name, query.if_exists); - if (database) + if (!database) + return {}; + + bool drop = query.kind == ASTDropQuery::Kind::Drop; + bool truncate = query.kind == ASTDropQuery::Kind::Truncate; + + getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); + + if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases"); + + if (query.if_empty) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); + + if (!truncate && database->hasReplicationThread()) + database->stopReplication(); + + if (database->shouldBeEmptyOnDetach()) { - if (query.kind == ASTDropQuery::Kind::Detach || query.kind == ASTDropQuery::Kind::Drop - || query.kind == ASTDropQuery::Kind::Truncate) + /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. + /// So it will not startup tables concurrently with the flushAndPrepareForShutdown call below. + auto restart_replica_lock = DatabaseCatalog::instance().getLockForDropDatabase(database_name); + + ASTDropQuery query_for_table; + query_for_table.kind = query.kind; + // For truncate operation on database, drop the tables + if (truncate) + query_for_table.kind = query.has_all_tables ? ASTDropQuery::Kind::Truncate : ASTDropQuery::Kind::Drop; + query_for_table.if_exists = true; + query_for_table.if_empty = false; + query_for_table.setDatabase(database_name); + query_for_table.sync = query.sync; + + /// Flush should not be done if shouldBeEmptyOnDetach() == false, + /// since in this case getTablesIterator() may do some additional work, + /// see DatabaseMaterializedMySQL::getTablesIterator() + auto table_context = Context::createCopy(getContext()); + table_context->setInternalQuery(true); + /// Do not hold extra shared pointers to tables + std::vector> tables_to_drop; + // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. + for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { - bool drop = query.kind == ASTDropQuery::Kind::Drop; - bool truncate = query.kind == ASTDropQuery::Kind::Truncate; + auto table_ptr = iterator->table(); + tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); + } - getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); - - if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases"); - - if (query.if_empty) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); - - if (!truncate && database->hasReplicationThread()) - database->stopReplication(); - - if (database->shouldBeEmptyOnDetach()) + /// Prepare tables for shutdown in parallel. + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); + for (const auto & [name, _] : tables_to_drop) + { + auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); + runner([my_table_ptr = std::move(table_ptr)]() { - /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. - /// So it will not startup tables concurrently with the flushAndPrepareForShutdown call below. - auto restart_replica_lock = DatabaseCatalog::instance().getLockForDropDatabase(database_name); + my_table_ptr->flushAndPrepareForShutdown(); + }); + } + runner.waitForAllToFinishAndRethrowFirstError(); - ASTDropQuery query_for_table; - query_for_table.kind = query.kind; - // For truncate operation on database, drop the tables - if (truncate) - query_for_table.kind = query.has_all_tables ? ASTDropQuery::Kind::Truncate : ASTDropQuery::Kind::Drop; - query_for_table.if_exists = true; - query_for_table.if_empty = false; - query_for_table.setDatabase(database_name); - query_for_table.sync = query.sync; - - /// Flush should not be done if shouldBeEmptyOnDetach() == false, - /// since in this case getTablesIterator() may do some additional work, - /// see DatabaseMaterializedMySQL::getTablesIterator() - auto table_context = Context::createCopy(getContext()); - table_context->setInternalQuery(true); - /// Do not hold extra shared pointers to tables - std::vector> tables_to_drop; - // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. - for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) - { - auto table_ptr = iterator->table(); - tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); - } - - /// Prepare tables for shutdown in parallel. - ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); - for (const auto & [name, _] : tables_to_drop) - { - auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); - runner([my_table_ptr = std::move(table_ptr)]() - { - my_table_ptr->flushAndPrepareForShutdown(); - }); - } - runner.waitForAllToFinishAndRethrowFirstError(); - - for (const auto & table : tables_to_drop) - { - query_for_table.setTable(table.first.getTableName()); - query_for_table.is_dictionary = table.second; - DatabasePtr db; - UUID table_to_wait = UUIDHelpers::Nil; - executeToTableImpl(table_context, query_for_table, db, table_to_wait); - uuids_to_wait.push_back(table_to_wait); - } - } - // only if operation is DETACH - if ((!drop || !truncate) && query.sync) - { - /// Avoid "some tables are still in use" when sync mode is enabled - for (const auto & table_uuid : uuids_to_wait) - database->waitDetachedTableNotInUse(table_uuid); - } - - /// Protects from concurrent CREATE TABLE queries - auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - // only if operation is DETACH - if (!drop || !truncate) - database->assertCanBeDetached(true); - - /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. - if (!truncate) - DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); + for (const auto & table : tables_to_drop) + { + query_for_table.setTable(table.first.getTableName()); + query_for_table.is_dictionary = table.second; + DatabasePtr db; + UUID table_to_wait = UUIDHelpers::Nil; + executeToTableImpl(table_context, query_for_table, db, table_to_wait); + uuids_to_wait.push_back(table_to_wait); } } + // only if operation is DETACH + if ((!drop || !truncate) && query.sync) + { + /// Avoid "some tables are still in use" when sync mode is enabled + for (const auto & table_uuid : uuids_to_wait) + database->waitDetachedTableNotInUse(table_uuid); + } + + /// Protects from concurrent CREATE TABLE queries + auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); + // only if operation is DETACH + if (!drop || !truncate) + database->assertCanBeDetached(true); + + /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. + if (!truncate) + DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); return {}; } From 1ccd461c97da60689bf5ab1cb646df7cc59b781e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 4 Sep 2024 23:43:53 +0200 Subject: [PATCH 32/44] Fix restoring access entities dependant on existing ones. --- src/Access/AccessBackup.cpp | 75 +++++++++++++++-------- src/Access/AccessBackup.h | 4 +- src/Access/AccessControl.cpp | 4 +- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 41 +++++-------- src/Access/DiskAccessStorage.h | 5 +- src/Access/IAccessEntity.cpp | 24 ++++++++ src/Access/IAccessEntity.h | 5 +- src/Access/IAccessStorage.cpp | 57 ++++++++++++++--- src/Access/IAccessStorage.h | 10 ++- src/Access/MemoryAccessStorage.cpp | 43 +++++-------- src/Access/MemoryAccessStorage.h | 5 +- src/Access/MultipleAccessStorage.cpp | 4 +- src/Access/MultipleAccessStorage.h | 2 +- src/Access/Quota.cpp | 2 +- src/Access/Quota.h | 2 +- src/Access/ReplicatedAccessStorage.cpp | 85 ++++++++++++++------------ src/Access/ReplicatedAccessStorage.h | 7 ++- src/Access/Role.cpp | 2 +- src/Access/Role.h | 2 +- src/Access/RowPolicy.cpp | 2 +- src/Access/RowPolicy.h | 2 +- src/Access/SettingsProfile.cpp | 2 +- src/Access/SettingsProfile.h | 2 +- src/Access/User.cpp | 2 +- src/Access/User.h | 2 +- 26 files changed, 238 insertions(+), 155 deletions(-) diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index d9ee89b45ce..e8ea21852b5 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -29,6 +29,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_RESTORE_TABLE; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int LOGICAL_ERROR; } @@ -175,9 +176,46 @@ namespace return res; } - std::unordered_map resolveDependencies(const std::unordered_map> & dependencies, const AccessControl & access_control, bool allow_unresolved_dependencies) + /// Checks if new entities (which we're going to restore) already exist, + /// and either skips them or throws an exception depending on the restore settings. + void checkExistingEntities(std::vector> & entities, + std::unordered_map & old_to_new_id, + const AccessControl & access_control, + RestoreAccessCreationMode creation_mode) + { + if (creation_mode == RestoreAccessCreationMode::kReplace) + return; + + auto should_skip = [&](const std::pair & id_and_entity) + { + const auto & id = id_and_entity.first; + const auto & entity = *id_and_entity.second; + auto existing_id = access_control.find(entity.getType(), entity.getName()); + if (!existing_id) + { + return false; + } + else if (creation_mode == RestoreAccessCreationMode::kCreateIfNotExists) + { + old_to_new_id[id] = *existing_id; + return true; + } + else + { + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists", entity.formatTypeWithName()); + } + }; + + std::erase_if(entities, should_skip); + } + + /// If new entities (which we're going to restore) depend on other entities which are not going to be restored or not present in the backup + /// then we should try to replace those dependencies with already existing entities. + void resolveDependencies(const std::unordered_map> & dependencies, + std::unordered_map & old_to_new_ids, + const AccessControl & access_control, + bool allow_unresolved_dependencies) { - std::unordered_map old_to_new_ids; for (const auto & [id, name_and_type] : dependencies) { std::optional new_id; @@ -188,9 +226,9 @@ namespace if (new_id) old_to_new_ids.emplace(id, *new_id); } - return old_to_new_ids; } + /// Generates random IDs for the new entities. void generateRandomIDs(std::vector> & entities, std::unordered_map & old_to_new_ids) { Poco::UUIDGenerator generator; @@ -203,27 +241,12 @@ namespace } } - void replaceDependencies(std::vector> & entities, const std::unordered_map & old_to_new_ids) + /// Updates dependencies of the new entities using a specified map. + void replaceDependencies(std::vector> & entities, + const std::unordered_map & old_to_new_ids) { for (auto & entity : entities | boost::adaptors::map_values) - { - bool need_replace = false; - for (const auto & dependency : entity->findDependencies()) - { - if (old_to_new_ids.contains(dependency)) - { - need_replace = true; - break; - } - } - - if (!need_replace) - continue; - - auto new_entity = entity->clone(); - new_entity->replaceDependencies(old_to_new_ids); - entity = new_entity; - } + IAccessEntity::replaceDependencies(entity, old_to_new_ids); } AccessRightsElements getRequiredAccessToRestore(const std::vector> & entities) @@ -314,7 +337,9 @@ std::pair makeBackupEntryForAccess( AccessRestorerFromBackup::AccessRestorerFromBackup( const BackupPtr & backup_, const RestoreSettings & restore_settings_) - : backup(backup_), allow_unresolved_access_dependencies(restore_settings_.allow_unresolved_access_dependencies) + : backup(backup_) + , creation_mode(restore_settings_.create_access) + , allow_unresolved_dependencies(restore_settings_.allow_unresolved_access_dependencies) { } @@ -362,7 +387,9 @@ std::vector> AccessRestorerFromBackup::getAcces { auto new_entities = entities; - auto old_to_new_ids = resolveDependencies(dependencies, access_control, allow_unresolved_access_dependencies); + std::unordered_map old_to_new_ids; + checkExistingEntities(new_entities, old_to_new_ids, access_control, creation_mode); + resolveDependencies(dependencies, old_to_new_ids, access_control, allow_unresolved_dependencies); generateRandomIDs(new_entities, old_to_new_ids); replaceDependencies(new_entities, old_to_new_ids); diff --git a/src/Access/AccessBackup.h b/src/Access/AccessBackup.h index aa59d6bf201..51a1112e5d5 100644 --- a/src/Access/AccessBackup.h +++ b/src/Access/AccessBackup.h @@ -17,6 +17,7 @@ using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::shared_ptr; struct RestoreSettings; +enum class RestoreAccessCreationMode : uint8_t; /// Makes a backup of access entities of a specified type. @@ -45,7 +46,8 @@ public: private: BackupPtr backup; - bool allow_unresolved_access_dependencies = false; + RestoreAccessCreationMode creation_mode; + bool allow_unresolved_dependencies = false; std::vector> entities; std::unordered_map> dependencies; std::unordered_set data_paths; diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 95a467bbbe5..ec513f0692d 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -544,9 +544,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists)) + if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) { changes_notifier->sendNotifications(); return true; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index bfaf256ad48..0c3bb9352f0 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -243,7 +243,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index ee422f7d8ff..046c532cf5c 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include #include #include #include @@ -418,7 +416,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector & ids_to_keep) @@ -507,14 +505,14 @@ std::optional> DiskAccessStorage::readNameWi } -bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true); } -bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) +bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) { const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -533,9 +531,15 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne if (name_collision && !replace_if_exists) { if (throw_if_exists) + { throwNameCollisionCannotInsert(type, name); + } else + { + if (conflicting_id) + *conflicting_id = id_by_name; return false; + } } auto it_by_id = entries_by_id.find(id); @@ -548,7 +552,11 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne throwIDCollisionCannotInsert(id, type, name, existing_entry.type, existing_entry.name); } else + { + if (conflicting_id) + *conflicting_id = id; return false; + } } if (write_on_disk) @@ -727,25 +735,4 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Couldn't delete {}", file_path); } - -void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) -{ - if (!isRestoreAllowed()) - throwRestoreNotAllowed(); - - auto entities = restorer.getAccessEntitiesToRestore(); - if (entities.empty()) - return; - - auto create_access = restorer.getRestoreSettings().create_access; - bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] - { - for (const auto & [id, entity] : my_entities) - insert(id, entity, replace_if_exists, throw_if_exists); - }); -} - } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 38172b26970..40f2017dd97 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -34,14 +34,13 @@ public: bool exists(const UUID & id) const override; bool isBackupAllowed() const override { return backup_allowed; } - void restoreFromBackup(RestorerFromBackup & restorer) override; private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; @@ -55,7 +54,7 @@ private: void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex); + bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); diff --git a/src/Access/IAccessEntity.cpp b/src/Access/IAccessEntity.cpp index 5dc566fe456..9afa1b73597 100644 --- a/src/Access/IAccessEntity.cpp +++ b/src/Access/IAccessEntity.cpp @@ -9,4 +9,28 @@ bool IAccessEntity::equal(const IAccessEntity & other) const return (name == other.name) && (getType() == other.getType()); } +void IAccessEntity::replaceDependencies(std::shared_ptr & entity, const std::unordered_map & old_to_new_ids) +{ + if (old_to_new_ids.empty()) + return; + + bool need_replace_dependencies = false; + auto dependencies = entity->findDependencies(); + for (const auto & dependency : dependencies) + { + if (old_to_new_ids.contains(dependency)) + { + need_replace_dependencies = true; + break; + } + } + + if (!need_replace_dependencies) + return; + + auto new_entity = entity->clone(); + new_entity->replaceDependencies(old_to_new_ids); + entity = new_entity; +} + } diff --git a/src/Access/IAccessEntity.h b/src/Access/IAccessEntity.h index 5614a172f6f..2c2df7353c5 100644 --- a/src/Access/IAccessEntity.h +++ b/src/Access/IAccessEntity.h @@ -50,7 +50,8 @@ struct IAccessEntity virtual std::vector findDependencies() const { return {}; } /// Replaces dependencies according to a specified map. - virtual void replaceDependencies(const std::unordered_map & /* old_to_new_ids */) {} + void replaceDependencies(const std::unordered_map & old_to_new_ids) { doReplaceDependencies(old_to_new_ids); } + static void replaceDependencies(std::shared_ptr & entity, const std::unordered_map & old_to_new_ids); /// Whether this access entity should be written to a backup. virtual bool isBackupAllowed() const { return false; } @@ -66,6 +67,8 @@ protected: { return std::make_shared(typeid_cast(*this)); } + + virtual void doReplaceDependencies(const std::unordered_map & /* old_to_new_ids */) {} }; using AccessEntityPtr = std::shared_ptr; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index e8e5d57722d..6c9b027fe53 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,10 +16,11 @@ #include #include #include +#include #include +#include #include - namespace DB { namespace ErrorCodes @@ -178,20 +181,20 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); } -std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { auto id = generateRandomID(); - if (insert(id, entity, replace_if_exists, throw_if_exists)) + if (insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) return id; return std::nullopt; } -bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - return insertImpl(id, entity, replace_if_exists, throw_if_exists); + return insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id); } @@ -285,7 +288,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vectorgetType(), entity->getName()); @@ -611,12 +614,50 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c } -void IAccessStorage::restoreFromBackup(RestorerFromBackup &) +void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "restoreFromBackup() is not implemented in {}", getStorageType()); + if (isReplicated() && !acquireReplicatedRestore(restorer)) + return; + + auto entities = restorer.getAccessEntitiesToRestore(); + if (entities.empty()) + return; + + auto create_access = restorer.getRestoreSettings().create_access; + bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); + bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); + + restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] mutable + { + std::unordered_map new_to_existing_ids; + for (auto & [id, entity] : my_entities) + { + UUID existing_entity_id; + if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id)) + { + /// Couldn't insert `entity` because there is an existing entity with the same name. + new_to_existing_ids[id] = existing_entity_id; + } + } + + if (!new_to_existing_ids.empty()) + { + /// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed, + /// then we should correct those dependencies. + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto res = entity; + IAccessEntity::replaceDependencies(res, new_to_existing_ids); + return res; + }; + std::vector ids; + boost::copy(my_entities | boost::adaptors::map_keys, std::back_inserter(ids)); + tryUpdate(ids, update_func); + } + }); } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index e193b65e77a..a8ac75075d3 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -64,6 +64,9 @@ public: /// Returns true if this entity is readonly. virtual bool isReadOnly(const UUID &) const { return isReadOnly(); } + /// Returns true if this storage is replicated. + virtual bool isReplicated() const { return false; } + /// Starts periodic reloading and updating of entities in this storage. virtual void startPeriodicReloading() {} @@ -153,8 +156,8 @@ public: /// Inserts an entity to the storage. Returns ID of a new entry in the storage. /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); - std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); - bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); + bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -218,7 +221,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl( @@ -240,6 +243,7 @@ protected: LoggerPtr getLogger() const; static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } static void clearConflictsInEntitiesList(std::vector> & entities, LoggerPtr log_); + virtual bool acquireReplicatedRestore(RestorerFromBackup &) const { return false; } [[noreturn]] void throwNotFound(const UUID & id) const; [[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const; [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type); diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 791030b9b12..3b5a987fc6e 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -63,14 +61,14 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); } -bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -86,9 +84,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & if (name_collision && !replace_if_exists) { if (throw_if_exists) + { throwNameCollisionCannotInsert(type, name); + } else + { + if (conflicting_id) + *conflicting_id = id_by_name; return false; + } } auto it_by_id = entries_by_id.find(id); @@ -97,9 +101,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & { const auto & existing_entry = it_by_id->second; if (throw_if_exists) + { throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getName()); + } else + { + if (conflicting_id) + *conflicting_id = id; return false; + } } /// Remove collisions if necessary. @@ -270,28 +280,7 @@ void MemoryAccessStorage::setAll(const std::vector findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); bool removeNoLock(const UUID & id, bool throw_if_not_exists); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index fda6601e4c6..f1da8359d48 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -353,7 +353,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::shared_ptr storage_for_insertion; @@ -376,7 +376,7 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & getStorageName()); } - if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists)) + if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) { std::lock_guard lock{mutex}; ids_cache.set(id, storage_for_insertion); diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index e1543c59b67..352cc7f7457 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -67,7 +67,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/Quota.cpp b/src/Access/Quota.cpp index 87b15e722c3..ead5f77ce57 100644 --- a/src/Access/Quota.cpp +++ b/src/Access/Quota.cpp @@ -24,7 +24,7 @@ std::vector Quota::findDependencies() const return to_roles.findDependencies(); } -void Quota::replaceDependencies(const std::unordered_map & old_to_new_ids) +void Quota::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { to_roles.replaceDependencies(old_to_new_ids); } diff --git a/src/Access/Quota.h b/src/Access/Quota.h index eb9edb14fb0..69ec2eb53a5 100644 --- a/src/Access/Quota.h +++ b/src/Access/Quota.h @@ -47,7 +47,7 @@ struct Quota : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return true; } }; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index ed114327041..9039a3b98b7 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -5,10 +5,9 @@ #include #include #include -#include -#include #include #include +#include #include #include #include @@ -120,7 +119,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); @@ -128,7 +127,7 @@ bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr auto zookeeper = getZooKeeper(); bool ok = false; - retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists); }); + retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); }); if (!ok) return false; @@ -143,7 +142,8 @@ bool ReplicatedAccessStorage::insertZooKeeper( const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, - bool throw_if_exists) + bool throw_if_exists, + UUID * conflicting_id) { const String & name = new_entity->getName(); const AccessEntityType type = new_entity->getType(); @@ -167,27 +167,52 @@ bool ReplicatedAccessStorage::insertZooKeeper( if (res == Coordination::Error::ZNODEEXISTS) { - if (!throw_if_exists && !replace_if_exists) - return false; /// Couldn't insert a new entity. - - if (throw_if_exists) + if (!replace_if_exists) { if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { - /// To fail with a nice error message, we need info about what already exists. - /// This itself could fail if the conflicting uuid disappears in the meantime. - /// If that happens, then we'll just retry from the start. - String existing_entity_definition = zookeeper->get(entity_path); + /// Couldn't insert the new entity because there is an existing entity with such UUID. + if (throw_if_exists) + { + /// To fail with a nice error message, we need info about what already exists. + /// This itself can fail if the conflicting uuid disappears in the meantime. + /// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start. + String existing_entity_definition = zookeeper->get(entity_path); - AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); - AccessEntityType existing_type = existing_entity->getType(); - String existing_name = existing_entity->getName(); - throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name); + AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); + AccessEntityType existing_type = existing_entity->getType(); + String existing_name = existing_entity->getName(); + throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name); + } + else + { + if (conflicting_id) + *conflicting_id = id; + return false; + } + } + else if (responses[1]->error == Coordination::Error::ZNODEEXISTS) + { + /// Couldn't insert the new entity because there is an existing entity with the same name. + if (throw_if_exists) + { + throwNameCollisionCannotInsert(type, name); + } + else + { + if (conflicting_id) + { + /// Get UUID of the existing entry with the same name. + /// This itself can fail if the conflicting name disappears in the meantime. + /// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start. + *conflicting_id = parseUUID(zookeeper->get(name_path)); + } + return false; + } } else { - /// Couldn't insert the new entity because there is an existing entity with such name. - throwNameCollisionCannotInsert(type, name); + zkutil::KeeperMultiException::check(res, ops, responses); } } @@ -693,28 +718,10 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col } -void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) +bool ReplicatedAccessStorage::acquireReplicatedRestore(RestorerFromBackup & restorer) const { - if (!isRestoreAllowed()) - throwRestoreNotAllowed(); - auto restore_coordination = restorer.getRestoreCoordination(); - if (!restore_coordination->acquireReplicatedAccessStorage(zookeeper_path)) - return; - - auto entities = restorer.getAccessEntitiesToRestore(); - if (entities.empty()) - return; - - auto create_access = restorer.getRestoreSettings().create_access; - bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] - { - for (const auto & [id, entity] : my_entities) - insert(id, entity, replace_if_exists, throw_if_exists); - }); + return restore_coordination->acquireReplicatedAccessStorage(zookeeper_path); } } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index f8518226997..528dbb31c24 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -26,6 +26,7 @@ public: void shutdown() override; const char * getStorageType() const override { return STORAGE_TYPE; } + bool isReplicated() const override { return true; } void startPeriodicReloading() override { startWatchingThread(); } void stopPeriodicReloading() override { stopWatchingThread(); } @@ -35,7 +36,6 @@ public: bool isBackupAllowed() const override { return backup_allowed; } void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; - void restoreFromBackup(RestorerFromBackup & restorer) override; private: String zookeeper_path; @@ -48,11 +48,11 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists); bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); @@ -80,6 +80,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; + bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override; mutable std::mutex mutex; MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex); diff --git a/src/Access/Role.cpp b/src/Access/Role.cpp index 089488e7aba..f6250594103 100644 --- a/src/Access/Role.cpp +++ b/src/Access/Role.cpp @@ -21,7 +21,7 @@ std::vector Role::findDependencies() const return res; } -void Role::replaceDependencies(const std::unordered_map & old_to_new_ids) +void Role::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { granted_roles.replaceDependencies(old_to_new_ids); settings.replaceDependencies(old_to_new_ids); diff --git a/src/Access/Role.h b/src/Access/Role.h index b2f879dc357..c7f98585a6c 100644 --- a/src/Access/Role.h +++ b/src/Access/Role.h @@ -21,7 +21,7 @@ struct Role : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return settings.isBackupAllowed(); } }; diff --git a/src/Access/RowPolicy.cpp b/src/Access/RowPolicy.cpp index d25b9e259b1..8724d0f513c 100644 --- a/src/Access/RowPolicy.cpp +++ b/src/Access/RowPolicy.cpp @@ -63,7 +63,7 @@ std::vector RowPolicy::findDependencies() const return to_roles.findDependencies(); } -void RowPolicy::replaceDependencies(const std::unordered_map & old_to_new_ids) +void RowPolicy::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { to_roles.replaceDependencies(old_to_new_ids); } diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 9c190458620..5cfe85c186a 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -50,7 +50,7 @@ struct RowPolicy : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return true; } /// Which roles or users should use this row policy. diff --git a/src/Access/SettingsProfile.cpp b/src/Access/SettingsProfile.cpp index 48aa48040ab..632bd97fbf5 100644 --- a/src/Access/SettingsProfile.cpp +++ b/src/Access/SettingsProfile.cpp @@ -21,7 +21,7 @@ std::vector SettingsProfile::findDependencies() const return res; } -void SettingsProfile::replaceDependencies(const std::unordered_map & old_to_new_ids) +void SettingsProfile::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { elements.replaceDependencies(old_to_new_ids); to_roles.replaceDependencies(old_to_new_ids); diff --git a/src/Access/SettingsProfile.h b/src/Access/SettingsProfile.h index f85630d324d..6bcaf6fef30 100644 --- a/src/Access/SettingsProfile.h +++ b/src/Access/SettingsProfile.h @@ -22,7 +22,7 @@ struct SettingsProfile : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return elements.isBackupAllowed(); } }; diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 0bd11000775..2052527f4ae 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -49,7 +49,7 @@ std::vector User::findDependencies() const return res; } -void User::replaceDependencies(const std::unordered_map & old_to_new_ids) +void User::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { default_roles.replaceDependencies(old_to_new_ids); granted_roles.replaceDependencies(old_to_new_ids); diff --git a/src/Access/User.h b/src/Access/User.h index 28f16a76b0c..7f91c1e3756 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -32,7 +32,7 @@ struct User : public IAccessEntity void setName(const String & name_) override; std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return settings.isBackupAllowed(); } }; From f8f72ccb00d3bc35212ec94a272e055b1fe32c76 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 6 Sep 2024 20:56:09 +0200 Subject: [PATCH 33/44] Add test. --- ..._restore_user_with_existing_role.reference | 6 ++ .../03231_restore_user_with_existing_role.sh | 77 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/03231_restore_user_with_existing_role.reference create mode 100755 tests/queries/0_stateless/03231_restore_user_with_existing_role.sh diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference b/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference new file mode 100644 index 00000000000..cad1bf13574 --- /dev/null +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference @@ -0,0 +1,6 @@ +Everything dropped +User dropped +Nothing dropped +Nothing dropped, mode=replace +Nothing dropped, mode=create +ACCESS_ENTITY_ALREADY_EXISTS diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh new file mode 100755 index 00000000000..7862911de04 --- /dev/null +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +# Disabled parallel since RESTORE can only restore either all users or no users +# (it can't restore only users added by the current test run), +# so a RESTORE from a parallel test run could recreate our users before we expect that. + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +user_a="user_a_${CLICKHOUSE_TEST_UNIQUE_NAME}" +role_b="role_b_${CLICKHOUSE_TEST_UNIQUE_NAME}" + +${CLICKHOUSE_CLIENT} -m --query " +CREATE ROLE ${role_b} SETTINGS custom_x=1; +CREATE USER ${user_a} DEFAULT ROLE ${role_b} SETTINGS custom_x=2; +" + +backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')" + +${CLICKHOUSE_CLIENT} --query "BACKUP TABLE system.users, TABLE system.roles TO ${backup_name} FORMAT Null" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" + +do_check() +{ + local replacements + replacements="s/${user_a}/user_a/g; s/${role_b}/role_b/g" + local check_info + check_info=$(${CLICKHOUSE_CLIENT} -mq " + SHOW CREATE USER ${user_a}; + SHOW GRANTS FOR ${user_a}; + SHOW CREATE ROLE ${role_b}; + SHOW GRANTS FOR ${role_b}; + " | sed "${replacements}") + local expected + expected=$'CREATE USER user_a DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' + if [[ "${check_info}" != "${expected}" ]]; then + echo "Assertion failed:" + echo "\"${check_info}\"" + echo "!=" + echo "\"${expected}\"" + echo "Test database: ${CLICKHOUSE_DATABASE}" >&2 + fi +} + +echo "Everything dropped" +${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}" +${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +echo "User dropped" +${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +# TODO: Cannot restore a dropped role granted to an existing user. The result after RESTORE ALL below is the following: +# CREATE USER user_a DEFAULT ROLE NONE SETTINGS custom_x = 2; GRANT NONE TO user_a; CREATE ROLE role_b SETTINGS custom_x = 1 +# because `role_b` is restored but not granted to existing user `user_a`. +# +# echo "Role dropped" +# ${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}" +# ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +# do_check + +echo "Nothing dropped" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +echo "Nothing dropped, mode=replace" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='replace' FORMAT Null" +do_check + +echo "Nothing dropped, mode=create" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='create' FORMAT Null" 2>&1 | grep -om1 "ACCESS_ENTITY_ALREADY_EXISTS" +do_check From 983b061b58c0d0c8421b07e2974795da16c7be0f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 17 Sep 2024 12:56:10 +0200 Subject: [PATCH 34/44] Corrections after review. --- src/Access/IAccessStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 6c9b027fe53..29475461c45 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -630,10 +630,10 @@ void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] mutable + restorer.addDataRestoreTask([this, entities_to_restore = std::move(entities), replace_if_exists, throw_if_exists] mutable { std::unordered_map new_to_existing_ids; - for (auto & [id, entity] : my_entities) + for (auto & [id, entity] : entities_to_restore) { UUID existing_entity_id; if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id)) @@ -654,7 +654,8 @@ void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) return res; }; std::vector ids; - boost::copy(my_entities | boost::adaptors::map_keys, std::back_inserter(ids)); + ids.reserve(entities_to_restore.size()); + boost::copy(entities_to_restore | boost::adaptors::map_keys, std::back_inserter(ids)); tryUpdate(ids, update_func); } }); From f768717be86d0198847783f10b4c9a8f662d78d5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 17 Sep 2024 13:05:02 +0200 Subject: [PATCH 35/44] Fix test. --- .../0_stateless/03231_restore_user_with_existing_role.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh index 7862911de04..04f907b719d 100755 --- a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh @@ -34,7 +34,7 @@ do_check() SHOW GRANTS FOR ${role_b}; " | sed "${replacements}") local expected - expected=$'CREATE USER user_a DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' + expected=$'CREATE USER user_a IDENTIFIED WITH no_password DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' if [[ "${check_info}" != "${expected}" ]]; then echo "Assertion failed:" echo "\"${check_info}\"" From 3a05282bced2b8e4a557ef452afdd5c3fc62471f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 14:26:31 +0200 Subject: [PATCH 36/44] Update assert --- src/Interpreters/Cache/FileCache.cpp | 7 ++++++- tests/clickhouse-test | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 60db406ca72..ffe9a611014 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -718,7 +718,12 @@ FileCache::getOrSet( } } - chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); + chassert(file_segments_limit + ? file_segments.back()->range().left <= result_range.right + : file_segments.back()->range().contains(result_range.right), + fmt::format("Unexpected state. Back: {}, result range: {}, limit: {}", + file_segments.back()->range().toString(), result_range.toString(), file_segments_limit)); + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 75a180f356b..a005fdcc439 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,7 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 3, 10, 50]), + "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From 5ce8604869e9c21bc5b707a985bf4247c1c5a0fd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 17 Sep 2024 12:37:23 +0000 Subject: [PATCH 37/44] Automatic style fix --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a005fdcc439..810bae86cb0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,9 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), + "filesystem_cache_segments_batch_size": lambda: random.choice( + [0, 1, 2, 3, 5, 10, 50, 100] + ), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From 4b69d8e2ca2a68e2030d31151d6adb63a79de836 Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 17 Sep 2024 15:52:20 +0300 Subject: [PATCH 38/44] Fix CE --- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 50c7a5bf588..2b728039632 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -291,11 +291,11 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) std::move(materialize_mode_settings)); }; - DatabaseFactory::Features features{ + DatabaseFactory::EngineFeatures features{ .supports_arguments = true, .supports_settings = true, .supports_table_overrides = true, - } + }; factory.registerDatabase("MaterializeMySQL", create_fn, features); factory.registerDatabase("MaterializedMySQL", create_fn, features); } From aba7de5091ffc52c864cc004c44ff4be966bb126 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:53:32 +0200 Subject: [PATCH 39/44] Verify that there are no intersecting parts in the resulting all_parts_to_read --- .../ParallelReplicasReadingCoordinator.cpp | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ddbed5db7dc..0a25874cfd9 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -911,6 +911,24 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } +#ifndef NDEBUG + /// Double check that there are no intersecting parts + { + auto part_it = all_parts_to_read.begin(); + auto next_part_it = part_it; + if (next_part_it != all_parts_to_read.end()) + ++next_part_it; + while (next_part_it != all_parts_to_read.end()) + { + chassert(part_it->description.info.isDisjoint(next_part_it->description.info), + fmt::format("Parts {} and {} intersect", + part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); + ++part_it; + ++next_part_it; + } + } +#endif + state_initialized = true; // progress_callback is not set when local plan is used for initiator From 190d3f04c9fb03e8a8c64ce1b25d5536e7835ad7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:54:49 +0200 Subject: [PATCH 40/44] More optimal check for intrsecting parts in DefaultCoordinator init --- .../ParallelReplicasReadingCoordinator.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 0a25874cfd9..603584af6c2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -376,17 +376,20 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (state_initialized) return; - for (auto && part : announcement.description) { - auto intersecting_it = std::find_if( - all_parts_to_read.begin(), - all_parts_to_read.end(), - [&part](const Part & other) { return !other.description.info.isDisjoint(part.info); }); + /// To speedup search for adjacent parts + Parts known_parts(all_parts_to_read.begin(), all_parts_to_read.end()); - if (intersecting_it != all_parts_to_read.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + for (auto && part : announcement.description) + { + auto intersecting_it = known_parts.lower_bound(Part{.description = part, .replicas = {}}); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + known_parts.emplace(Part{.description = part, .replicas = {}}); + } } std::ranges::sort( From 3674c97ebba63bc88c6bc03f630124afa314053a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:49:02 +0200 Subject: [PATCH 41/44] Fix for using part after std::move from it --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 603584af6c2..98f28430ecc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -387,8 +387,8 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); known_parts.emplace(Part{.description = part, .replicas = {}}); + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); } } From 574a26c63ba24e4632b428827642b40db48424e4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:56:44 +0200 Subject: [PATCH 42/44] Use adjacent_find to check adjacent parts --- .../ParallelReplicasReadingCoordinator.cpp | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 98f28430ecc..26f2273d196 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -917,18 +917,15 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa #ifndef NDEBUG /// Double check that there are no intersecting parts { - auto part_it = all_parts_to_read.begin(); - auto next_part_it = part_it; - if (next_part_it != all_parts_to_read.end()) - ++next_part_it; - while (next_part_it != all_parts_to_read.end()) - { - chassert(part_it->description.info.isDisjoint(next_part_it->description.info), - fmt::format("Parts {} and {} intersect", - part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); - ++part_it; - ++next_part_it; - } + auto intersecting_part_it = std::adjacent_find(all_parts_to_read.begin(), all_parts_to_read.end(), + [] (const Part & lhs, const Part & rhs) + { + return !lhs.description.info.isDisjoint(rhs.description.info); + }); + + if (intersecting_part_it != all_parts_to_read.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parts {} and {} intersect", + intersecting_part_it->description.info.getPartNameV1(), std::next(intersecting_part_it)->description.info.getPartNameV1()); } #endif From 665f362601a2ee1869cd5c91102d48818f7f0145 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Sep 2024 16:10:03 +0000 Subject: [PATCH 43/44] Prohibit ALTER TABLE ... ADD INDEX ... TYPE inverted if setting = 0 --- src/Storages/AlterCommands.cpp | 10 +++ src/Storages/AlterCommands.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++ ...02346_inverted_index_experimental_flag.sql | 66 +++++++++++++++---- 4 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ef76bc691ec..68778243371 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1142,6 +1142,16 @@ bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) return false; } +bool AlterCommands::hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata) +{ + for (const auto & index : metadata.secondary_indices) + { + if (index.type == INVERTED_INDEX_NAME) + return true; + } + return false; +} + bool AlterCommands::hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata) { for (const auto & index : metadata.secondary_indices) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index c4c792e7dec..be1b31f3d20 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -235,8 +235,9 @@ public: /// additional mutation command (MATERIALIZE_TTL) will be returned. MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const; - /// Check if commands have any full-text index + /// Check if commands have any full-text index or a (legacy) inverted index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + static bool hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata); /// Check if commands have any vector similarity index static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ca619d4d208..80d61058d08 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3230,6 +3230,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); + if (AlterCommands::hasLegacyInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental inverted index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')"); diff --git a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index f2d294ff9e4..84188337a8d 100644 --- a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -1,16 +1,60 @@ --- Tests that the inverted index can only be supported when allow_experimental_full_text_index = 1. - -SET allow_experimental_full_text_index = 0; +-- Tests that CREATE TABLE and ADD INDEX respect settings 'allow_experimental_full_text_index' and `allow_experimental_inverted_index` DROP TABLE IF EXISTS tab; -CREATE TABLE tab -( - `key` UInt64, - `str` String -) -ENGINE = MergeTree -ORDER BY key; -ALTER TABLE tab ADD INDEX inv_idx(str) TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +-- Test CREATE TABLE + full_text index setting +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } DROP TABLE tab; + +SET allow_experimental_full_text_index = 0; -- reset to default + +-- Test CREATE TABLE + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); +DROP TABLE tab; + +SET allow_experimental_inverted_index = 0; -- reset to default + +-- Test ADD INDEX + full_text index setting + +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; +SET allow_experimental_full_text_index = 0; -- reset to default + + +-- Test ADD INDEX + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); +DROP TABLE tab; +SET allow_experimental_inverted_index = 0; -- reset to default From 474499d240425872fadbd59ee97750a67f41390f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Sep 2024 21:48:19 +0200 Subject: [PATCH 44/44] Revert "Add user-level settings min_free_diskspace_bytes_to_throw_insert and min_free_diskspace_ratio_to_throw_insert" --- .../settings/merge-tree-settings.md | 20 ------ src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 30 +-------- src/Storages/MergeTree/MergeTreeSettings.h | 2 - .../__init__.py | 0 .../config.d/storage_configuration.xml | 19 ------ .../test.py | 61 ------------------- 8 files changed, 3 insertions(+), 135 deletions(-) delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 376c1c66ad5..a13aacc76e6 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -156,26 +156,6 @@ Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. -## min_free_disk_bytes_to_throw_insert {#min_free_disk_bytes_to_throw_insert} - -The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes - `keep_free_space_bytes` is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting does not take into account the amount of data that will be written by the `INSERT` operation. - -Possible values: - -- Any positive integer. - -Default value: 0 bytes. - -## min_free_disk_ratio_to_throw_insert {#min_free_disk_ratio_to_throw_insert} - -The minimum free to total disk space ratio to perform an `INSERT`. The free space is calculated by subtracting `keep_free_space_bytes` from the total available space in disk. - -Possible values: - -- Float, 0.0 - 1.0 - -Default value: 0.0 - ## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 95f585d1a27..fadc4079fe0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,8 +343,6 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index da0fbfad255..560f144866b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,9 +85,7 @@ static std::initializer_listgetVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); - const auto & data_settings = data.getSettings(); - const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; - - if (min_bytes > 0 || min_ratio > 0.0) - { - const auto disk = data_part_volume->getDisk(); - const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getAvailableSpace(); - - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); - const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); - - if (needed_free_bytes > free_disk_bytes) - { - throw Exception( - ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space ({}). " - "Configure this limit with user settings {} or {}", - needed_free_bytes, - free_disk_bytes, - "min_free_disk_bytes_to_throw_insert", - "min_free_disk_ratio_to_throw_insert"); - } - } - auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -591,6 +564,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); + const auto & data_settings = data.getSettings(); + SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -713,7 +688,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataPartType part_type; /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); - // just check if there is enough space on parent volume MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b2ebfa1dfda..dcb18155114 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -99,8 +99,6 @@ struct Settings; M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ \ diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml deleted file mode 100644 index d4031ff656c..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - local - /disk1/ - - - - - -
- disk1 -
-
-
-
-
-
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py deleted file mode 100644 index 328de674de1..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ /dev/null @@ -1,61 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", - main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/disk1:size=7M"], - macros={"shard": 0, "replica": 1}, -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_insert_stops_when_disk_full(start_cluster): - min_free_bytes = 3 * 1024 * 1024 # 3 MiB - - node.query( - f""" - CREATE TABLE test_table ( - id UInt32, - data String - ) ENGINE = MergeTree() - ORDER BY id - SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} - """ - ) - - count = 0 - - # Insert data to fill up disk - try: - for _ in range(100000): - node.query( - "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" - ) - count += 1 - except QueryRuntimeException as e: - assert "Could not perform insert" in str(e) - assert "free bytes in disk space" in str(e) - - free_space = int( - node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() - ) - assert ( - free_space <= min_free_bytes - ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" - - rows = int(node.query("SELECT count() from test_table").strip()) - assert rows == count - - node.query("DROP TABLE test_table")