From b2a8f414197ba4f44972075e263f10c276ff643c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Jun 2025 09:40:42 +0000 Subject: [PATCH 1/9] Merge pull request #81069 from ClickHouse/move_read_from_object_storage Move ReadStepFromObjectStorage to a separate file --- .../QueryPlan/ReadFromObjectStorageStep.cpp | 120 ++++++++++++++++++ .../QueryPlan/ReadFromObjectStorageStep.h | 54 ++++++++ .../ObjectStorage/S3/Configuration.cpp | 74 ++++++----- src/Storages/ObjectStorage/S3/Configuration.h | 8 +- .../ObjectStorage/StorageObjectStorage.cpp | 116 +---------------- .../integration/test_storage_iceberg/test.py | 2 +- ...functions_with_parallel_replicas.reference | 6 +- 7 files changed, 228 insertions(+), 152 deletions(-) create mode 100644 src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp create mode 100644 src/Processors/QueryPlan/ReadFromObjectStorageStep.h diff --git a/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp b/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp new file mode 100644 index 000000000000..c152d991d62d --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp @@ -0,0 +1,120 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace Setting +{ + extern const SettingsMaxThreads max_threads; +} + + +ReadFromObjectStorageStep::ReadFromObjectStorageStep( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const Names & columns_to_read, + const NamesAndTypesList & virtual_columns_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const std::optional & format_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(info_.source_header, columns_to_read, query_info_, storage_snapshot_, context_) + , object_storage(object_storage_) + , configuration(configuration_) + , info(std::move(info_)) + , virtual_columns(virtual_columns_) + , format_settings(format_settings_) + , name(name_ + "ReadStep") + , need_only_count(need_only_count_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , distributed_processing(distributed_processing_) +{ +} + +void ReadFromObjectStorageStep::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + createIterator(); +} + +void ReadFromObjectStorageStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createIterator(); + + Pipes pipes; + auto context = getContext(); + const size_t max_threads = context->getSettingsRef()[Setting::max_threads]; + size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); + + if (estimated_keys_count > 1) + num_streams = std::min(num_streams, estimated_keys_count); + else + { + /// The amount of keys (zero) was probably underestimated. + /// We will keep one stream for this particular case. + num_streams = 1; + } + + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared( + getName(), object_storage, configuration, info, format_settings, + context, max_block_size, iterator_wrapper, max_parsing_threads, need_only_count); + + source->setKeyCondition(filter_actions_dag, context); + pipes.emplace_back(std::move(source)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); +} + +void ReadFromObjectStorageStep::createIterator() +{ + if (iterator_wrapper) + return; + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag.has_value()) + predicate = filter_actions_dag->getOutputs().at(0); + + auto context = getContext(); + iterator_wrapper = StorageObjectStorageSource::createFileIterator( + configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, + context, predicate, filter_actions_dag, virtual_columns, info.hive_partition_columns_to_read_from_file_path, nullptr, context->getFileProgressCallback()); +} + + +} diff --git a/src/Processors/QueryPlan/ReadFromObjectStorageStep.h b/src/Processors/QueryPlan/ReadFromObjectStorageStep.h new file mode 100644 index 000000000000..1fe3f6b21b01 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromObjectStorageStep.h @@ -0,0 +1,54 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadFromObjectStorageStep : public SourceStepWithFilter +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + ReadFromObjectStorageStep( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const Names & columns_to_read, + const NamesAndTypesList & virtual_columns_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const std::optional & format_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_); + + std::string getName() const override { return name; } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + +private: + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::shared_ptr iterator_wrapper; + + const ReadFromFormatInfo info; + const NamesAndTypesList virtual_columns; + const std::optional format_settings; + const std::string name; + const bool need_only_count; + const size_t max_block_size; + size_t num_streams; + const bool distributed_processing; + + void createIterator(); +}; + +} diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index dc552fad8518..0ba94d243aa8 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -127,20 +127,6 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, { assertInitialized(); - const auto & config = context->getConfigRef(); - const auto & settings = context->getSettingsRef(); - - auto s3_settings = getSettings(config, "s3" /* config_prefix */, context, url.uri_str, settings[Setting::s3_validate_request_settings]); - - if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) - { - s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); - s3_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); - } - - s3_settings->auth_settings.updateIfChanged(auth_settings); - s3_settings->request_settings.updateIfChanged(request_settings); - if (!headers_from_ast.empty()) { s3_settings->auth_settings.headers.insert( @@ -150,11 +136,15 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, auto client = getClient(url, *s3_settings, context, /* for_disk_s3 */false); auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); - auto s3_capabilities = getCapabilitiesFromConfig(config, "s3"); return std::make_shared( - std::move(client), std::move(s3_settings), url, s3_capabilities, - key_generator, "StorageS3", false); + std::move(client), + std::make_unique(*s3_settings), + url, + *s3_capabilities, + key_generator, + "StorageS3", + false); } void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection, ContextPtr context) @@ -168,12 +158,21 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect else url = S3::URI(collection.get("url"), settings[Setting::allow_archive_path_syntax]); - auth_settings[S3AuthSetting::access_key_id] = collection.getOrDefault("access_key_id", ""); - auth_settings[S3AuthSetting::secret_access_key] = collection.getOrDefault("secret_access_key", ""); - auth_settings[S3AuthSetting::use_environment_credentials] = collection.getOrDefault("use_environment_credentials", 1); - auth_settings[S3AuthSetting::no_sign_request] = collection.getOrDefault("no_sign_request", false); - auth_settings[S3AuthSetting::expiration_window_seconds] = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); - auth_settings[S3AuthSetting::session_token] = collection.getOrDefault("session_token", ""); + const auto & config = context->getConfigRef(); + s3_settings = getSettings(config, "s3" /* config_prefix */, context, url.uri_str, settings[Setting::s3_validate_request_settings]); + + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) + { + s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); + s3_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); + } + + s3_settings->auth_settings[S3AuthSetting::access_key_id] = collection.getOrDefault("access_key_id", ""); + s3_settings->auth_settings[S3AuthSetting::secret_access_key] = collection.getOrDefault("secret_access_key", ""); + s3_settings->auth_settings[S3AuthSetting::use_environment_credentials] = collection.getOrDefault("use_environment_credentials", 1); + s3_settings->auth_settings[S3AuthSetting::no_sign_request] = collection.getOrDefault("no_sign_request", false); + s3_settings->auth_settings[S3AuthSetting::expiration_window_seconds] = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); + s3_settings->auth_settings[S3AuthSetting::session_token] = collection.getOrDefault("session_token", ""); if (collection.has("partition_strategy")) { @@ -194,11 +193,14 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); - request_settings = S3::S3RequestSettings(collection, settings, /* validate_settings */true); + s3_settings->request_settings = S3::S3RequestSettings(collection, settings, /* validate_settings */true); - static_configuration = !auth_settings[S3AuthSetting::access_key_id].value.empty() || auth_settings[S3AuthSetting::no_sign_request].changed; + static_configuration = !s3_settings->auth_settings[S3AuthSetting::access_key_id].value.empty() || s3_settings->auth_settings[S3AuthSetting::no_sign_request].changed; + + s3_capabilities = std::make_unique(getCapabilitiesFromConfig(config, "s3")); keys = {url.key}; + } void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) @@ -211,6 +213,9 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ getMaxNumberOfArguments(with_structure), getSignatures(with_structure)); + const auto & config = context->getConfigRef(); + s3_capabilities = std::make_unique(getCapabilitiesFromConfig(config, "s3")); + std::unordered_map engine_args_to_idx; bool no_sign_request = false; @@ -423,6 +428,14 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ /// This argument is always the first url = S3::URI(checkAndGetLiteralArgument(args[0], "url"), context->getSettingsRef()[Setting::allow_archive_path_syntax]); + s3_settings = getSettings(config, "s3" /* config_prefix */, context, url.uri_str, context->getSettingsRef()[Setting::s3_validate_request_settings]); + + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) + { + s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); + s3_settings->request_settings.updateIfChanged(endpoint_settings->request_settings); + } + if (engine_args_to_idx.contains("format")) { format = checkAndGetLiteralArgument(args[engine_args_to_idx["format"]], "format"); @@ -457,18 +470,19 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ partition_columns_in_data_file = partition_strategy_type != PartitionStrategyFactory::StrategyType::HIVE; if (engine_args_to_idx.contains("access_key_id")) - auth_settings[S3AuthSetting::access_key_id] = checkAndGetLiteralArgument(args[engine_args_to_idx["access_key_id"]], "access_key_id"); + s3_settings->auth_settings[S3AuthSetting::access_key_id] = checkAndGetLiteralArgument(args[engine_args_to_idx["access_key_id"]], "access_key_id"); if (engine_args_to_idx.contains("secret_access_key")) - auth_settings[S3AuthSetting::secret_access_key] = checkAndGetLiteralArgument(args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); + s3_settings->auth_settings[S3AuthSetting::secret_access_key] = checkAndGetLiteralArgument(args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); if (engine_args_to_idx.contains("session_token")) - auth_settings[S3AuthSetting::session_token] = checkAndGetLiteralArgument(args[engine_args_to_idx["session_token"]], "session_token"); + s3_settings->auth_settings[S3AuthSetting::session_token] = checkAndGetLiteralArgument(args[engine_args_to_idx["session_token"]], "session_token"); if (no_sign_request) - auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; + s3_settings->auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; - static_configuration = !auth_settings[S3AuthSetting::access_key_id].value.empty() || auth_settings[S3AuthSetting::no_sign_request].changed; + static_configuration = !s3_settings->auth_settings[S3AuthSetting::access_key_id].value.empty() || s3_settings->auth_settings[S3AuthSetting::no_sign_request].changed; + s3_settings->auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; keys = {url.key}; } diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 2804fd2a1104..14b07c249515 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -5,6 +5,7 @@ #if USE_AWS_S3 #include #include +#include #include namespace DB @@ -73,7 +74,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; } S3::URI getURL() const { return url; } - const S3::S3AuthSettings & getAuthSettings() const { return auth_settings; } + const S3::S3AuthSettings & getAuthSettings() const { return s3_settings->auth_settings; } Path getRawPath() const override { return url.key; } @@ -107,8 +108,9 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration S3::URI url; Paths keys; - S3::S3AuthSettings auth_settings; - S3::S3RequestSettings request_settings; + std::unique_ptr s3_settings; + std::unique_ptr s3_capabilities; + HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f910691767aa..06db9350c79e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include #include @@ -341,121 +342,6 @@ std::optional StorageObjectStorage::totalBytes(ContextPtr query_context) return configuration->totalBytes(query_context); } -namespace -{ -class ReadFromObjectStorageStep : public SourceStepWithFilter -{ -public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - - ReadFromObjectStorageStep( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - const String & name_, - const Names & columns_to_read, - const NamesAndTypesList & virtual_columns_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const std::optional & format_settings_, - bool distributed_processing_, - ReadFromFormatInfo info_, - const bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(info_.source_header, columns_to_read, query_info_, storage_snapshot_, context_) - , object_storage(object_storage_) - , configuration(configuration_) - , info(std::move(info_)) - , virtual_columns(virtual_columns_) - , format_settings(format_settings_) - , name(name_ + "Source") - , need_only_count(need_only_count_) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - , distributed_processing(distributed_processing_) - { - } - - std::string getName() const override { return name; } - - void applyFilters(ActionDAGNodes added_filter_nodes) override - { - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - createIterator(); - } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override - { - createIterator(); - - Pipes pipes; - auto context = getContext(); - const size_t max_threads = context->getSettingsRef()[Setting::max_threads]; - size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); - - if (estimated_keys_count > 1) - num_streams = std::min(num_streams, estimated_keys_count); - else - { - /// The amount of keys (zero) was probably underestimated. - /// We will keep one stream for this particular case. - num_streams = 1; - } - - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); - - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - getName(), object_storage, configuration, info, format_settings, - context, max_block_size, iterator_wrapper, max_parsing_threads, need_only_count); - - source->setKeyCondition(filter_actions_dag, context); - pipes.emplace_back(std::move(source)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); - } - -private: - ObjectStoragePtr object_storage; - ConfigurationPtr configuration; - std::shared_ptr iterator_wrapper; - - const ReadFromFormatInfo info; - const NamesAndTypesList virtual_columns; - const std::optional format_settings; - const String name; - const bool need_only_count; - const size_t max_block_size; - size_t num_streams; - const bool distributed_processing; - - void createIterator() - { - if (iterator_wrapper) - return; - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag.has_value()) - predicate = filter_actions_dag->getOutputs().at(0); - - auto context = getContext(); - iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, - context, predicate, filter_actions_dag, virtual_columns, info.hive_partition_columns_to_read_from_file_path, nullptr, context->getFileProgressCallback()); - } -}; -} - ReadFromFormatInfo StorageObjectStorage::Configuration::prepareReadingFromFormat( ObjectStoragePtr, const Strings & requested_columns, diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index f733018d650a..e0e957b70653 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -1725,7 +1725,7 @@ def test_explanation(started_cluster, format_version, storage_type): [ "Expression ((Project names + (Projection + Change column names to column identifiers)))" ], - [f" Iceberg{storage_type.title()}(default.{TABLE_NAME})Source"], + [f" Iceberg{storage_type.title()}(default.{TABLE_NAME})ReadStep"], ] assert res == expected diff --git a/tests/queries/0_stateless/03275_auto_cluster_functions_with_parallel_replicas.reference b/tests/queries/0_stateless/03275_auto_cluster_functions_with_parallel_replicas.reference index 2616b41f689e..c0cd8408e9b0 100644 --- a/tests/queries/0_stateless/03275_auto_cluster_functions_with_parallel_replicas.reference +++ b/tests/queries/0_stateless/03275_auto_cluster_functions_with_parallel_replicas.reference @@ -5,11 +5,11 @@ Expression (Project names) CreatingSets (Create sets before main query execution) Expression ((Project names + Projection)) Filter ((WHERE + Change column names to column identifiers)) - S3(_table_function.s3)Source + S3(_table_function.s3)ReadStep CreatingSet (Create set for subquery) Expression ((Project names + (Projection + Change column names to column identifiers))) - S3(_table_function.s3)Source + S3(_table_function.s3)ReadStep Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromURL Expression ((Project names + (Projection + Change column names to column identifiers))) - S3(_table_function.s3)Source + S3(_table_function.s3)ReadStep From 2d9c9fa744ee8efacd727d3b0b460024d4f26fd8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jun 2025 17:02:14 +0000 Subject: [PATCH 2/9] Merge pull request #82307 from azat/fix-stress-tests ci: fix stress tests (disable auth injection in keeper) --- tests/config/install.sh | 29 +++++++++++++++++++++++---- tests/docker_scripts/stress_tests.lib | 5 +++-- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 07758307ea1a..4f28eb5b21c6 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -18,6 +18,7 @@ USE_AZURE_STORAGE_FOR_MERGE_TREE=${USE_AZURE_STORAGE_FOR_MERGE_TREE:0} USE_ASYNC_INSERT=${USE_ASYNC_INSERT:0} BUGFIX_VALIDATE_CHECK=0 NO_AZURE=0 +KEEPER_INJECT_AUTH=1 while [[ "$#" -gt 0 ]]; do case $1 in @@ -36,6 +37,8 @@ while [[ "$#" -gt 0 ]]; do --async-insert) USE_ASYNC_INSERT=1 ;; --bugfix-validation) BUGFIX_VALIDATE_CHECK=1 ;; + + --no-keeper-inject-auth) KEEPER_INJECT_AUTH=0 ;; *) echo "Unknown option: $1" ; exit 1 ;; esac shift @@ -213,18 +216,29 @@ else fi # We randomize creating the snapshot on exit for Keeper to test out using older snapshots -value=$(($RANDOM % 2)) +value=$((RANDOM % 2)) +echo "Replacing create_snapshot_on_exit with $value" sed --follow-symlinks -i "s|[01]|$value|" $DEST_SERVER_PATH/config.d/keeper_port.xml -value=$((($RANDOM + 100) * 2048)) +value=$(((RANDOM + 100) * 2048)) +echo "Replacing latest_logs_cache_size_threshold with $value" sed --follow-symlinks -i "s|[[:digit:]]\+|$value|" $DEST_SERVER_PATH/config.d/keeper_port.xml -value=$((($RANDOM + 100) * 2048)) +value=$(((RANDOM + 100) * 2048)) +echo "Replacing commit_logs_cache_size_threshold with $value" sed --follow-symlinks -i "s|[[:digit:]]\+|$value|" $DEST_SERVER_PATH/config.d/keeper_port.xml -value=$(($RANDOM % 2)) +value=$((RANDOM % 2)) +echo "Replacing digest_enabled_on_commit with $value" sed --follow-symlinks -i "s|[01]|$value|" $DEST_SERVER_PATH/config.d/keeper_port.xml +value=$((RANDOM % 2)) +if [[ $KEEPER_INJECT_AUTH -eq 0 ]]; then + value=0 +fi +echo "Replacing inject_auth with $value" +sed --follow-symlinks -i "s|[01]|$value|" $DEST_SERVER_PATH/config.d/keeper_port.xml + if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -sf $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ fi @@ -294,6 +308,13 @@ if [[ "$USE_DATABASE_REPLICATED" == "1" ]]; then rm $DEST_SERVER_PATH/config.d/zookeeper.xml rm $DEST_SERVER_PATH/config.d/keeper_port.xml + value=$((RANDOM % 2)) + if [[ $KEEPER_INJECT_AUTH -eq 0 ]]; then + value=0 + fi + echo "Replacing inject_auth with $value (for Replicated database)" + sed --follow-symlinks -i "s|[01]|$value|" $DEST_SERVER_PATH/config.d/database_replicated.xml + # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 # And we have to copy configs... ch_server_1_path=$DEST_SERVER_PATH/../clickhouse-server1 diff --git a/tests/docker_scripts/stress_tests.lib b/tests/docker_scripts/stress_tests.lib index 59e547304132..17369592192e 100644 --- a/tests/docker_scripts/stress_tests.lib +++ b/tests/docker_scripts/stress_tests.lib @@ -60,7 +60,7 @@ function configure() # install test configs export USE_DATABASE_ORDINARY=1 export EXPORT_S3_STORAGE_POLICIES=1 - /repo/tests/config/install.sh + /repo/tests/config/install.sh /etc/clickhouse-server /etc/clickhouse-client --no-keeper-inject-auth # avoid too slow startup sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ @@ -69,7 +69,8 @@ function configure() sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml function randomize_config_boolean_value { - value=$((RANDOM % 2)) + local value=$((RANDOM % 2)) + echo "Replacing $1 with $value" sudo cat "/etc/clickhouse-server/config.d/$2.xml" \ | sed "s|<$1>[01]|<$1>$value|" \ > "/etc/clickhouse-server/config.d/$2.xml.tmp" From c3cc1d1bc57a34f05f7e99c37a4b6c878c56b98d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 4 Jul 2025 09:19:32 +0000 Subject: [PATCH 3/9] Merge pull request #82604 from ClickHouse/get_rid_of_s3_settings Trying to get rid of second S3Settings --- .../ObjectStorages/ObjectStorageFactory.cpp | 9 +++-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 31 +++++++++------ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 38 ++++--------------- src/Disks/ObjectStorages/S3/diskSettings.cpp | 28 +------------- src/Disks/ObjectStorages/S3/diskSettings.h | 14 ++----- src/IO/S3Defines.h | 2 + src/IO/S3RequestSettings.cpp | 5 ++- src/IO/S3Settings.cpp | 32 ++++++++++++++++ src/IO/S3Settings.h | 16 ++++++++ .../ObjectStorage/S3/Configuration.cpp | 10 +++-- src/Storages/ObjectStorage/S3/Configuration.h | 2 +- 11 files changed, 100 insertions(+), 87 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 39200b1571cb..bd9470eb14f5 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -184,7 +184,8 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); + auto settings = std::make_unique(); + settings->loadFromConfigForObjectStorage(config, config_prefix, context->getSettingsRef(), uri.uri.getScheme(), true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -217,7 +218,8 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); + auto settings = std::make_unique(); + settings->loadFromConfigForObjectStorage(config, config_prefix, context->getSettingsRef(), uri.uri.getScheme(), true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); @@ -248,7 +250,8 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); - auto settings = getSettings(config, config_prefix, context, endpoint, /* validate_settings */true); + auto settings = std::make_unique(); + settings->loadFromConfigForObjectStorage(config, config_prefix, context->getSettingsRef(), uri.uri.getScheme(), true); auto client = getClient(endpoint, *settings, context, /* for_disk_s3 */true); auto key_generator = getKeyGenerator(uri, config, config_prefix); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 23277f868d6a..6d8dacd40fe9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -52,6 +53,13 @@ namespace Setting extern const SettingsBool s3_validate_request_settings; } +namespace S3RequestSetting +{ + extern const S3RequestSettingsUInt64 list_object_keys_size; + extern const S3RequestSettingsUInt64 objects_chunk_size_to_delete; +} + + namespace ErrorCodes { extern const int S3_ERROR; @@ -233,7 +241,7 @@ ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefi { auto settings_ptr = s3_settings.get(); if (!max_keys) - max_keys = settings_ptr->list_object_keys_size; + max_keys = settings_ptr->request_settings[S3RequestSetting::list_object_keys_size]; return std::make_shared(uri.bucket, path_prefix, client.get(), max_keys); } @@ -247,7 +255,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet if (max_keys) request.SetMaxKeys(static_cast(max_keys)); else - request.SetMaxKeys(settings_ptr->list_object_keys_size); + request.SetMaxKeys(settings_ptr->request_settings[S3RequestSetting::list_object_keys_size]); Aws::S3::Model::ListObjectsV2Outcome outcome; do @@ -321,7 +329,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e auto settings_ptr = s3_settings.get(); deleteFilesFromS3(client.get(), uri.bucket, keys, if_exists, - s3_capabilities, settings_ptr->objects_chunk_size_to_delete, + s3_capabilities, settings_ptr->request_settings[S3RequestSetting::objects_chunk_size_to_delete], blob_storage_log, local_paths_for_blob_storage_log, file_sizes_for_blob_storage_log, ProfileEvents::DiskS3DeleteObjects); } @@ -457,11 +465,6 @@ void S3ObjectStorage::copyObject( // NOLINT object_to_attributes); } -void S3ObjectStorage::setNewSettings(std::unique_ptr && s3_settings_) -{ - s3_settings.set(std::move(s3_settings_)); -} - void S3ObjectStorage::shutdown() { /// This call stops any next retry attempts for ongoing S3 requests. @@ -483,12 +486,18 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto settings_from_config - = getSettings(config, config_prefix, context, uri.uri_str, context->getSettingsRef()[Setting::s3_validate_request_settings]); - auto modified_settings = std::make_unique(*s3_settings.get()); + std::unique_ptr settings_from_config = std::make_unique(); + + settings_from_config->loadFromConfigForObjectStorage( + config, config_prefix, context->getSettingsRef(), uri.uri.getScheme(), context->getSettingsRef()[Setting::s3_validate_request_settings]); + + auto modified_settings = std::make_unique(*s3_settings.get()); modified_settings->auth_settings.updateIfChanged(settings_from_config->auth_settings); modified_settings->request_settings.updateIfChanged(settings_from_config->request_settings); + modified_settings->request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(uri.uri.getScheme()), config_prefix, config); + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) { modified_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index f8811785ed39..2283ebe54c06 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -15,33 +15,11 @@ namespace DB { -struct S3ObjectStorageSettings +namespace S3RequestSetting { - S3ObjectStorageSettings() = default; - - S3ObjectStorageSettings( - const S3::S3RequestSettings & request_settings_, - const S3::S3AuthSettings & auth_settings_, - uint64_t min_bytes_for_seek_, - int32_t list_object_keys_size_, - int32_t objects_chunk_size_to_delete_, - bool read_only_) - : request_settings(request_settings_) - , auth_settings(auth_settings_) - , min_bytes_for_seek(min_bytes_for_seek_) - , list_object_keys_size(list_object_keys_size_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) - , read_only(read_only_) - {} - - S3::S3RequestSettings request_settings; - S3::S3AuthSettings auth_settings; - - uint64_t min_bytes_for_seek; - int32_t list_object_keys_size; - int32_t objects_chunk_size_to_delete; - bool read_only; -}; + extern const S3RequestSettingsBool read_only; +} + class S3ObjectStorage : public IObjectStorage { @@ -49,7 +27,7 @@ class S3ObjectStorage : public IObjectStorage S3ObjectStorage( const char * logger_name, std::unique_ptr && client_, - std::unique_ptr && s3_settings_, + std::unique_ptr && s3_settings_, S3::URI uri_, const S3Capabilities & s3_capabilities_, ObjectStorageKeysGeneratorPtr key_generator_, @@ -153,13 +131,11 @@ class S3ObjectStorage : public IObjectStorage bool areObjectKeysRandom() const override; - bool isReadOnly() const override { return s3_settings.get()->read_only; } + bool isReadOnly() const override { return s3_settings.get()->request_settings[S3RequestSetting::read_only]; } std::shared_ptr getS3StorageClient() override; std::shared_ptr tryGetS3StorageClient() override; private: - void setNewSettings(std::unique_ptr && s3_settings_); - void removeObjectImpl(const StoredObject & object, bool if_exists); void removeObjectsImpl(const StoredObjects & objects, bool if_exists); @@ -168,7 +144,7 @@ class S3ObjectStorage : public IObjectStorage std::string disk_name; MultiVersion client; - MultiVersion s3_settings; + MultiVersion s3_settings; S3Capabilities s3_capabilities; ObjectStorageKeysGeneratorPtr key_generator; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 6117827868ee..f0c9447c55b3 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -60,33 +60,9 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; } -std::unique_ptr getSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context, - const std::string & endpoint, - bool validate_settings) -{ - const auto & settings = context->getSettingsRef(); - - auto auth_settings = S3::S3AuthSettings(config, settings, config_prefix); - auto request_settings = S3::S3RequestSettings(config, settings, config_prefix, "s3_", validate_settings); - - request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( - ProxyConfiguration::protocolFromString(S3::URI(endpoint).uri.getScheme()), config_prefix, config); - - return std::make_unique( - request_settings, - auth_settings, - config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), - config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), - config.getBool(config_prefix + ".readonly", false)); -} - std::unique_ptr getClient( const std::string & endpoint, - const S3ObjectStorageSettings & settings, + const S3Settings & settings, ContextPtr context, bool for_disk_s3) { @@ -98,7 +74,7 @@ std::unique_ptr getClient( std::unique_ptr getClient( const S3::URI & url, - const S3ObjectStorageSettings & settings, + const S3Settings & settings, ContextPtr context, bool for_disk_s3) { diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index aa427bee41ae..26023c345124 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -6,30 +6,22 @@ #include #include +#include #include namespace DB { -struct S3ObjectStorageSettings; - -std::unique_ptr getSettings( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context, - const std::string & endpoint, - bool validate_settings); - std::unique_ptr getClient( const std::string & endpoint, - const S3ObjectStorageSettings & settings, + const S3Settings & settings, ContextPtr context, bool for_disk_s3); std::unique_ptr getClient( const S3::URI & url_, - const S3ObjectStorageSettings & settings, + const S3Settings & settings, ContextPtr context, bool for_disk_s3); diff --git a/src/IO/S3Defines.h b/src/IO/S3Defines.h index 332ebcfea92d..4a45e19f8a31 100644 --- a/src/IO/S3Defines.h +++ b/src/IO/S3Defines.h @@ -34,6 +34,8 @@ inline static constexpr uint64_t DEFAULT_MAX_SINGLE_READ_TRIES = 4; inline static constexpr uint64_t DEFAULT_MAX_UNEXPECTED_WRITE_ERROR_RETRIES = 4; inline static constexpr uint64_t DEFAULT_MAX_REDIRECTS = 10; inline static constexpr uint64_t DEFAULT_RETRY_ATTEMPTS = 100; +inline static constexpr uint64_t DEFAULT_MIN_BYTES_FOR_SEEK = 1024 * 1024; +inline static constexpr uint64_t DEFAULT_OBJECTS_CHUNK_SIZE_TO_DELETE = 1000; inline static constexpr bool DEFAULT_ALLOW_NATIVE_COPY = true; inline static constexpr bool DEFAULT_CHECK_OBJECTS_AFTER_UPLOAD = false; diff --git a/src/IO/S3RequestSettings.cpp b/src/IO/S3RequestSettings.cpp index f483c5adcaf7..d823dc5cc5c6 100644 --- a/src/IO/S3RequestSettings.cpp +++ b/src/IO/S3RequestSettings.cpp @@ -41,7 +41,10 @@ namespace ErrorCodes DECLARE(String, storage_class_name, "", "", 0) \ DECLARE(UInt64, http_max_fields, 1000000, "", 0) \ DECLARE(UInt64, http_max_field_name_size, 128 * 1024, "", 0) \ - DECLARE(UInt64, http_max_field_value_size, 128 * 1024, "", 0) + DECLARE(UInt64, http_max_field_value_size, 128 * 1024, "", 0) \ + DECLARE(UInt64, min_bytes_for_seek, S3::DEFAULT_MIN_BYTES_FOR_SEEK, "", 0) \ + DECLARE(UInt64, objects_chunk_size_to_delete, S3::DEFAULT_OBJECTS_CHUNK_SIZE_TO_DELETE, "", 0) \ + DECLARE(Bool, read_only, false, "", 0) #define PART_UPLOAD_SETTINGS(DECLARE, ALIAS) \ DECLARE(UInt64, strict_upload_part_size, 0, "", 0) \ diff --git a/src/IO/S3Settings.cpp b/src/IO/S3Settings.cpp index 431e770fd3bf..11c3a17160af 100644 --- a/src/IO/S3Settings.cpp +++ b/src/IO/S3Settings.cpp @@ -4,6 +4,7 @@ #include #include +#include #include @@ -14,6 +15,15 @@ namespace Setting extern const SettingsBool s3_validate_request_settings; } +namespace S3RequestSetting +{ + extern const S3RequestSettingsBool read_only; + extern const S3RequestSettingsUInt64 min_bytes_for_seek; + extern const S3RequestSettingsUInt64 list_object_keys_size; + extern const S3RequestSettingsUInt64 objects_chunk_size_to_delete; +} + + void S3Settings::loadFromConfig( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, @@ -23,6 +33,28 @@ void S3Settings::loadFromConfig( request_settings = S3::S3RequestSettings(config, settings, config_prefix); } +void S3Settings::loadFromConfigForObjectStorage( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + const std::string & scheme, + bool validate_settings) +{ + auth_settings = S3::S3AuthSettings(config, settings, config_prefix); + request_settings = S3::S3RequestSettings(config, settings, config_prefix, "s3_", validate_settings); + + request_settings.proxy_resolver = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(scheme), config_prefix, config); + + /// We override these request settings from configuration, because they are related to disk configuration, + /// which shouldn't be changed from user query + request_settings[S3RequestSetting::read_only] = config.getBool(config_prefix + ".readonly", false); + request_settings[S3RequestSetting::min_bytes_for_seek] = config.getUInt64(config_prefix + ".min_bytes_for_seek", S3::DEFAULT_MIN_BYTES_FOR_SEEK); + request_settings[S3RequestSetting::list_object_keys_size] = config.getUInt64(config_prefix + ".list_object_keys_size", S3::DEFAULT_LIST_OBJECT_KEYS_SIZE); + request_settings[S3RequestSetting::objects_chunk_size_to_delete] = config.getUInt(config_prefix + ".objects_chunk_size_to_delete", S3::DEFAULT_OBJECTS_CHUNK_SIZE_TO_DELETE); +} + + void S3Settings::updateIfChanged(const S3Settings & settings) { auth_settings.updateIfChanged(settings.auth_settings); diff --git a/src/IO/S3Settings.h b/src/IO/S3Settings.h index c58903324f6d..3309c2118211 100644 --- a/src/IO/S3Settings.h +++ b/src/IO/S3Settings.h @@ -24,11 +24,27 @@ struct S3Settings S3::S3AuthSettings auth_settings; S3::S3RequestSettings request_settings; + /// Read all settings with from specified prefix in config. Expect settings name to start with "s3_". + /// This method is useful when you need to get S3Settings for standalone S3 Client. void loadFromConfig( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DB::Settings & settings); + /// Read all settings with from specified prefix in config. Expect settings name to start with "s3_". + /// Override 4 settings: readonly, min_bytes_for_seek, list_object_keys_size, objects_chunk_size_to_delete from + /// settings values from config without "s3_" prefix. This method exists for historical reasons. Initially we added + /// some settings which don't directly affect S3 requests in terms of S3 API, but they affect ClickHouse logic + /// when it reads from ObjectStorage. + /// + /// This method is useful when you need to get S3Settings for Disk or ObjectStorage, not a standalone S3Client. + void loadFromConfigForObjectStorage( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const DB::Settings & settings, + const std::string & scheme, + bool validate_settings); + void updateIfChanged(const S3Settings & settings); }; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 0ba94d243aa8..02371dec4211 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include @@ -139,7 +140,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, return std::make_shared( std::move(client), - std::make_unique(*s3_settings), + std::make_unique(*s3_settings), url, *s3_capabilities, key_generator, @@ -159,7 +160,9 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect url = S3::URI(collection.get("url"), settings[Setting::allow_archive_path_syntax]); const auto & config = context->getConfigRef(); - s3_settings = getSettings(config, "s3" /* config_prefix */, context, url.uri_str, settings[Setting::s3_validate_request_settings]); + + s3_settings = std::make_unique(); + s3_settings->loadFromConfigForObjectStorage(config, "s3", context->getSettingsRef(), url.uri.getScheme(), context->getSettingsRef()[Setting::s3_validate_request_settings]); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) { @@ -428,7 +431,8 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ /// This argument is always the first url = S3::URI(checkAndGetLiteralArgument(args[0], "url"), context->getSettingsRef()[Setting::allow_archive_path_syntax]); - s3_settings = getSettings(config, "s3" /* config_prefix */, context, url.uri_str, context->getSettingsRef()[Setting::s3_validate_request_settings]); + s3_settings = std::make_unique(); + s3_settings->loadFromConfigForObjectStorage(config, "s3", context->getSettingsRef(), url.uri.getScheme(), context->getSettingsRef()[Setting::s3_validate_request_settings]); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) { diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 14b07c249515..170e510aa49b 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -108,7 +108,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration S3::URI url; Paths keys; - std::unique_ptr s3_settings; + std::unique_ptr s3_settings; std::unique_ptr s3_capabilities; HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. From 6c5cfd9ebc5a53f46284cf6c8791711b88041be5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 24 Jul 2025 11:06:21 +0000 Subject: [PATCH 4/9] Merge pull request #59898 from ClickHouse/encryption-s3-by-default Use encrypted disks for tests with S3 by default --- ci/jobs/functional_tests.py | 8 + src/Backups/BackupIO_S3.cpp | 6 +- src/Disks/DiskEncrypted.h | 3 +- src/Disks/DiskEncryptedTransaction.h | 14 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 4 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 23 ++ src/Disks/MetadataStorageWithPathWrapper.h | 262 ++++++++++++++++++ .../ObjectStorages/MetadataStorageFromDisk.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +- tests/clickhouse-test | 14 + ...re_encrypted_storage_policy_by_default.xml | 6 + tests/config/config.d/azure_storage_conf.xml | 16 ++ .../azure_storage_policy_by_default.xml | 1 + ...orage_policy_for_merge_tree_by_default.xml | 6 + .../config.d/s3_storage_policy_by_default.xml | 15 +- ...orage_policy_for_merge_tree_by_default.xml | 6 + ...torage_policy_with_template_object_key.xml | 15 +- tests/config/config.d/transactions.xml | 9 - .../config/config.d/transactions_info_log.xml | 7 + tests/config/install.sh | 49 +++- .../0_stateless/01133_begin_commit_race.sh | 2 +- .../0_stateless/01167_isolation_hermitage.sh | 2 +- .../0_stateless/01168_mutations_isolation.sh | 2 +- .../01169_alter_partition_isolation_stress.sh | 2 +- ...69_old_alter_partition_isolation_stress.sh | 2 +- .../01170_alter_partition_isolation.sh | 2 +- .../01171_mv_select_insert_isolation_long.sh | 2 +- .../01172_transaction_counters.sql | 2 +- .../01173_transaction_control_queries.sql | 2 +- .../01174_select_insert_isolation.sh | 2 +- .../02345_implicit_transaction.sql | 2 +- .../02421_truncate_isolation_no_merges.sh | 2 +- ...02421_truncate_isolation_with_mutations.sh | 2 +- .../02435_rollback_cancelled_queries.sh | 2 +- ...97_source_part_is_intact_when_mutation.sql | 2 +- .../02980_s3_plain_DROP_TABLE_MergeTree.sh | 2 +- ...s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 2 +- ...03167_transactions_are_really_disabled.sql | 2 + ...se_backup_merge_tree_table_file_engine.sql | 2 +- ...7_database_backup_database_file_engine.sql | 2 +- ...se_backup_merge_tree_table_disk_engine.sql | 2 +- ...9_database_backup_database_disk_engine.sql | 2 +- .../03282_memory_transaction_crash.sql | 2 +- ...med_session_try_recreate_before_timeout.sh | 1 + 44 files changed, 450 insertions(+), 66 deletions(-) create mode 100644 src/Disks/MetadataStorageWithPathWrapper.h create mode 100644 tests/config/config.d/azure_encrypted_storage_policy_by_default.xml create mode 100644 tests/config/config.d/s3_encrypted_storage_policy_for_merge_tree_by_default.xml create mode 100644 tests/config/config.d/s3_storage_policy_for_merge_tree_by_default.xml create mode 100644 tests/config/config.d/transactions_info_log.xml diff --git a/ci/jobs/functional_tests.py b/ci/jobs/functional_tests.py index 31b1b506fc16..3c73fefe5e20 100644 --- a/ci/jobs/functional_tests.py +++ b/ci/jobs/functional_tests.py @@ -136,8 +136,10 @@ def main(): is_flaky_check = False is_bugfix_validation = False is_s3_storage = False + is_azure_storage = False is_database_replicated = False is_shared_catalog = False + is_encrypted_storage = True # randomize it runner_options = "" info = Info() @@ -163,6 +165,8 @@ def main(): if "s3 storage" in to: is_s3_storage = True + if "azure" in to: + is_azure_storage = True if "DatabaseReplicated" in to: is_database_replicated = True if "SharedCatalog" in to: @@ -180,6 +184,10 @@ def main(): print("Disable azure for a local run") config_installs_args += " --no-azure" + if (is_azure_storage or is_s3_storage) and is_encrypted_storage: + config_installs_args += " --encrypted-storage" + runner_options += f" --encrypted-storage" + ch_path = args.ch_path stop_watch = Utils.Stopwatch() diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 0a1faac4240e..69305917c48b 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -308,6 +308,10 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src [&] { LOG_TRACE(log, "Falling back to copy file {} from disk {} to S3 through buffers", src_path, src_disk->getName()); + + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, read_settings); + return src_disk->readFile(src_path, read_settings); }); return; /// copied! @@ -320,7 +324,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src void BackupWriterS3::copyFile(const String & destination, const String & source, size_t size) { - LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); + LOG_TRACE(log, "Copying file inside backup from {} to {}", source, destination); const auto source_key = fs::path(s3_uri.key) / source; copyS3File( diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 6791dad46f31..f920d21205f1 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -357,7 +358,7 @@ class DiskEncrypted : public IDisk MetadataStoragePtr getMetadataStorage() override { - return delegate->getMetadataStorage(); + return std::make_shared(delegate->getMetadataStorage(), disk_path); } std::unordered_map getSerializedMetadata(const std::vector & paths) const override; diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 26f4c979bf87..2e1fc4f7c38c 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -188,15 +188,11 @@ class DiskEncryptedTransaction : public IDiskTransaction /// Third param determines which files cannot be removed even if second is true. void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override { - for (const auto & file : files) - { - auto wrapped_path = wrappedPath(file.path); - bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); - if (file.if_exists) - delegate_transaction->removeSharedFileIfExists(wrapped_path, keep); - else - delegate_transaction->removeSharedFile(wrapped_path, keep); - } + auto wrapped_path_files = files; + for (auto & file : wrapped_path_files) + file.path = wrappedPath(file.path); + + delegate_transaction->removeSharedFiles(wrapped_path_files, keep_all_batch_data, file_names_remove_metadata_only); } /// Set last modified time to file or directory at `path`. diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 64638d3c16d2..d63fb8a17a42 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -821,7 +821,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { last_caller_id = FileSegment::getCallerId(); - chassert(file_offset_of_buffer_end <= read_until_position); + chassert( + file_offset_of_buffer_end <= read_until_position, + fmt::format("file_offset_of_buffer_end {}, read_until_position {}", file_offset_of_buffer_end, read_until_position)); if (file_offset_of_buffer_end == read_until_position) return false; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 519d1874f717..4e1e40c11b88 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -16,6 +16,8 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int LOGICAL_ERROR; + } ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( @@ -135,6 +137,27 @@ void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) if (position == read_until_position) return; + if (!use_external_buffer && position < file_offset_of_buffer_end) + { + /// file has been read beyond new read until position already + if (available() >= file_offset_of_buffer_end - position) + { + /// new read until position is after the current position in the working buffer + working_buffer.resize(working_buffer.size() - (file_offset_of_buffer_end - position)); + file_offset_of_buffer_end = position; + pos = std::min(pos, working_buffer.end()); + } + else + { + /// new read until position is before the current position in the working buffer + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to set read until position before already read data ({} > {})", + position, + getPosition()); + } + } + reset(); read_until_position = position; } diff --git a/src/Disks/MetadataStorageWithPathWrapper.h b/src/Disks/MetadataStorageWithPathWrapper.h new file mode 100644 index 000000000000..7d9cceaee840 --- /dev/null +++ b/src/Disks/MetadataStorageWithPathWrapper.h @@ -0,0 +1,262 @@ +#pragma once + +#include "config.h" + +#if USE_SSL + +#include +#include + +namespace DB +{ + +class MetadataStorageWithPathWrapperTransaction final : public IMetadataTransaction +{ +private: + MetadataTransactionPtr delegate; + std::string metadata_path; + + String wrappedPath(const String & path) const { return wrappedPath(metadata_path, path); } + +public: + static String wrappedPath(const String & path_wrapper, const String & path) + { + // if path starts_with metadata_path -> got already wrapped path + if (!path_wrapper.empty() && path.starts_with(path_wrapper)) + return path; + return path_wrapper + path; + } + + MetadataStorageWithPathWrapperTransaction(MetadataTransactionPtr delegate_, const std::string & metadata_path_) + : delegate(std::move(delegate_)) + , metadata_path(metadata_path_) + { + } + + ~MetadataStorageWithPathWrapperTransaction() override = default; + + const IMetadataStorage & getStorageForNonTransactionalReads() const final + { + return delegate->getStorageForNonTransactionalReads(); + } + + void commit() final + { + delegate->commit(); + } + + void writeStringToFile(const std::string & path, const std::string & data) override + { + delegate->writeStringToFile(wrappedPath(path), data); + } + + void writeInlineDataToFile(const std::string & path, const std::string & data) override + { + delegate->writeInlineDataToFile(wrappedPath(path), data); + } + + void createEmptyMetadataFile(const std::string & path) override + { + delegate->createEmptyMetadataFile(wrappedPath(path)); + } + + void createMetadataFile(const std::string & path, ObjectStorageKey object_key, uint64_t size_in_bytes) override + { + delegate->createMetadataFile(wrappedPath(path), object_key, size_in_bytes); + } + + void addBlobToMetadata(const std::string & path, ObjectStorageKey object_key, uint64_t size_in_bytes) override + { + delegate->addBlobToMetadata(wrappedPath(path), object_key, size_in_bytes); + } + + void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override + { + delegate->setLastModified(wrappedPath(path), timestamp); + } + + bool supportsChmod() const override { return delegate->supportsChmod(); } + + void chmod(const String & path, mode_t mode) override + { + delegate->chmod(wrappedPath(path), mode); + } + + void setReadOnly(const std::string & path) override + { + delegate->setReadOnly(wrappedPath(path)); + } + + void unlinkFile(const std::string & path) override + { + delegate->unlinkFile(wrappedPath(path)); + } + + void createDirectory(const std::string & path) override + { + delegate->createDirectory(wrappedPath(path)); + } + + void createDirectoryRecursive(const std::string & path) override + { + delegate->createDirectoryRecursive(wrappedPath(path)); + } + + void removeDirectory(const std::string & path) override + { + delegate->removeDirectory(wrappedPath(path)); + } + + void removeRecursive(const std::string & path) override + { + delegate->removeRecursive(wrappedPath(path)); + } + + void createHardLink(const std::string & path_from, const std::string & path_to) override + { + delegate->createHardLink(wrappedPath(path_from), wrappedPath(path_to)); + } + + void moveFile(const std::string & path_from, const std::string & path_to) override + { + delegate->moveFile(wrappedPath(path_from), wrappedPath(path_to)); + } + + void moveDirectory(const std::string & path_from, const std::string & path_to) override + { + delegate->moveDirectory(wrappedPath(path_from), wrappedPath(path_to)); + } + + void replaceFile(const std::string & path_from, const std::string & path_to) override + { + delegate->replaceFile(wrappedPath(path_from), wrappedPath(path_to)); + } + + UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override + { + return delegate->unlinkMetadata(wrappedPath(path)); + } + + TruncateFileOperationOutcomePtr truncateFile(const std::string & src_path, size_t target_size) override + { + return delegate->truncateFile(wrappedPath(src_path), target_size); + } +}; + +class MetadataStorageWithPathWrapper final : public IMetadataStorage +{ +private: + MetadataStoragePtr delegate; + std::string metadata_path; + std::string metadata_absolute_path; + + String wrappedPath(const String & path) const { return MetadataStorageWithPathWrapperTransaction::wrappedPath(metadata_path, path); } + +public: + MetadataStorageWithPathWrapper(MetadataStoragePtr delegate_, const std::string & metadata_path_) + : delegate(std::move(delegate_)) + , metadata_path(metadata_path_) + , metadata_absolute_path(wrappedPath(delegate->getPath())) + { + } + + MetadataTransactionPtr createTransaction() override + { + return std::make_shared(delegate->createTransaction(), metadata_path); + } + + const std::string & getPath() const override + { + return metadata_absolute_path; + } + + MetadataStorageType getType() const override { return delegate->getType(); } + + /// Metadata on disk for an empty file can store empty list of blobs and size=0 + bool supportsEmptyFilesWithoutBlobs() const override { return delegate->supportsEmptyFilesWithoutBlobs(); } + + bool existsFile(const std::string & path) const override + { + return delegate->existsFile(wrappedPath(path)); + } + + bool existsDirectory(const std::string & path) const override + { + return delegate->existsDirectory(wrappedPath(path)); + } + + bool existsFileOrDirectory(const std::string & path) const override + { + return delegate->existsFileOrDirectory(wrappedPath(path)); + } + + uint64_t getFileSize(const String & path) const override + { + return delegate->getFileSize(wrappedPath(path)); + } + + Poco::Timestamp getLastModified(const std::string & path) const override + { + return delegate->getLastModified(wrappedPath(path)); + } + + time_t getLastChanged(const std::string & path) const override + { + return delegate->getLastChanged(wrappedPath(path)); + } + + bool supportsChmod() const override { return delegate->supportsChmod(); } + + bool supportsStat() const override { return delegate->supportsStat(); } + + bool supportsPartitionCommand(const PartitionCommand & command) const override + { + return delegate->supportsPartitionCommand(command); + } + + struct stat stat(const String & path) const override { return delegate->stat(wrappedPath(path)); } + + std::vector listDirectory(const std::string & path) const override + { + return delegate->listDirectory(wrappedPath(path)); + } + + DirectoryIteratorPtr iterateDirectory(const std::string & path) const override + { + return delegate->iterateDirectory(wrappedPath(path)); + } + + std::string readFileToString(const std::string & path) const override + { + return delegate->readFileToString(wrappedPath(path)); + } + + std::string readInlineDataToString(const std::string & path) const override + { + return delegate->readInlineDataToString(wrappedPath(path)); + } + + std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override + { + std::vector wrapped_paths; + wrapped_paths.reserve(file_paths.size()); + for (const auto & path : file_paths) + wrapped_paths.push_back(wrappedPath(path)); + return delegate->getSerializedMetadata(wrapped_paths); + } + + uint32_t getHardlinkCount(const std::string & path) const override + { + return delegate->getHardlinkCount(wrappedPath(path)); + } + + StoredObjects getStorageObjects(const std::string & path) const override + { + return delegate->getStorageObjects(wrappedPath(path)); + } +}; + + +} + +#endif diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 1bc2b36f5042..4377ca4bcf82 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -135,7 +135,6 @@ class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, pr UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; TruncateFileOperationOutcomePtr truncateFile(const std::string & src_path, size_t target_size) override; - }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index bd4ddf6e5a29..ad1135253db2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -663,7 +663,7 @@ bool IMergeTreeDataPart::mayStoreDataInCaches() const void IMergeTreeDataPart::removeIfNeeded() { - assert(assertHasValidVersionMetadata()); + chassert(assertHasValidVersionMetadata()); std::string path; try @@ -1979,7 +1979,7 @@ void IMergeTreeDataPart::initializeIndexGranularityInfo() void IMergeTreeDataPart::remove() { - assert(assertHasValidVersionMetadata()); + chassert(assertHasValidVersionMetadata()); part_is_probably_removed_from_disk = true; auto can_remove_callback = [this] () diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 58d7f321933c..2d86ba656859 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -931,6 +931,7 @@ class FailureReason(enum.Enum): NO_STATELESS = "no-stateless" NO_STATEFUL = "no-stateful" NO_OPENSSL_FIPS = "no-openssl-fips" + ENCRYPTED_STORAGE = "encrypted-storage" # UNKNOWN reasons NO_REFERENCE = "no reference file" @@ -1577,6 +1578,9 @@ class TestCase: if tags and ("no-openssl-fips" in tags) and args.openssl_fips: return FailureReason.NO_OPENSSL_FIPS + if tags and ("no-encrypted-storage" in tags) and args.encrypted_storage: + return FailureReason.ENCRYPTED_STORAGE + if tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -3736,6 +3740,12 @@ def parse_args(): action="store_true", help="Display $ characters after line with trailing whitespaces in diff output", ) + parser.add_argument( + "--encrypted-storage", + action="store_true", + default=False, + help="Run tests over encrypted storage", + ) group = parser.add_mutually_exclusive_group(required=False) group.add_argument( @@ -4060,6 +4070,10 @@ if __name__ == "__main__": if args.replace_replicated_with_shared: args.s3_storage = True + if args.encrypted_storage and not args.s3_storage and not args.azure_blob_storage: + print("Disabling encrypted storage. To run tests with encrypted storage, either s3_storage or azure_blob_storage should be enabled.") + args.encrypted_storage = False + try: main(args) except ServerDied as e: diff --git a/tests/config/config.d/azure_encrypted_storage_policy_by_default.xml b/tests/config/config.d/azure_encrypted_storage_policy_by_default.xml new file mode 100644 index 000000000000..4cdd81306e79 --- /dev/null +++ b/tests/config/config.d/azure_encrypted_storage_policy_by_default.xml @@ -0,0 +1,6 @@ + + + azure + + cached_azure + diff --git a/tests/config/config.d/azure_storage_conf.xml b/tests/config/config.d/azure_storage_conf.xml index d68615c8da22..25a2e7b10627 100644 --- a/tests/config/config.d/azure_storage_conf.xml +++ b/tests/config/config.d/azure_storage_conf.xml @@ -19,13 +19,29 @@ 100000000000 1 + + encrypted + cached_azure + 1234567812345678 + cached_azure_encrypted/ + + + +
azure
+
+
cached_azure
+ + +
cached_azure_encrypted
+
+
diff --git a/tests/config/config.d/azure_storage_policy_by_default.xml b/tests/config/config.d/azure_storage_policy_by_default.xml index cab8a106f1b8..fdaf93aa2d90 100644 --- a/tests/config/config.d/azure_storage_policy_by_default.xml +++ b/tests/config/config.d/azure_storage_policy_by_default.xml @@ -2,4 +2,5 @@ azure_cache + cached_azure diff --git a/tests/config/config.d/s3_encrypted_storage_policy_for_merge_tree_by_default.xml b/tests/config/config.d/s3_encrypted_storage_policy_for_merge_tree_by_default.xml new file mode 100644 index 000000000000..98b7ce1861a7 --- /dev/null +++ b/tests/config/config.d/s3_encrypted_storage_policy_for_merge_tree_by_default.xml @@ -0,0 +1,6 @@ + + + s3_encrypted + + cached_s3 + \ No newline at end of file diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index 4e3d9636daf6..224e3b7c97e5 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -13,6 +13,12 @@ cached_s3/ s3 + + encrypted + cached_s3 + 1234567812345678 + cached_s3_encrypted/ +
@@ -20,10 +26,11 @@
cached_s3
+ + +
cached_s3_encrypted
+
+
- - s3 - - cached_s3 diff --git a/tests/config/config.d/s3_storage_policy_for_merge_tree_by_default.xml b/tests/config/config.d/s3_storage_policy_for_merge_tree_by_default.xml new file mode 100644 index 000000000000..b2f9b1567906 --- /dev/null +++ b/tests/config/config.d/s3_storage_policy_for_merge_tree_by_default.xml @@ -0,0 +1,6 @@ + + + s3 + + cached_s3 + \ No newline at end of file diff --git a/tests/config/config.d/s3_storage_policy_with_template_object_key.xml b/tests/config/config.d/s3_storage_policy_with_template_object_key.xml index 834f5102da16..7b4ec7b66b38 100644 --- a/tests/config/config.d/s3_storage_policy_with_template_object_key.xml +++ b/tests/config/config.d/s3_storage_policy_with_template_object_key.xml @@ -16,6 +16,12 @@ cached_s3/ s3 + + encrypted + cached_s3 + 1234567812345678 + cached_s3_encrypted/ +
@@ -23,10 +29,11 @@
cached_s3
+ + +
cached_s3_encrypted
+
+
- - s3 - - cached_s3 diff --git a/tests/config/config.d/transactions.xml b/tests/config/config.d/transactions.xml index 64e166b81b53..b23e76ec4c36 100644 --- a/tests/config/config.d/transactions.xml +++ b/tests/config/config.d/transactions.xml @@ -1,15 +1,6 @@ 42 - - - - - system - transactions_info_log
- 7500 -
- /test/clickhouse/txn diff --git a/tests/config/config.d/transactions_info_log.xml b/tests/config/config.d/transactions_info_log.xml new file mode 100644 index 000000000000..52fb870ee796 --- /dev/null +++ b/tests/config/config.d/transactions_info_log.xml @@ -0,0 +1,7 @@ + + + system + transactions_info_log
+ 7500 +
+
diff --git a/tests/config/install.sh b/tests/config/install.sh index 4f28eb5b21c6..7d5116ca9c3f 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -19,6 +19,7 @@ USE_ASYNC_INSERT=${USE_ASYNC_INSERT:0} BUGFIX_VALIDATE_CHECK=0 NO_AZURE=0 KEEPER_INJECT_AUTH=1 +USE_ENCRYPTED_STORAGE=0 while [[ "$#" -gt 0 ]]; do case $1 in @@ -39,6 +40,8 @@ while [[ "$#" -gt 0 ]]; do --bugfix-validation) BUGFIX_VALIDATE_CHECK=1 ;; --no-keeper-inject-auth) KEEPER_INJECT_AUTH=0 ;; + + --encrypted-storage) USE_ENCRYPTED_STORAGE=1 ;; *) echo "Unknown option: $1" ; exit 1 ;; esac shift @@ -107,7 +110,12 @@ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/prometheus.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ + +ln -sf $SRC_PATH/config.d/transactions_info_log.xml $DEST_SERVER_PATH/config.d/ +if [[ "$USE_ENCRYPTED_STORAGE" == "0" ]]; then + ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ +fi + ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logger_trace.xml $DEST_SERVER_PATH/config.d/ @@ -246,14 +254,15 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th ln -sf $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/ fi -if [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" == "1" ]]; then - object_key_types_options=("generate-suffix" "generate-full-key" "generate-template-key") - object_key_type="${object_key_types_options[0]}" +object_key_types_options=("generate-suffix" "generate-full-key" "generate-template-key") +object_key_type="${object_key_types_options[0]}" - if [[ -n "$RANDOMIZE_OBJECT_KEY_TYPE" ]] && [[ "$RANDOMIZE_OBJECT_KEY_TYPE" -eq 1 ]]; then - object_key_type="${object_key_types_options[$(($RANDOM % ${#object_key_types_options[@]}))]}" - fi +if [[ -n "$RANDOMIZE_OBJECT_KEY_TYPE" ]] && [[ "$RANDOMIZE_OBJECT_KEY_TYPE" -eq 1 ]]; then + object_key_type="${object_key_types_options[$(($RANDOM % ${#object_key_types_options[@]}))]}" +fi +function setup_storage_policy() +{ case $object_key_type in "generate-full-key") ln -sf $SRC_PATH/config.d/storage_metadata_with_full_object_key.xml $DEST_SERVER_PATH/config.d/ @@ -267,8 +276,22 @@ if [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" == "1" ]]; then ln -sf $SRC_PATH/config.d/s3_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ ;; esac +} + +if [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" == "1" ]]; then + setup_storage_policy + + if [[ "$USE_ENCRYPTED_STORAGE" == "1" ]]; then + ln -sf $SRC_PATH/config.d/s3_encrypted_storage_policy_for_merge_tree_by_default.xml $DEST_SERVER_PATH/config.d/ + else + ln -sf $SRC_PATH/config.d/s3_storage_policy_for_merge_tree_by_default.xml $DEST_SERVER_PATH/config.d/ + fi elif [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" == "1" ]]; then - ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ + if [[ "$USE_ENCRYPTED_STORAGE" == "1" ]]; then + ln -sf $SRC_PATH/config.d/azure_encrypted_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ + else + ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ + fi fi if [[ "$EXPORT_S3_STORAGE_POLICIES" == "1" ]]; then @@ -332,10 +355,12 @@ if [[ "$USE_DATABASE_REPLICATED" == "1" ]]; then cat $DEST_SERVER_PATH/config.d/macros.xml | sed "s|r1|r2|" > $ch_server_1_path/config.d/macros.xml cat $DEST_SERVER_PATH/config.d/macros.xml | sed "s|s1|s2|" > $ch_server_2_path/config.d/macros.xml - rm $ch_server_1_path/config.d/transactions.xml - rm $ch_server_2_path/config.d/transactions.xml - cat $DEST_SERVER_PATH/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn1|" > $ch_server_1_path/config.d/transactions.xml - cat $DEST_SERVER_PATH/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn2|" > $ch_server_2_path/config.d/transactions.xml + if [[ "$USE_ENCRYPTED_STORAGE" == "0" ]]; then + rm $ch_server_1_path/config.d/transactions.xml + rm $ch_server_2_path/config.d/transactions.xml + cat $DEST_SERVER_PATH/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn1|" > $ch_server_1_path/config.d/transactions.xml + cat $DEST_SERVER_PATH/config.d/transactions.xml | sed "s|/test/clickhouse/txn|/test/clickhouse/txn2|" > $ch_server_2_path/config.d/transactions.xml + fi # ch_server_lib_1=$DEST_SERVER_PATH/../../var/lib/clickhouse1 # ch_server_lib_2=$DEST_SERVER_PATH/../../var/lib/clickhouse2 diff --git a/tests/queries/0_stateless/01133_begin_commit_race.sh b/tests/queries/0_stateless/01133_begin_commit_race.sh index 81e88e387ee2..3ba14a8eeaee 100755 --- a/tests/queries/0_stateless/01133_begin_commit_race.sh +++ b/tests/queries/0_stateless/01133_begin_commit_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-ordinary-database +# Tags: long, no-ordinary-database, no-encrypted-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01167_isolation_hermitage.sh b/tests/queries/0_stateless/01167_isolation_hermitage.sh index 15dc78f9f3cd..4f1fd9004470 100755 --- a/tests/queries/0_stateless/01167_isolation_hermitage.sh +++ b/tests/queries/0_stateless/01167_isolation_hermitage.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-replicated-database, no-ordinary-database +# Tags: long, no-fasttest, no-replicated-database, no-ordinary-database, no-encrypted-storage # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 1adfd4b2bff8..1fd1fdb64ae6 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database, no-ordinary-database +# Tags: no-fasttest, no-replicated-database, no-ordinary-database, no-encrypted-storage # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 8873fd88f0e6..65f443872018 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database, no-ordinary-database +# Tags: long, no-replicated-database, no-ordinary-database, no-encrypted-storage # shellcheck disable=SC2015 # shellcheck disable=SC2119 diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh index 0c1b19c1d5f3..fb41fa0554cd 100755 --- a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database, no-ordinary-database, zookeeper +# Tags: long, no-replicated-database, no-ordinary-database, zookeeper, no-encrypted-storage # shellcheck disable=SC2015 diff --git a/tests/queries/0_stateless/01170_alter_partition_isolation.sh b/tests/queries/0_stateless/01170_alter_partition_isolation.sh index 6ac957138004..ebe8c677e2d0 100755 --- a/tests/queries/0_stateless/01170_alter_partition_isolation.sh +++ b/tests/queries/0_stateless/01170_alter_partition_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database, no-ordinary-database +# Tags: no-fasttest, no-replicated-database, no-ordinary-database, no-encrypted-storage # Looks like server does not listen https port in fasttest # FIXME Replicated database executes ALTERs in separate context, so transaction info is lost diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index f6a88c205c19..47349248e2bc 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-ordinary-database +# Tags: long, no-ordinary-database, no-encrypted-storage # shellcheck disable=SC2119 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index 436a2427d448..0ecff8be03ed 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database +-- Tags: no-ordinary-database, no-encrypted-storage drop table if exists txn_counters; diff --git a/tests/queries/0_stateless/01173_transaction_control_queries.sql b/tests/queries/0_stateless/01173_transaction_control_queries.sql index a59abf30947f..c4191fdb9c2f 100644 --- a/tests/queries/0_stateless/01173_transaction_control_queries.sql +++ b/tests/queries/0_stateless/01173_transaction_control_queries.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database +-- Tags: no-ordinary-database, no-encrypted-storage drop table if exists mt1; drop table if exists mt2; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 235d98fb5def..9b193416856b 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-ordinary-database +# Tags: long, no-ordinary-database, no-encrypted-storage # shellcheck disable=SC2015 # shellcheck disable=SC2119 diff --git a/tests/queries/0_stateless/02345_implicit_transaction.sql b/tests/queries/0_stateless/02345_implicit_transaction.sql index a9b9e62b4902..7c206b298daf 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.sql +++ b/tests/queries/0_stateless/02345_implicit_transaction.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-fasttest +-- Tags: no-ordinary-database, no-fasttest, no-encrypted-storage DROP TABLE IF EXISTS landing_to_target; DROP TABLE IF EXISTS target; DROP TABLE IF EXISTS landing; diff --git a/tests/queries/0_stateless/02421_truncate_isolation_no_merges.sh b/tests/queries/0_stateless/02421_truncate_isolation_no_merges.sh index 4f625df5409b..21d01a4b4218 100755 --- a/tests/queries/0_stateless/02421_truncate_isolation_no_merges.sh +++ b/tests/queries/0_stateless/02421_truncate_isolation_no_merges.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database, no-ordinary-database, long +# Tags: no-fasttest, no-replicated-database, no-ordinary-database, long, no-encrypted-storage set -e -o pipefail diff --git a/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh b/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh index da0b132bcbc2..288822874310 100755 --- a/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh +++ b/tests/queries/0_stateless/02421_truncate_isolation_with_mutations.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-replicated-database, no-ordinary-database, long +# Tags: no-fasttest, no-replicated-database, no-ordinary-database, long, no-encrypted-storage set -e -o pipefail diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 385d7624b82c..0f7fdbf3c2f3 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-ordinary-database, no-fasttest, no-azure-blob-storage +# Tags: no-random-settings, no-ordinary-database, no-fasttest, no-azure-blob-storage, no-encrypted-storage # no-fasttest: The test is slow (too many small blocks) # no-azure-blob-storage: The test uploads many parts to Azure (5k+), and it runs in parallel with other tests. # As a result, they may interfere, and some queries won't be able to finish in 30 seconds timeout leading to a test failure. diff --git a/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql b/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql index 960c6c168a12..4702f9252b1a 100644 --- a/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql +++ b/tests/queries/0_stateless/02497_source_part_is_intact_when_mutation.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database, no-ordinary-database +-- Tags: no-replicated-database, no-ordinary-database, no-encrypted-storage SET mutations_sync = 1; SET check_query_single_value_result = 0; diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh index 2ffc55b5f088..7c5bbf344ca9 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings +# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings, no-encrypted-storage # Tag no-fasttest: requires S3 # Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index c504a6c1e559..a0d41f042639 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings, no-shared-merge-tree +# Tags: no-fasttest, no-random-settings, no-random-merge-tree-settings, no-shared-merge-tree, no-encrypted-storage # Tag no-fasttest: requires S3 # Tag no-random-settings, no-random-merge-tree-settings: to avoid creating extra files like serialization.json, this test too exocit anyway # Tag no-shared-merge-tree: use database ordinary diff --git a/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql index e3c86a2d5be8..793bc1885052 100644 --- a/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql +++ b/tests/queries/0_stateless/03167_transactions_are_really_disabled.sql @@ -1,3 +1,5 @@ +-- Tags: no-encrypted-storage + DROP TABLE IF EXISTS mv_table; DROP TABLE IF EXISTS null_table; diff --git a/tests/queries/0_stateless/03276_database_backup_merge_tree_table_file_engine.sql b/tests/queries/0_stateless/03276_database_backup_merge_tree_table_file_engine.sql index 6cd7c68014b1..5596c69995be 100644 --- a/tests/queries/0_stateless/03276_database_backup_merge_tree_table_file_engine.sql +++ b/tests/queries/0_stateless/03276_database_backup_merge_tree_table_file_engine.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-flaky-check +-- Tags: no-parallel, no-fasttest, no-flaky-check, no-encrypted-storage -- Because we are creating a backup with fixed path. DROP DATABASE IF EXISTS 03276_test_database; diff --git a/tests/queries/0_stateless/03277_database_backup_database_file_engine.sql b/tests/queries/0_stateless/03277_database_backup_database_file_engine.sql index 5fb775db4bb4..ea594998da2d 100644 --- a/tests/queries/0_stateless/03277_database_backup_database_file_engine.sql +++ b/tests/queries/0_stateless/03277_database_backup_database_file_engine.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-flaky-check +-- Tags: no-parallel, no-fasttest, no-flaky-check, no-encrypted-storage -- Because we are creating a backup with fixed path. DROP DATABASE IF EXISTS 03277_test_database; diff --git a/tests/queries/0_stateless/03278_database_backup_merge_tree_table_disk_engine.sql b/tests/queries/0_stateless/03278_database_backup_merge_tree_table_disk_engine.sql index 412f9f0f762e..4a4fa04f47d6 100644 --- a/tests/queries/0_stateless/03278_database_backup_merge_tree_table_disk_engine.sql +++ b/tests/queries/0_stateless/03278_database_backup_merge_tree_table_disk_engine.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-flaky-check +-- Tags: no-parallel, no-fasttest, no-flaky-check, no-encrypted-storage -- Because we are creating a backup with fixed path. DROP DATABASE IF EXISTS 03278_test_database; diff --git a/tests/queries/0_stateless/03279_database_backup_database_disk_engine.sql b/tests/queries/0_stateless/03279_database_backup_database_disk_engine.sql index 8435e33a38d6..399471d23f0e 100644 --- a/tests/queries/0_stateless/03279_database_backup_database_disk_engine.sql +++ b/tests/queries/0_stateless/03279_database_backup_database_disk_engine.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest, no-flaky-check +-- Tags: no-parallel, no-fasttest, no-flaky-check, no-encrypted-storage -- Because we are creating a backup with fixed path. DROP DATABASE IF EXISTS 03279_test_database; diff --git a/tests/queries/0_stateless/03282_memory_transaction_crash.sql b/tests/queries/0_stateless/03282_memory_transaction_crash.sql index e90c842825e3..d8bd54412b12 100644 --- a/tests/queries/0_stateless/03282_memory_transaction_crash.sql +++ b/tests/queries/0_stateless/03282_memory_transaction_crash.sql @@ -1,4 +1,4 @@ --- Tags: zookeeper +-- Tags: zookeeper, no-encrypted-storage -- https://github.com/ClickHouse/ClickHouse/issues/72887 DROP TABLE IF EXISTS t0; diff --git a/tests/queries/0_stateless/03373_named_session_try_recreate_before_timeout.sh b/tests/queries/0_stateless/03373_named_session_try_recreate_before_timeout.sh index 7cfe9e2c2e01..de2ee8a99dbb 100755 --- a/tests/queries/0_stateless/03373_named_session_try_recreate_before_timeout.sh +++ b/tests/queries/0_stateless/03373_named_session_try_recreate_before_timeout.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-encrypted-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0e415c735bfacd711bf17a1e89a04ef0347bd2eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 25 Jun 2025 14:47:38 +0000 Subject: [PATCH 5/9] Merge pull request #82508 from ClickHouse/remove_zero_copy_metadata Remove send_metadata logic related to zero-copy replication --- src/Coordination/KeeperContext.cpp | 6 +- src/Disks/DiskBackup.cpp | 4 - src/Disks/DiskBackup.h | 2 - src/Disks/DiskEncrypted.cpp | 4 +- src/Disks/DiskEncrypted.h | 6 - src/Disks/DiskLocal.cpp | 4 +- src/Disks/DiskLocal.h | 2 +- src/Disks/IDisk.cpp | 4 +- src/Disks/IDisk.h | 7 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 21 - .../AzureBlobStorage/AzureObjectStorage.h | 6 - .../Cached/CachedObjectStorage.cpp | 9 - .../Cached/CachedObjectStorage.h | 6 - .../ObjectStorages/DiskObjectStorage.cpp | 82 +-- src/Disks/ObjectStorages/DiskObjectStorage.h | 19 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 579 ------------------ ...ObjectStorageRemoteMetadataRestoreHelper.h | 103 ---- .../DiskObjectStorageTransaction.cpp | 56 +- .../DiskObjectStorageTransaction.h | 16 +- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 8 - .../ObjectStorages/HDFS/HDFSObjectStorage.h | 6 - src/Disks/ObjectStorages/IObjectStorage.h | 7 - .../Local/LocalObjectStorage.cpp | 9 - .../ObjectStorages/Local/LocalObjectStorage.h | 6 - .../ObjectStorages/ObjectStorageFactory.cpp | 13 - .../RegisterDiskObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 13 - .../ObjectStorages/S3/S3ObjectStorage.cpp | 17 - src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 6 - .../ObjectStorages/Web/WebObjectStorage.cpp | 8 - .../ObjectStorages/Web/WebObjectStorage.h | 6 - .../MergeTree/DataPartStorageOnDiskBase.cpp | 10 - .../MergeTree/DataPartStorageOnDiskBase.h | 2 - src/Storages/MergeTree/DataPartsExchange.cpp | 21 - src/Storages/MergeTree/IDataPartStorage.h | 4 - src/Storages/MergeTree/MergeTreeData.cpp | 3 - .../configs/config.d/s3.xml | 3 - .../configs/conf.xml | 1 - 38 files changed, 29 insertions(+), 1052 deletions(-) delete mode 100644 src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp delete mode 100644 src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 378f76ea80c2..01d95f2ad36c 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -430,7 +430,7 @@ KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::Ab fs::create_directories(path); auto disk = std::make_shared("LocalLogDisk", path); - disk->startup(Context::getGlobalContextInstance(), false); + disk->startup(false); return disk; }; @@ -457,7 +457,7 @@ KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Uti fs::create_directories(path); auto disk = std::make_shared("LocalSnapshotDisk", path); - disk->startup(Context::getGlobalContextInstance(), false); + disk->startup(false); return disk; }; @@ -484,7 +484,7 @@ KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::A fs::create_directories(path); auto disk = std::make_shared("LocalStateFileDisk", path); - disk->startup(Context::getGlobalContextInstance(), false); + disk->startup(false); return disk; }; diff --git a/src/Disks/DiskBackup.cpp b/src/Disks/DiskBackup.cpp index fe741b3ec4d0..1ea3085b6b44 100644 --- a/src/Disks/DiskBackup.cpp +++ b/src/Disks/DiskBackup.cpp @@ -269,10 +269,6 @@ DataSourceDescription DiskBackup::getDataSourceDescription() const return description; } -void DiskBackup::startupImpl(ContextPtr) -{ -} - void DiskBackup::shutdown() { } diff --git a/src/Disks/DiskBackup.h b/src/Disks/DiskBackup.h index c730747c5f61..00f67181c796 100644 --- a/src/Disks/DiskBackup.h +++ b/src/Disks/DiskBackup.h @@ -125,8 +125,6 @@ class DiskBackup final : public IDisk bool isReadOnly() const override { return true; } - void startupImpl(ContextPtr context) override; - void shutdown() override; private: diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 5c1f9b26b51f..ac41b32fda1f 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -452,13 +452,13 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check) const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context, + ContextPtr, const DisksMap & map, bool, bool) -> DiskPtr { bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); DiskPtr disk = std::make_shared(name, config, config_prefix, map); - disk->startup(context, skip_access_check); + disk->startup(skip_access_check); return disk; }; factory.registerDiskType("encrypted", creator); diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index f920d21205f1..268939038202 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -299,12 +299,6 @@ class DiskEncrypted : public IDisk return delegate->checkUniqueId(id); } - void onFreeze(const String & path) override - { - auto wrapped_path = wrappedPath(path); - delegate->onFreeze(wrapped_path); - } - void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override; DataSourceDescription getDataSourceDescription() const override diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cfdc65c6dde6..f9eed66c8def 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -745,7 +745,7 @@ void DiskLocal::setup() throw Exception(ErrorCodes::LOGICAL_ERROR, "disk_checker_magic_number is not initialized. It's a bug"); } -void DiskLocal::startupImpl(ContextPtr) +void DiskLocal::startupImpl() { broken = false; disk_checker_magic_number = -1; @@ -804,7 +804,7 @@ void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check) bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); std::shared_ptr disk = std::make_shared(name, path, keep_free_space_bytes, context, config, config_prefix); - disk->startup(context, skip_access_check); + disk->startup(skip_access_check); return disk; }; factory.registerDiskType("local", creator); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 79147eac6210..bad75b5d86a2 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -140,7 +140,7 @@ class DiskLocal : public IDisk bool isBroken() const override { return broken; } bool isReadOnly() const override { return readonly; } - void startupImpl(ContextPtr context) override; + void startupImpl() override; void shutdown() override; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index cbfc086de7f7..99527985aac6 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -202,7 +202,7 @@ SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const return nullptr; } -void IDisk::startup(ContextPtr context, bool skip_access_check) +void IDisk::startup(bool skip_access_check) { if (!skip_access_check) { @@ -215,7 +215,7 @@ void IDisk::startup(ContextPtr context, bool skip_access_check) else checkAccess(); } - startupImpl(context); + startupImpl(); } void IDisk::checkAccess() diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 09f8fb4ea41e..0ae5979660ee 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -458,10 +458,10 @@ class IDisk : public Space virtual void shutdown() {} /// Performs access check and custom action on disk startup. - void startup(ContextPtr context, bool skip_access_check); + void startup(bool skip_access_check); /// Performs custom action on disk startup. - virtual void startupImpl(ContextPtr) {} + virtual void startupImpl() {} /// If the state can be changed under the hood and become outdated in memory, perform a reload if necessary. /// but don't do it more frequently than the specified parameter. @@ -481,9 +481,6 @@ class IDisk : public Space /// Required for remote disk to ensure that the replica has access to data written by other node virtual bool checkUniqueId(const String & id) const { return existsFile(id); } - /// Invoked on partitions freeze query. - virtual void onFreeze(const String &) {} - /// Returns guard, that insures synchronization of directory metadata with storage device. virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 25d789bef792..71b72006a0ad 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -364,27 +364,6 @@ void AzureObjectStorage::applyNewSettings( client.set(std::move(new_client)); } - -std::unique_ptr AzureObjectStorage::cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) -{ - auto new_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context->getSettingsRef()); - bool is_client_for_disk = client.get()->IsClientForDisk(); - - AzureBlobStorage::ConnectionParams params - { - .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), - .auth_method = AzureBlobStorage::getAuthMethod(config, config_prefix), - .client_options = AzureBlobStorage::getClientOptions(*new_settings, is_client_for_disk), - }; - - auto new_client = AzureBlobStorage::getContainerClient(params, /*readonly=*/ true); - return std::make_unique(name, params.auth_method, std::move(new_client), std::move(new_settings), new_namespace, params.endpoint.getServiceEndpoint()); -} - } #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 8038bfcf54c1..8af163ddb2e6 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -91,12 +91,6 @@ class AzureObjectStorage : public IObjectStorage String getObjectsNamespace() const override { return object_namespace ; } - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool areObjectKeysRandom() const override { return true; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 7fe9a4e15f51..92e6d413ac5f 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -193,15 +193,6 @@ void CachedObjectStorage::copyObject( // NOLINT object_storage->copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes); } -std::unique_ptr CachedObjectStorage::cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) -{ - return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); -} - void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { object_storage->listObjects(path, children, max_keys); diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 087f02d67539..63302d092439 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -64,12 +64,6 @@ class CachedObjectStorage final : public IObjectStorage IObjectStorage & object_storage_to, std::optional object_to_attributes = {}) override; - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 5063c1d8499a..6d316712db6d 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -47,18 +46,16 @@ DiskTransactionPtr DiskObjectStorage::createObjectStorageTransaction() { return std::make_shared( *object_storage, - *metadata_storage, - send_metadata ? metadata_helper.get() : nullptr); + *metadata_storage); } -DiskTransactionPtr DiskObjectStorage::createObjectStorageTransactionToAnotherDisk(DiskObjectStorage& to_disk) +DiskTransactionPtr DiskObjectStorage::createObjectStorageTransactionToAnotherDisk(DiskObjectStorage & to_disk) { return std::make_shared( *object_storage, *metadata_storage, *to_disk.getObjectStorage(), - *to_disk.getMetadataStorage(), - send_metadata ? metadata_helper.get() : nullptr); + *to_disk.getMetadataStorage()); } @@ -74,10 +71,8 @@ DiskObjectStorage::DiskObjectStorage( , log(getLogger("DiskObjectStorage(" + name + ")")) , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) - , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) - , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) , remove_shared_recursive_file_limit(config.getUInt64(config_prefix + ".remove_shared_recursive_file_limit", DEFAULT_REMOVE_SHARED_RECURSIVE_FILE_LIMIT)) { data_source_description = DataSourceDescription{ @@ -220,20 +215,13 @@ size_t DiskObjectStorage::getFileSize(const String & path) const void DiskObjectStorage::moveDirectory(const String & from_path, const String & to_path) { - if (send_metadata) - sendMoveMetadata(from_path, to_path); - auto transaction = createObjectStorageTransaction(); transaction->moveDirectory(from_path, to_path); transaction->commit(); } -void DiskObjectStorage::moveFile(const String & from_path, const String & to_path, bool should_send_metadata) +void DiskObjectStorage::moveFile(const String & from_path, const String & to_path) { - - if (should_send_metadata) - sendMoveMetadata(from_path, to_path); - auto transaction = createObjectStorageTransaction(); transaction->moveFile(from_path, to_path); transaction->commit(); @@ -271,11 +259,6 @@ void DiskObjectStorage::copyFile( /// NOLINT } } -void DiskObjectStorage::moveFile(const String & from_path, const String & to_path) -{ - moveFile(from_path, to_path, send_metadata); -} - void DiskObjectStorage::replaceFile(const String & from_path, const String & to_path) { if (existsFile(to_path)) @@ -348,30 +331,13 @@ bool DiskObjectStorage::checkUniqueId(const String & id) const return object_storage->exists(object); } -void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata) +void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path) { - if (should_send_metadata && !dst_path.starts_with("shadow/")) - { - auto revision = metadata_helper->revision_counter + 1; - metadata_helper->revision_counter += 1; - const ObjectAttributes object_metadata { - {"src_path", src_path}, - {"dst_path", dst_path} - }; - metadata_helper->createFileOperationObject("hardlink", revision, object_metadata); - } - auto transaction = createObjectStorageTransaction(); transaction->createHardLink(src_path, dst_path); transaction->commit(); } -void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path) -{ - createHardLink(src_path, dst_path, send_metadata); -} - - void DiskObjectStorage::setReadOnly(const String & path) { /// We should store read only flag inside metadata file (instead of using FS flag), @@ -478,13 +444,11 @@ void DiskObjectStorage::shutdown() LOG_INFO(log, "Disk {} shut down", name); } -void DiskObjectStorage::startupImpl(ContextPtr context) +void DiskObjectStorage::startupImpl() { LOG_INFO(log, "Starting up disk {}", name); object_storage->startup(); - restoreMetadataIfNeeded(context->getConfigRef(), "storage_configuration.disks." + name, context); - LOG_INFO(log, "Disk {} started up", name); } @@ -610,15 +574,6 @@ bool DiskObjectStorage::tryReserve(UInt64 bytes) return false; } -void DiskObjectStorage::sendMoveMetadata(const String & from_path, const String & to_path) -{ - chassert(send_metadata); - auto revision = metadata_helper->revision_counter + 1; - metadata_helper->revision_counter += 1; - - const ObjectAttributes object_metadata{{"from_path", from_path}, {"to_path", to_path}}; - metadata_helper->createFileOperationObject("rename", revision, object_metadata); -} bool DiskObjectStorage::supportsCache() const { @@ -885,31 +840,6 @@ void DiskObjectStorage::applyNewSettings( IDisk::applyNewSettings(config, context_, config_prefix, disk_map); } -void DiskObjectStorage::restoreMetadataIfNeeded( - const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) -{ - if (send_metadata) - { - metadata_helper->restore(config, config_prefix, context); - - auto current_schema_version = DB::DiskObjectStorageRemoteMetadataRestoreHelper::readSchemaVersion(object_storage.get(), object_key_prefix); - if (current_schema_version < DiskObjectStorageRemoteMetadataRestoreHelper::RESTORABLE_SCHEMA_VERSION) - metadata_helper->migrateToRestorableSchema(); - - metadata_helper->findLastRevision(); - } -} - -void DiskObjectStorage::syncRevision(UInt64 revision) -{ - metadata_helper->syncRevision(revision); -} - -UInt64 DiskObjectStorage::getRevision() const -{ - return metadata_helper->getRevision(); -} - #if USE_AWS_S3 std::shared_ptr DiskObjectStorage::getS3StorageClient() const { diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 108116f9c73f..1a0705f47776 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -28,7 +27,6 @@ class DiskObjectStorage : public IDisk { friend class DiskObjectStorageReservation; -friend class DiskObjectStorageRemoteMetadataRestoreHelper; public: DiskObjectStorage( @@ -70,8 +68,6 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; void moveFile(const String & from_path, const String & to_path) override; - void moveFile(const String & from_path, const String & to_path, bool should_send_metadata); - void replaceFile(const String & from_path, const String & to_path) override; void renameExchange(const std::string & old_path, const std::string & new_path) override; @@ -112,7 +108,6 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; bool checkUniqueId(const String & id) const override; void createHardLink(const String & src_path, const String & dst_path) override; - void createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata); void listFiles(const String & path, std::vector & file_names) const override; @@ -144,7 +139,7 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; void shutdown() override; - void startupImpl(ContextPtr context) override; + void startupImpl() override; void refresh(UInt64 not_sooner_than_milliseconds) override { @@ -186,14 +181,6 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override; - void restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context); - - void onFreeze(const String & path) override; - - void syncRevision(UInt64 revision) override; - - UInt64 getRevision() const override; - ObjectStoragePtr getObjectStorage() override; DiskObjectStoragePtr createDiskObjectStorage() override; @@ -269,8 +256,6 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; bool tryReserve(UInt64 bytes); void sendMoveMetadata(const String & from_path, const String & to_path); - const bool send_metadata; - mutable std::mutex resource_mutex; String read_resource_name_from_config; // specified in disk config.xml read_resource element String write_resource_name_from_config; // specified in disk config.xml write_resource element @@ -280,8 +265,6 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper; String write_resource_name_from_sql_any; // described by CREATE RESOURCE query with WRITE ANY DISK clause scope_guard resource_changes_subscription; - std::unique_ptr metadata_helper; - UInt64 remove_shared_recursive_file_limit; }; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp deleted file mode 100644 index c6dde668fe31..000000000000 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ /dev/null @@ -1,579 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace CurrentMetrics -{ - extern const Metric LocalThread; - extern const Metric LocalThreadActive; - extern const Metric LocalThreadScheduled; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_FORMAT; - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} - -static String revisionToString(UInt64 revision) -{ - return std::bitset<64>(revision).to_string(); -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::createFileOperationObject( - const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const -{ - const String relative_path = "operations/r" + revisionToString(revision) + operation_log_suffix + "-" + operation_name; - StoredObject object(fs::path(disk->object_key_prefix) / relative_path); - auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite, metadata); - buf->write('0'); - buf->finalize(); -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::findLastRevision() -{ - /// Construct revision number from high to low bits. - String revision; - revision.reserve(64); - for (int bit = 0; bit < 64; ++bit) - { - auto revision_prefix = revision + "1"; - - LOG_TRACE(disk->log, "Check object exists with revision prefix {}", revision_prefix); - - const auto & object_storage = disk->object_storage; - StoredObject revision_object{disk->object_key_prefix + "r" + revision_prefix}; - StoredObject revision_operation_object{disk->object_key_prefix + "operations/r" + revision_prefix}; - - /// Check file or operation with such revision prefix exists. - if (object_storage->exists(revision_object) || object_storage->exists(revision_operation_object)) - revision += "1"; - else - revision += "0"; - } - revision_counter = static_cast(std::bitset<64>(revision).to_ullong()); - LOG_INFO(disk->log, "Found last revision number {} for disk {}", revision_counter.load(), disk->name); -} - -int DiskObjectStorageRemoteMetadataRestoreHelper::readSchemaVersion(IObjectStorage * object_storage, const String & source_path) -{ - StoredObject object(fs::path(source_path) / SCHEMA_VERSION_OBJECT); - int version = 0; - if (!object_storage->exists(object)) - return version; - - auto buf = object_storage->readObject(object, ReadSettings{}); - readIntText(version, *buf); - - return version; -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::saveSchemaVersion(const int & version) const -{ - StoredObject object{fs::path(disk->object_key_prefix) / SCHEMA_VERSION_OBJECT}; - - auto buf = disk->object_storage->writeObject(object, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings); - writeIntText(version, *buf); - buf->finalize(); - -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const -{ - StoredObject object{key}; - disk->object_storage->copyObject(object, object, read_settings, write_settings, metadata); -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::migrateFileToRestorableSchema(const String & path) const -{ - LOG_TRACE(disk->log, "Migrate file {} to restorable schema", disk->metadata_storage->getPath() + path); - - auto objects = disk->metadata_storage->getStorageObjects(path); - for (const auto & object : objects) - { - ObjectAttributes metadata { - {"path", path} - }; - updateObjectMetadata(object.remote_path, metadata); - } -} -void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchemaRecursive(const String & path, ThreadPool & pool) -{ - checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - - LOG_TRACE(disk->log, "Migrate directory {} to restorable schema", disk->metadata_storage->getPath() + path); - - bool dir_contains_only_files = true; - for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) - { - if (disk->existsDirectory(it->path())) - { - dir_contains_only_files = false; - break; - } - } - - /// The whole directory can be migrated asynchronously. - if (dir_contains_only_files) - { - pool.scheduleOrThrowOnError([this, path] - { - setThreadName("BackupWorker"); - for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) - migrateFileToRestorableSchema(it->path()); - }); - } - else - { - for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) - { - if (disk->existsDirectory(it->path())) - { - migrateToRestorableSchemaRecursive(it->path(), pool); - } - else - { - auto source_path = it->path(); - pool.scheduleOrThrowOnError([this, source_path] { migrateFileToRestorableSchema(source_path); }); - } - } - } - -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::migrateToRestorableSchema() -{ - try - { - LOG_INFO(disk->log, "Start migration to restorable schema for disk {}", disk->name); - - ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled}; - - for (const auto & root : data_roots) - if (disk->existsDirectory(root)) - migrateToRestorableSchemaRecursive(root + '/', pool); - - pool.wait(); - - saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); - } - catch (const Exception &) - { - tryLogCurrentException(disk->log, fmt::format("Failed to migrate to restorable schema for disk {}", disk->name)); - - throw; - } -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) -{ - LOG_INFO(disk->log, "Restore operation for disk {} called", disk->name); - - if (!disk->existsFile(RESTORE_FILE_NAME)) - { - LOG_INFO(disk->log, "No restore file '{}' exists, finishing restore", RESTORE_FILE_NAME); - return; - } - - try - { - RestoreInformation information; - information.source_path = disk->object_key_prefix; - information.source_namespace = disk->object_storage->getObjectsNamespace(); - - readRestoreInformation(information); - if (information.revision == 0) - information.revision = LATEST_REVISION; - if (!information.source_path.ends_with('/')) - information.source_path += '/'; - - IObjectStorage * source_object_storage = disk->object_storage.get(); - if (information.source_namespace == disk->object_storage->getObjectsNamespace()) - { - /// In this case we need to additionally cleanup S3 from objects with later revision. - /// Will be simply just restore to different path. - if (information.source_path == disk->object_key_prefix && information.revision != LATEST_REVISION) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Restoring to the same bucket and path is allowed if revision is latest (0)"); - - /// This case complicates S3 cleanup in case of unsuccessful restore. - if (information.source_path != disk->object_key_prefix && disk->object_key_prefix.starts_with(information.source_path)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Restoring to the same bucket is allowed only if source path is not a sub-path of configured path in S3 disk"); - } - else - { - object_storage_from_another_namespace = disk->object_storage->cloneObjectStorage(information.source_namespace, config, config_prefix, context); - source_object_storage = object_storage_from_another_namespace.get(); - } - - LOG_INFO(disk->log, "Starting to restore disk {}. Revision: {}, Source path: {}", - disk->name, information.revision, information.source_path); - - if (readSchemaVersion(source_object_storage, information.source_path) < RESTORABLE_SCHEMA_VERSION) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Source bucket doesn't have restorable schema."); - - LOG_INFO(disk->log, "Removing old metadata..."); - - bool cleanup_s3 = information.source_path != disk->object_key_prefix; - for (const auto & root : data_roots) - if (disk->existsDirectory(root)) - disk->removeSharedRecursive(root + '/', !cleanup_s3, {}); - - LOG_INFO(disk->log, "Old metadata removed, restoring new one"); - restoreFiles(source_object_storage, information); - restoreFileOperations(source_object_storage, information); - - auto tx = disk->metadata_storage->createTransaction(); - tx->unlinkFile(RESTORE_FILE_NAME); - tx->commit(); - - saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); - - LOG_INFO(disk->log, "Restore disk {} finished", disk->name); - } - catch (const Exception &) - { - tryLogCurrentException(disk->log, fmt::format("Failed to restore disk {}", disk->name)); - - throw; - } -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::readRestoreInformation(RestoreInformation & restore_information) /// NOLINT -{ - auto metadata_str = disk->metadata_storage->readFileToString(RESTORE_FILE_NAME); - ReadBufferFromString buffer(metadata_str); - - try - { - std::map properties; - - while (buffer.hasPendingData()) - { - String property; - readText(property, buffer); - assertChar('\n', buffer); - - auto pos = property.find('='); - if (pos == std::string::npos || pos == 0 || pos == property.length()) - throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Invalid property {} in restore file", property); - - auto key = property.substr(0, pos); - auto value = property.substr(pos + 1); - - auto it = properties.find(key); - if (it != properties.end()) - throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Property key duplication {} in restore file", key); - - properties[key] = value; - } - - for (const auto & [key, value] : properties) - { - ReadBufferFromString value_buffer(value); - - if (key == "revision") - readIntText(restore_information.revision, value_buffer); - else if (key == "source_bucket" || key == "source_namespace") - readText(restore_information.source_namespace, value_buffer); - else if (key == "source_path") - readText(restore_information.source_path, value_buffer); - else if (key == "detached") - readBoolTextWord(restore_information.detached, value_buffer); - else - throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown key {} in restore file", key); - } - } - catch (const Exception &) - { - tryLogCurrentException(disk->log, "Failed to read restore information"); - throw; - } -} - -static String shrinkKey(const String & path, const String & key) -{ - if (!key.starts_with(path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} prefix mismatch with given {}", key, path); - - return key.substr(path.length()); -} - -static std::tuple extractRevisionAndOperationFromKey(const String & key) -{ - String revision_str; - String suffix; - String operation; - /// Key has format: ../../r{revision}(-{hostname})-{operation} - static const re2::RE2 key_regexp{R"(.*/r(\d+)(-[\w\d\-\.]+)?-(\w+)$)"}; - - re2::RE2::FullMatch(key, key_regexp, &revision_str, &suffix, &operation); - - return {(revision_str.empty() ? 0 : static_cast(std::bitset<64>(revision_str).to_ullong())), operation}; -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata) -{ - if (disk->existsFileOrDirectory(to_path)) - { - if (send_metadata) - { - auto revision = ++revision_counter; - const ObjectAttributes object_metadata { - {"from_path", from_path}, - {"to_path", to_path} - }; - createFileOperationObject("rename", revision, object_metadata); - } - if (disk->existsDirectory(from_path)) - { - for (auto it = disk->iterateDirectory(from_path); it->isValid(); it->next()) - moveRecursiveOrRemove(it->path(), fs::path(to_path) / it->name(), false); - } - else - { - disk->removeFile(from_path); - } - } - else - { - disk->moveFile(from_path, to_path, send_metadata); - } -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information) -{ - LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name); - - ThreadPool pool{CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled}; - auto restore_files = [this, &source_object_storage, &restore_information, &pool](const RelativePathsWithMetadata & objects) - { - std::vector keys_names; - for (const auto & object : objects) - { - - LOG_INFO(disk->log, "Calling restore for key for disk {}", object->relative_path); - - /// Skip file operations objects. They will be processed separately. - if (object->relative_path.contains("/operations/")) - continue; - - const auto [revision, _] = extractRevisionAndOperationFromKey(object->relative_path); - /// Filter early if it's possible to get revision from key. - if (revision > restore_information.revision) - continue; - - keys_names.push_back(object->relative_path); - } - - if (!keys_names.empty()) - { - pool.scheduleOrThrowOnError([this, &source_object_storage, &restore_information, keys_names]() - { - processRestoreFiles(source_object_storage, restore_information.source_path, keys_names); - }); - } - - return true; - }; - - RelativePathsWithMetadata children; - source_object_storage->listObjects(restore_information.source_path, children, /* max_keys= */ 0); - - restore_files(children); - - pool.wait(); - - LOG_INFO(disk->log, "Files are restored for disk {}", disk->name); - -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( - IObjectStorage * source_object_storage, const String & source_path, const std::vector & keys) const -{ - for (const auto & key : keys) - { - auto metadata = source_object_storage->getObjectMetadata(key); - auto object_attributes = metadata.attributes; - - String path; - /// Restore file if object has 'path' in metadata. - auto path_entry = object_attributes.find("path"); - if (path_entry == object_attributes.end()) - { - /// Such keys can remain after migration, we can skip them. - LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key); - continue; - } - - path = path_entry->second; - disk->createDirectories(directoryPath(path)); - auto object_key = ObjectStorageKey::createAsRelative(disk->object_key_prefix, shrinkKey(source_path, key)); - - StoredObject object_from{key}; - StoredObject object_to{object_key.serialize()}; - - /// Copy object if we restore to different bucket / path. - if (source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->object_key_prefix != source_path) - source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, *disk->object_storage); - - auto tx = disk->metadata_storage->createTransaction(); - tx->addBlobToMetadata(path, object_key, metadata.size_bytes); - tx->commit(); - - LOG_TRACE(disk->log, "Restored file {}", path); - } - -} - -void DiskObjectStorage::onFreeze(const String & path) -{ - createDirectories(path); - auto tx = metadata_storage->createTransaction(); - WriteBufferFromOwnString revision_file_buf ; - writeIntText(metadata_helper->revision_counter.load(), revision_file_buf); - tx->writeStringToFile(path + "revision.txt", revision_file_buf.str()); - tx->commit(); -} - -static String pathToDetached(const String & source_path) -{ - if (source_path.ends_with('/')) - return fs::path(source_path).parent_path().parent_path() / "detached/"; - return fs::path(source_path).parent_path() / "detached/"; -} - -void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information) -{ - /// Enable recording file operations if we restore to different bucket / path. - bool send_metadata = source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() - || disk->object_key_prefix != restore_information.source_path; - - std::set renames; - auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithMetadata & objects) - { - const String rename = "rename"; - const String hardlink = "hardlink"; - - for (const auto & object : objects) - { - const auto [revision, operation] = extractRevisionAndOperationFromKey(object->relative_path); - if (revision == UNKNOWN_REVISION) - { - LOG_WARNING(disk->log, "Skip key {} with unknown revision", object->relative_path); - continue; - } - - /// S3 ensures that keys will be listed in ascending UTF-8 bytes order (revision order). - /// We can stop processing if revision of the object is already more than required. - if (revision > restore_information.revision) - return false; - - /// Keep original revision if restore to different bucket / path. - if (send_metadata) - revision_counter = revision - 1; - - auto object_attributes = source_object_storage->getObjectMetadata(object->relative_path).attributes; - if (operation == rename) - { - auto from_path = object_attributes["from_path"]; - auto to_path = object_attributes["to_path"]; - if (disk->existsFileOrDirectory(from_path)) - { - moveRecursiveOrRemove(from_path, to_path, send_metadata); - - LOG_TRACE(disk->log, "Revision {}. Restored rename {} -> {}", revision, from_path, to_path); - - if (restore_information.detached && disk->existsDirectory(to_path)) - { - /// Sometimes directory paths are passed without trailing '/'. We should keep them in one consistent way. - if (!from_path.ends_with('/')) - from_path += '/'; - if (!to_path.ends_with('/')) - to_path += '/'; - - /// Always keep latest actual directory path to avoid 'detaching' not existing paths. - auto it = renames.find(from_path); - if (it != renames.end()) - renames.erase(it); - - renames.insert(to_path); - } - } - } - else if (operation == hardlink) - { - auto src_path = object_attributes["src_path"]; - auto dst_path = object_attributes["dst_path"]; - if (disk->existsFile(src_path)) - { - disk->createDirectories(directoryPath(dst_path)); - disk->createHardLink(src_path, dst_path, send_metadata); - LOG_TRACE(disk->log, "Revision {}. Restored hardlink {} -> {}", revision, src_path, dst_path); - } - } - } - - return true; - }; - - RelativePathsWithMetadata children; - source_object_storage->listObjects(restore_information.source_path + "operations/", children, /* max_keys= */ 0); - restore_file_operations(children); - - if (restore_information.detached) - { - Strings not_finished_prefixes{"tmp_", "delete_tmp_", "attaching_", "deleting_"}; - - auto tx = disk->metadata_storage->createTransaction(); - for (const auto & path : renames) - { - /// Skip already detached parts. - if (path.contains("/detached/")) - continue; - - /// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them. - fs::path directory_path(path); - auto directory_name = directory_path.parent_path().filename().string(); - - auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); }; - if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate)) - continue; - - auto detached_path = pathToDetached(path); - - LOG_TRACE(disk->log, "Move directory to 'detached' {} -> {}", path, detached_path); - - fs::path from_path = fs::path(path); - fs::path to_path = fs::path(detached_path); - if (path.ends_with('/')) - to_path /= from_path.parent_path().filename(); - else - to_path /= from_path.filename(); - - /// to_path may exist and non-empty in case for example abrupt restart, so remove it before rename - if (disk->metadata_storage->existsFileOrDirectory(to_path)) - tx->removeRecursive(to_path); - - disk->createDirectories(directoryPath(to_path)); - tx->moveDirectory(from_path, to_path); - } - tx->commit(); - } - - LOG_INFO(disk->log, "File operations restored for disk {}", disk->name); -} - -} diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h deleted file mode 100644 index ee81e8a209e3..000000000000 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h +++ /dev/null @@ -1,103 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class DiskObjectStorage; - -/// Class implements storage of ObjectStorage metadata inside object storage itself, -/// so it's possible to recover from this remote information in case of local disk loss. -/// -/// This mechanism can be enabled with `true` option inside -/// disk configuration. Implemented only for S3 and Azure Blob storage. Other object storages -/// don't support metadata for blobs. -/// -/// FIXME: this class is very intrusive and use a lot of DiskObjectStorage internals. -/// FIXME: it's very complex and unreliable, need to implement something better. -class DiskObjectStorageRemoteMetadataRestoreHelper -{ -public: - static constexpr UInt64 LATEST_REVISION = std::numeric_limits::max(); - static constexpr UInt64 UNKNOWN_REVISION = 0; - - DiskObjectStorageRemoteMetadataRestoreHelper(DiskObjectStorage * disk_, ReadSettings read_settings_, WriteSettings write_settings_) - : disk(disk_) - , read_settings(std::move(read_settings_)) - , write_settings(std::move(write_settings_)) - , operation_log_suffix("-" + getFQDNOrHostName()) - { - } - - /// Most important method, called on DiskObjectStorage startup - void restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context); - - void syncRevision(UInt64 revision) - { - UInt64 local_revision = revision_counter.load(); - while ((revision > local_revision) && revision_counter.compare_exchange_weak(local_revision, revision)); - } - - UInt64 getRevision() const - { - return revision_counter.load(); - } - - static int readSchemaVersion(IObjectStorage * object_storage, const String & source_path); - - void migrateToRestorableSchema(); - - void findLastRevision(); - - void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const; - - /// Version with possibility to backup-restore metadata. - static constexpr int RESTORABLE_SCHEMA_VERSION = 1; - - std::atomic revision_counter = 0; -private: - struct RestoreInformation - { - UInt64 revision = LATEST_REVISION; - String source_namespace; - String source_path; - bool detached = false; - }; - - using Futures = std::vector>; - - /// Move file or files in directory when possible and remove files in other case - /// to restore by S3 operation log with same operations from different replicas - void moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata); - - void saveSchemaVersion(const int & version) const; - void updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const; - void migrateFileToRestorableSchema(const String & path) const; - void migrateToRestorableSchemaRecursive(const String & path, ThreadPool & pool); - - void readRestoreInformation(RestoreInformation & restore_information); - void restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information); - void processRestoreFiles(IObjectStorage * source_object_storage, const String & source_path, const std::vector & keys) const; - void restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information); - - inline static const String RESTORE_FILE_NAME = "restore"; - - /// Object contains information about schema version. - inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION"; - /// Directories with data. - const std::vector data_roots {"data", "store"}; - - DiskObjectStorage * disk; - - ObjectStoragePtr object_storage_from_another_namespace; - - ReadSettings read_settings; - WriteSettings write_settings; - - String operation_log_suffix; -}; - -} diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 5a11fd0d50ab..bd713dc987cf 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -26,38 +26,32 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int FILE_DOESNT_EXIST; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int LOGICAL_ERROR; } DiskObjectStorageTransaction::DiskObjectStorageTransaction( IObjectStorage & object_storage_, - IMetadataStorage & metadata_storage_, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_) + IMetadataStorage & metadata_storage_) : object_storage(object_storage_) , metadata_storage(metadata_storage_) , metadata_transaction(metadata_storage.createTransaction()) - , metadata_helper(metadata_helper_) {} DiskObjectStorageTransaction::DiskObjectStorageTransaction( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_, MetadataTransactionPtr metadata_transaction_) : object_storage(object_storage_) , metadata_storage(metadata_storage_) , metadata_transaction(metadata_transaction_) - , metadata_helper(metadata_helper_) {} MultipleDisksObjectStorageTransaction::MultipleDisksObjectStorageTransaction( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, - IObjectStorage& destination_object_storage_, - IMetadataStorage& destination_metadata_storage_, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_) - : DiskObjectStorageTransaction(object_storage_, metadata_storage_, metadata_helper_, destination_metadata_storage_.createTransaction()) + IObjectStorage & destination_object_storage_, + IMetadataStorage & destination_metadata_storage_) + : DiskObjectStorageTransaction(object_storage_, metadata_storage_, destination_metadata_storage_.createTransaction()) , destination_object_storage(destination_object_storage_) , destination_metadata_storage(destination_metadata_storage_) {} @@ -743,16 +737,6 @@ void DiskObjectStorageTransaction::removeSharedFiles( operations_to_execute.emplace_back(std::move(operation)); } -namespace -{ - -String revisionToString(UInt64 revision) -{ - return std::bitset<64>(revision).to_string(); -} - -} - std::unique_ptr DiskObjectStorageTransaction::writeFile( /// NOLINT const std::string & path, size_t buf_size, @@ -763,22 +747,6 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; - if (metadata_helper) - { - if (!object_key.hasPrefix()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata helper is not supported with absolute paths"); - - auto revision = ++metadata_helper->revision_counter; - object_attributes = - { - {"path", path} - }; - - object_key = ObjectStorageKey::createAsRelative( - object_key.getPrefix(), - "r" + revisionToString(revision) + "-file-" + object_key.getSuffix()); - } - /// Does metadata_storage support empty files without actual blobs in the object_storage? const bool do_not_write_empty_blob = metadata_storage.supportsEmptyFilesWithoutBlobs(); @@ -882,22 +850,6 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction( auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */); std::optional object_attributes; - if (metadata_helper) - { - if (!object_key.hasPrefix()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "metadata helper is not supported with abs paths"); - - auto revision = metadata_helper->revision_counter + 1; - metadata_helper->revision_counter++; - object_attributes = { - {"path", path} - }; - - object_key = ObjectStorageKey::createAsRelative( - object_key.getPrefix(), - "r" + revisionToString(revision) + "-file-" + object_key.getSuffix()); - } - /// seems ok auto object = StoredObject(object_key.serialize(), path); auto write_operation = std::make_unique(object_storage, metadata_storage, object); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index d69378b8cba8..a8a7bb74e2d7 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -59,22 +59,17 @@ struct DiskObjectStorageTransaction : public IDiskTransaction, std::enable_share MetadataTransactionPtr metadata_transaction; - /// TODO we can get rid of this params - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper; - DiskObjectStorageOperations operations_to_execute; DiskObjectStorageTransaction( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_, MetadataTransactionPtr metadata_transaction_); public: DiskObjectStorageTransaction( IObjectStorage & object_storage_, - IMetadataStorage & metadata_storage_, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_); + IMetadataStorage & metadata_storage_); void commit() override; void undo() override; @@ -129,15 +124,14 @@ struct DiskObjectStorageTransaction : public IDiskTransaction, std::enable_share struct MultipleDisksObjectStorageTransaction final : public DiskObjectStorageTransaction, std::enable_shared_from_this { - IObjectStorage& destination_object_storage; - IMetadataStorage& destination_metadata_storage; + IObjectStorage & destination_object_storage; + IMetadataStorage & destination_metadata_storage; MultipleDisksObjectStorageTransaction( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, - IObjectStorage& destination_object_storage, - IMetadataStorage& destination_metadata_storage, - DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_); + IObjectStorage & destination_object_storage, + IMetadataStorage & destination_metadata_storage); void copyFile(const std::string & from_file_path, const std::string & to_file_path, const ReadSettings & read_settings, const WriteSettings &) override; }; diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index c2f14fb43633..5dea86c49027 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -237,14 +237,6 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -std::unique_ptr HDFSObjectStorage::cloneObjectStorage( - const std::string &, - const Poco::Util::AbstractConfiguration &, - const std::string &, ContextPtr) -{ - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning"); -} - } #endif diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 28b75d44f21e..733407e236ef 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -96,12 +96,6 @@ class HDFSObjectStorage : public IObjectStorage, public HDFSErrorWrapper String getObjectsNamespace() const override { return ""; } - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool areObjectKeysRandom() const override { return true; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 0d9464b1ad7e..523ca03a036a 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -237,13 +237,6 @@ class IObjectStorage /// buckets in S3. If object storage doesn't have any namepaces return empty string. virtual String getObjectsNamespace() const = 0; - /// FIXME: confusing function required for a very specific case. Create new instance of object storage - /// in different namespace. - virtual std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, ContextPtr context) = 0; - /// Generate blob name for passed absolute local path. /// Path can be generated either independently or based on `path`. virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const = 0; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index d21b4464300b..2c444954d538 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; @@ -204,14 +203,6 @@ void LocalObjectStorage::throwIfReadonly() const throw Exception(ErrorCodes::READONLY, "Local object storage `{}` is readonly", getName()); } -std::unique_ptr LocalObjectStorage::cloneObjectStorage( - const std::string & /* new_namespace */, - const Poco::Util::AbstractConfiguration & /* config */, - const std::string & /* config_prefix */, ContextPtr /* context */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage is not implemented for LocalObjectStorage"); -} - ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional & /* key_prefix */) const { diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index cbb82c8e5393..a8a9fe321894 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -79,12 +79,6 @@ class LocalObjectStorage : public IObjectStorage String getObjectsNamespace() const override { return ""; } - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; bool areObjectKeysRandom() const override { return true; } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index bd9470eb14f5..ceebde13ae0a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -207,14 +207,6 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) const ContextPtr & context, bool /* skip_access_check */) -> ObjectStoragePtr { - /// send_metadata changes the filenames (includes revision), while - /// s3_plain do not care about this, and expect that the file name - /// will not be changed. - /// - /// And besides, send_metadata does not make sense for s3_plain. - if (config.getBool(config_prefix + ".send_metadata", false)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain does not supports send_metadata"); - auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); @@ -242,11 +234,6 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) const ContextPtr & context, bool /* skip_access_check */) -> ObjectStoragePtr { - /// send_metadata changes the filenames (includes revision), while - /// s3_plain_rewritable does not support file renaming. - if (config.getBool(config_prefix + ".send_metadata", false)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain_rewritable does not supports send_metadata"); - auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto endpoint = getEndpoint(config, config_prefix, context); diff --git a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp index 0963dd37974e..a1baafe8b6b0 100644 --- a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp @@ -48,7 +48,7 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch config, config_prefix); - disk->startup(context, skip_access_check); + disk->startup(skip_access_check); return disk; }; diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index 1bb33f564450..5ea712cbd56a 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -17,13 +17,6 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator( const String & config_prefix) { bool storage_metadata_write_full_object_key = DiskObjectStorageMetadata::getWriteFullObjectKeySetting(); - bool send_metadata = config.getBool(config_prefix + ".send_metadata", false); - - if (send_metadata && storage_metadata_write_full_object_key) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Wrong configuration in {}. " - "s3 does not supports feature 'send_metadata' with feature 'storage_metadata_write_full_object_key'.", - config_prefix); String object_key_compatibility_prefix = config.getString(config_prefix + ".key_compatibility_prefix", String()); String object_key_template = config.getString(config_prefix + ".key_template", String()); @@ -39,12 +32,6 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator( return createObjectStorageKeysGeneratorByPrefix(uri.key); } - if (send_metadata) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Wrong configuration in {}. " - "s3 does not supports send_metadata with setting 'key_template'.", - config_prefix); - if (!storage_metadata_write_full_object_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong configuration in {}. " diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 6d8dacd40fe9..1308fea6b2fb 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -514,23 +514,6 @@ void S3ObjectStorage::applyNewSettings( s3_settings.set(std::move(modified_settings)); } -std::unique_ptr S3ObjectStorage::cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) -{ - const auto & settings = context->getSettingsRef(); - auto new_s3_settings = getSettings(config, config_prefix, context, uri.uri_str, settings[Setting::s3_validate_request_settings]); - auto new_client = getClient(uri, *new_s3_settings, context, for_disk_s3); - - auto new_uri{uri}; - new_uri.bucket = new_namespace; - - return std::make_unique( - std::move(new_client), std::move(new_s3_settings), new_uri, s3_capabilities, key_generator, disk_name); -} - ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { if (!key_generator) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 2283ebe54c06..9817d66fb965 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -119,12 +119,6 @@ class S3ObjectStorage : public IObjectStorage bool isRemote() const override { return true; } - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - bool supportParallelWrite() const override { return true; } ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 5d1d95143fb5..6559be348da3 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -283,12 +283,4 @@ ObjectMetadata WebObjectStorage::getObjectMetadata(const std::string & /* path * throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Metadata is not supported for {}", getName()); } -std::unique_ptr WebObjectStorage::cloneObjectStorage( - const std::string & /* new_namespace */, - const Poco::Util::AbstractConfiguration & /* config */, - const std::string & /* config_prefix */, ContextPtr /* context */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage is not implemented for WebObjectStorage"); -} - } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index c96b0bd1e724..2aa59afbcdcf 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -66,12 +66,6 @@ class WebObjectStorage : public IObjectStorage, WithContext String getObjectsNamespace() const override { return ""; } - std::unique_ptr cloneObjectStorage( - const std::string & new_namespace, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & /* key_prefix */) const override { return ObjectStorageKey::createAsRelative(path); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 79a3c24678aa..bf9998b8a5f6 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -270,16 +270,6 @@ bool DataPartStorageOnDiskBase::isReadonly() const return volume->getDisk()->isReadOnly() || volume->getDisk()->isWriteOnce(); } -void DataPartStorageOnDiskBase::syncRevision(UInt64 revision) const -{ - volume->getDisk()->syncRevision(revision); -} - -UInt64 DataPartStorageOnDiskBase::getRevision() const -{ - return volume->getDisk()->getRevision(); -} - std::string DataPartStorageOnDiskBase::getDiskPath() const { return volume->getDisk()->getPath(); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 919b6bf0f205..7b5785f75b25 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -42,8 +42,6 @@ class DataPartStorageOnDiskBase : public IDataPartStorage bool supportParallelWrite() const override; bool isBroken() const override; bool isReadonly() const override; - void syncRevision(UInt64 revision) const override; - UInt64 getRevision() const override; std::string getDiskPath() const override; ReservationPtr reserve(UInt64 bytes) const override; ReservationPtr tryReserve(UInt64 bytes) const override; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 11161be0bc2a..5056b4d17892 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -158,17 +158,6 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend}; - if (part->getDataPartStorage().isStoredOnRemoteDisk()) - { - UInt64 revision = parse(params.get("disk_revision", "0")); - if (revision) - part->getDataPartStorage().syncRevision(revision); - - revision = part->getDataPartStorage().getRevision(); - if (revision) - response.addCookie({"disk_revision", toString(revision)}); - } - if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) writeBinary(part->checksums.getTotalSizeOnDisk(), out); @@ -457,12 +446,7 @@ std::pair Fetcher::fetchSelected }); if (disk) - { LOG_TRACE(log, "Will fetch to disk {} with type {}", disk->getName(), disk->getDataSourceDescription().toString()); - UInt64 revision = disk->getRevision(); - if (revision) - uri.addQueryParameter("disk_revision", toString(revision)); - } Strings capability; if (try_zero_copy && (*data_settings)[MergeTreeSetting::allow_remote_fs_zero_copy_replication]) @@ -597,11 +581,6 @@ std::pair Fetcher::fetchSelected LOG_TEST(log, "Disk for fetch is disk {} with type {}", disk->getName(), disk->getDataSourceDescription().toString()); } - UInt64 revision = parse(in->getResponseCookie("disk_revision", "0")); - - if (revision) - disk->syncRevision(revision); - bool sync = ((*data_settings)[MergeTreeSetting::min_compressed_bytes_to_fsync_after_fetch] && sum_files_size >= (*data_settings)[MergeTreeSetting::min_compressed_bytes_to_fsync_after_fetch]); diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index cf728128a875..eb98d3496b89 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -187,10 +187,6 @@ class IDataPartStorage : public boost::noncopyable virtual bool isBroken() const = 0; virtual bool isReadonly() const = 0; - /// TODO: remove or at least remove const. - virtual void syncRevision(UInt64 revision) const = 0; - virtual UInt64 getRevision() const = 0; - /// Get a path for internal disk if relevant. It is used mainly for logging. virtual std::string getDiskPath() const = 0; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aa3ac075a5a7..29f22dde5b5a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8326,9 +8326,6 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( String backup_name = (!with_name.empty() ? escapeForFileName(with_name) : toString(increment)); String backup_path = fs::path(shadow_path) / backup_name / ""; - for (const auto & disk : getStoragePolicy()->getDisks()) - disk->onFreeze(backup_path); - PartitionCommandsResultInfo result; size_t parts_processed = 0; diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 452e31ddbaa2..f5f3bb90ecaf 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -7,21 +7,18 @@ http://minio1:9001/root/data/ minio ClickHouse_Minio_P@ssw0rd - false s3 http://minio1:9001/root/data/ minio ClickHouse_Minio_P@ssw0rd - false s3 http://minio1:9001/root/data2/ minio ClickHouse_Minio_P@ssw0rd - false diff --git a/tests/integration/test_userspace_page_cache/configs/conf.xml b/tests/integration/test_userspace_page_cache/configs/conf.xml index 0a919499ff16..d4490f4b3c42 100644 --- a/tests/integration/test_userspace_page_cache/configs/conf.xml +++ b/tests/integration/test_userspace_page_cache/configs/conf.xml @@ -6,7 +6,6 @@ http://minio1:9001/root/data/ minio ClickHouse_Minio_P@ssw0rd - true From 31d09b7a0b6b6cd130b0a029bcc91f4a2db03e6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Jul 2025 10:05:00 +0200 Subject: [PATCH 6/9] Merge pull request #84011 from ClickHouse/sync-disks Implement AWS S3 authentication with an explicitly provided IAM role; implement OAuth for GCS. --- .../BackupCoordinationReplicatedTables.cpp | 28 +- .../BackupCoordinationReplicatedTables.h | 1 - src/Backups/BackupEntriesCollector.cpp | 8 +- src/Backups/BackupEntryWrappedWith.h | 4 + src/Backups/BackupFileInfo.cpp | 6 + src/Backups/BackupIO_AzureBlobStorage.cpp | 23 +- src/Backups/BackupIO_S3.cpp | 38 ++- src/Backups/BackupIO_S3.h | 4 + src/Backups/BackupImpl.cpp | 128 ++++++++- src/Backups/BackupImpl.h | 23 +- src/Backups/BackupInfo.cpp | 8 + src/Backups/BackupInfo.h | 1 + src/Backups/BackupsWorker.cpp | 24 +- src/Backups/BackupsWorker.h | 5 + src/Backups/IBackup.h | 3 + src/Backups/IBackupEntriesLazyBatch.cpp | 2 + src/Backups/IRestoreCoordination.h | 3 + src/Backups/RestoreCoordinationLocal.cpp | 6 + src/Backups/RestoreCoordinationLocal.h | 4 + src/Backups/RestoreCoordinationOnCluster.cpp | 28 ++ src/Backups/RestoreCoordinationOnCluster.h | 3 + src/Backups/RestorerFromBackup.cpp | 75 ++++- .../registerBackupEngineAzureBlobStorage.cpp | 40 ++- src/Backups/registerBackupEngineS3.cpp | 111 +++++++- src/CMakeLists.txt | 28 ++ src/Common/FailPoint.cpp | 18 +- src/Common/Macros.cpp | 6 + src/Common/Macros.h | 1 + src/Common/OpenSSLHelpers.cpp | 50 ++++ src/Common/OpenSSLHelpers.h | 4 + src/Common/SystemLogBase.cpp | 10 + src/Common/SystemLogBase.h | 6 + src/Core/DistributedCacheDefines.h | 12 +- src/Dictionaries/SSDCacheDictionaryStorage.h | 2 +- src/Disks/DiskCommitTransactionOptions.h | 116 ++++++++ src/Disks/DiskEncrypted.cpp | 77 +++-- src/Disks/DiskEncryptedTransaction.h | 11 +- src/Disks/DiskType.cpp | 11 +- src/Disks/DiskType.h | 18 +- src/Disks/FakeDiskTransaction.h | 2 +- src/Disks/IDiskTransaction.h | 25 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- src/Disks/MetadataStorageWithPathWrapper.h | 14 +- .../AzureBlobStorage/AzureBlobStorageCommon.h | 1 + .../AzureBlobStorage/AzureObjectStorage.cpp | 12 +- .../AzureBlobStorage/AzureObjectStorage.h | 13 +- .../AzureObjectStorageConnectionInfo.cpp | 148 ++++++++++ .../AzureObjectStorageConnectionInfo.h | 78 +++++ .../Cached/CachedObjectStorage.h | 2 + .../ObjectStorages/DiskObjectStorage.cpp | 153 ++++++++-- src/Disks/ObjectStorages/DiskObjectStorage.h | 13 +- .../DiskObjectStorageMetadata.cpp | 16 +- .../DiskObjectStorageTransaction.cpp | 187 +++++++++++- .../DiskObjectStorageTransaction.h | 6 +- src/Disks/ObjectStorages/IMetadataStorage.h | 39 ++- src/Disks/ObjectStorages/IObjectStorage.h | 26 +- .../IObjectStorageConnectionInfo.cpp | 75 +++++ .../IObjectStorageConnectionInfo.h | 96 +++++++ .../MetadataOperationsHolder.cpp | 5 +- .../ObjectStorages/MetadataOperationsHolder.h | 3 +- .../ObjectStorages/MetadataStorageFactory.cpp | 38 ++- .../MetadataStorageFromDisk.cpp | 11 +- .../ObjectStorages/MetadataStorageFromDisk.h | 6 +- .../MetadataStorageFromPlainObjectStorage.cpp | 12 +- .../MetadataStorageFromPlainObjectStorage.h | 9 +- .../ObjectStorageClientsCache.h | 99 +++++++ .../ObjectStorages/ObjectStorageFactory.cpp | 25 +- .../RegisterDiskObjectStorage.cpp | 22 +- src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 10 + .../ObjectStorages/S3/S3ObjectStorage.cpp | 6 + src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 + .../S3/S3ObjectStorageConnectionInfo.cpp | 266 ++++++++++++++++++ .../S3/S3ObjectStorageConnectionInfo.h | 75 +++++ src/Disks/ObjectStorages/S3/diskSettings.cpp | 15 + .../MetadataStorageFromStaticFilesWebServer.h | 11 +- .../ObjectStorages/Web/WebObjectStorage.cpp | 6 + .../ObjectStorages/Web/WebObjectStorage.h | 2 + .../Web/WebObjectStorageConnectionInfo.cpp | 70 +++++ .../Web/WebObjectStorageConnectionInfo.h | 11 + src/Disks/ReadOnlyDiskWrapper.h | 134 +++++++++ src/Disks/tests/gtest_disk_encrypted.cpp | 2 +- src/IO/BufferWithOwnMemory.h | 33 ++- src/IO/DistributedCacheSettings.h | 14 +- src/IO/ReadBufferFromS3.cpp | 13 + src/IO/S3/Client.cpp | 22 +- src/IO/S3/Client.h | 10 + src/IO/S3/Credentials.cpp | 165 ++++++++++- src/IO/S3/Credentials.h | 98 ++++++- src/IO/S3/PocoHTTPClient.cpp | 107 ++++++- src/IO/S3/PocoHTTPClient.h | 50 +++- src/IO/S3/PocoHTTPClientFactory.cpp | 11 +- src/IO/S3/tests/TestPocoHTTPServer.h | 115 ++++++++ src/IO/S3/tests/gtest_aws_s3_client.cpp | 186 ++++++++++++ src/IO/S3AuthSettings.cpp | 8 +- src/IO/WriteBufferFromS3.cpp | 2 +- src/Interpreters/SystemLog.cpp | 17 ++ src/Interpreters/SystemLog.h | 12 + .../Formats/Impl/AvroRowInputFormat.cpp | 2 +- .../ObjectStorage/Azure/Configuration.cpp | 4 +- .../ObjectStorage/S3/Configuration.cpp | 104 ++++++- src/Storages/ObjectStorage/S3/Configuration.h | 3 + .../StorageObjectStorageSource.cpp | 85 +++++- 102 files changed, 3531 insertions(+), 226 deletions(-) create mode 100644 src/Disks/DiskCommitTransactionOptions.h create mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp create mode 100644 src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.h create mode 100644 src/Disks/ObjectStorages/IObjectStorageConnectionInfo.cpp create mode 100644 src/Disks/ObjectStorages/IObjectStorageConnectionInfo.h create mode 100644 src/Disks/ObjectStorages/ObjectStorageClientsCache.h create mode 100644 src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.cpp create mode 100644 src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.h create mode 100644 src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.cpp create mode 100644 src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.h create mode 100644 src/Disks/ReadOnlyDiskWrapper.h diff --git a/src/Backups/BackupCoordinationReplicatedTables.cpp b/src/Backups/BackupCoordinationReplicatedTables.cpp index b444dd4bb90c..224415439bce 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.cpp +++ b/src/Backups/BackupCoordinationReplicatedTables.cpp @@ -1,6 +1,9 @@ #include #include #include +#if CLICKHOUSE_CLOUD + #include +#endif #include #include @@ -292,13 +295,6 @@ void BackupCoordinationReplicatedTables::prepare() const auto part_info = MergeTreePartInfo::fromPartName(part_name, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); [[maybe_unused]] const auto & partition_id = part_info.getPartitionId(); - auto & min_data_versions_by_partition = table_info.min_data_versions_by_partition; - auto it2 = min_data_versions_by_partition.find(partition_id); - if (it2 == min_data_versions_by_partition.end()) - min_data_versions_by_partition[partition_id] = part_info.getDataVersion(); - else - it2->second = std::min(it2->second, part_info.getDataVersion()); - table_info.covered_parts_finder->addPartInfo(std::move(part_info), part_replicas.replica_names[0]); } @@ -310,24 +306,6 @@ void BackupCoordinationReplicatedTables::prepare() const const auto & chosen_replica_name = *part_replicas.replica_names[chosen_index]; table_info.part_names_by_replica_name[chosen_replica_name].push_back(part_name); } - - /// Remove finished or unrelated mutations. - std::unordered_map unfinished_mutations; - for (const auto & [mutation_id, mutation_entry_str] : table_info.mutations) - { - auto mutation_entry = ReplicatedMergeTreeMutationEntry::parse(mutation_entry_str, mutation_id); - std::map new_block_numbers; - for (const auto & [partition_id, block_number] : mutation_entry.block_numbers) - { - auto it = table_info.min_data_versions_by_partition.find(partition_id); - if ((it != table_info.min_data_versions_by_partition.end()) && (it->second < block_number)) - new_block_numbers[partition_id] = block_number; - } - mutation_entry.block_numbers = std::move(new_block_numbers); - if (!mutation_entry.block_numbers.empty()) - unfinished_mutations[mutation_id] = mutation_entry.toString(); - } - table_info.mutations = unfinished_mutations; } catch (Exception & e) { diff --git a/src/Backups/BackupCoordinationReplicatedTables.h b/src/Backups/BackupCoordinationReplicatedTables.h index 50ab56aef756..0613570dd25c 100644 --- a/src/Backups/BackupCoordinationReplicatedTables.h +++ b/src/Backups/BackupCoordinationReplicatedTables.h @@ -104,7 +104,6 @@ class BackupCoordinationReplicatedTables std::map replicas_by_part_name; /// Should be ordered because we need this map to be in the same order on every replica. mutable std::unordered_map */, Strings> part_names_by_replica_name; std::unique_ptr covered_parts_finder; - mutable std::unordered_map min_data_versions_by_partition; mutable std::unordered_map mutations; String replica_name_to_store_mutations; std::unordered_set data_paths; diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 22dbffe61c1e..4ce7a5f99edc 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -45,6 +45,9 @@ namespace Setting extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms; extern const SettingsUInt64 backup_restore_keeper_max_retries; extern const SettingsSeconds lock_acquire_timeout; + + /// Cloud only + extern const SettingsBool cloud_mode; } namespace ErrorCodes @@ -112,7 +115,10 @@ BackupEntriesCollector::BackupEntriesCollector( context->getConfigRef().getUInt64("backups.min_sleep_before_next_attempt_to_collect_metadata", 100)) , max_sleep_before_next_attempt_to_collect_metadata( context->getConfigRef().getUInt64("backups.max_sleep_before_next_attempt_to_collect_metadata", 5000)) - , compare_collected_metadata(context->getConfigRef().getBool("backups.compare_collected_metadata", true)) + , compare_collected_metadata( + context->getConfigRef().getBool("backups.compare_collected_metadata", + !context->getSettingsRef()[Setting::cloud_mode])) /// Collected metadata shouldn't be compared by default in our Cloud + /// (because in the Cloud only Replicated databases are used) , log(getLogger("BackupEntriesCollector")) , zookeeper_retries_info( context->getSettingsRef()[Setting::backup_restore_keeper_max_retries], diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index 6a609a4582ad..9bb27e92da12 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -23,6 +23,10 @@ class BackupEntryWrappedWith : public IBackupEntry bool isEncryptedByDisk() const override { return entry->isEncryptedByDisk(); } bool isFromFile() const override { return entry->isFromFile(); } bool isFromImmutableFile() const override { return entry->isFromImmutableFile(); } + bool isFromRemoteFile() const override { return entry->isFromRemoteFile(); } + String getEndpointURI() const override { return entry->getEndpointURI(); } + String getNamespace() const override { return entry->getNamespace(); } + String getRemotePath() const override { return entry->getRemotePath(); } String getFilePath() const override { return entry->getFilePath(); } DiskPtr getDisk() const override { return entry->getDisk(); } diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index 0fce155ace3e..374750e33786 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -124,6 +124,12 @@ BackupFileInfo buildFileInfoForBackupEntry( info.size = backup_entry->getSize(); info.encrypted_by_disk = backup_entry->isEncryptedByDisk(); + if (backup_entry->isFromRemoteFile()) + { + info.object_key = backup_entry->getRemotePath(); + return info; + } + /// We don't set `info.data_file_name` and `info.data_file_index` in this function because they're set during backup coordination /// (see the class BackupCoordinationFileInfos). diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index bf13f254eac3..cdc6b194f84a 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -13,6 +13,7 @@ #include #include +#include #include @@ -65,6 +66,16 @@ static bool compareAuthMethod (AzureBlobStorage::AuthMethod auth_method_a, Azure Azure::Core::Credentials::TokenRequestContext tokenRequestContext; return managed_identity_a->get()->GetToken(tokenRequestContext, {}).Token == managed_identity_b->get()->GetToken(tokenRequestContext, {}).Token; } + + const auto * static_credential_a = std::get_if>(&auth_method_a); + const auto * static_credential_b = std::get_if>(&auth_method_b); + + if (static_credential_a && static_credential_b) + { + Azure::Core::Credentials::TokenRequestContext tokenRequestContext; + auto az_context = Azure::Core::Context(); + return static_credential_a->get()->GetToken(tokenRequestContext, az_context).Token == static_credential_b->get()->GetToken(tokenRequestContext, az_context).Token; + } } catch (const Azure::Core::Credentials::AuthenticationException & e) { @@ -83,7 +94,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false, ""} , connection_params(connection_params_) , blob_path(blob_path_) { @@ -95,8 +106,10 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( connection_params.auth_method, std::move(client_ptr), std::move(settings_ptr), + connection_params, connection_params.getContainer(), - connection_params.getConnectionURL()); + connection_params.getConnectionURL(), + /*common_key_prefix*/ ""); client = object_storage->getAzureBlobStorageClient(); settings = object_storage->getSettings(); @@ -177,7 +190,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const ContextPtr & context_, bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, connection_params_.getConnectionURL(), false, false, ""} , connection_params(connection_params_) , blob_path(blob_path_) { @@ -192,8 +205,10 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( connection_params.auth_method, std::move(client_ptr), std::move(settings_ptr), + connection_params, connection_params.getContainer(), - connection_params.getConnectionURL()); + connection_params.getConnectionURL(), + /*common_key_prefix*/ ""); client = object_storage->getAzureBlobStorageClient(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 69305917c48b..47e89b65c872 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -46,6 +46,13 @@ namespace S3AuthSetting extern const S3AuthSettingsString server_side_encryption_customer_key_base64; extern const S3AuthSettingsBool use_environment_credentials; extern const S3AuthSettingsBool use_insecure_imds_request; + + extern const S3AuthSettingsString role_arn; + extern const S3AuthSettingsString role_session_name; + extern const S3AuthSettingsString http_client; + extern const S3AuthSettingsString service_account; + extern const S3AuthSettingsString metadata_service; + extern const S3AuthSettingsString request_token_path; } namespace S3RequestSetting @@ -69,6 +76,8 @@ namespace const S3::URI & s3_uri, const String & access_key_id, const String & secret_access_key, + String role_arn, + String role_session_name, const S3Settings & settings, const ContextPtr & context) { @@ -84,6 +93,12 @@ namespace const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); const Settings & local_settings = context->getSettingsRef(); + if (role_arn.empty()) + { + role_arn = settings.auth_settings[S3AuthSetting::role_arn]; + role_session_name = settings.auth_settings[S3AuthSetting::role_session_name]; + } + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings[S3AuthSetting::region], context->getRemoteHostFilter(), @@ -108,6 +123,11 @@ namespace client_configuration.http_max_field_name_size = request_settings[S3RequestSetting::http_max_field_name_size]; client_configuration.http_max_field_value_size = request_settings[S3RequestSetting::http_max_field_value_size]; + client_configuration.http_client = settings.auth_settings[S3AuthSetting::http_client]; + client_configuration.service_account = settings.auth_settings[S3AuthSetting::service_account]; + client_configuration.metadata_service = settings.auth_settings[S3AuthSetting::metadata_service]; + client_configuration.request_token_path = settings.auth_settings[S3AuthSetting::request_token_path]; + S3::ClientSettings client_settings{ .use_virtual_addressing = s3_uri.is_virtual_hosted_style, .disable_checksum = local_settings[Setting::s3_disable_checksum], @@ -128,7 +148,10 @@ namespace settings.auth_settings[S3AuthSetting::use_environment_credentials], settings.auth_settings[S3AuthSetting::use_insecure_imds_request], settings.auth_settings[S3AuthSetting::expiration_window_seconds], - settings.auth_settings[S3AuthSetting::no_sign_request] + settings.auth_settings[S3AuthSetting::no_sign_request], + std::move(role_arn), + std::move(role_session_name), + /*sts_endpoint_override=*/"" }); } @@ -150,6 +173,8 @@ BackupReaderS3::BackupReaderS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, + const String & role_arn, + const String & role_session_name, bool allow_s3_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, @@ -157,7 +182,7 @@ BackupReaderS3::BackupReaderS3( bool is_internal_backup) : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3")) , s3_uri(s3_uri_) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false, ""} { s3_settings.loadFromConfig(context_->getConfigRef(), "s3", context_->getSettingsRef()); @@ -170,7 +195,7 @@ BackupReaderS3::BackupReaderS3( s3_settings.request_settings.updateFromSettings(context_->getSettingsRef(), /* if_changed */true); s3_settings.request_settings[S3RequestSetting::allow_native_copy] = allow_s3_native_copy; - client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); + client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, role_arn, role_session_name, s3_settings, context_); if (auto blob_storage_system_log = context_->getBlobStorageLog()) blob_storage_log = std::make_shared(blob_storage_system_log); @@ -247,6 +272,8 @@ BackupWriterS3::BackupWriterS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, + const String & role_arn, + const String & role_session_name, bool allow_s3_native_copy, const String & storage_class_name, const ReadSettings & read_settings_, @@ -255,7 +282,7 @@ BackupWriterS3::BackupWriterS3( bool is_internal_backup) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3")) , s3_uri(s3_uri_) - , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false, ""} , s3_capabilities(getCapabilitiesFromConfig(context_->getConfigRef(), "s3")) { s3_settings.loadFromConfig(context_->getConfigRef(), "s3", context_->getSettingsRef()); @@ -270,7 +297,8 @@ BackupWriterS3::BackupWriterS3( s3_settings.request_settings[S3RequestSetting::allow_native_copy] = allow_s3_native_copy; s3_settings.request_settings[S3RequestSetting::storage_class_name] = storage_class_name; - client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); + client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, role_arn, role_session_name, s3_settings, context_); + if (auto blob_storage_system_log = context_->getBlobStorageLog()) { blob_storage_log = std::make_shared(blob_storage_system_log); diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 9f58e7553157..6cedf04a67af 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -23,6 +23,8 @@ class BackupReaderS3 : public BackupReaderDefault const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, + const String & role_arn, + const String & role_session_name, bool allow_s3_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, @@ -54,6 +56,8 @@ class BackupWriterS3 : public BackupWriterDefault const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, + const String & role_arn, + const String & role_session_name, bool allow_s3_native_copy, const String & storage_class_name, const ReadSettings & read_settings_, diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index cc58fa590e53..c77423b064fd 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -30,6 +31,7 @@ namespace ProfileEvents { extern const Event BackupsOpenedForRead; extern const Event BackupsOpenedForWrite; + extern const Event BackupsOpenedForUnlock; extern const Event BackupReadMetadataMicroseconds; extern const Event BackupWriteMetadataMicroseconds; extern const Event BackupLockFileReads; @@ -37,12 +39,14 @@ namespace ProfileEvents namespace DB { + namespace ErrorCodes { extern const int BACKUP_NOT_FOUND; extern const int BACKUP_ALREADY_EXISTS; extern const int BACKUP_VERSION_NOT_SUPPORTED; extern const int BACKUP_DAMAGED; + extern const int BAD_ARGUMENTS; extern const int NO_BASE_BACKUP; extern const int WRONG_BASE_BACKUP; extern const int BACKUP_ENTRY_NOT_FOUND; @@ -129,6 +133,22 @@ BackupImpl::BackupImpl( open(); } +BackupImpl::BackupImpl( + const BackupInfo & backup_info_, + const ArchiveParams & archive_params_, + std::shared_ptr reader_, + std::shared_ptr lightweight_snapshot_writer_) + : backup_info(backup_info_) + , backup_name_for_logging(backup_info.toStringForLogging()) + , use_archive(!archive_params_.archive_name.empty()) + , archive_params(archive_params_) + , open_mode(OpenMode::UNLOCK) + , reader(reader_) + , lightweight_snapshot_writer(lightweight_snapshot_writer_) + , log(getLogger("BackupImpl")) +{ + open(); +} BackupImpl::~BackupImpl() { @@ -153,7 +173,12 @@ void BackupImpl::open() { std::lock_guard lock{mutex}; - if (open_mode == OpenMode::READ) + if (open_mode == OpenMode::UNLOCK) + { + ProfileEvents::increment(ProfileEvents::BackupsOpenedForUnlock); + LOG_INFO(log, "Unlocking backup: {}", backup_name_for_logging); + } + else if (open_mode == OpenMode::READ) { ProfileEvents::increment(ProfileEvents::BackupsOpenedForRead); LOG_INFO(log, "Reading backup: {}", backup_name_for_logging); @@ -179,7 +204,7 @@ void BackupImpl::open() if (use_archive) openArchive(); - if (open_mode == OpenMode::READ) + if (open_mode == OpenMode::READ || open_mode == OpenMode::UNLOCK) readBackupMetadata(); } @@ -351,6 +376,12 @@ void BackupImpl::writeBackupMetadata() } } + if (params.is_lightweight_snapshot) + { + *out << "" << original_endpoint << ""; + *out << "" << original_namespace << ""; + } + num_files = all_file_infos.size(); total_size = 0; num_entries = 0; @@ -364,6 +395,13 @@ void BackupImpl::writeBackupMetadata() *out << "" << xml << info.file_name << ""; *out << "" << info.size << ""; + if (!info.object_key.empty()) + { + *out << "" << info.object_key << ""; + if (original_endpoint.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "In lightweight snapshot backup, the endpoint should not be empty. Do not run this command with `ON CLUSTER`"); + } + if (info.size) { *out << "" << hexChecksum(info.checksum) << ""; @@ -431,7 +469,6 @@ void BackupImpl::readBackupMetadata() Poco::XML::DOMParser dom_parser; Poco::AutoPtr config = dom_parser.parseMemory(str.data(), str.size()); const Poco::XML::Node * config_root = getRootNode(config); - version = getInt(config_root, "version"); if ((version < INITIAL_BACKUP_VERSION) || (version > CURRENT_BACKUP_VERSION)) throw Exception( @@ -446,6 +483,11 @@ void BackupImpl::readBackupMetadata() if (config_root->getNodeByPath("base_backup_uuid")) base_backup_uuid = parse(getString(config_root, "base_backup_uuid")); + if (config_root->getNodeByPath("original_endpoint")) + original_endpoint = getString(config_root, "original_endpoint"); + if (config_root->getNodeByPath("original_namespace")) + original_namespace = getString(config_root, "original_namespace"); + num_files = 0; total_size = 0; num_entries = 0; @@ -459,7 +501,7 @@ void BackupImpl::readBackupMetadata() const Poco::XML::Node * file_config = child; BackupFileInfo info; info.file_name = getString(file_config, "name"); - + info.object_key = getString(file_config, "object_key", ""); info.size = getUInt64(file_config, "size"); if (info.size) { @@ -495,7 +537,18 @@ void BackupImpl::readBackupMetadata() } file_names.emplace(info.file_name, std::pair{info.size, info.checksum}); - if (info.size) + if (!info.object_key.empty()) + { + if (original_endpoint.empty() || original_namespace.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "In lightweight snapshot backup, the endpoint or namespace should be not empty. We cannot restore this file."); + + if (open_mode == OpenMode::READ) + lightweight_snapshot_reader = lightweight_snapshot_reader_creator(original_endpoint, original_namespace); + + file_object_keys.emplace(info.file_name, info.object_key); + lightweight_snapshot_file_infos.try_emplace(info.object_key, info); + } + else if (info.size) file_infos.try_emplace(std::pair{info.size, info.checksum}, info); ++num_files; @@ -693,8 +746,20 @@ std::unique_ptr BackupImpl::readFile(const String & file return readFileImpl(file_name, size_and_checksum, /* read_encrypted= */ false); } +std::unique_ptr BackupImpl::readFileByObjectKey(const BackupFileInfo & info) const +{ + if (open_mode == OpenMode::WRITE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The backup file should not be opened for writing. Something is wrong internally"); + + if (info.object_key.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Object key of {} is empty string", info.data_file_name); + + return lightweight_snapshot_reader->readFile(info.object_key); +} + std::unique_ptr BackupImpl::readFileImpl(const String & file_name, const SizeAndChecksum & size_and_checksum, bool read_encrypted) const + { if (open_mode == OpenMode::WRITE) throw Exception(ErrorCodes::LOGICAL_ERROR, "The backup file should not be opened for writing. Something is wrong internally"); @@ -790,9 +855,28 @@ BackupImpl::readFileImpl(const String & file_name, const SizeAndChecksum & size_ info.data_file_name, std::move(base_read_buffer), info.base_size, std::move(read_buffer), info.size - info.base_size); } +String BackupImpl::getObjectKey(const String & file_name) const +{ + if (open_mode == OpenMode::WRITE) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The backup file should not be opened for writing. Something is wrong internally"); + + auto adjusted_path = removeLeadingSlash(file_name); + + std::lock_guard lock{mutex}; + auto it = file_object_keys.find(adjusted_path); + if (it != file_object_keys.end()) + return it->second; + return ""; +} + size_t BackupImpl::copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const { +#if CLICKHOUSE_CLOUD + String object_key = getObjectKey(file_name); + if (!object_key.empty()) + return copyFileToDiskByObjectKey(object_key, destination_disk, destination_path, write_mode); +#endif return copyFileToDisk(getFileSizeAndChecksum(file_name), destination_disk, destination_path, write_mode); } @@ -884,6 +968,15 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) if (entry->isReference()) return; + if (entry->isFromRemoteFile()) + { + LOG_TRACE(log, "Writing backup for file {} : skipped because of lightweight snapshot", info.data_file_name); + std::lock_guard lock{mutex}; + original_endpoint = entry->getEndpointURI(); + original_namespace = entry->getNamespace(); + return; + } + if (open_mode == OpenMode::READ) throw Exception(ErrorCodes::LOGICAL_ERROR, "The backup file should not be opened for reading. Something is wrong internally"); @@ -1082,4 +1175,29 @@ bool BackupImpl::tryRemoveAllFiles() noexcept } } +bool BackupImpl::tryRemoveAllFilesUnderDirectory(const String & directory) const noexcept +{ + try + { + LOG_INFO(log, "Removing all files of under directory {}", directory); + + Strings files_to_remove = listFiles(directory, true); + Strings objects_to_remove; + for (const String & file_name : files_to_remove) + { + std::lock_guard lock(mutex); + String file_object_key = file_object_keys.at(fs::path(removeLeadingSlash(directory)) / file_name); + objects_to_remove.push_back(file_object_key); + } + + lightweight_snapshot_writer->removeFiles(objects_to_remove); + return true; + } + catch (...) + { + DB::tryLogCurrentException(log, "Caught exception while removing files of a corrupted backup"); + return false; + } +} + } diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 0fdcf78229d2..91d46103a97a 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -1,5 +1,6 @@ #pragma once +#include "config.h" #include #include #include @@ -48,6 +49,13 @@ class BackupImpl : public IBackup const ArchiveParams & archive_params_, std::shared_ptr writer_); + /// UNLOCK + BackupImpl( + const BackupInfo & backup_info_, + const ArchiveParams & archive_params_, + std::shared_ptr reader_, + std::shared_ptr lightweight_snapshot_writer_); + ~BackupImpl() override; const String & getNameForLogging() const override { return backup_name_for_logging; } @@ -80,6 +88,7 @@ class BackupImpl : public IBackup void finalizeWriting() override; bool setIsCorrupted() noexcept override; bool tryRemoveAllFiles() noexcept override; + bool tryRemoveAllFilesUnderDirectory(const String & directory) const noexcept override; private: void open(); @@ -92,6 +101,13 @@ class BackupImpl : public IBackup void writeBackupMetadata() TSA_REQUIRES(mutex); void readBackupMetadata() TSA_REQUIRES(mutex); +#if CLICKHOUSE_CLOUD + size_t copyFileToDiskByObjectKey(const String & object_key, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const; +#endif + + String getObjectKey(const String & file_name) const; + std::unique_ptr readFileByObjectKey(const BackupFileInfo & info) const; + /// Returns the base backup or null if there is no base backup. std::shared_ptr getBaseBackupUnlocked() const TSA_REQUIRES(mutex); @@ -116,12 +132,17 @@ class BackupImpl : public IBackup const bool use_archive; const ArchiveParams archive_params; const OpenMode open_mode; + /// Used to write data to destinated object storage. std::shared_ptr writer; + /// Used to read data from backup files. std::shared_ptr reader; /// Only used for lightweight backup, we read data from original object storage so the endpoint may be different from the backup files. std::shared_ptr lightweight_snapshot_reader; std::shared_ptr lightweight_snapshot_writer; - SnapshotReaderCreator lightweight_snapshot_reader_creator; + SnapshotReaderCreator lightweight_snapshot_reader_creator; + String original_endpoint; /// endpoint of source disk, we need to write it to metafile to restore a snapshot. + String original_namespace; /// namespace of source disk, we need to write it to metafile to restore a snapshot. + std::shared_ptr coordination; mutable std::mutex mutex; diff --git a/src/Backups/BackupInfo.cpp b/src/Backups/BackupInfo.cpp index 424947e26de2..6e582fb77146 100644 --- a/src/Backups/BackupInfo.cpp +++ b/src/Backups/BackupInfo.cpp @@ -47,6 +47,9 @@ ASTPtr BackupInfo::toAST() const for (const auto & arg : args) list->children.push_back(std::make_shared(arg)); + if (function_arg) + list->children.push_back(function_arg); + return func; } @@ -85,6 +88,11 @@ BackupInfo BackupInfo::fromAST(const IAST & ast) const auto * lit = elem->as(); if (!lit) { + if (index == args_size - 1 && elem->as()) + { + res.function_arg = elem; + break; + } throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected literal, got {}", elem->formatForErrorMessage()); } res.args.push_back(lit->value); diff --git a/src/Backups/BackupInfo.h b/src/Backups/BackupInfo.h index 67af4efa98c6..08218ece35a5 100644 --- a/src/Backups/BackupInfo.h +++ b/src/Backups/BackupInfo.h @@ -15,6 +15,7 @@ struct BackupInfo String backup_engine_name; String id_arg; std::vector args; + ASTPtr function_arg; String toString() const; static BackupInfo fromString(const String & str); diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index eddf7aacb53e..dfed48f20c0a 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -15,11 +15,15 @@ #include #include #include +#if CLICKHOUSE_CLOUD +#include +#endif #include #include #include #include #include +#include #include #include #include @@ -568,6 +572,15 @@ void BackupsWorker::doBackup( bool on_cluster, const ClusterPtr & cluster) { +#if CLICKHOUSE_CLOUD + if (backup_settings.experimental_lightweight_snapshot) + { + auto zookeeper = context->getGlobalContext()->getZooKeeper(); + if (zookeeper->exists(fs::path(LIGHTWEIGHT_SNAPSHOT_COMMIT_PATH) / backup_id)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Backup ID {} has existed. Please unlock this backup or change another name", backup_id); + } +#endif + bool is_internal_backup = backup_settings.internal; /// Checks access rights if this is not ON CLUSTER query. @@ -633,6 +646,16 @@ void BackupsWorker::doBackup( compressed_size = backup->getCompressedSize(); } +#if CLICKHOUSE_CLOUD + /// We need to commit the lightweight backup in keeper indicating the transaction of the backup is done. + if (backup_settings.experimental_lightweight_snapshot && !is_internal_backup) + { + auto zookeeper = context->getGlobalContext()->getZooKeeper(); + zookeeper->create(fs::path(LIGHTWEIGHT_SNAPSHOT_COMMIT_PATH) / backup_id, "", zkutil::CreateMode::Persistent); + LOG_INFO(log, "Snapshot {} has been created", backup_id); + } +#endif + /// NOTE: we need to update metadata again after backup->finalizeWriting(), because backup metadata is written there. setNumFilesAndSize(backup_id, num_files, total_size, num_entries, uncompressed_size, compressed_size, 0, 0); } @@ -900,7 +923,6 @@ BackupPtr BackupsWorker::openBackupForReading(const BackupInfo & backup_info, co return backup; } - void BackupsWorker::doRestore( const std::shared_ptr & restore_query, const OperationID & restore_id, diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index ade057c2fb57..9050f4386526 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -1,5 +1,6 @@ #pragma once +#include "config.h" #include #include #include @@ -99,6 +100,10 @@ class BackupsWorker BackupPtr openBackupForReading(const BackupInfo & backup_info, const RestoreSettings & restore_settings, const ContextPtr & context) const; +#if CLICKHOUSE_CLOUD + BackupMutablePtr openBackupForUnlockSnapshot(const BackupInfo & backup_info, const ContextPtr & context) const; +#endif + void doRestore( const std::shared_ptr & restore_query, const BackupOperationID & restore_id, diff --git a/src/Backups/IBackup.h b/src/Backups/IBackup.h index e47a87ffd0f4..a9b68ef88341 100644 --- a/src/Backups/IBackup.h +++ b/src/Backups/IBackup.h @@ -135,6 +135,9 @@ class IBackup : public std::enable_shared_from_this /// Try to remove all files copied to the backup. Could be used after setIsCorrupted(). virtual bool tryRemoveAllFiles() noexcept = 0; + + /// Try to remove all files of a directory from original object storage. + virtual bool tryRemoveAllFilesUnderDirectory(const String & directory) const noexcept = 0; }; using BackupPtr = std::shared_ptr; diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index f55bcd088b3e..5ce1a972e9fe 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -27,6 +27,8 @@ class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry bool isFromImmutableFile() const override { return getInternalBackupEntry()->isFromImmutableFile(); } bool isFromRemoteFile() const override { return getInternalBackupEntry()->isFromRemoteFile(); } String getRemotePath() const override { return getInternalBackupEntry()->getRemotePath(); } + String getEndpointURI() const override { return getInternalBackupEntry()->getEndpointURI(); } + String getNamespace() const override { return getInternalBackupEntry()->getNamespace(); } String getFilePath() const override { return getInternalBackupEntry()->getFilePath(); } DiskPtr getDisk() const override { return getInternalBackupEntry()->getDisk(); } bool isReference() const override { return getInternalBackupEntry()->isReference(); } diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index e2992915662f..9d00c4bbcd44 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -41,6 +41,9 @@ class IRestoreCoordination /// Removes temporary nodes in ZooKeeper. virtual bool cleanup(bool throw_if_error) = 0; + /// Starts creating a shared database. Returns false if there is another host which is already creating this database. + virtual bool acquireCreatingSharedDatabase(const String & database_name) = 0; + /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. virtual bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) = 0; diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index ef679d2d620f..f6a341870184 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -22,6 +22,12 @@ ZooKeeperRetriesInfo RestoreCoordinationLocal::getOnClusterInitializationKeeperR return {}; } +bool RestoreCoordinationLocal::acquireCreatingSharedDatabase(const String & database_name) +{ + std::lock_guard lock{mutex}; + return acquired_shared_databases.emplace(database_name).second; +} + bool RestoreCoordinationLocal::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) { std::lock_guard lock{mutex}; diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index 6e3262a8a2ea..2eb87c6942cf 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -28,6 +28,9 @@ class RestoreCoordinationLocal : public IRestoreCoordination bool finish(bool) override { return true; } bool cleanup(bool) override { return true; } + /// Starts creating a shared database. Returns false if there is another host which is already creating this database. + bool acquireCreatingSharedDatabase(const String & database_name) override; + /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; @@ -61,6 +64,7 @@ class RestoreCoordinationLocal : public IRestoreCoordination std::unordered_set acquired_data_in_replicated_tables TSA_GUARDED_BY(mutex); std::unordered_map create_query_uuids TSA_GUARDED_BY(mutex); std::unordered_set acquired_data_in_keeper_map_tables TSA_GUARDED_BY(mutex); + std::unordered_set acquired_shared_databases; mutable std::mutex mutex; }; diff --git a/src/Backups/RestoreCoordinationOnCluster.cpp b/src/Backups/RestoreCoordinationOnCluster.cpp index accd4cf4dfe0..550449faa10e 100644 --- a/src/Backups/RestoreCoordinationOnCluster.cpp +++ b/src/Backups/RestoreCoordinationOnCluster.cpp @@ -67,6 +67,8 @@ void RestoreCoordinationOnCluster::createRootNodes() zk->createIfNotExists(zookeeper_path + "/repl_sql_objects_acquired", ""); zk->createIfNotExists(zookeeper_path + "/keeper_map_tables", ""); zk->createIfNotExists(zookeeper_path + "/table_uuids", ""); + + zk->createIfNotExists(zookeeper_path + "/shared_databases_acquired", ""); }); } @@ -126,6 +128,32 @@ ZooKeeperRetriesInfo RestoreCoordinationOnCluster::getOnClusterInitializationKee process_list_element}; } +bool RestoreCoordinationOnCluster::acquireCreatingSharedDatabase(const String & database_name) +{ + bool result = false; + auto holder = with_retries.createRetriesControlHolder("acquireCreatingTableInReplicatedDatabase"); + holder.retries_ctl.retryLoop( + [&, &zk = holder.faulty_zookeeper]() + { + with_retries.renewZooKeeper(zk); + + String path = fs::path(zookeeper_path) / "shared_databases_acquired" / escapeForFileName(database_name); + auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) + throw zkutil::KeeperException::fromPath(code, path); + + if (code == Coordination::Error::ZOK) + { + result = true; + return; + } + + /// We need to check who created that node + result = zk->get(path) == toString(current_host_index); + }); + return result; +} + bool RestoreCoordinationOnCluster::acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) { bool result = false; diff --git a/src/Backups/RestoreCoordinationOnCluster.h b/src/Backups/RestoreCoordinationOnCluster.h index 890117e63e8c..b9b29d32f4d3 100644 --- a/src/Backups/RestoreCoordinationOnCluster.h +++ b/src/Backups/RestoreCoordinationOnCluster.h @@ -38,6 +38,9 @@ class RestoreCoordinationOnCluster : public IRestoreCoordination bool finish(bool throw_if_error) override; bool cleanup(bool throw_if_error) override; + /// Starts creating a shared database. Returns false if there is another host which is already creating this database. + bool acquireCreatingSharedDatabase(const String & database_name) override; + /// Starts creating a table in a replicated database. Returns false if there is another host which is already creating this table. bool acquireCreatingTableInReplicatedDatabase(const String & database_zk_path, const String & table_name) override; diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 5973fe370e95..ba0e6718d6f1 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -34,6 +34,12 @@ #include #include +#include + +#if CLICKHOUSE_CLOUD +#include +#endif + namespace fs = std::filesystem; @@ -45,6 +51,7 @@ namespace Setting extern const SettingsUInt64 backup_restore_keeper_retry_max_backoff_ms; extern const SettingsUInt64 backup_restore_keeper_max_retries; extern const SettingsSeconds lock_acquire_timeout; + extern const SettingsBool restore_replicated_merge_tree_to_shared_merge_tree; } namespace ErrorCodes @@ -794,11 +801,52 @@ void RestorerFromBackup::createDatabase(const String & database_name) const /// Add the clause `IF NOT EXISTS` if that is specified in the restore settings. create_database_query->if_not_exists = (restore_settings.create_database == RestoreTableCreationMode::kCreateIfNotExists); +#if CLICKHOUSE_CLOUD + bool shared_catalog = SharedDatabaseCatalog::initialized(); + auto & create = create_database_query->as(); + auto engine_name = create.storage != nullptr && create.storage->engine != nullptr ? create.storage->engine->name : ""; + + if (shared_catalog && engine_name == "Replicated") + { + auto engine = std::make_shared(); + + engine->name = "Shared"; + engine->no_empty_args = true; + + create.storage->set(create.storage->engine, engine); + } + else if (!shared_catalog && engine_name == "Shared") + { + // Change engine to Replicated + auto engine = makeASTFunction("Replicated", + std::make_shared("/clickhouse/databases/{uuid}"), + std::make_shared("{shard}"), + std::make_shared("{replica}") + ); + + create.storage->set(create.storage->engine, engine); + } +#endif + LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), create_database_query->formatForLogging()); auto create_query_context = Context::createCopy(query_context); create_query_context->setSetting("allow_deprecated_database_ordinary", 1); +#if CLICKHOUSE_CLOUD + if (shared_catalog && SharedDatabaseCatalog::instance().shouldRestoreDatabase(create_database_query)) + { + SharedDatabaseCatalog::instance().createDatabaseRestoredFromBackup( + database_name, + create_database_query, + create_query_context, + restore_coordination, + std::chrono::duration_cast(create_table_timeout).count()); + + return; + } +#endif + /// Execute CREATE DATABASE query. InterpreterCreateQuery interpreter{create_database_query, create_query_context}; interpreter.setInternal(true); @@ -829,8 +877,15 @@ void RestorerFromBackup::checkDatabase(const String & database_name) is_predefined_database = database_info.is_predefined_database; } + auto is_shared = [](ASTPtr ast) -> bool + { + auto create = ast->as(); + return create.storage && create.storage->engine && create.storage->engine->name == "Shared"; + }; + bool shared_migration = is_shared(database_def_from_backup) != is_shared(database->getCreateDatabaseQuery()); + /// Check that the database's definition is the same as expected. - if (!restore_settings.allow_different_database_def && !is_predefined_database) + if (!restore_settings.allow_different_database_def && !is_predefined_database && !shared_migration) { ASTPtr existing_database_def = database->getCreateDatabaseQuery(); if (!BackupUtils::compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext())) @@ -981,6 +1036,20 @@ void RestorerFromBackup::createTable(const QualifiedTableName & table_name) /// Add the clause `IF NOT EXISTS` if that is specified in the restore settings. create_table_query->if_not_exists = (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists); + if (query_context->getSettingsRef()[Setting::restore_replicated_merge_tree_to_shared_merge_tree]) + { + LOG_INFO(log, "`restore_replicated_merge_tree_to_shared_merge_tree` enabled, will try to replace Replicated engine with Shared"); + ASTStorage * storage = create_table_query->storage; + if (storage != nullptr && storage->engine != nullptr) + boost::replace_first(storage->engine->name, "Replicated", "Shared"); + else if (create_table_query->is_materialized_view_with_inner_table()) + { + storage = create_table_query->targets->getInnerEngine(ViewTarget::To).get(); + if (storage != nullptr && storage->engine != nullptr) + boost::replace_first(storage->engine->name, "Replicated", "Shared"); + } + } + LOG_TRACE(log, "Creating {}: {}", tableNameWithTypeToString(table_name.database, table_name.table, false), create_table_query->formatForLogging()); @@ -1060,8 +1129,8 @@ void RestorerFromBackup::checkTable(const QualifiedTableName & table_name) is_predefined_table = table_info.is_predefined_table; } - /// Check that the table's definition is the same as expected. - if (!restore_settings.allow_different_table_def && !is_predefined_table) + if (!restore_settings.allow_different_table_def && !is_predefined_table && + !query_context->getSettingsRef()[Setting::restore_replicated_merge_tree_to_shared_merge_tree]) { ASTPtr existing_table_def = database->getCreateTableQuery(resolved_id.table_name, context); if (!BackupUtils::compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext())) diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index b8ef5bdf848e..38cf10532f86 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -126,6 +126,31 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); } + if (params.open_mode == IBackup::OpenMode::UNLOCK) + { + auto reader = std::make_shared( + connection_params, + blob_path, + params.allow_azure_native_copy, + params.read_settings, + params.write_settings, + params.context); + + auto lightweight_snapshot_writer = std::make_shared( + connection_params, + "", + params.allow_azure_native_copy, + params.read_settings, + params.write_settings, + params.context, + params.azure_attempt_to_create_container); + + return std::make_unique( + params.backup_info, + archive_params, + reader, + lightweight_snapshot_writer); + } params.use_same_s3_credentials_for_base_backup = false; @@ -139,7 +164,20 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) params.write_settings, params.context); - return std::make_unique(params, archive_params, reader); + auto snapshot_reader_creator = [&](const String & endpoint, const String & container_name) + { + connection_params.endpoint.storage_account_url = endpoint; + connection_params.endpoint.container_name = container_name; + return std::make_shared( + connection_params, + "", + params.allow_azure_native_copy, + params.read_settings, + params.write_settings, + params.context); + }; + + return std::make_unique(params, archive_params, reader, snapshot_reader_creator); } auto writer = std::make_shared( diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 38c04466c6a7..8c0feb17e89a 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -10,6 +10,15 @@ #include #include #include + +#include + +namespace DB::S3AuthSetting +{ + extern const S3AuthSettingsString role_arn; + extern const S3AuthSettingsString role_session_name; +} + #endif @@ -51,6 +60,8 @@ void registerBackupEngineS3(BackupFactory & factory) String s3_uri; String access_key_id; String secret_access_key; + String role_arn; + String role_session_name; if (!id_arg.empty()) { @@ -63,6 +74,8 @@ void registerBackupEngineS3(BackupFactory & factory) s3_uri = config.getString(config_prefix + ".url"); access_key_id = config.getString(config_prefix + ".access_key_id", ""); secret_access_key = config.getString(config_prefix + ".secret_access_key", ""); + role_arn = config.getString(config_prefix + ".role_arn", ""); + role_session_name = config.getString(config_prefix + ".role_session_name", ""); if (config.has(config_prefix + ".filename")) s3_uri = std::filesystem::path(s3_uri) / config.getString(config_prefix + ".filename"); @@ -85,6 +98,17 @@ void registerBackupEngineS3(BackupFactory & factory) access_key_id = args[1].safeGet(); secret_access_key = args[2].safeGet(); } + + if (params.backup_info.function_arg) + { + S3::S3AuthSettings auth_settings; + + if (!StorageS3Configuration::collectCredentials(params.backup_info.function_arg, auth_settings, params.context)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid argument: {}", params.backup_info.function_arg->formatForErrorMessage()); + + role_arn = std::move(auth_settings[S3AuthSetting::role_arn]); + role_session_name = std::move(auth_settings[S3AuthSetting::role_session_name]); + } } BackupImpl::ArchiveParams archive_params; @@ -104,34 +128,95 @@ void registerBackupEngineS3(BackupFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); } - if (params.open_mode == IBackup::OpenMode::READ) + if (params.open_mode == IBackup::OpenMode::UNLOCK) { auto reader = std::make_shared( S3::URI{s3_uri}, access_key_id, secret_access_key, + role_arn, + role_session_name, params.allow_s3_native_copy, params.read_settings, params.write_settings, params.context, params.is_internal_backup); + /// We assume object storage of backup files and original disk use same endpoint, bucket and credentials. + auto uri_for_lightweight = S3::URI{s3_uri}; + /// We set the prefix to "" because in meta file, object key is absolute path. + uri_for_lightweight.key = ""; + auto lightweight_snapshot_writer = std::make_shared( + uri_for_lightweight, + access_key_id, + secret_access_key, + role_arn, + role_session_name, + params.allow_s3_native_copy, + params.s3_storage_class, + params.read_settings, + params.write_settings, + params.context, + params.is_internal_backup); - return std::make_unique(params, archive_params, reader); + return std::make_unique( + params.backup_info, + archive_params, + reader, + lightweight_snapshot_writer); } + else if (params.open_mode == IBackup::OpenMode::READ) + { + auto reader = std::make_shared( + S3::URI{s3_uri}, + access_key_id, + secret_access_key, + role_arn, + role_session_name, + params.allow_s3_native_copy, + params.read_settings, + params.write_settings, + params.context, + params.is_internal_backup); - auto writer = std::make_shared( - S3::URI{s3_uri}, - access_key_id, - secret_access_key, - params.allow_s3_native_copy, - params.s3_storage_class, - params.read_settings, - params.write_settings, - params.context, - params.is_internal_backup); - return std::make_unique(params, archive_params, writer); + auto snapshot_reader_creator = [&](const String & s3_uri_, const String & s3_bucket_) + { + String full_uri = std::filesystem::path(s3_uri_) / s3_bucket_; + auto uri_for_lightweight = S3::URI{full_uri}; + /// We set the prefix to "" because in meta file, object key is absolute path. + uri_for_lightweight.key = ""; + return std::make_shared( + uri_for_lightweight, + access_key_id, + secret_access_key, + role_arn, + role_session_name, + params.allow_s3_native_copy, + params.read_settings, + params.write_settings, + params.context, + params.is_internal_backup); + }; + + return std::make_unique(params, archive_params, reader, snapshot_reader_creator); + } + else + { + auto writer = std::make_shared( + S3::URI{s3_uri}, + access_key_id, + secret_access_key, + std::move(role_arn), + std::move(role_session_name), + params.allow_s3_native_copy, + params.s3_storage_class, + params.read_settings, + params.write_settings, + params.context, + params.is_internal_backup); + return std::make_unique(params, archive_params, writer); + } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "S3 support is disabled"); #endif diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0f7a3cf851b1..92a949f28009 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -67,6 +67,14 @@ add_subdirectory (Processors) add_subdirectory (Compression) add_subdirectory (Server) add_subdirectory (Coordination) + +if (ENABLE_CLICKHOUSE_DISTRIBUTED_CACHE) + message(STATUS "ClickHouse distributed cache is enabled, will build with it") + add_subdirectory (DistributedCache) +else() + message(STATUS "ClickHouse distributed cache is disabled, will build without it") +endif() + add_subdirectory (Daemon) add_subdirectory (Loggers) add_subdirectory (Formats) @@ -198,6 +206,13 @@ endif () add_library(clickhouse_common_io ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) +if (ENABLE_CLICKHOUSE_DISTRIBUTED_CACHE) + target_compile_definitions(clickhouse_common_io PUBLIC ENABLE_DISTRIBUTED_CACHE=1) +else() + target_compile_definitions(clickhouse_common_io PUBLIC ENABLE_DISTRIBUTED_CACHE=0) +endif() + + set_source_files_properties(Common/ThreadFuzzer.cpp PROPERTIES COMPILE_FLAGS "-fomit-frame-pointer -momit-leaf-frame-pointer") add_library (clickhouse_malloc OBJECT Common/malloc.cpp) @@ -216,6 +231,9 @@ if (TARGET ch_contrib::azure_sdk) endif() target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash ch_contrib::incbin) +if (TARGET ch_contrib::consistent_hashing) + target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::consistent_hashing) +endif() add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) @@ -271,6 +289,12 @@ add_object_library(clickhouse_client Client) # https://github.com/ClickHouse/ClickHouse/issues/65745 set_source_files_properties(Client/ClientBaseOptimizedParts.cpp PROPERTIES COMPILE_FLAGS "-O3") add_object_library(clickhouse_bridge BridgeHelper) + +if (ENABLE_CLICKHOUSE_DISTRIBUTED_CACHE) + add_object_library(clickhouse_distributed_cache DistributedCache) + add_object_library(clickhouse_server_distributed_cache Server/DistributedCache) +endif() + add_object_library(clickhouse_server Server) add_object_library(clickhouse_server_http Server/HTTP) if (TARGET ch_contrib::ssh) @@ -408,6 +432,10 @@ if (TARGET ch_contrib::crc32-vpmsum) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::crc32-vpmsum) endif() +if (TARGET ch_contrib::jwt-cpp) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::jwt-cpp) +endif() + if (TARGET ch_contrib::ssh) target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh) endif() diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 0bf2dcb24575..e2ace0de76d0 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -46,6 +46,8 @@ static struct InitFiu ONCE(smt_commit_merge_mutate_zk_fail_before_op) \ ONCE(smt_commit_write_zk_fail_after_op) \ ONCE(smt_commit_write_zk_fail_before_op) \ + PAUSEABLE_ONCE(smt_commit_tweaks_gate_open) \ + PAUSEABLE_ONCE(smt_commit_tweaks_gate_close) \ ONCE(smt_commit_merge_change_version_before_op) \ ONCE(smt_merge_mutate_intention_freeze_in_destructor) \ ONCE(smt_add_part_sleep_after_add_before_commit) \ @@ -56,8 +58,17 @@ static struct InitFiu ONCE(smt_sleep_after_hardware_in_insert) \ ONCE(smt_throw_keeper_exception_after_successful_insert) \ ONCE(smt_lightweight_snapshot_fail) \ + ONCE(smt_lightweight_update_sleep_after_block_allocation) \ + ONCE(smt_merge_task_sleep_in_prepare) \ + ONCE(rmt_lightweight_update_sleep_after_block_allocation) \ + ONCE(rmt_merge_task_sleep_in_prepare) \ + ONCE(s3_read_buffer_throw_expired_token) \ + ONCE(distributed_cache_fail_request_in_the_middle_of_request) \ + REGULAR(distributed_cache_fail_connect_non_retriable) \ + REGULAR(distributed_cache_fail_connect_retriable) \ REGULAR(object_storage_queue_fail_commit) \ REGULAR(smt_dont_merge_first_part) \ + REGULAR(smt_mutate_only_second_part) \ REGULAR(smt_sleep_in_schedule_data_processing_job) \ REGULAR(cache_warmer_stall) \ REGULAR(file_cache_dynamic_resize_fail_to_evict) \ @@ -70,6 +81,7 @@ static struct InitFiu PAUSEABLE_ONCE(replicated_merge_tree_insert_retry_pause) \ PAUSEABLE_ONCE(finish_set_quorum_failed_parts) \ PAUSEABLE_ONCE(finish_clean_quorum_failed_parts) \ + PAUSEABLE_ONCE(smt_wait_next_mutation) \ PAUSEABLE(dummy_pausable_failpoint) \ ONCE(execute_query_calling_empty_set_result_func_on_exception) \ ONCE(receive_timeout_on_table_status_response) \ @@ -91,7 +103,11 @@ static struct InitFiu ONCE(keeper_leader_sets_invalid_digest) \ ONCE(parallel_replicas_wait_for_unused_replicas) \ REGULAR(database_replicated_delay_recovery) \ - REGULAR(database_replicated_delay_entry_execution) + REGULAR(database_replicated_delay_entry_execution) \ + REGULAR(plain_object_storage_copy_temp_source_file_fail_on_file_move) \ + REGULAR(plain_object_storage_copy_temp_target_file_fail_on_file_move) \ + REGULAR(output_format_sleep_on_progress) \ + ONCE(smt_commit_exception_before_op) namespace FailPoints diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index f3936a4ec170..aec2f113066b 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -167,6 +167,12 @@ String Macros::getValue(const String & key) const throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No macro {} in config", key); } +std::optional Macros::tryGetValue(const String & key) const +{ + if (auto it = macros.find(key); it != macros.end()) + return it->second; + return {}; +} String Macros::expand(const String & s) const { diff --git a/src/Common/Macros.h b/src/Common/Macros.h index a87eda4035e5..a1a425cdb0db 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -68,6 +68,7 @@ class Macros MacroMap getMacroMap() const { return macros; } String getValue(const String & key) const; + std::optional tryGetValue(const String & key) const; private: MacroMap macros; diff --git a/src/Common/OpenSSLHelpers.cpp b/src/Common/OpenSSLHelpers.cpp index 6305477b91b9..b32f5811efcb 100644 --- a/src/Common/OpenSSLHelpers.cpp +++ b/src/Common/OpenSSLHelpers.cpp @@ -58,6 +58,56 @@ void encodeSHA256(const void * text, size_t size, unsigned char * out) throw Exception(ErrorCodes::OPENSSL_ERROR, "EVP_DigestFinal failed: {}", getOpenSSLErrors()); } +std::string rsaSHA256Sign(EVP_PKEY * pkey, const std::string & data) +{ + EVP_MD_CTX* ctx = EVP_MD_CTX_new(); + + if (!ctx) + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + if (EVP_DigestSignInit(ctx, nullptr, EVP_sha256(), nullptr, pkey) != 1) + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + if (EVP_DigestSignUpdate(ctx, data.data(), data.size()) != 1) + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + size_t signature_length = 0; + if (EVP_DigestSignFinal(ctx, nullptr, &signature_length) != 1) + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + std::string signature(signature_length, 0); + if (EVP_DigestSignFinal(ctx, reinterpret_cast(signature.data()), &signature_length) != 1) + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + EVP_MD_CTX_free(ctx); + signature.resize(signature_length); + + return signature; +} + +bool rsaSHA256Verify(EVP_PKEY * pkey, const std::string & data, const std::string & signature) +{ + EVP_MD_CTX* ctx = EVP_MD_CTX_new(); + + if (!ctx) + throw DB::Exception(DB::ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors()); + + if (EVP_DigestVerifyInit(ctx, nullptr, EVP_sha256(), nullptr, pkey) != 1) + throw DB::Exception(DB::ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors); + + if (EVP_DigestVerifyUpdate(ctx, data.data(), data.size()) != 1) + throw DB::Exception(DB::ErrorCodes::OPENSSL_ERROR, "{}", getOpenSSLErrors); + + int result = EVP_DigestVerifyFinal( + ctx, + reinterpret_cast(signature.data()), + static_cast(signature.size()) + ); + EVP_MD_CTX_free(ctx); + + return result == 1; +} + std::vector hmacSHA256(const std::vector & key, const std::string & data) { std::vector result(EVP_MAX_MD_SIZE); diff --git a/src/Common/OpenSSLHelpers.h b/src/Common/OpenSSLHelpers.h index e143adab2731..8c2b34cf553a 100644 --- a/src/Common/OpenSSLHelpers.h +++ b/src/Common/OpenSSLHelpers.h @@ -6,6 +6,7 @@ #if USE_SSL # include +# include namespace DB @@ -23,6 +24,9 @@ void encodeSHA256(const void * text, size_t size, unsigned char * out); std::vector hmacSHA256(const std::vector & key, const std::string & data); std::vector pbkdf2SHA256(std::string_view password, const std::vector& salt, int iterations); +std::string rsaSHA256Sign(EVP_PKEY * pkey, const std::string & data); +bool rsaSHA256Verify(EVP_PKEY * pkey, const std::string & data, const std::string & signature); + /// Returns concatenation of error strings for all errors that OpenSSL has recorded, emptying the error queue. String getOpenSSLErrors(); diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 0528ddfe5e74..4eb7754adaab 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -15,6 +15,10 @@ #include #include #include +#if CLICKHOUSE_CLOUD +#include +#include +#endif #include #include #include @@ -309,10 +313,16 @@ void SystemLogBase::add(LogElement element) #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#if CLICKHOUSE_CLOUD + SYSTEM_LOG_ELEMENTS_CLOUD(INSTANTIATE_SYSTEM_LOG_BASE) +#endif SYSTEM_PERIODIC_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) #define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) +#if CLICKHOUSE_CLOUD +SYSTEM_LOG_ELEMENTS_CLOUD(INSTANTIATE_SYSTEM_LOG_QUEUE) +#endif SYSTEM_PERIODIC_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index fe09b747ea90..fdadcdf8b0ae 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -10,6 +10,7 @@ #include #include + #define SYSTEM_LOG_ELEMENTS(M) \ M(AsynchronousMetricLogElement) \ M(CrashLogElement) \ @@ -32,6 +33,11 @@ M(BlobStorageLogElement) \ M(QueryMetricLogElement) +#define SYSTEM_LOG_ELEMENTS_CLOUD(M) \ + M(DistributedCacheLogElement) \ + M(DistributedCacheServerLogElement) \ + + namespace Poco { diff --git a/src/Core/DistributedCacheDefines.h b/src/Core/DistributedCacheDefines.h index ac7faff08db7..85558f5ad91d 100644 --- a/src/Core/DistributedCacheDefines.h +++ b/src/Core/DistributedCacheDefines.h @@ -18,8 +18,11 @@ static constexpr auto MAX_UNACKED_INFLIGHT_PACKETS = 10; static constexpr auto ACK_DATA_PACKET_WINDOW = 5; static constexpr auto DEFAULT_CONNECTION_POOL_SIZE = 15000; static constexpr auto DEFAULT_CONNECTION_TTL_SEC = 200; -static constexpr auto DEFAULT_CONNECT_MAX_TRIES = 20; -static constexpr auto DEFAULT_READ_REQUEST_MAX_TRIES = 20; +static constexpr auto DEFAULT_CONNECT_MAX_TRIES = 5; +static constexpr auto DEFAULT_READ_REQUEST_MAX_TRIES = 10; +static constexpr auto DEFAULT_CREDENTIALS_REFRESH_PERIOD_SECONDS = 5; +static constexpr auto DEFAULT_CONNECT_BACKOFF_MIN_MS = 0; +static constexpr auto DEFAULT_CONNECT_BACKOFF_MAX_MS = 50; static constexpr auto INITIAL_PROTOCOL_VERSION = 0; static constexpr auto PROTOCOL_VERSION_WITH_QUERY_ID = 1; @@ -28,7 +31,10 @@ static constexpr auto PROTOCOL_VERSION_WITH_GCS_TOKEN = 3; static constexpr auto PROTOCOL_VERSION_WITH_AZURE_AUTH = 4; static constexpr auto PROTOCOL_VERSION_WITH_TEMPORATY_DATA = 5; static constexpr auto PROTOCOL_VERSION_WITH_READ_RANGE_ID = 6; +static constexpr auto PROTOCOL_VERSION_WITH_CREDENTIALS_REFRESH = 7; +static constexpr auto PROTOCOL_VERSION_WITH_CACHE_STATS_DATA_PACKET = 8; +static constexpr auto PROTOCOL_VERSION_WITH_STORAGE_INFO_IN_CACHE_KEY = 9; -static constexpr UInt32 CURRENT_PROTOCOL_VERSION = PROTOCOL_VERSION_WITH_READ_RANGE_ID; +static constexpr UInt32 CURRENT_PROTOCOL_VERSION = PROTOCOL_VERSION_WITH_STORAGE_INFO_IN_CACHE_KEY; } diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index cd27f464e1d8..062d13918b2f 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -572,7 +572,7 @@ class SSDCacheFileBuffer : private boost::noncopyable size_t buffer_size_in_bytes = blocks_length * block_size; - Memory read_buffer_memory(block_size * blocks_length, block_size); + Memory<> read_buffer_memory(block_size * blocks_length, block_size); iocb request{}; iocb * request_ptr = &request; diff --git a/src/Disks/DiskCommitTransactionOptions.h b/src/Disks/DiskCommitTransactionOptions.h new file mode 100644 index 000000000000..6d43e992f7bb --- /dev/null +++ b/src/Disks/DiskCommitTransactionOptions.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +using NoCommitOptions = std::monostate; + +template +struct MetaInKeeperCommitOptions +{ + bool need_rollback_blobs = true; + bool may_retry = false; + ZooKeeperConnection * zookeeper; + Coordination::Requests additional_requests; +}; + +using TransactionCommitOptionsVariant = std::variant< + NoCommitOptions, + MetaInKeeperCommitOptions, + MetaInKeeperCommitOptions>; + +inline String getFirstFailedPath(Coordination::Error code, const Coordination::Responses & responses, + size_t first_additional_request_idx, const Coordination::Requests & additional_ops) +{ + size_t failed_op_index = zkutil::getFailedOpIndex(code, responses); + if (failed_op_index < first_additional_request_idx) + return ""; + return additional_ops.at(failed_op_index - first_additional_request_idx)->getPath(); +} + +struct MetaInKeeperCommitOutcome +{ + Coordination::Error code; + + /// Responses for the keeper multi-op performed by this transaction. + /// The order is: disk's internal ops, then `additional_requests`. + /// Populated only if isUserError(code). + Coordination::Responses responses; + size_t first_additional_request_idx; + + size_t additionalOpIdx(size_t idx) const { return first_additional_request_idx + idx; } +}; + +using TransactionCommitOutcomeVariant = std::variant< + bool, + MetaInKeeperCommitOutcome>; + +inline bool isSuccessfulOutcome(const TransactionCommitOutcomeVariant & outcome) +{ + if (const auto * success = std::get_if(&outcome); success) + return *success; + else if (const auto * result = std::get_if(&outcome); result) + return result->code == Coordination::Error::ZOK; + + UNREACHABLE(); +} + +inline bool needRollbackBlobs(const TransactionCommitOptionsVariant & options) +{ + if (const auto * with_faults = std::get_if>(&options); with_faults) + return with_faults->need_rollback_blobs; + else if (const auto * just_keeper = std::get_if>(&options); just_keeper) + return just_keeper->need_rollback_blobs; + + return true; +} + +inline bool canRollbackBlobs(const TransactionCommitOptionsVariant & options, const TransactionCommitOutcomeVariant & outcome) +{ + if (needRollbackBlobs(options)) + { + if (const auto * result = std::get_if(&outcome); result) + { + return !Coordination::isHardwareError(result->code); + } + return true; + } + return false; +} + +inline bool mayRetryCommit(const TransactionCommitOptionsVariant & options, const TransactionCommitOutcomeVariant & outcome) +{ + if (isSuccessfulOutcome(outcome)) + return false; + + if (const auto * with_faults = std::get_if>(&options); with_faults) + { + return with_faults->may_retry; + } + else if (const auto * just_keeper = std::get_if>(&options); just_keeper) + { + return just_keeper->may_retry; + } + + return false; +} + +inline MetaInKeeperCommitOutcome getKeeperOutcome(const TransactionCommitOutcomeVariant & outcome) +{ + if (const auto * result = std::get_if(&outcome); result) + return *result; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not keeper outcome"); +} + +} diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index ac41b32fda1f..a28ac3cfaae4 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -2,14 +2,20 @@ #if USE_SSL #include +#include +#include #include #include #include #include -#include +#include +#include #include #include #include +#if CLICKHOUSE_CLOUD +#include +#endif namespace DB @@ -41,22 +47,28 @@ namespace } } + struct Key + { + String plain; + std::optional encrypted; + }; + /// Reads encryption keys from the configuration. void getKeysFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - std::map & out_keys_by_id, Strings & out_keys_without_id) + std::map & out_keys_by_id, std::vector& out_keys_without_id) { Strings config_keys; config.keys(config_prefix, config_keys); for (const std::string & config_key : config_keys) { - String key; + Key key; std::optional key_id; if ((config_key == "key") || config_key.starts_with("key[")) { String key_path = config_prefix + "." + config_key; - key = config.getString(key_path); + key.plain = toString(config.getString(key_path)); String key_id_path = key_path + "[@id]"; if (config.has(key_id_path)) key_id = config.getUInt64(key_id_path); @@ -64,11 +76,23 @@ namespace else if ((config_key == "key_hex") || config_key.starts_with("key_hex[")) { String key_path = config_prefix + "." + config_key; - key = unhexKey(config.getString(key_path)); + key.plain = unhexKey(toString(config.getString(key_path))); String key_id_path = key_path + "[@id]"; if (config.has(key_id_path)) key_id = config.getUInt64(key_id_path); } +#if CLICKHOUSE_CLOUD + else if ((config_key == "key_aws") || config_key.starts_with("key_aws[") || (config_key == "key_gcp") || config_key.starts_with("key_gcp[") || + (config_key == "key_azure") || config_key.starts_with("key_azure[")) + { + String key_path = config_prefix + "." + config_key; + String key_id_path = key_path + "[@id]"; + if (config.has(key_id_path)) + key_id = config.getUInt64(key_id_path); + key.plain = decryptKeyUsingCloudKeyManagementService(config,config_prefix, key_path, config_key); + key.encrypted = config.getString(key_path); + } +#endif else continue; @@ -95,7 +119,7 @@ namespace /// Reads the current encryption key from the configuration. String getCurrentKeyFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const std::map & keys_by_id, const Strings & keys_without_id) + const std::map & keys_by_id, const std::vector & keys_without_id) { String key_path = config_prefix + ".current_key"; String key_hex_path = config_prefix + ".current_key_hex"; @@ -108,28 +132,32 @@ namespace { for (const auto & [_, key] : keys_by_id) { - if (key == current_key_) - return; + if (key.plain == current_key_) + return current_key_; + + if (std::string_view(key.encrypted.value_or("")) == std::string_view(current_key_)) + return key.plain; } for (const auto & key : keys_without_id) { - if (key == current_key_) - return; + if (key.plain == current_key_) + return current_key_; + + if (std::string_view(key.encrypted.value_or("")) == std::string_view(current_key_)) + return key.plain; } throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not found in keys"); }; if (config.has(key_path)) { - String current_key = config.getString(key_path); - check_current_key_found(current_key); - return current_key; + String current_key = toString(config.getString(key_path)); + return check_current_key_found(current_key); } if (config.has(key_hex_path)) { - String current_key = unhexKey(config.getString(key_hex_path)); - check_current_key_found(current_key); - return current_key; + String current_key = unhexKey(toString(config.getString(key_hex_path))); + return check_current_key_found(current_key); } if (config.has(key_id_path)) { @@ -137,12 +165,12 @@ namespace auto it = keys_by_id.find(current_key_id); if (it == keys_by_id.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", current_key_id); - return it->second; + return it->second.plain; } if (keys_by_id.size() == 1 && keys_without_id.empty() && keys_by_id.begin()->first == 0) { /// There is only a single key defined with id=0, so we can choose it as current. - return keys_by_id.begin()->second; + return keys_by_id.begin()->second.plain; } throw Exception(ErrorCodes::BAD_ARGUMENTS, "The current key is not specified"); @@ -178,7 +206,7 @@ namespace throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk path must ends with '/', but '{}' doesn't.", quoteString(out_path)); } - /// Parses the settings of an ecnrypted disk from the configuration. + /// Parses the settings of an encrypted disk from the configuration. std::unique_ptr parseDiskEncryptedSettings( const String & disk_name, const Poco::Util::AbstractConfiguration & config, @@ -189,12 +217,13 @@ namespace { auto res = std::make_unique(); - std::map keys_by_id; - Strings keys_without_id; + std::map keys_by_id; + std::vector keys_without_id; getKeysFromConfig(config, config_prefix, keys_by_id, keys_without_id); - for (const auto & [key_id, key] : keys_by_id) + for (const auto & [key_id, key_entry] : keys_by_id) { + const auto key = key_entry.plain; auto fingerprint = calculateKeyFingerprint(key); res->all_keys[fingerprint] = key; @@ -206,8 +235,8 @@ namespace for (const auto & key : keys_without_id) { - auto fingerprint = calculateKeyFingerprint(key); - res->all_keys[fingerprint] = key; + auto fingerprint = calculateKeyFingerprint(key.plain); + res->all_keys[fingerprint] = key.plain; } String current_key = getCurrentKeyFromConfig(config, config_prefix, keys_by_id, keys_without_id); diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 2e1fc4f7c38c..56fe5d25c481 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -48,9 +49,9 @@ class DiskEncryptedTransaction : public IDiskTransaction /// Tries to commit all accumulated operations simultaneously. /// If something fails rollback and throw exception. - void commit() override // NOLINT + void commit(const TransactionCommitOptionsVariant & options = NoCommitOptions{}) override // NOLINT { - delegate_transaction->commit(); + delegate_transaction->commit(options); } void undo() override @@ -58,6 +59,11 @@ class DiskEncryptedTransaction : public IDiskTransaction delegate_transaction->undo(); } + TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override + { + return delegate_transaction->tryCommit(options); + } + ~DiskEncryptedTransaction() override = default; /// Create directory. @@ -247,7 +253,6 @@ class DiskEncryptedTransaction : public IDiskTransaction delegate_transaction->truncateFile(wrapped_path, target_size); } - private: String wrappedPath(const String & path) const diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index 07a7099419ba..e4d8cf803fd8 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -2,11 +2,14 @@ #include #include +#include + namespace DB { namespace ErrorCodes { extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int LOGICAL_ERROR; } MetadataStorageType metadataTypeFromString(const String & type) @@ -20,6 +23,10 @@ MetadataStorageType metadataTypeFromString(const String & type) return MetadataStorageType::PlainRewritable; if (check_type == "web") return MetadataStorageType::StaticWeb; + if (check_type == "keeper") + return MetadataStorageType::Keeper; + if (check_type == "memory") + return MetadataStorageType::Memory; throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "MetadataStorageFactory: unknown metadata storage type: {}", type); @@ -27,7 +34,7 @@ MetadataStorageType metadataTypeFromString(const String & type) bool DataSourceDescription::operator==(const DataSourceDescription & other) const { - return std::tie(type, object_storage_type, description, is_encrypted) == std::tie(other.type, other.object_storage_type, other.description, other.is_encrypted); + return std::tie(type, object_storage_type, description, is_encrypted, zookeeper_name) == std::tie(other.type, other.object_storage_type, other.description, other.is_encrypted, other.zookeeper_name); } bool DataSourceDescription::sameKind(const DataSourceDescription & other) const @@ -68,6 +75,8 @@ std::string DataSourceDescription::toString() const return "web"; case ObjectStorageType::None: return "none"; + case ObjectStorageType::Max: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected object storage type: Max"); } } } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index bf7ef3d30eb0..f8eb31b2f988 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -15,12 +15,14 @@ enum class DataSourceType : uint8_t enum class ObjectStorageType : uint8_t { - None, - S3, - Azure, - HDFS, - Web, - Local, + None = 0, + S3 = 1, + Azure = 2, + HDFS = 3, + Web = 4, + Local = 5, + + Max = 6, }; enum class MetadataStorageType : uint8_t @@ -48,10 +50,14 @@ struct DataSourceDescription bool is_encrypted = false; bool is_cached = false; + std::string zookeeper_name; + bool operator==(const DataSourceDescription & other) const; bool sameKind(const DataSourceDescription & other) const; std::string toString() const; }; +bool canUseNativeCopy(const DataSourceDescription & left, const DataSourceDescription & right); + } diff --git a/src/Disks/FakeDiskTransaction.h b/src/Disks/FakeDiskTransaction.h index 69f08de25173..6a5e0b1c8663 100644 --- a/src/Disks/FakeDiskTransaction.h +++ b/src/Disks/FakeDiskTransaction.h @@ -22,7 +22,7 @@ struct FakeDiskTransaction final : public IDiskTransaction : disk(disk_) {} - void commit() override {} + void commit(const TransactionCommitOptionsVariant &) override {} void undo() override {} void createDirectory(const std::string & path) override diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index fc84281baea2..e59377c1fc7e 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -3,16 +3,17 @@ #include #include #include -#include #include - +#include +#include namespace DB { -struct ReadSettings; -struct WriteSettings; -class WriteBufferFromFileBase; +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} struct RemoveRequest { @@ -32,12 +33,18 @@ using RemoveBatchRequest = std::vector; struct IDiskTransaction : private boost::noncopyable { public: + /// Tries to commit all accumulated operations simultaneously. /// If something fails rollback and throw exception. - virtual void commit() = 0; + virtual void commit(const TransactionCommitOptionsVariant & = NoCommitOptions{}) = 0; // NOLINT virtual void undo() = 0; + virtual TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant &) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Commit with ZK connection not implemented"); + } + virtual ~IDiskTransaction() = default; /// Create directory. @@ -52,7 +59,7 @@ struct IDiskTransaction : private boost::noncopyable /// Move directory from `from_path` to `to_path`. virtual void moveDirectory(const std::string & from_path, const std::string & to_path) = 0; - virtual void moveFile(const std::string & from_path, const std::string & to_path) = 0; + virtual void moveFile(const String & from_path, const String & to_path) = 0; virtual void createFile(const String & path) = 0; @@ -78,10 +85,10 @@ struct IDiskTransaction : private boost::noncopyable const WriteSettings & settings = {}, bool autocommit = true) = 0; - using WriteBlobFunction = std::function & blob_path, WriteMode mode, const std::optional & object_attributes)>; + using WriteBlobFunction = std::function & object_attributes)>; /// Write a file using a custom function to write an object to the disk's object storage. - virtual void writeFileUsingBlobWritingFunction(const std::string & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; + virtual void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) = 0; /// Remove file. Throws exception if file doesn't exists or it's a directory. virtual void removeFile(const std::string & path) = 0; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index e767fbf932bc..e6103b19b7e1 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -302,7 +302,7 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() } auto size = buffer_allocation_policy->getBufferSize(); - memory = Memory(size); + memory = Memory<>(size); WriteBuffer::set(memory.data(), memory.size()); } diff --git a/src/Disks/MetadataStorageWithPathWrapper.h b/src/Disks/MetadataStorageWithPathWrapper.h index 7d9cceaee840..473aa35681ff 100644 --- a/src/Disks/MetadataStorageWithPathWrapper.h +++ b/src/Disks/MetadataStorageWithPathWrapper.h @@ -40,9 +40,9 @@ class MetadataStorageWithPathWrapperTransaction final : public IMetadataTransact return delegate->getStorageForNonTransactionalReads(); } - void commit() final + void commit(const TransactionCommitOptionsVariant & options) final { - delegate->commit(); + delegate->commit(options); } void writeStringToFile(const std::string & path, const std::string & data) override @@ -141,6 +141,11 @@ class MetadataStorageWithPathWrapperTransaction final : public IMetadataTransact { return delegate->truncateFile(wrappedPath(src_path), target_size); } + + std::optional tryGetBlobsFromTransactionIfExists(const std::string & path) const override + { + return delegate->tryGetBlobsFromTransactionIfExists(path); + } }; class MetadataStorageWithPathWrapper final : public IMetadataStorage @@ -254,6 +259,11 @@ class MetadataStorageWithPathWrapper final : public IMetadataStorage { return delegate->getStorageObjects(wrappedPath(path)); } + + bool isReadOnly() const override + { + return delegate->isReadOnly(); + } }; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index 2808ffc4bf63..9cd87593816d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -147,6 +147,7 @@ struct ConnectionParams std::unique_ptr createForContainer() const; }; + Endpoint processEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); void processURL(const String & url, const String & container_name, Endpoint & endpoint, AuthMethod & auth_method); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 71b72006a0ad..3bd43f410861 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -108,14 +108,18 @@ AzureObjectStorage::AzureObjectStorage( AzureBlobStorage::AuthMethod auth_method_, ClientPtr && client_, SettingsPtr && settings_, + const AzureBlobStorage::ConnectionParams & connection_params_, const String & object_namespace_, - const String & description_) + const String & description_, + const String & common_key_prefix_) : name(name_) , auth_method(std::move(auth_method_)) , client(std::move(client_)) , settings(std::move(settings_)) , object_namespace(object_namespace_) , description(description_) + , common_key_prefix(common_key_prefix_) + , connection_params(connection_params_) , log(getLogger("AzureObjectStorage")) { } @@ -364,6 +368,12 @@ void AzureObjectStorage::applyNewSettings( client.set(std::move(new_client)); } + +ObjectStorageConnectionInfoPtr AzureObjectStorage::getConnectionInfo() const +{ + return DB::getAzureObjectStorageConnectionInfo(connection_params); +} + } #endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 8af163ddb2e6..5a23deb9b65b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace Poco { @@ -29,8 +30,10 @@ class AzureObjectStorage : public IObjectStorage AzureBlobStorage::AuthMethod auth_method, ClientPtr && client_, SettingsPtr && settings_, + const AzureBlobStorage::ConnectionParams & connection_params_, const String & object_namespace_, - const String & description_); + const String & description_, + const String & common_key_prefix_); void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; @@ -44,7 +47,7 @@ class AzureObjectStorage : public IObjectStorage std::string getRootPrefix() const override { return object_namespace; } /// Object keys are unique within the object namespace (container + prefix). - std::string getCommonKeyPrefix() const override { return ""; } + std::string getCommonKeyPrefix() const override { return common_key_prefix; } std::string getDescription() const override { return description; } @@ -72,6 +75,8 @@ class AzureObjectStorage : public IObjectStorage ObjectMetadata getObjectMetadata(const std::string & path) const override; + ObjectStorageConnectionInfoPtr getConnectionInfo() const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, @@ -120,6 +125,10 @@ class AzureObjectStorage : public IObjectStorage /// We use source url without container and prefix as description, because in Azure there are no limitations for operations between different containers. const String description; + const String common_key_prefix; + + const AzureBlobStorage::ConnectionParams connection_params; + LoggerPtr log; }; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp new file mode 100644 index 000000000000..8473de860e0d --- /dev/null +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp @@ -0,0 +1,148 @@ +#include + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include + +#if ENABLE_DISTRIBUTED_CACHE +#include +#endif + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace CurrentMetrics +{ + extern const Metric DistrCacheServerS3CachedClients; +} + +namespace ProfileEvents +{ + extern const Event DistrCacheServerNewS3CachedClients; + extern const Event DistrCacheServerReusedS3CachedClients; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int PROTOCOL_VERSION_MISMATCH; +} + +AzureObjectStorageConnectionInfo::AzureObjectStorageConnectionInfo(const AzureBlobStorage::ConnectionParams & connection_params) +{ + auto identity = std::get>(connection_params.auth_method); + + auto az_context = Azure::Core::Context(); + auto token_request_context = Azure::Core::Credentials::TokenRequestContext{ + .Scopes = { Azure::Storage::_internal::StorageScope }, + .TenantId = "", + }; + + auto access_token = identity->GetToken(token_request_context, az_context); + + client_info = AzureClientInfo{ + .endpoint = connection_params.endpoint.getServiceEndpoint(), + .container_name = connection_params.getContainer() + "/" + connection_params.endpoint.prefix, + .session_token = access_token.Token, + .expires_on = static_cast(access_token.ExpiresOn), + }; +} + +AzureObjectStorageConnectionInfo::AzureObjectStorageConnectionInfo(const std::string & user_info_) + : user_info(user_info_) +{ +} + +bool AzureObjectStorageConnectionInfo::equals(const IObjectStorageConnectionInfo & other) const +{ + const auto * azure_info = dynamic_cast(&other); + if (azure_info) + return client_info == azure_info->client_info; + return false; +} + +void AzureObjectStorageConnectionInfo::readBinaryImpl(ReadBuffer & in) +{ + DB::readBinary(client_info.endpoint, in); + DB::readBinary(client_info.container_name, in); + DB::readBinary(client_info.session_token, in); +} + +void AzureObjectStorageConnectionInfo::writeBinaryImpl(size_t protocol_version, WriteBuffer & out) +{ + if (protocol_version < DistributedCache::PROTOCOL_VERSION_WITH_AZURE_AUTH) + throw Exception(ErrorCodes::PROTOCOL_VERSION_MISMATCH, "Protocol version does not support Azure"); + + DB::writeBinary(client_info.endpoint, out); + DB::writeBinary(client_info.container_name, out); + DB::writeBinary(client_info.session_token, out); +} + +std::unique_ptr AzureObjectStorageConnectionInfo::createReader( + const StoredObject & object, + const ReadSettings & read_settings) +{ + if (!client) + client = ObjectStorageClientsCache::instance().getClient(user_info, client_info); + + AzureBlobStorage::RequestSettings settings; + return std::make_unique( + client, + object.remote_path, + read_settings, + settings.max_single_read_retries, + settings.max_single_download_retries, + /* restricted_seek */true, + /* use_external_buffer */true, + /* read_until_position */0 + ); +} + +std::shared_ptr AzureObjectStorageConnectionInfo::makeClient(const AzureClientInfo & info) +{ + auto params = AzureBlobStorage::ConnectionParams{}; + AzureBlobStorage::processURL(info.endpoint, info.container_name, params.endpoint, params.auth_method); + + auto settings = AzureBlobStorage::getRequestSettings(Context::getGlobalContextInstance()->getSettingsRef()); + params.client_options = AzureBlobStorage::getClientOptions(Context::getGlobalContextInstance(), *settings, /*for_disk*/true); + params.auth_method = std::make_shared(info.session_token, info.expires_on); + + return params.createForContainer(); +} + +ObjectStorageConnectionInfoPtr getAzureObjectStorageConnectionInfo(const AzureBlobStorage::ConnectionParams & connection_params) +{ + try + { + std::get>(connection_params.auth_method); + } + catch (const std::bad_variant_access & e) + { + LOG_WARNING(getLogger("AzureObjectStorageConnectionInfo"), + "AzureObjectStorageConnectionInfo: only Workload Identity is supported; {}", e.what()); + + return nullptr; + } + + return std::make_shared(connection_params); +} + +} +#endif diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.h new file mode 100644 index 000000000000..a2f9fc5ae1de --- /dev/null +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.h @@ -0,0 +1,78 @@ +#pragma once +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include + + +namespace DB +{ +class ReadBuffer; + + +ObjectStorageConnectionInfoPtr getAzureObjectStorageConnectionInfo(const AzureBlobStorage::ConnectionParams & connection_params); + +struct AzureClientInfo +{ + std::string endpoint; + std::string container_name; + std::string session_token; + std::chrono::system_clock::time_point expires_on; + + bool operator == (const AzureClientInfo & other) const + { + /// We do not include `expires_on`. + return endpoint == other.endpoint + && container_name == other.container_name + && session_token == other.session_token; + } + + void updateHash(SipHash & hash, bool include_credentials) const + { + hash.update(endpoint); + hash.update(container_name); + if (include_credentials) + hash.update(session_token); + } +}; + +class AzureObjectStorageConnectionInfo : public IObjectStorageConnectionInfo +{ + friend class IObjectStorageConnectionInfo; +public: + using Client = AzureBlobStorage::ContainerClient; + using ClientInfo = AzureClientInfo; + + /// Writer constructor. + explicit AzureObjectStorageConnectionInfo(const AzureBlobStorage::ConnectionParams & connection_params); + /// Reader constructor. + explicit AzureObjectStorageConnectionInfo(const std::string & user_info_); + + bool equals(const IObjectStorageConnectionInfo &) const override; + + ObjectStorageType getType() const override { return ObjectStorageType::Azure; } + + void updateHash(SipHash & hash, bool include_credentials) const override { client_info.updateHash(hash, include_credentials); } + + std::unique_ptr createReader( + const StoredObject & object, + const ReadSettings & read_settings) override; + + static std::shared_ptr makeClient(const AzureClientInfo & info); + +protected: + void writeBinaryImpl(size_t protocol_version, WriteBuffer & out) override; + + void readBinaryImpl(ReadBuffer & in); + +private: + AzureClientInfo client_info; + const std::string user_info; + std::shared_ptr client; +}; +} + +#endif diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 63302d092439..68cce9f2ccb6 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -68,6 +68,8 @@ class CachedObjectStorage final : public IObjectStorage ObjectMetadata getObjectMetadata(const std::string & path) const override; + ObjectStorageConnectionInfoPtr getConnectionInfo() const override { return object_storage->getConnectionInfo(); } + void shutdown() override; void startup() override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 6d316712db6d..37d993441db3 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -16,10 +16,17 @@ #include #include #include +#include #include #include #include #include +#if ENABLE_DISTRIBUTED_CACHE +#include +#include +#endif +#include + namespace DB { @@ -34,7 +41,9 @@ namespace ErrorCodes DiskTransactionPtr DiskObjectStorage::createTransaction() { - return std::make_shared(*this); + if (use_fake_transaction) + return std::make_shared(*this); + return createObjectStorageTransaction(); } ObjectStoragePtr DiskObjectStorage::getObjectStorage() @@ -65,7 +74,8 @@ DiskObjectStorage::DiskObjectStorage( MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, const Poco::Util::AbstractConfiguration & config, - const String & config_prefix) + const String & config_prefix, + bool use_fake_transaction_) : IDisk(name_, config, config_prefix) , object_key_prefix(object_key_prefix_) , log(getLogger("DiskObjectStorage(" + name + ")")) @@ -73,6 +83,8 @@ DiskObjectStorage::DiskObjectStorage( , object_storage(std::move(object_storage_)) , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) + , enable_distributed_cache(config.getBool(config_prefix + ".enable_distributed_cache", true)) + , use_fake_transaction(use_fake_transaction_) , remove_shared_recursive_file_limit(config.getUInt64(config_prefix + ".remove_shared_recursive_file_limit", DEFAULT_REMOVE_SHARED_RECURSIVE_FILE_LIMIT)) { data_source_description = DataSourceDescription{ @@ -82,6 +94,7 @@ DiskObjectStorage::DiskObjectStorage( .description = object_storage->getDescription(), .is_encrypted = false, .is_cached = object_storage->supportsCache(), + .zookeeper_name = metadata_storage->getZooKeeperName(), }; resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe( [this] (const std::vector & events) @@ -358,9 +371,23 @@ void DiskObjectStorage::createDirectory(const String & path) void DiskObjectStorage::createDirectories(const String & path) { - auto transaction = createObjectStorageTransaction(); - transaction->createDirectories(path); - transaction->commit(); + /// Super clumsy code which allows to avoid race condition in MetaInKeeper + if (metadata_storage->isTransactional()) + { + while (!metadata_storage->existsFileOrDirectory(path)) + { + auto transaction = createObjectStorageTransaction(); + transaction->createDirectories(path); + if (isSuccessfulOutcome(transaction->tryCommit(TransactionCommitOptionsVariant{}))) + break; + } + } + else + { + auto transaction = createObjectStorageTransaction(); + transaction->createDirectories(path); + transaction->commit(); + } } @@ -447,6 +474,7 @@ void DiskObjectStorage::shutdown() void DiskObjectStorage::startupImpl() { LOG_INFO(log, "Starting up disk {}", name); + metadata_storage->startup(); object_storage->startup(); LOG_INFO(log, "Disk {} started up", name); @@ -595,6 +623,29 @@ bool DiskObjectStorage::isWriteOnce() const return object_storage->isWriteOnce(); } +bool DiskObjectStorage::isSharedCompatible() const +{ + switch (object_storage->getType()) + { + case ObjectStorageType::S3: + case ObjectStorageType::Azure: + case ObjectStorageType::Web: + break; + default: + return false; + } + + switch (metadata_storage->getType()) + { + case MetadataStorageType::Plain: + case MetadataStorageType::PlainRewritable: + case MetadataStorageType::StaticWeb: + return true; + default: + return false; + } +} + bool DiskObjectStorage::supportsHardLinks() const { return !isWriteOnce() && !object_storage->isPlain(); @@ -614,7 +665,8 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() metadata_storage, object_storage, Context::getGlobalContextInstance()->getConfigRef(), - config_prefix); + config_prefix, + use_fake_transaction); } template @@ -683,30 +735,46 @@ std::unique_ptr DiskObjectStorage::readFile( /// /// [AsynchronousBoundedReadBuffer] (if use_async_buffer) /// [CachedInMemoryReadBufferFromFile] (if use_page_cache) - /// ReadBufferFromRemoteFSGather - /// [CachedOnDiskReadBufferFromFile] (if fs cache is enabled) - /// ReadBufferFromS3 or similar + /// [ReadBufferFromDistributedCache] (if use_distributed_cache) + /// ReadBufferFromRemoteFSGather + /// [CachedOnDiskReadBufferFromFile] (if fs cache is enabled) + /// ReadBufferFromS3 or similar /// /// Some of them have special requirements: /// * use_external_buffer = true is required for the buffer nested directly inside - /// AsynchronousBoundedReadBuffer, CachedInMemoryReadBufferFromFile, and - /// ReadBufferFromRemoteFSGather. - /// * The buffer directly inside CachedInMemoryReadBufferFromFile must be freely seekable. - /// I.e. either remote_read_buffer_restrict_seek = false or buffer implementation that - /// ignores that setting. - /// Note: ReadBufferFromRemoteFSGather ignores this setting. + /// AsynchronousBoundedReadBuffer, CachedInMemoryReadBufferFromFile, + /// ReadBufferFromDistributedCache, and ReadBufferFromRemoteFSGather. + /// * The buffer directly inside CachedInMemoryReadBufferFromFile or + /// ReadBufferFromDistributedCache must be freely seekable. I.e. either + /// remote_read_buffer_restrict_seek = false or buffer implementation that ignores this setting. + /// Note: ReadBufferFromRemoteFSGather and ReadBufferFromDistributedCache ignore this setting. auto read_settings = updateIOSchedulingSettings(settings, getReadResourceName(), getWriteResourceName()); /// We wrap read buffer from object storage (read_buf = object_storage->readObject()) /// inside ReadBufferFromRemoteFSGather, so add nested buffer setting. read_settings = read_settings.withNestedBuffer(); + bool use_distributed_cache = false; +#if ENABLE_DISTRIBUTED_CACHE + ObjectStorageConnectionInfoPtr connection_info; + if (enable_distributed_cache + && settings.read_through_distributed_cache + && DistributedCache::Registry::instance().isReady(settings.distributed_cache_settings.read_only_from_current_az)) + { + connection_info = object_storage->getConnectionInfo(); + if (connection_info) + use_distributed_cache = true; + } +#endif + const bool use_async_buffer = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; + const bool file_cache_enabled = object_storage->supportsCache() && read_settings.enable_filesystem_cache; const bool use_page_cache = - (!object_storage->supportsCache() || !read_settings.enable_filesystem_cache) - && read_settings.page_cache && read_settings.use_page_cache_for_disks_without_file_cache; + read_settings.page_cache + && (use_distributed_cache ? read_settings.use_page_cache_with_distributed_cache + : (read_settings.use_page_cache_for_disks_without_file_cache && !file_cache_enabled)); - const bool use_external_buffer_for_gather = use_async_buffer || use_page_cache; + const bool use_external_buffer_for_gather = use_async_buffer || use_page_cache || use_distributed_cache; auto read_buffer_creator = [this, read_settings, read_hint, file_size] @@ -721,7 +789,8 @@ std::unique_ptr DiskObjectStorage::readFile( bool prefer_bigger_buffer_size = read_settings.filesystem_cache_prefer_bigger_buffer_size && !read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache && object_storage->supportsCache() - && read_settings.enable_filesystem_cache; + && read_settings.enable_filesystem_cache + && !use_distributed_cache; size_t buffer_size = prefer_bigger_buffer_size ? std::max(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) @@ -731,13 +800,39 @@ std::unique_ptr DiskObjectStorage::readFile( if (total_objects_size) buffer_size = std::min(buffer_size, total_objects_size); + auto read_from_object_storage_gather = [=]() + { + return std::make_unique( + std::move(read_buffer_creator), + storage_objects, + read_settings, + use_external_buffer_for_gather, + /* buffer_size */use_external_buffer_for_gather ? 0 : buffer_size); + }; + std::unique_ptr impl; - impl = std::make_unique( - std::move(read_buffer_creator), - storage_objects, - read_settings, - use_external_buffer_for_gather, - /* buffer_size */use_external_buffer_for_gather ? 0 : buffer_size); + +#if ENABLE_DISTRIBUTED_CACHE + if (use_distributed_cache) + { + LOG_TEST(log, "Reading from distributed cache"); + + auto query_context = CurrentThread::isInitialized() ? CurrentThread::get().getQueryContext() : nullptr; + impl = std::make_unique( + path, + storage_objects, + read_settings, + connection_info, + ConnectionTimeouts::getTCPTimeoutsWithoutFailover(query_context ? query_context->getSettingsRef() : global_context->getSettingsRef()), + read_from_object_storage_gather, + /*use_external_buffer*/use_page_cache || use_async_buffer, + global_context->getDistributedCacheLog(), + /* include_credentials_in_cache_key */false); + } +#endif + + if (!impl) + impl = read_from_object_storage_gather(); if (use_page_cache) { @@ -756,12 +851,16 @@ std::unique_ptr DiskObjectStorage::readFile( if (use_async_buffer) { auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + const size_t min_bytes_for_seek = use_distributed_cache + ? read_settings.distributed_cache_settings.min_bytes_for_seek + : read_settings.remote_read_min_bytes_for_seek; + return std::make_unique( std::move(impl), reader, read_settings, buffer_size, - read_settings.remote_read_min_bytes_for_seek, /// Modified in private repo. + min_bytes_for_seek, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); @@ -833,11 +932,13 @@ void DiskObjectStorage::applyNewSettings( read_resource_name_from_config = new_read_resource_name; if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name_from_config) write_resource_name_from_config = new_write_resource_name; + enable_distributed_cache = config.getBool(config_prefix + ".enable_distributed_cache", true); } remove_shared_recursive_file_limit = config.getUInt64(config_prefix + ".remove_shared_recursive_file_limit", DEFAULT_REMOVE_SHARED_RECURSIVE_FILE_LIMIT); IDisk::applyNewSettings(config, context_, config_prefix, disk_map); + metadata_storage->applyNewSettings(config, config_prefix, context_); } #if USE_AWS_S3 diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 1a0705f47776..95fabdeb784f 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -35,7 +35,8 @@ friend class DiskObjectStorageReservation; MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, const Poco::Util::AbstractConfiguration & config, - const String & config_prefix); + const String & config_prefix, + bool use_fake_transaction_ = true); /// Create fake transaction DiskTransactionPtr createTransaction() override; @@ -200,6 +201,9 @@ friend class DiskObjectStorageReservation; /// MergeTree table on this disk. bool isWriteOnce() const override; + /// Return true if the disk is "shared-compatible", i.e. does not uses local disks + bool isSharedCompatible() const; + bool supportsHardLinks() const override; bool supportsPartitionCommand(const PartitionCommand & command) const override; @@ -210,6 +214,11 @@ friend class DiskObjectStorageReservation; /// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage))) String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); } + std::string getObjectsKeyPrefix() const + { + return object_key_prefix; + } + /// Add a cache layer. /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// There can be any number of cache layers: @@ -264,7 +273,9 @@ friend class DiskObjectStorageReservation; String read_resource_name_from_sql_any; // described by CREATE RESOURCE query with READ ANY DISK clause String write_resource_name_from_sql_any; // described by CREATE RESOURCE query with WRITE ANY DISK clause scope_guard resource_changes_subscription; + std::atomic_bool enable_distributed_cache; + bool use_fake_transaction; UInt64 remove_shared_recursive_file_limit; }; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index a9b868f72ce7..70047e9330fc 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -93,11 +93,25 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) } } -void DiskObjectStorageMetadata::deserializeFromString(const String & data) +void DiskObjectStorageMetadata::createFromSingleObject(ObjectStorageKey object_key, size_t bytes_size, size_t ref_count_, bool read_only_) +{ + keys_with_meta.emplace_back(std::move(object_key), ObjectMetadata{.size_bytes = bytes_size, .last_modified = {}, .etag = "", .attributes = {}}); + total_size = bytes_size; + ref_count = static_cast(ref_count_); + read_only = read_only_; +} + +void DiskObjectStorageMetadata::deserializeFromString(const std::string & data) +try { ReadBufferFromString buf(data); deserialize(buf); } +catch (Exception & e) +{ + e.addMessage("while parsing: '{}'", data); + throw; +} void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const { diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index bd713dc987cf..5c33eb330898 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -1,5 +1,10 @@ #include #include +#if ENABLE_DISTRIBUTED_CACHE +#include +#include +#endif +#include #include #include #include @@ -8,7 +13,8 @@ #include #include #include - +#include +#include #include #include #include @@ -18,6 +24,11 @@ namespace DB { +namespace FailPoints +{ + extern const char smt_insert_fake_hardware_error[]; +} + namespace ErrorCodes { extern const int UNKNOWN_FORMAT; @@ -124,6 +135,16 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation { if (!metadata_storage.existsFile(path)) { + auto maybe_blobs = tx->tryGetBlobsFromTransactionIfExists(path); + if (maybe_blobs.has_value()) + { + auto unlink_outcome = tx->unlinkMetadata(path); + if (unlink_outcome) + objects_to_remove = ObjectsToRemove{std::move(*maybe_blobs), std::move(unlink_outcome)}; + + return; + } + if (if_exists) return; @@ -206,6 +227,19 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati { if (!metadata_storage.existsFile(path)) { + auto maybe_blobs = tx->tryGetBlobsFromTransactionIfExists(path); + if (maybe_blobs.has_value()) + { + auto unlink_outcome = tx->unlinkMetadata(path); + if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) + { + objects_to_remove.emplace_back(ObjectsToRemove{std::move(*maybe_blobs), std::move(unlink_outcome)}); + paths_removed_with_objects.push_back(path); + } + + continue; + } + if (if_exists) continue; @@ -762,8 +796,14 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile { /// Otherwise we will produce lost blobs which nobody points to /// WriteOnce storages are not affected by the issue - if (!tx->object_storage.isPlain() && tx->metadata_storage.existsFile(path)) - tx->object_storage.removeObjectsIfExist(tx->metadata_storage.getStorageObjects(path)); + if (!tx->object_storage.isPlain()) + { + auto maybe_blobs = tx->metadata_transaction->tryGetBlobsFromTransactionIfExists(path); + if (maybe_blobs.has_value()) + tx->object_storage.removeObjectsIfExist(*maybe_blobs); + /// NOTE: In the best case it would be better to remove file from metadata transaction + /// as well, nevertheless it will be overridden in createMetadataFilePath + } if (do_not_write_empty_blob && count == 0) { @@ -799,15 +839,21 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile /// ... /// buf1->finalize() // shouldn't do anything with metadata operations, just memoize what to do /// tx->commit() - write_op->setOnExecute([object_storage_tx, mode, path, key_, count, do_not_write_empty_blob](MetadataTransactionPtr tx) + /// NOTE: we should not take shared_ptr, only raw pointer, because transaction object always lives + /// longer than buffer with this callback. Otherwise we can get cyclic shared_ptr. + write_op->setOnExecute([object_storage_tx_raw_ptr = object_storage_tx.get(), mode, path, key_, count, do_not_write_empty_blob](MetadataTransactionPtr tx) { if (mode == WriteMode::Rewrite) { /// Otherwise we will produce lost blobs which nobody points to /// WriteOnce storages are not affected by the issue - if (!object_storage_tx->object_storage.isPlain() && object_storage_tx->metadata_storage.existsFile(path)) + if (!object_storage_tx_raw_ptr->object_storage.isPlain()) { - object_storage_tx->object_storage.removeObjectsIfExist(object_storage_tx->metadata_storage.getStorageObjects(path)); + auto maybe_blobs = tx->tryGetBlobsFromTransactionIfExists(path); + if (maybe_blobs.has_value()) + object_storage_tx_raw_ptr->object_storage.removeObjectsIfExist(*maybe_blobs); + /// NOTE: In the best case it would be better to remove file from metadata transaction + /// as well, nevertheless it will be overridden in createMetadataFilePath } if (do_not_write_empty_blob && count == 0) @@ -830,14 +876,46 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile operations_to_execute.emplace_back(std::move(write_operation)); } + [[maybe_unused]] bool use_distributed_cache = false; + size_t use_buffer_size = buf_size; +#if ENABLE_DISTRIBUTED_CACHE + use_distributed_cache = settings.write_through_distributed_cache + && DistributedCache::Registry::instance().isReady(settings.distributed_cache_settings.read_only_from_current_az); + if (use_distributed_cache && settings.distributed_cache_settings.write_through_cache_buffer_size) + use_buffer_size = settings.distributed_cache_settings.write_through_cache_buffer_size; +#endif + auto impl = object_storage.writeObject( object, /// We always use mode Rewrite because we simulate append using metadata and different files WriteMode::Rewrite, object_attributes, - buf_size, + use_buffer_size, settings); +#if ENABLE_DISTRIBUTED_CACHE + if (use_distributed_cache) + { + auto connection_info = object_storage.getConnectionInfo(); + if (connection_info) + { + auto global_context = Context::getGlobalContextInstance(); + auto query_context = CurrentThread::isInitialized() ? CurrentThread::get().getQueryContext() : nullptr; + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover( + query_context ? query_context->getSettingsRef() : global_context->getSettingsRef()); + + impl = std::make_unique( + path, + object, + settings, + connection_info, + std::move(impl), + std::move(timeouts)); + } + } +#endif + return std::make_unique( std::move(impl), std::move(create_metadata_callback), object.remote_path, do_not_write_empty_blob); } @@ -945,7 +1023,7 @@ void MultipleDisksObjectStorageTransaction::copyFile(const std::string & from_fi object_storage, metadata_storage, destination_object_storage, read_settings, write_settings, from_file_path, to_file_path)); } -void DiskObjectStorageTransaction::commit() +void DiskObjectStorageTransaction::commit(const TransactionCommitOptionsVariant & options) { for (size_t i = 0; i < operations_to_execute.size(); ++i) { @@ -980,24 +1058,107 @@ void DiskObjectStorageTransaction::commit() try { - metadata_transaction->commit(); + metadata_transaction->commit(options); } catch (...) { - for (const auto & operation : operations_to_execute | std::views::reverse) - operation->undo(); + if (needRollbackBlobs(options)) + undo(); throw; } for (const auto & operation : operations_to_execute) operation->finalize(); + + operations_to_execute.clear(); + + is_committed = true; +} + + +TransactionCommitOutcomeVariant DiskObjectStorageTransaction::tryCommit(const TransactionCommitOptionsVariant & options) +{ + for (size_t i = 0; i < operations_to_execute.size(); ++i) + { + try + { + operations_to_execute[i]->execute(metadata_transaction); + } + catch (Exception & ex) + { + /// Reset metadata transaction, it will be refilled in operations_to_execute[i]->execute on the next retry if needed + metadata_transaction = metadata_storage.createTransaction(); + + ex.addMessage(fmt::format("While executing operation #{}", i)); + + if (needRollbackBlobs(options)) + { + for (int64_t j = i; j >= 0; --j) + { + try + { + operations_to_execute[j]->undo(); + } + catch (Exception & rollback_ex) + { + rollback_ex.addMessage(fmt::format("While undoing operation #{}", i)); + throw; + } + } + } + + throw; + } + } + + TransactionCommitOutcomeVariant outcome = metadata_transaction->tryCommit(options); + try + { + fiu_do_on(FailPoints::smt_insert_fake_hardware_error, + { + auto * result = std::get_if(&outcome); + result->code = Coordination::Error::ZOPERATIONTIMEOUT; + }); + + if (!isSuccessfulOutcome(outcome)) + { + /// Reset metadata transaction, it will be refilled in operations_to_execute[i]->execute on the next retry if needed + metadata_transaction = metadata_storage.createTransaction(); + + if (canRollbackBlobs(options, outcome)) + { + undo(); + } + + return outcome; + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + undo(); + throw; + } + + for (const auto & operation : operations_to_execute) + operation->finalize(); + + operations_to_execute.clear(); + + is_committed = true; + return outcome; } void DiskObjectStorageTransaction::undo() { - for (const auto & operation : operations_to_execute | std::views::reverse) - operation->undo(); + if (!is_committed) + { + for (const auto & operation : operations_to_execute | std::views::reverse) + operation->undo(); + } + + operations_to_execute.clear(); } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index a8a7bb74e2d7..7b8e3c2549f1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -66,12 +66,14 @@ struct DiskObjectStorageTransaction : public IDiskTransaction, std::enable_share IMetadataStorage & metadata_storage_, MetadataTransactionPtr metadata_transaction_); + bool is_committed = false; + public: DiskObjectStorageTransaction( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_); - void commit() override; + void commit(const TransactionCommitOptionsVariant & options) override; void undo() override; void createDirectory(const std::string & path) override; @@ -120,6 +122,8 @@ struct DiskObjectStorageTransaction : public IDiskTransaction, std::enable_share void chmod(const String & path, mode_t mode) override; void setReadOnly(const std::string & path) override; void createHardLink(const std::string & src_path, const std::string & dst_path) override; + + TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override; }; struct MultipleDisksObjectStorageTransaction final : public DiskObjectStorageTransaction, std::enable_shared_from_this diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 2fc5a5723c3f..0f5a953bd603 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -52,7 +53,12 @@ using TruncateFileOperationOutcomePtr = std::shared_ptr tryGetBlobsFromTransactionIfExists(const std::string &) const = 0; + virtual ~IMetadataTransaction() = default; protected: @@ -184,6 +193,9 @@ class IMetadataStorage : private boost::noncopyable virtual MetadataStorageType getType() const = 0; + virtual std::string getZooKeeperName() const { return ""; } + virtual std::string getZooKeeperPath() const { return ""; } + /// Returns true if empty file can be created without any blobs in the corresponding object storage. /// E.g. metadata storage can store the empty list of blobs corresponding to a file without actually storing any blobs. /// But if the metadata storage just relies on for example local FS to store data under logical path, then a file has to be created even if it's empty. @@ -251,6 +263,7 @@ class IMetadataStorage : private boost::noncopyable throwNotImplemented(); } + virtual void startup() {} virtual void shutdown() { /// This method is overridden for specific metadata implementations in ClickHouse Cloud. @@ -286,6 +299,30 @@ class IMetadataStorage : private boost::noncopyable return std::nullopt; } + virtual bool isReadOnly() const = 0; + + virtual bool isTransactional() const + { + return false; + } + + /// Re-read paths or their full subtrees from disk and update cache. + /// Can return serialized description of cache update which can be used to populate cache on other nodes. + virtual void updateCache(const std::vector & /* paths */, bool /* recursive */, bool /* enforce_fresh */, + std::string * /* serialized_cache_update_description */) {} + /// Allows to apply cache update from serialized description. + virtual void updateCacheFromSerializedDescription(const std::string & /* serialized_cache_update_description */) {} + virtual void invalidateCache(const std::string & /* path */) {} + + /// Clear all cache content. + virtual void dropCache() {} + + /// Apply configuration changes. + virtual void applyNewSettings( + const Poco::Util::AbstractConfiguration & /* config */, + const std::string & /* config_prefix */, + ContextPtr /* context */) {} + protected: [[noreturn]] static void throwNotImplemented() { diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 523ca03a036a..aec2d2f6b6ea 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -39,14 +40,33 @@ namespace DB::AzureBlobStorage class ContainerClientWrapper; using ContainerClient = ContainerClientWrapper; +class StaticCredential : public Azure::Core::Credentials::TokenCredential +{ +public: + StaticCredential(std::string token_, std::chrono::system_clock::time_point expires_on_) + : token(std::move(token_)), expires_on(expires_on_) + {} + + Azure::Core::Credentials::AccessToken GetToken( + Azure::Core::Credentials::TokenRequestContext const &, + Azure::Core::Context const &) const override + { + return Azure::Core::Credentials::AccessToken { .Token = token, .ExpiresOn = expires_on }; + } + +private: + std::string token; + std::chrono::system_clock::time_point expires_on; +}; + using ConnectionString = StrongTypedef; using AuthMethod = std::variant< ConnectionString, std::shared_ptr, std::shared_ptr, - std::shared_ptr>; - + std::shared_ptr, + std::shared_ptr>; } @@ -170,6 +190,8 @@ class IObjectStorage /// at least size of object virtual ObjectMetadata getObjectMetadata(const std::string & path) const = 0; + virtual ObjectStorageConnectionInfoPtr getConnectionInfo() const { return nullptr; } + /// Read single object virtual std::unique_ptr readObject( /// NOLINT const StoredObject & object, diff --git a/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.cpp new file mode 100644 index 000000000000..e309933bb550 --- /dev/null +++ b/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include +#include +#include +#include "config.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; +} + +void IObjectStorageConnectionInfo::writeBinary(size_t protocol_version, WriteBuffer & out) +{ + DB::writeBinary(UInt8(getType()), out); + writeBinaryImpl(protocol_version, out); +} + +ObjectStorageConnectionInfoPtr IObjectStorageConnectionInfo::readBinary(ReadBuffer & in, const std::string & user_id [[maybe_unused]]) +{ + UInt8 type = 0; + DB::readBinary(type, in); + + if (type >= static_cast(ObjectStorageType::Max)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected value smaller than {}, got {}", + toString(static_cast(ObjectStorageType::Max)), toString(type)); + } + + switch (ObjectStorageType(type)) + { + case ObjectStorageType::S3: + { +#if USE_AWS_S3 + auto info = std::make_unique(user_id); + info->readBinaryImpl(in); + return info; +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Server is built without S3 support"); +#endif + } + case ObjectStorageType::Azure: + { +#if USE_AZURE_BLOB_STORAGE + auto info = std::make_unique(user_id); + info->readBinaryImpl(in); + return info; +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Server is built without Azure support"); +#endif + } + case ObjectStorageType::Web: + { + return getWebObjectStorageConnectionInfo(in); + } + case ObjectStorageType::None: + { + /// Data is not stored in object storage, but written by client directly to cache. + /// It's used for temporary data. + return std::make_shared(); + } + default: + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Storage {} is not supported / implemented", ObjectStorageType(type)); + } + UNREACHABLE(); +} + +} diff --git a/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.h b/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.h new file mode 100644 index 000000000000..aa7615972b03 --- /dev/null +++ b/src/Disks/ObjectStorages/IObjectStorageConnectionInfo.h @@ -0,0 +1,96 @@ +#pragma once +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class ReadBuffer; +class WriteBuffer; +class ReadBufferFromFileBase; +class WriteBufferFromFileBase; +struct ReadSettings; +struct WriteSettings; +class RemoteHostFilter; + +class IObjectStorageConnectionInfo; +using ObjectStorageConnectionInfoPtr = std::shared_ptr; + +/// A class which contains information about how to connect +/// and later read/write from/to object storage. +/// Exposes methods to serialize this information to later be send over network. +class IObjectStorageConnectionInfo +{ +public: + virtual ~IObjectStorageConnectionInfo() = default; + + /// Deserialize connection info from `in`. + /// No protocol version is passed, because it is expected that + /// in writeBinary we used "mutual" protocol version between sender and receiver. + static ObjectStorageConnectionInfoPtr readBinary(ReadBuffer & in, const std::string & user_id); + + virtual bool equals(const IObjectStorageConnectionInfo &) const = 0; + + /// Serialize connection info into `out`. + /// Take into account protocol version if needed. + /// `protocol_version` must be a mutual version between sender and receiver. + void writeBinary(size_t protocol_version, WriteBuffer & out); + + /// Get object storage type: S3, Azure, GCP, etc. + virtual ObjectStorageType getType() const = 0; + + /// Calculate hash of the stored connection info. + virtual void updateHash(SipHash & hash, bool include_credentials) const = 0; + + /// Refresh credentials if changed. + virtual bool refreshCredentials() { return false; } + + /// Create reader to be able to read from object storage. + virtual std::unique_ptr createReader( + const StoredObject & object, + const ReadSettings & read_settings) = 0; + +protected: + /// Serialize object storage specific connection info. + virtual void writeBinaryImpl(size_t protocol_version, WriteBuffer & out) = 0; +}; + +/// Used for temporary data in cache, +/// where data is not stored in object storage, +/// but written by client directly to cache. +class NoneStorageConnectionInfo : public IObjectStorageConnectionInfo +{ +public: + ObjectStorageType getType() const override { return ObjectStorageType::None; } + + void updateHash(SipHash &, bool) const override {} + + bool equals(const IObjectStorageConnectionInfo & other) const override + { + const auto * none_info = dynamic_cast(&other); + return none_info; + } + + std::unique_ptr createReader( + const StoredObject & object, + const ReadSettings &) override + { + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "File {} (local_path: {}) should be read from local disk", + object.remote_path, object.local_path); + } + +protected: + void writeBinaryImpl(size_t, WriteBuffer &) override {} +}; + +} diff --git a/src/Disks/ObjectStorages/MetadataOperationsHolder.cpp b/src/Disks/ObjectStorages/MetadataOperationsHolder.cpp index 4bd816895500..49a740fa445b 100644 --- a/src/Disks/ObjectStorages/MetadataOperationsHolder.cpp +++ b/src/Disks/ObjectStorages/MetadataOperationsHolder.cpp @@ -47,7 +47,7 @@ void MetadataOperationsHolder::addOperation(MetadataOperationPtr && operation) operations.emplace_back(std::move(operation)); } -void MetadataOperationsHolder::commitImpl(SharedMutex & metadata_mutex) +void MetadataOperationsHolder::commitImpl(const TransactionCommitOptionsVariant & options, SharedMutex & metadata_mutex) { if (state != MetadataStorageTransactionState::PREPARING) throw Exception( @@ -69,7 +69,8 @@ void MetadataOperationsHolder::commitImpl(SharedMutex & metadata_mutex) tryLogCurrentException(__PRETTY_FUNCTION__); ex.addMessage(fmt::format("While committing metadata operation #{}", i)); state = MetadataStorageTransactionState::FAILED; - rollback(lock, i); + if (needRollbackBlobs(options)) + rollback(lock, i); throw; } } diff --git a/src/Disks/ObjectStorages/MetadataOperationsHolder.h b/src/Disks/ObjectStorages/MetadataOperationsHolder.h index a042f4bd8b9b..2b94b230431a 100644 --- a/src/Disks/ObjectStorages/MetadataOperationsHolder.h +++ b/src/Disks/ObjectStorages/MetadataOperationsHolder.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -24,7 +25,7 @@ class MetadataOperationsHolder protected: void addOperation(MetadataOperationPtr && operation); - void commitImpl(SharedMutex & metadata_mutex); + void commitImpl(const TransactionCommitOptionsVariant & options, SharedMutex & metadata_mutex); }; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp index 4112ee6a2790..18ffb3a013ea 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp @@ -1,12 +1,17 @@ #include +#include #include #include +#if CLICKHOUSE_CLOUD + #include +#endif #include #include #include #include #include + namespace DB { namespace ErrorCodes @@ -85,7 +90,12 @@ static std::string getObjectKeyCompatiblePrefix( const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - return config.getString(config_prefix + ".key_compatibility_prefix", object_storage.getCommonKeyPrefix()); + std::string prefix = config.getString(config_prefix + ".key_compatibility_prefix", object_storage.getCommonKeyPrefix()); + Macros::MacroExpansionInfo info; + info.ignore_unknown = true; + info.expand_special_macros_only = true; + info.replica = Context::getGlobalContextInstance()->getMacros()->tryGetValue("replica"); + return Context::getGlobalContextInstance()->getMacros()->expand(prefix, info); } void registerMetadataStorageFromDisk(MetadataStorageFactory & factory) @@ -108,6 +118,29 @@ void registerMetadataStorageFromDisk(MetadataStorageFactory & factory) }); } +#if CLICKHOUSE_CLOUD +void registerMetadataStorageFromKeeper(MetadataStorageFactory & factory) +{ + factory.registerMetadataStorageType("keeper", []( + const std::string & /* name */, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ObjectStoragePtr object_storage) -> MetadataStoragePtr + { + LOG_INFO(getLogger("registerDiskS3"), "Using DiskS3 with metadata keeper"); + + std::string zookeeper_name = config.getString(config_prefix + ".zookeeper_name", "default"); + auto key_compatibility_prefix = getObjectKeyCompatiblePrefix(*object_storage, config, config_prefix); + /// Yes, we place objects in metadata storage from keeper by prefix from s3 object keys. + /// No reason, it just happened. Now it has to be preserved. + auto keeper_prefix = key_compatibility_prefix; + + return std::make_shared( + zookeeper_name, keeper_prefix, key_compatibility_prefix, config, config_prefix, Context::getGlobalContextInstance()); + }); +} +#endif + void registerPlainMetadataStorage(MetadataStorageFactory & factory) { factory.registerMetadataStorageType("plain", []( @@ -156,6 +189,9 @@ void registerMetadataStorages() registerPlainMetadataStorage(factory); registerPlainRewritableMetadataStorage(factory); registerMetadataStorageFromStaticFilesWebServer(factory); +#if CLICKHOUSE_CLOUD + registerMetadataStorageFromKeeper(factory); +#endif } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 459836afdb10..a30d3081c639 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -160,9 +160,9 @@ const IMetadataStorage & MetadataStorageFromDiskTransaction::getStorageForNonTra return metadata_storage; } -void MetadataStorageFromDiskTransaction::commit() +void MetadataStorageFromDiskTransaction::commit(const TransactionCommitOptionsVariant & options) { - MetadataOperationsHolder::commitImpl(metadata_storage.metadata_mutex); + MetadataOperationsHolder::commitImpl(options, metadata_storage.metadata_mutex); } void MetadataStorageFromDiskTransaction::writeStringToFile( @@ -268,4 +268,11 @@ TruncateFileOperationOutcomePtr MetadataStorageFromDiskTransaction::truncateFile return result; } +std::optional MetadataStorageFromDiskTransaction::tryGetBlobsFromTransactionIfExists(const std::string & path) const +{ + if (metadata_storage.existsFileOrDirectory(path)) + return metadata_storage.getStorageObjects(path); + return std::nullopt; +} + } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 4377ca4bcf82..11fe535c6d60 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -78,6 +78,8 @@ class MetadataStorageFromDisk final : public IMetadataStorage DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::unique_lock & lock) const; DiskObjectStorageMetadataPtr readMetadataUnlocked(const std::string & path, std::shared_lock & lock) const; + + bool isReadOnly() const override { return disk->isReadOnly(); } }; class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, private MetadataOperationsHolder @@ -94,7 +96,7 @@ class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, pr const IMetadataStorage & getStorageForNonTransactionalReads() const final; - void commit() final; + void commit(const TransactionCommitOptionsVariant & options) final; void writeStringToFile(const std::string & path, const std::string & data) override; @@ -135,6 +137,8 @@ class MetadataStorageFromDiskTransaction final : public IMetadataTransaction, pr UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; TruncateFileOperationOutcomePtr truncateFile(const std::string & src_path, size_t target_size) override; + + std::optional tryGetBlobsFromTransactionIfExists(const std::string & path) const override; }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index fc7027f6518b..168615ab0c79 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -310,8 +310,16 @@ UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTrans return result; } -void MetadataStorageFromPlainObjectStorageTransaction::commit() +void MetadataStorageFromPlainObjectStorageTransaction::commit(const TransactionCommitOptionsVariant & options) { - MetadataOperationsHolder::commitImpl(metadata_storage.metadata_mutex); + MetadataOperationsHolder::commitImpl(options, metadata_storage.metadata_mutex); } + + +std::optional +MetadataStorageFromPlainObjectStorageTransaction::tryGetBlobsFromTransactionIfExists(const std::string & path) const +{ + return metadata_storage.getStorageObjectsIfExist(path); +} + } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 80eb37dfe4d0..72b4ce0bdeb7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -90,6 +90,11 @@ class MetadataStorageFromPlainObjectStorage : public IMetadataStorage bool supportsStat() const override { return false; } bool supportsPartitionCommand(const PartitionCommand & command) const override; + bool isReadOnly() const override { return true; } + +private: + ObjectStorageKey getObjectKeyForPath(const std::string & path) const; + protected: /// Get the object storage prefix for storing metadata files. virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); } @@ -153,7 +158,9 @@ class MetadataStorageFromPlainObjectStorageTransaction : public IMetadataTransac UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; - void commit() override; + std::optional tryGetBlobsFromTransactionIfExists(const std::string & path) const override; + + void commit(const TransactionCommitOptionsVariant & options) override; bool supportsChmod() const override { return false; } }; diff --git a/src/Disks/ObjectStorages/ObjectStorageClientsCache.h b/src/Disks/ObjectStorages/ObjectStorageClientsCache.h new file mode 100644 index 000000000000..3d83c617f8cc --- /dev/null +++ b/src/Disks/ObjectStorages/ObjectStorageClientsCache.h @@ -0,0 +1,99 @@ +#pragma once + +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric DistrCacheServerS3CachedClients; +} + +namespace ProfileEvents +{ + extern const Event DistrCacheServerNewS3CachedClients; + extern const Event DistrCacheServerReusedS3CachedClients; +} + +namespace DB +{ + +/// A cache for Object Storage clients. +/// All S3 clients are separated by users. +/// Users are associated with concrete ClickHouse services (shared among different replicas). +/// For each such user we store a set of clients. +/// Each client is uniqly identified by ObjectStorageClientInfo::Hash. +/// We use LRU to manage the number of clients. +template +class ObjectStorageClientsCache : boost::noncopyable +{ +public: + using User = std::string; + using ClientPtr = std::shared_ptr; + using ClientInfoHash = UInt128; + + explicit ObjectStorageClientsCache(size_t max_clients_) : max_clients(max_clients_) { } + + static ObjectStorageClientsCache & instance() + { + static ObjectStorageClientsCache ret(/* max_clients */ 100); + return ret; + } + + std::shared_ptr getClient(const std::string & user, const typename T::ClientInfo & info) + { + std::lock_guard lock(mutex); + + SipHash hash; + hash.update(user); + info.updateHash(hash, /* include_credentials */true); + auto client_hash = hash.get128(); + + auto it = clients.find(client_hash); + if (it == clients.end()) + { + LOG_TEST(getLogger("ObjectStorageClientsCache"), "Total clients size: {}, adding client for user: {}", clients.size(), user); + + if (clients_lru.size() == max_clients) + { + auto client = clients_lru.front(); + clients.erase(client.hash); + clients_lru.pop_front(); + chassert(clients_lru.size() < max_clients); + } + else + CurrentMetrics::add(CurrentMetrics::DistrCacheServerS3CachedClients); + + ProfileEvents::increment(ProfileEvents::DistrCacheServerNewS3CachedClients); + auto client = T::makeClient(info); + clients_lru.emplace_back(client_hash, client); + clients.emplace(client_hash, std::prev(clients_lru.end())); + return client; + } + else + { + LOG_TEST(getLogger("ObjectStorageClientsCache"), "Total clients size: {}, reusing client for user: {}", clients.size(), user); + + ProfileEvents::increment(ProfileEvents::DistrCacheServerReusedS3CachedClients); + clients_lru.splice(clients_lru.end(), clients_lru, it->second); + return it->second->client; + } + } + +private: + struct ClientData + { + ClientData(const ClientInfoHash & hash_, ClientPtr client_) : hash(hash_), client(client_) { } + ClientInfoHash hash; + ClientPtr client; + }; + using ClientsLRUQueue = std::list; + + const size_t max_clients; + std::mutex mutex; + std::unordered_map clients; + ClientsLRUQueue clients_lru; +}; + +} diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index ceebde13ae0a..f33cd60236da 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -88,7 +88,12 @@ ObjectStoragePtr createObjectStorage( throw Exception( ErrorCodes::NOT_IMPLEMENTED, "plain_rewritable metadata storage support is not implemented for '{}' object storage", - DataSourceDescription{DataSourceType::ObjectStorage, type, MetadataStorageType::PlainRewritable, /*description*/ ""} + DataSourceDescription{ + .type = DataSourceType::ObjectStorage, + .object_storage_type = type, + .metadata_type = MetadataStorageType::PlainRewritable, + .description = "", + .zookeeper_name = ""} .toString()); auto metadata_storage_metrics = DB::MetadataStorageMetrics::create(); @@ -172,9 +177,7 @@ static std::string getEndpoint( void registerS3ObjectStorage(ObjectStorageFactory & factory) { - static constexpr auto disk_type = "s3"; - - factory.registerObjectStorageType(disk_type, []( + auto creator = []( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, @@ -193,7 +196,9 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) ObjectStorageType::S3, config, config_prefix, std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); return object_storage; - }); + }; + factory.registerObjectStorageType("s3", creator); + factory.registerObjectStorageType("s3_with_keeper", creator); } void registerS3PlainObjectStorage(ObjectStorageFactory & factory) @@ -288,6 +293,12 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) { auto azure_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context->getSettingsRef()); + /// AzureObjectStorage::getCommonKeyPrefix() was not implemented previousely by mistake and was always returning an empty string. + /// However, we use this string as ZooKeeper path for disk with metadata in Keeper. + /// So, all instances of azure-with-keeper were using the same (empty, or root) path. + /// We keep using empty prefix by default for compatibility, but allow to configure another one + const String & common_key_prefix = config.getString(config_prefix + ".common_key_prefix_for_azure", ""); + AzureBlobStorage::ConnectionParams params { .endpoint = AzureBlobStorage::processEndpoint(config, config_prefix), @@ -298,8 +309,8 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) return createObjectStorage( ObjectStorageType::Azure, config, config_prefix, name, params.auth_method, AzureBlobStorage::getContainerClient(params, /*readonly=*/ false), std::move(azure_settings), - params.endpoint.prefix.empty() ? params.endpoint.container_name : params.endpoint.container_name + "/" + params.endpoint.prefix, - params.endpoint.getServiceEndpoint()); + params, params.endpoint.prefix.empty() ? params.endpoint.container_name : params.endpoint.container_name + "/" + params.endpoint.prefix, + params.endpoint.getServiceEndpoint(), common_key_prefix); }; factory.registerObjectStorageType("azure_blob_storage", creator); diff --git a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp index a1baafe8b6b0..37f56a78af4e 100644 --- a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -25,10 +26,13 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch { bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); auto object_storage = ObjectStorageFactory::instance().create(name, config, config_prefix, context, skip_access_check); + std::string compatibility_metadata_type_hint; if (!config.has(config_prefix + ".metadata_type")) { - if (object_storage->isPlain()) + if (config.getString(config_prefix + ".type", "") == "s3_with_keeper") + compatibility_metadata_type_hint = "keeper"; + else if (object_storage->isPlain()) if (object_storage->isWriteOnce()) compatibility_metadata_type_hint = "plain"; else @@ -40,13 +44,24 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch auto metadata_storage = MetadataStorageFactory::instance().create( name, config, config_prefix, object_storage, compatibility_metadata_type_hint); - DiskObjectStoragePtr disk = std::make_shared( + bool use_fake_transaction = metadata_storage->getType() != MetadataStorageType::Keeper; + + DiskPtr disk = std::make_shared( name, object_storage->getCommonKeyPrefix(), std::move(metadata_storage), std::move(object_storage), config, - config_prefix); + config_prefix, + use_fake_transaction); + + /// If this disk was created "on the fly" in order to serve as a temporary read-only disk. + bool is_read_only_disk = config.getBool(config_prefix + ".read_only", false); + if (is_read_only_disk) + { + LOG_DEBUG(getLogger("registerDiskObjectStorage"), "Using read-only disk wrapper"); + disk = std::make_shared(disk); + } disk->startup(skip_access_check); return disk; @@ -56,6 +71,7 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch #if USE_AWS_S3 factory.registerDiskType("s3", creator); /// For compatibility factory.registerDiskType("s3_plain", creator); /// For compatibility + factory.registerDiskType("s3_with_keeper", creator); /// For compatibility factory.registerDiskType("s3_plain_rewritable", creator); // For compatibility #endif #if USE_HDFS diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index 5ea712cbd56a..978d1b042144 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -1,7 +1,9 @@ #include "DiskS3Utils.h" #if USE_AWS_S3 +#include #include +#include #include namespace DB @@ -21,6 +23,14 @@ ObjectStorageKeysGeneratorPtr getKeyGenerator( String object_key_compatibility_prefix = config.getString(config_prefix + ".key_compatibility_prefix", String()); String object_key_template = config.getString(config_prefix + ".key_template", String()); + Macros::MacroExpansionInfo info; + info.ignore_unknown = true; + info.expand_special_macros_only = true; + info.replica = Context::getGlobalContextInstance()->getMacros()->tryGetValue("replica"); + object_key_compatibility_prefix = Context::getGlobalContextInstance()->getMacros()->expand(object_key_compatibility_prefix, info); + info.level = 0; + object_key_template = Context::getGlobalContextInstance()->getMacros()->expand(object_key_template, info); + if (object_key_template.empty()) { if (!object_key_compatibility_prefix.empty()) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 1308fea6b2fb..430d99231461 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -23,6 +23,7 @@ #include #include +#include #include #include @@ -514,6 +515,11 @@ void S3ObjectStorage::applyNewSettings( s3_settings.set(std::move(modified_settings)); } +ObjectStorageConnectionInfoPtr S3ObjectStorage::getConnectionInfo() const +{ + return getS3ObjectStorageConnectionInfo(*client.get(), uri.bucket); +} + ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { if (!key_generator) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 9817d66fb965..7b64d3b0f9fe 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -24,6 +24,8 @@ namespace S3RequestSetting class S3ObjectStorage : public IObjectStorage { private: + friend class S3PlainObjectStorage; + S3ObjectStorage( const char * logger_name, std::unique_ptr && client_, @@ -88,6 +90,8 @@ class S3ObjectStorage : public IObjectStorage ObjectMetadata getObjectMetadata(const std::string & path) const override; + ObjectStorageConnectionInfoPtr getConnectionInfo() const override; + std::optional tryGetObjectMetadata(const std::string & path) const override; void copyObject( /// NOLINT diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.cpp new file mode 100644 index 000000000000..29f512c08231 --- /dev/null +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.cpp @@ -0,0 +1,266 @@ +#include + +#if USE_AWS_S3 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#if ENABLE_DISTRIBUTED_CACHE +#include +#endif +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int PROTOCOL_VERSION_MISMATCH; + extern const int LOGICAL_ERROR; +} + +namespace Setting +{ + extern const SettingsBool s3_validate_request_settings; +} +namespace S3AuthSetting +{ + extern const S3AuthSettingsString access_key_id; + extern const S3AuthSettingsString secret_access_key; + extern const S3AuthSettingsString session_token; + extern const S3AuthSettingsString region; +} + +void S3ObjectStorageClientInfo::updateHash(SipHash & hash, bool include_credentials) const +{ + hash.update(provider_type); + hash.update(endpoint); + hash.update(region); + hash.update(bucket); + + if (include_credentials) + { + switch (provider_type) + { + case S3::ProviderType::AWS: + { + hash.update(access_key_id); + hash.update(secret_key); + break; + } + case S3::ProviderType::GCS: + { + hash.update(gcs_token); + break; + } + case S3::ProviderType::UNKNOWN: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown S3 provider type"); + } + } +} + +S3ObjectStorageConnectionInfo::S3ObjectStorageConnectionInfo(const S3::Client & client_, const std::string & bucket_) + : client(client_.clone()) + , log(getLogger("S3ObjectStorageConnectionInfo")) +{ + const auto credentials = client->getCredentials(); + auto provider_type = client->getProviderType(); + + if (provider_type == S3::ProviderType::UNKNOWN) + provider_type = S3::ProviderType::AWS; + + client_info = ClientInfo{ + .provider_type = provider_type, + .endpoint = client->getInitialEndpoint(), + .region = client->getRegion(), + .bucket = bucket_, + .access_key_id = credentials.GetAWSAccessKeyId(), + .secret_key = credentials.GetAWSSecretKey(), + .session_token = credentials.GetSessionToken(), + .gcs_token = client->getGCSOAuthToken(), + }; +} + +S3ObjectStorageConnectionInfo::S3ObjectStorageConnectionInfo(const std::string & user_info_) + : log(getLogger("S3ObjectStorageConnectionInfo")) + , user_info(user_info_) +{ +} + +bool S3ObjectStorageConnectionInfo::equals(const IObjectStorageConnectionInfo & other) const +{ + const auto * s3_info = dynamic_cast(&other); + if (s3_info) + return client_info == s3_info->client_info; + return false; +} + +void S3ObjectStorageConnectionInfo::readBinaryImpl(ReadBuffer & in) +{ + UInt8 type; + readVarUInt(type, in); + client_info.provider_type = S3::ProviderType{type}; + switch (client_info.provider_type) + { + case S3::ProviderType::AWS: + { + DB::readBinary(client_info.endpoint, in); + DB::readBinary(client_info.region, in); + DB::readBinary(client_info.bucket, in); + DB::readBinary(client_info.access_key_id, in); + DB::readBinary(client_info.secret_key, in); + DB::readBinary(client_info.session_token, in); + break; + } + case S3::ProviderType::GCS: + { + DB::readBinary(client_info.endpoint, in); + DB::readBinary(client_info.region, in); + DB::readBinary(client_info.bucket, in); + DB::readBinary(client_info.access_key_id, in); + DB::readBinary(client_info.secret_key, in); + + DB::readBinary(client_info.gcs_token, in); + + break; + } + case S3::ProviderType::UNKNOWN: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown provider type"); + } + } +} + +bool S3ObjectStorageConnectionInfo::refreshCredentials() +{ + if (!client) + client = ObjectStorageClientsCache::instance().getClient(user_info, client_info); + + const auto credentials = client->getCredentials(); + const auto & access_key_id = credentials.GetAWSAccessKeyId(); + const auto & secret_key = credentials.GetAWSSecretKey(); + const auto & session_token = credentials.GetSessionToken(); + const auto & gcs_token = client->getGCSOAuthToken(); + + if (client_info.access_key_id == access_key_id + && client_info.secret_key == secret_key + && client_info.session_token == session_token + && client_info.gcs_token == gcs_token) + { + LOG_TEST(log, "Credentials did not change, will not update"); + return false; + } + + LOG_TRACE(log, "Credentials changed, will update"); + + client_info.access_key_id = access_key_id; + client_info.secret_key = secret_key; + client_info.session_token = session_token; + client_info.gcs_token = gcs_token; + + return true; +} + +void S3ObjectStorageConnectionInfo::writeBinaryImpl(size_t mutual_protocol_version, WriteBuffer & out) +{ + if (client_info.provider_type == S3::ProviderType::GCS + && mutual_protocol_version < DistributedCache::PROTOCOL_VERSION_WITH_GCS_TOKEN) + throw Exception(ErrorCodes::PROTOCOL_VERSION_MISMATCH, "Protocol version does not support GCS token"); + + writeVarUInt(UInt8(client_info.provider_type), out); + switch (client_info.provider_type) + { + case S3::ProviderType::AWS: + { + DB::writeBinary(client_info.endpoint, out); + DB::writeBinary(client_info.region, out); + DB::writeBinary(client_info.bucket, out); + DB::writeBinary(client_info.access_key_id, out); + DB::writeBinary(client_info.secret_key, out); + DB::writeBinary(client_info.session_token, out); + break; + } + case S3::ProviderType::GCS: + { + DB::writeBinary(client_info.endpoint, out); + DB::writeBinary(client_info.region, out); + DB::writeBinary(client_info.bucket, out); + DB::writeBinary(client_info.access_key_id, out); + DB::writeBinary(client_info.secret_key, out); + + DB::writeBinary(client_info.gcs_token, out); + + break; + } + case S3::ProviderType::UNKNOWN: + UNREACHABLE(); + } +} + +std::unique_ptr S3ObjectStorageConnectionInfo::createReader( + const StoredObject & object, + const ReadSettings & read_settings) +{ + if (!client) + client = ObjectStorageClientsCache::instance().getClient(user_info, client_info); + + S3::S3RequestSettings request_settings; + return std::make_unique( + client, + client_info.bucket, + object.remote_path, + /* version_id */"", + request_settings, + read_settings, + /* use_external_buffer */true, + /* offset */0, + /* read_until_position */0, + /* restricted_seek */true); +} + + +std::shared_ptr S3ObjectStorageConnectionInfo::makeClient(const S3ObjectStorageClientInfo & info) +{ + auto global_context = Context::getGlobalContextInstance(); + std::unique_ptr s3_settings= std::make_unique(); + + S3::URI url(info.endpoint); + if (url.bucket.empty() || url.bucket == "default") + url.bucket = info.bucket; + chassert(url.bucket == info.bucket); + + s3_settings->loadFromConfigForObjectStorage( + global_context->getConfigRef(), "s3", global_context->getSettingsRef(), url.uri.getScheme(), global_context->getSettingsRef()[Setting::s3_validate_request_settings]); + + s3_settings->auth_settings[S3AuthSetting::access_key_id] = info.access_key_id; + s3_settings->auth_settings[S3AuthSetting::secret_access_key] = info.secret_key; + s3_settings->auth_settings[S3AuthSetting::session_token] = info.session_token; + s3_settings->auth_settings[S3AuthSetting::region] = info.region; + + if (!info.gcs_token.empty()) + { + s3_settings->auth_settings.headers.push_back( + {"Authorization", fmt::format("Bearer {}", info.gcs_token)}); + } + + return getClient(url, *s3_settings, global_context, true /* for_disk_s3 */); +} + + +ObjectStorageConnectionInfoPtr getS3ObjectStorageConnectionInfo(const S3::Client & client, const std::string & bucket) +{ + return std::make_shared(client, bucket); +} + +} +#endif diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.h b/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.h new file mode 100644 index 000000000000..344d4529731c --- /dev/null +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorageConnectionInfo.h @@ -0,0 +1,75 @@ +#pragma once +#include "config.h" + +#if USE_AWS_S3 +#include +#include +#include + + +namespace DB +{ + +namespace S3 { class Client; } + +class ReadBuffer; + +ObjectStorageConnectionInfoPtr getS3ObjectStorageConnectionInfo(const S3::Client & client, const std::string & bucket); + +struct S3ObjectStorageClientInfo +{ + S3::ProviderType provider_type = S3::ProviderType::UNKNOWN; + + std::string endpoint; + std::string region; + std::string bucket; + std::string access_key_id; + std::string secret_key; + std::string session_token; + std::string gcs_token; + + bool operator == (const S3ObjectStorageClientInfo & other) const = default; + + void updateHash(SipHash & hash, bool include_credentials) const; +}; + +class S3ObjectStorageConnectionInfo : public IObjectStorageConnectionInfo +{ + friend class IObjectStorageConnectionInfo; +public: + using Client = S3::Client; + using ClientInfo = S3ObjectStorageClientInfo; + + /// Writer constructor. + S3ObjectStorageConnectionInfo(const S3::Client & client, const std::string & bucket); + /// Reader constructor. + explicit S3ObjectStorageConnectionInfo(const std::string & user_info_); + + ObjectStorageType getType() const override { return ObjectStorageType::S3; } + + bool equals(const IObjectStorageConnectionInfo &) const override; + + bool refreshCredentials() override; + + void updateHash(SipHash & hash, bool include_credentials) const override { client_info.updateHash(hash, include_credentials); } + + std::unique_ptr createReader( + const StoredObject & object, + const ReadSettings & read_settings) override; + + static std::shared_ptr makeClient(const S3ObjectStorageClientInfo &); + +protected: + void writeBinaryImpl(size_t protocol_version, WriteBuffer & out) override; + + void readBinaryImpl(ReadBuffer & in); + +private: + S3ObjectStorageClientInfo client_info; + std::shared_ptr client; + const LoggerPtr log; + const std::string user_info; +}; +} + +#endif diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index f0c9447c55b3..a04c5b88fb81 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -53,6 +53,13 @@ namespace S3AuthSetting extern const S3AuthSettingsBool use_adaptive_timeouts; extern const S3AuthSettingsBool use_environment_credentials; extern const S3AuthSettingsBool use_insecure_imds_request; + + extern const S3AuthSettingsString role_arn; + extern const S3AuthSettingsString role_session_name; + extern const S3AuthSettingsString http_client; + extern const S3AuthSettingsString service_account; + extern const S3AuthSettingsString metadata_service; + extern const S3AuthSettingsString request_token_path; } namespace ErrorCodes @@ -126,6 +133,11 @@ std::unique_ptr getClient( client_configuration.http_keep_alive_timeout = auth_settings[S3AuthSetting::http_keep_alive_timeout]; client_configuration.http_keep_alive_max_requests = auth_settings[S3AuthSetting::http_keep_alive_max_requests]; + client_configuration.http_client = auth_settings[S3AuthSetting::http_client]; + client_configuration.service_account = auth_settings[S3AuthSetting::service_account]; + client_configuration.metadata_service = auth_settings[S3AuthSetting::metadata_service]; + client_configuration.request_token_path = auth_settings[S3AuthSetting::request_token_path]; + client_configuration.endpointOverride = url.endpoint; client_configuration.s3_use_adaptive_timeouts = auth_settings[S3AuthSetting::use_adaptive_timeouts]; @@ -151,6 +163,9 @@ std::unique_ptr getClient( auth_settings[S3AuthSetting::use_insecure_imds_request], auth_settings[S3AuthSetting::expiration_window_seconds], auth_settings[S3AuthSetting::no_sign_request], + auth_settings[S3AuthSetting::role_arn], + auth_settings[S3AuthSetting::role_session_name], + /*sts_endpoint_override=*/"" }; return S3::ClientFactory::instance().create( diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index 4ba071231e23..14a18b00cdb5 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -59,6 +59,8 @@ class MetadataStorageFromStaticFilesWebServer final : public IMetadataStorage bool supportsChmod() const override { return false; } bool supportsStat() const override { return false; } bool supportsPartitionCommand(const PartitionCommand & command) const override; + + bool isReadOnly() const override { return true; } }; class MetadataStorageFromStaticFilesWebServerTransaction final : public IMetadataTransaction @@ -89,12 +91,19 @@ class MetadataStorageFromStaticFilesWebServerTransaction final : public IMetadat void createDirectoryRecursive(const std::string & path) override; - void commit() override + void commit(const TransactionCommitOptionsVariant &) override { /// Nothing to commit. } bool supportsChmod() const override { return false; } + + std::optional tryGetBlobsFromTransactionIfExists(const std::string & path) const override + { + if (metadata_storage.existsFileOrDirectory(path)) + return metadata_storage.getStorageObjects(path); + return std::nullopt; + } }; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 6559be348da3..461f9482812c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include @@ -283,4 +284,9 @@ ObjectMetadata WebObjectStorage::getObjectMetadata(const std::string & /* path * throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Metadata is not supported for {}", getName()); } +ObjectStorageConnectionInfoPtr WebObjectStorage::getConnectionInfo() const +{ + return getWebObjectStorageConnectionInfo(url); +} + } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 2aa59afbcdcf..f678b4b9ceef 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -53,6 +53,8 @@ class WebObjectStorage : public IObjectStorage, WithContext ObjectMetadata getObjectMetadata(const std::string & path) const override; + ObjectStorageConnectionInfoPtr getConnectionInfo() const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.cpp new file mode 100644 index 000000000000..8661749d7b1e --- /dev/null +++ b/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.cpp @@ -0,0 +1,70 @@ + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct WebObjectStorageConnectionInfo : public IObjectStorageConnectionInfo +{ + explicit WebObjectStorageConnectionInfo(const std::string & url_) : url(url_) {} + + explicit WebObjectStorageConnectionInfo(ReadBuffer & in) + { + DB::readBinary(url, in); + } + + ObjectStorageType getType() const override { return ObjectStorageType::Web; } + + bool equals(const IObjectStorageConnectionInfo & other) const override + { + const auto * web_info = dynamic_cast(&other); + if (web_info) + return url == web_info->url; + return false; + } + + void updateHash(SipHash & hash, bool /* include_credentials */) const override + { + hash.update(url); + } + + void writeBinaryImpl(size_t /*mutual_protocol_version*/, WriteBuffer & out) override + { + DB::writeBinary(url, out); + } + + std::unique_ptr createReader( + const StoredObject & object, + const ReadSettings & read_settings) override + { + return std::make_unique( + std::filesystem::path(url) / object.remote_path, + Context::getGlobalContextInstance(), + object.bytes_size, + read_settings, + /* use_external_buffer */true, + /* read_until_position */0); + } + +private: + std::string url; +}; + +ObjectStorageConnectionInfoPtr getWebObjectStorageConnectionInfo(const std::string & url) +{ + return std::make_shared(url); +} + +ObjectStorageConnectionInfoPtr getWebObjectStorageConnectionInfo(ReadBuffer & in) +{ + return std::make_shared(in); +} + +} diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.h b/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.h new file mode 100644 index 000000000000..90c8e28b5bdf --- /dev/null +++ b/src/Disks/ObjectStorages/Web/WebObjectStorageConnectionInfo.h @@ -0,0 +1,11 @@ +#pragma once +#include + +namespace DB +{ +class ReadBuffer; + +ObjectStorageConnectionInfoPtr getWebObjectStorageConnectionInfo(const std::string & url); +ObjectStorageConnectionInfoPtr getWebObjectStorageConnectionInfo(ReadBuffer & in); + +} diff --git a/src/Disks/ReadOnlyDiskWrapper.h b/src/Disks/ReadOnlyDiskWrapper.h new file mode 100644 index 000000000000..cf00ade7af58 --- /dev/null +++ b/src/Disks/ReadOnlyDiskWrapper.h @@ -0,0 +1,134 @@ +#pragma once + +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class ReadOnlyDiskWrapper : public IDisk +{ +public: + explicit ReadOnlyDiskWrapper(const DiskPtr & delegate_) : IDisk(fmt::format("{}-readonly", delegate_->getName())), delegate(delegate_) {} + ~ReadOnlyDiskWrapper() override = default; + + DiskTransactionPtr createTransaction() override { return delegate->createTransaction(); } + const String & getName() const override { return delegate->getName(); } + const String & getPath() const override { return delegate->getPath(); } + std::optional getTotalSpace() const override { return delegate->getTotalSpace(); } + std::optional getAvailableSpace() const override { return delegate->getAvailableSpace(); } + std::optional getUnreservedSpace() const override { return delegate->getUnreservedSpace(); } + UInt64 getKeepingFreeSpace() const override { return delegate->getKeepingFreeSpace(); } + bool existsFile(const String & path) const override { return delegate->existsFile(path); } + bool existsDirectory(const String & path) const override { return delegate->existsDirectory(path); } + bool existsFileOrDirectory(const String & path) const override { return delegate->existsFileOrDirectory(path); } + size_t getFileSize(const String & path) const override { return delegate->getFileSize(path); } + + Strings getBlobPath(const String & path) const override { return delegate->getBlobPath(path); } + bool areBlobPathsRandom() const override { return delegate->areBlobPathsRandom(); } + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override + { + delegate->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); + } + + DirectoryIteratorPtr iterateDirectory(const String & path) const override { return delegate->iterateDirectory(path); } + + void copyDirectoryContent( + const String & from_dir, + const std::shared_ptr & to_disk, + const String & to_dir, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const std::function & cancellation_hook) override + { + delegate->copyDirectoryContent(from_dir, to_disk, to_dir, read_settings, write_settings, cancellation_hook); + } + + void listFiles(const String & path, std::vector & file_names) const override { delegate->listFiles(path, file_names); } + + std::unique_ptr readFile( + const String & path, + const ReadSettings & settings, + std::optional read_hint, + std::optional file_size) const override { return delegate->readFile(path, settings, read_hint, file_size); } + + time_t getLastChanged(const String & path) const override { return delegate->getLastChanged(path); } + Poco::Timestamp getLastModified(const String & path) const override { return delegate->getLastModified(path); } + void setReadOnly(const String & path) override { delegate->setReadOnly(path); } + void truncateFile(const String & path, size_t size) override { delegate->truncateFile(path, size); } + String getUniqueId(const String & path) const override { return delegate->getUniqueId(path); } + bool checkUniqueId(const String & id) const override { return delegate->checkUniqueId(id); } + DataSourceDescription getDataSourceDescription() const override { return delegate->getDataSourceDescription(); } + bool isRemote() const override { return delegate->isRemote(); } + + bool isWriteOnce() const override { return delegate->isWriteOnce(); } + bool supportZeroCopyReplication() const override { return delegate->supportZeroCopyReplication(); } + bool supportParallelWrite() const override { return delegate->supportParallelWrite(); } + SyncGuardPtr getDirectorySyncGuard(const String & path) const override { return delegate->getDirectorySyncGuard(path); } + void shutdown() override { delegate->shutdown(); } + void startupImpl() override { delegate->startupImpl(); } + void applyNewSettings( + const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override + { + delegate->applyNewSettings(config, context, config_prefix, map); + } + + bool supportsCache() const override { return delegate->supportsCache(); } + + StoredObjects getStorageObjects(const String & path) const override { return delegate->getStorageObjects(path); } + + DiskObjectStoragePtr createDiskObjectStorage() override { return delegate->createDiskObjectStorage(); } + ObjectStoragePtr getObjectStorage() override { return delegate->getObjectStorage(); } + NameSet getCacheLayersNames() const override { return delegate->getCacheLayersNames(); } + + MetadataStoragePtr getMetadataStorage() override { return delegate->getMetadataStorage(); } + + std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override { return delegate->getSerializedMetadata(file_paths); } + + UInt32 getRefCount(const String & path) const override { return delegate->getRefCount(path); } + + void syncRevision(UInt64 revision) override { delegate->syncRevision(revision); } + + UInt64 getRevision() const override { return delegate->getRevision(); } + + bool supportsStat() const override { return delegate->supportsStat(); } + struct stat stat(const String & path) const override { return delegate->stat(path); } + + bool supportsChmod() const override { return delegate->supportsChmod(); } + void chmod(const String & path, mode_t mode) override { delegate->chmod(path, mode); } + + bool isReadOnly() const override { return true; } + std::unique_ptr writeFile(const String &, size_t, WriteMode, const WriteSettings &) override { throwNotAllowed(); } + void moveFile(const String &, const String &) override { throwNotAllowed(); } + void replaceFile(const String &, const String &) override { throwNotAllowed(); } + void removeFile(const String &) override { throwNotAllowed(); } + void removeFileIfExists(const String &) override { throwNotAllowed(); } + ReservationPtr reserve(UInt64 /*bytes*/) override { throwNotAllowed(); } + void removeRecursive(const String &) override { throwNotAllowed(); } + void removeSharedFile(const String &, bool) override { throwNotAllowed(); } + void removeSharedFileIfExists(const String &, bool) override { throwNotAllowed(); } + void removeSharedRecursive(const String &, bool, const NameSet &) override { throwNotAllowed(); } + void clearDirectory(const String &) override { throwNotAllowed(); } + void moveDirectory(const String &, const String &) override { throwNotAllowed(); } + void removeDirectory(const String &) override { throwNotAllowed(); } + void setLastModified(const String &, const Poco::Timestamp &) override { throwNotAllowed(); } + void createFile(const String &) override { throwNotAllowed(); } + void createDirectory(const String &) override { throwNotAllowed(); } + void createDirectories(const String &) override { throwNotAllowed(); } + void createHardLink(const String &, const String &) override { throwNotAllowed(); } + +private: + + [[noreturn]] void throwNotAllowed() const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Operation not allowed, disk {} is read-only", getName()); + } + + DiskPtr delegate; +}; + +} diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 1a3579669bde..ce7b37c953dc 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -463,7 +463,7 @@ TEST_F(DiskEncryptedTest, LocalBlobs) makeEncryptedDisk(FileEncryption::Algorithm::AES_128_CTR, "1234567890123456", local_blobs); testSeekAndReadUntilPosition(encrypted_disk, "a.txt", {}); - + { ReadSettings read_settings; read_settings.local_fs_buffer_size = 1; diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index bd256f533219..23a0de76a7c7 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -24,6 +24,7 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -76,7 +77,7 @@ struct Memory : boost::noncopyable, Allocator const char * data() const { return m_data; } char * data() { return m_data; } - void resize(size_t new_size) + void resize(size_t new_size, bool deallocate_if_empty = false) { if (!m_data) { @@ -84,6 +85,13 @@ struct Memory : boost::noncopyable, Allocator return; } + if (new_size == 0 && deallocate_if_empty) + { + dealloc(); + m_size = m_capacity = 0; + return; + } + if (new_size <= m_capacity - pad_right) { m_size = new_size; @@ -148,14 +156,33 @@ class BufferWithOwnMemory : public Base { protected: Memory<> memory; + const bool use_existing_memory; + public: - /// If non-nullptr 'existing_memory' is passed, then buffer will not create its own memory and will use existing_memory without ownership. - explicit BufferWithOwnMemory(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) + /// If non-nullptr 'existing_memory' is passed, + /// then buffer will not create its own memory and will use existing_memory without ownership. + explicit BufferWithOwnMemory( + size_t size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0) : Base(nullptr, 0), memory(existing_memory ? 0 : size, alignment) + , use_existing_memory(existing_memory != nullptr) { Base::set(existing_memory ? existing_memory : memory.data(), size); Base::padded = !existing_memory; } + + void resize(size_t size, bool deallocate_if_empty = false) + { + if (use_existing_memory) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Existing memory was used for the buffer, resize is not allowed"); + } + memory.resize(size, deallocate_if_empty); + Base::set(memory.data(), size); + } }; diff --git a/src/IO/DistributedCacheSettings.h b/src/IO/DistributedCacheSettings.h index 8d391d1b336c..c62402ccdf82 100644 --- a/src/IO/DistributedCacheSettings.h +++ b/src/IO/DistributedCacheSettings.h @@ -7,6 +7,7 @@ namespace DB { +struct Settings; struct DistributedCacheSettings { @@ -14,16 +15,21 @@ struct DistributedCacheSettings bool bypass_connection_pool = false; size_t wait_connection_from_pool_milliseconds = 100; - size_t connect_max_tries = 100; - size_t read_request_max_tries = 100; - size_t read_alignment = 0; + size_t connect_max_tries = ::DistributedCache::DEFAULT_CONNECT_MAX_TRIES; + size_t connect_backoff_min_ms = ::DistributedCache::DEFAULT_CONNECT_BACKOFF_MIN_MS; + size_t connect_backoff_max_ms = ::DistributedCache::DEFAULT_CONNECT_BACKOFF_MAX_MS; + size_t read_request_max_tries = ::DistributedCache::DEFAULT_READ_REQUEST_MAX_TRIES; + size_t alignment = 0; size_t max_unacked_inflight_packets = ::DistributedCache::MAX_UNACKED_INFLIGHT_PACKETS; size_t data_packet_ack_window = ::DistributedCache::ACK_DATA_PACKET_WINDOW; + size_t credentials_refresh_period_seconds = ::DistributedCache::DEFAULT_CREDENTIALS_REFRESH_PERIOD_SECONDS; + size_t write_through_cache_buffer_size = 0; DistributedCachePoolBehaviourOnLimit pool_behaviour_on_limit = DistributedCachePoolBehaviourOnLimit::ALLOCATE_NEW_BYPASSING_POOL; size_t receive_response_wait_milliseconds = 10000; size_t receive_timeout_milliseconds = 1000; bool discard_connection_if_unread_data = true; + bool read_only_from_current_az = true; size_t min_bytes_for_seek = DBMS_DEFAULT_BUFFER_SIZE; @@ -31,6 +37,8 @@ struct DistributedCacheSettings bool operator ==(const DistributedCacheSettings &) const = default; + void load(const Settings & settings); + void validate() const; }; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index df2d5f089534..1a27e98c4ae1 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -38,6 +39,11 @@ namespace S3RequestSetting extern const S3RequestSettingsUInt64 max_single_read_retries; } +namespace FailPoints +{ + extern const char s3_read_buffer_throw_expired_token[]; +} + namespace ErrorCodes { extern const int S3_ERROR; @@ -88,6 +94,13 @@ bool ReadBufferFromS3::nextImpl() if (impl) { + fiu_do_on(FailPoints::s3_read_buffer_throw_expired_token, + { + throw Exception( + ErrorCodes::S3_ERROR, + "Unable to parse ExceptionName: ExpiredToken Message: The provided token has expired. This error happened for S3 disk"); + }); + if (impl->isResultReleased()) return false; diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 4c2be7f4673d..879b90624953 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -1,9 +1,13 @@ #include -#include -#include #if USE_AWS_S3 +#include +#include +#include +#include +#include + #include #include #include @@ -33,6 +37,7 @@ #include + namespace ProfileEvents { extern const Event S3WriteRequestsErrors; @@ -1029,11 +1034,22 @@ std::unique_ptr ClientFactory::create( // NOLINT client_configuration.extra_headers = std::move(headers); Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key, session_token); - auto credentials_provider = std::make_shared( + + // we need to force environment credentials if explicit credentials are empty and we have role_arn + // this is a crutch because we know that we have environment credentials on our Cloud + credentials_configuration.use_environment_credentials = + credentials_configuration.use_environment_credentials || (credentials.IsEmpty() && !credentials_configuration.role_arn.empty()); + + std::shared_ptr credentials_provider = std::make_shared( client_configuration, std::move(credentials), credentials_configuration); + if (!credentials_configuration.role_arn.empty()) + credentials_provider = std::make_shared(credentials_configuration.role_arn, + credentials_configuration.role_session_name, credentials_configuration.expiration_window_seconds, + std::move(credentials_provider), client_configuration, credentials_configuration.sts_endpoint_override); + client_configuration.retryStrategy = std::make_shared(client_configuration.s3_retry_attempts); /// Use virtual addressing if endpoint is not specified. diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 33ae0435e90c..d03bd70ca310 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -221,6 +221,16 @@ class Client : private Aws::S3::S3Client return client_configuration.for_disk_s3; } + ProviderType getProviderType() const { return provider_type; } + std::string getGCSOAuthToken() const + { + if (provider_type != ProviderType::GCS) + return ""; + + const auto & client = PocoHTTPClientGCPOAuth(client_configuration); + return client.getBearerToken(); + } + ThrottlerPtr getPutRequestThrottler() const { return client_configuration.put_request_throttler; } ThrottlerPtr getGetRequestThrottler() const { return client_configuration.get_request_throttler; } diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index 37cec351aba4..5b2a2ed85736 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -30,6 +30,13 @@ namespace S3 #if USE_AWS_S3 +# include + +# include +# include +# include +# include +# include # include # include # include @@ -429,13 +436,13 @@ void AWSInstanceProfileCredentialsProvider::refreshIfExpired() } AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider::AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider( - DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_) + DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_, String role_arn_) : logger(getLogger("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) , expiration_window_seconds(expiration_window_seconds_) { // check environment variables String tmp_region = Aws::Environment::GetEnv("AWS_DEFAULT_REGION"); - role_arn = Aws::Environment::GetEnv("AWS_ROLE_ARN"); + role_arn = role_arn_.empty() ? Aws::Environment::GetEnv("AWS_ROLE_ARN") : role_arn_; token_file = Aws::Environment::GetEnv("AWS_WEB_IDENTITY_TOKEN_FILE"); session_name = Aws::Environment::GetEnv("AWS_ROLE_SESSION_NAME"); @@ -736,7 +743,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( configuration.for_disk_s3, configuration.get_request_throttler, configuration.put_request_throttler); - AddProvider(std::make_shared(aws_client_configuration, credentials_configuration.expiration_window_seconds)); + AddProvider(std::make_shared(aws_client_configuration, credentials_configuration.expiration_window_seconds, credentials_configuration.kms_role_arn)); } AddProvider(std::make_shared()); @@ -838,6 +845,158 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( AddProvider(std::make_shared()); } +AssumeRoleRequest::AssumeRoleRequest(std::string role_arn_, std::string role_session_name_) + : role_arn(std::move(role_arn_)), role_session_name(std::move(role_session_name_)) +{ +} + +Aws::Http::HeaderValueCollection AssumeRoleRequest::GetHeaders() const +{ + return {{Aws::Http::HeaderValuePair(Aws::Http::CONTENT_TYPE_HEADER, Aws::AMZN_XML_CONTENT_TYPE)}}; +} + +void AssumeRoleRequest::AddQueryStringParameters(Aws::Http::URI & uri) const +{ + uri.AddQueryStringParameter("RoleArn", role_arn); + uri.AddQueryStringParameter("RoleSessionName", role_session_name); +} + +AssumeRoleResult::AssumeRoleResult(Aws::AmazonWebServiceResult result) +{ + using namespace Aws::Utils::Xml; + const auto & xml_document = result.GetPayload(); + auto response_node = xml_document.GetRootElement(); + + if (response_node.IsNull()) + { + LOG_WARNING(log, "Response is empty"); + return; + } + + auto assume_role_result_node = response_node.FirstChild("AssumeRoleResult"); + if (assume_role_result_node.IsNull()) + { + LOG_WARNING(log, "AssumeRoleResult node is missing"); + return; + } + + auto credentials_node = assume_role_result_node.FirstChild("Credentials"); + if (credentials_node.IsNull()) + { + LOG_WARNING(log, "Credentials node is missing"); + return; + } + + const auto get_credential_value_from_node = [&](auto & dest, const std::string & node_name) + { + if (auto node = credentials_node.FirstChild(node_name); + !node.IsNull()) + dest = DecodeEscapedXmlText(node.GetText()); + else + LOG_WARNING(log, "{} node is missing", node_name); + }; + + get_credential_value_from_node(access_key_id, "AccessKeyId"); + get_credential_value_from_node(secret_access_key, "SecretAccessKey"); + get_credential_value_from_node(session_token, "SessionToken"); + get_credential_value_from_node(expiration, "Expiration"); +} + +AWSAssumeRoleClient::AWSAssumeRoleClient( + const std::shared_ptr & credentials_provider, + const Aws::Client::ClientConfiguration & client_configuration, + const std::string & sts_endpoint_override) + : Aws::Client::AWSXMLClient( + client_configuration, + std::make_shared( + credentials_provider, + "sts", + Aws::Region::ComputeSignerRegion(client_configuration.region), + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Always, + /*doubleEncodeValue*/ false), + std::make_shared()) +{ + if (!sts_endpoint_override.empty()) + { + endpoint.SetURL(sts_endpoint_override); + return; + } + + std::string endpoint_str; + if (client_configuration.scheme == Aws::Http::Scheme::HTTP) + endpoint_str = "http://sts."; + else + endpoint_str = "https://sts."; + + + if (client_configuration.region != "aws-global") + endpoint_str += client_configuration.region + "."; + + endpoint_str += "amazonaws.com?Action=AssumeRole&Version=2011-06-15"; + + endpoint.SetURL(endpoint_str); +} + +AssumeRoleOutcome AWSAssumeRoleClient::assumeRole(const AssumeRoleRequest & request) const +{ + return AssumeRoleOutcome(MakeRequest(request, endpoint, Aws::Http::HttpMethod::HTTP_POST)); +} + +AwsAuthSTSAssumeRoleCredentialsProvider::AwsAuthSTSAssumeRoleCredentialsProvider( + std::string role_arn_, + std::string session_name_, + uint64_t expiration_window_seconds_, + std::shared_ptr credentials_provider, + DB::S3::PocoHTTPClientConfiguration & client_configuration, + const std::string & sts_endpoint_override) + : role_arn(std::move(role_arn_)) + , session_name(session_name_.empty() ? "ClickHouseSession" : std::move(session_name_)) + , expiration_window_seconds(expiration_window_seconds_) + , client(std::make_shared(credentials_provider, client_configuration, sts_endpoint_override)) + , logger(getLogger("AwsAuthSTSAssumeRoleCredentialsProvider")) +{} + +Aws::Auth::AWSCredentials AwsAuthSTSAssumeRoleCredentialsProvider::GetAWSCredentials() +{ + refreshIfExpired(); + Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); + return credentials; +} + +void AwsAuthSTSAssumeRoleCredentialsProvider::Reload() +{ + LOG_INFO(logger, "Credentials are empty or expired, attempting to renew with AssumeRole"); + + AssumeRoleRequest request(role_arn, session_name); + auto outcome = client->assumeRole(request); + if (!outcome.IsSuccess()) + { + LOG_WARNING(logger, "Failed to get credentials using AssumeRule. Error: {}", outcome.GetError().GetMessage()); + return; + } + + const auto & result = outcome.GetResult(); + credentials.SetAWSAccessKeyId(result.getAccessKeyID()); + credentials.SetAWSSecretKey(result.getSecretAccessKey()); + credentials.SetSessionToken(result.getSessionToken()); + credentials.SetExpiration(result.getExpiration()); + + LOG_TRACE(logger, "Successfully retrieved credentials"); +} + +void AwsAuthSTSAssumeRoleCredentialsProvider::refreshIfExpired() +{ + Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); + if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds)) + return; + + guard.UpgradeToWriterLock(); + if (!areCredentialsEmptyOrExpired(credentials, expiration_window_seconds)) // double-checked lock to avoid refreshing twice + return; + + Reload(); +} + } } diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 69349bf5d510..0444446d0fdb 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -6,6 +6,16 @@ # include +# include +# include +# include +# include +# include +# include +# include +# include +# include + # include # include # include @@ -15,7 +25,6 @@ # include # include - namespace DB::S3 { @@ -114,7 +123,7 @@ class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWS public: explicit AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider( - DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_); + DB::S3::PocoHTTPClientConfiguration & aws_client_configuration, uint64_t expiration_window_seconds_, String role_arn_ = ""); Aws::Auth::AWSCredentials GetAWSCredentials() override; @@ -174,6 +183,10 @@ struct CredentialsConfiguration bool use_insecure_imds_request = false; uint64_t expiration_window_seconds = DEFAULT_EXPIRATION_WINDOW_SECONDS; bool no_sign_request = false; + std::string role_arn{}; + std::string role_session_name{}; + std::string sts_endpoint_override{}; + std::string kms_role_arn{}; }; class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain @@ -185,6 +198,87 @@ class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain CredentialsConfiguration credentials_configuration); }; +class AssumeRoleRequest : public Aws::AmazonSerializableWebServiceRequest +{ +public: + AssumeRoleRequest(std::string role_arn_, std::string role_session_name_); + + Aws::Http::HeaderValueCollection GetHeaders() const override; + + const char * GetServiceRequestName() const override { return "AssumeRole"; } + + Aws::String SerializePayload() const override { return {}; } + + void AddQueryStringParameters(Aws::Http::URI & uri) const override; + +private: + std::string role_arn; + std::string role_session_name; +}; +class AssumeRoleResult +{ +public: + /// NOLINTNEXTLINE + AssumeRoleResult(Aws::AmazonWebServiceResult result); + + const std::string & getAccessKeyID() const { return access_key_id; } + + const std::string & getSecretAccessKey() const { return secret_access_key; } + + const std::string & getSessionToken() const { return session_token; } + + const Aws::Utils::DateTime & getExpiration() const { return expiration; } + +private: + std::string access_key_id; + std::string secret_access_key; + std::string session_token; + Aws::Utils::DateTime expiration; + + LoggerPtr log{getLogger("AssumeRoleResult")}; +}; + +using AssumeRoleOutcome = Aws::Utils::Outcome; + +class AWSAssumeRoleClient : public Aws::Client::AWSXMLClient +{ +public: + AWSAssumeRoleClient( + const std::shared_ptr & credentials_provider, + const Aws::Client::ClientConfiguration & client_configuration, + const std::string & sts_endpoint_override = ""); + + AssumeRoleOutcome assumeRole(const AssumeRoleRequest & request) const; +private: + Aws::Endpoint::AWSEndpoint endpoint; +}; + +class AwsAuthSTSAssumeRoleCredentialsProvider : public Aws::Auth::AWSCredentialsProvider +{ +public: + AwsAuthSTSAssumeRoleCredentialsProvider( + std::string role_arn_, + std::string session_name_, + uint64_t expiration_window_seconds_, + std::shared_ptr credentials_provider, + DB::S3::PocoHTTPClientConfiguration & client_configuration, + const std::string & sts_endpoint_override = ""); + + Aws::Auth::AWSCredentials GetAWSCredentials() override; + +protected: + void Reload() override; +private: + void refreshIfExpired(); + + std::string role_arn; + std::string session_name; + uint64_t expiration_window_seconds; + std::shared_ptr client; + Aws::Auth::AWSCredentials credentials; + LoggerPtr logger; +}; + } #else diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index a47d577cb930..da0cb33fdf22 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -31,6 +31,9 @@ #include #include #include +#include +#include +#include #include @@ -107,6 +110,7 @@ namespace DB::ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TOO_MANY_REDIRECTS; extern const int DNS_ERROR; + extern const int AUTHENTICATION_FAILED; } namespace DB::S3 @@ -530,7 +534,7 @@ void PocoHTTPClient::makeRequestInternalImpl( /// Headers coming from SDK are lower-cased. for (const auto & [header_name, header_value] : request.GetHeaders()) - poco_request.set(header_name, header_value); + poco_request.set(boost::algorithm::to_lower_copy(header_name), header_value); for (const auto & [header_name, header_value] : extra_headers) { // AWS S3 canonical headers must include `Host`, `Content-Type` and any `x-amz-*`. @@ -707,6 +711,107 @@ void PocoHTTPClient::makeRequestInternalImpl( } } +namespace +{ + +String getStringOrDefault(const String & str, const String & default_str) +{ + return str.empty() ? default_str : str; +} + +constexpr auto DEFAULT_SERVICE_ACCOUNT = "default"; +constexpr auto DEFAULT_METADATA_SERVICE = "metadata.google.internal"; +constexpr auto DEFAULT_REQUEST_TOKEN_PATH = "computeMetadata/v1/instance/service-accounts"; + +} + +PocoHTTPClientGCPOAuth::PocoHTTPClientGCPOAuth(const PocoHTTPClientConfiguration & client_configuration) + : PocoHTTPClient(client_configuration) + , service_account(getStringOrDefault(client_configuration.service_account, DEFAULT_SERVICE_ACCOUNT)) + , metadata_service(getStringOrDefault(client_configuration.metadata_service, DEFAULT_METADATA_SERVICE)) + , request_token_path(getStringOrDefault(client_configuration.request_token_path, DEFAULT_REQUEST_TOKEN_PATH)) +{ +} + +void PocoHTTPClientGCPOAuth::makeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const +{ + { + std::lock_guard lock(mutex); + if (!bearer_token || std::chrono::system_clock::now() > bearer_token->is_valid_to) + bearer_token = requestBearerToken(); + + request.SetHeaderValue("Authorization", fmt::format("Bearer {}", bearer_token->token)); + } + + PocoHTTPClient::makeRequestInternal(request, response, readLimiter, writeLimiter); +} + +std::string PocoHTTPClientGCPOAuth::getBearerToken() const +{ + std::lock_guard lock(mutex); + if (!bearer_token || std::chrono::system_clock::now() > bearer_token->is_valid_to) + bearer_token = requestBearerToken(); + + return bearer_token->token; +} + +PocoHTTPClientGCPOAuth::BearerToken PocoHTTPClientGCPOAuth::requestBearerToken() const +{ + assert(!request_token_path.empty()); + assert(!metadata_service.empty()); + assert(!service_account.empty()); + + Poco::URI url; + url.setScheme("http"); + url.setHost(metadata_service); + url.setPath(fmt::format("{}/{}/token", request_token_path, service_account)); + + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.toString(), Poco::Net::HTTPRequest::HTTP_1_1); + request.add("metadata-flavor", "Google"); + + auto log = getLogger("PocoHTTPClientGCPOAuth"); + if (enable_s3_requests_logging) + LOG_TEST(log, "Make request to: {}", url.toString()); + + auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE; + auto session = makeHTTPSession(group, url, timeouts); + session->sendRequest(request); + + Poco::Net::HTTPResponse response; + auto & in = session->receiveResponse(response); + + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Failed to request bearer token: {}", response.getReason()); + + String token_json_raw; + Poco::StreamCopier::copyToString(in, token_json_raw); + + if (enable_s3_requests_logging) + LOG_TEST(log, "Received token in response: {}", token_json_raw); + + Poco::JSON::Parser parser; + auto object = parser.parse(token_json_raw).extract(); + + if (!object->has("access_token") || !object->has("expires_in") || !object->has("token_type")) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Unexpected structure of response. Response should have fields: 'access_token', 'expires_in', 'token_type'"); + + auto token_type = object->getValue("token_type"); + if (token_type != "Bearer") + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Unexpected structure of response. Expected Bearer token, got {}", token_type); + + return + { + .token = object->getValue("access_token"), + .is_valid_to = std::chrono::system_clock::now() + std::chrono::seconds(object->getValue("expires_in")) + }; +} + } #endif diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 3d3c17ee7ab3..1513ea177fc6 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -52,7 +52,12 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration bool for_disk_s3; ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; + HTTPHeaderEntries extra_headers; + String http_client; + String service_account; + String metadata_service; + String request_token_path; /// See PoolBase::BehaviourOnLimit bool s3_use_adaptive_timeouts = true; @@ -141,12 +146,6 @@ class PocoHTTPClient : public Aws::Http::HttpClient private: - void makeRequestInternal( - Aws::Http::HttpRequest & request, - std::shared_ptr & response, - Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, - Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; - enum class S3MetricType : uint8_t { Microseconds, @@ -176,17 +175,23 @@ class PocoHTTPClient : public Aws::Http::HttpClient EnumSize, }; + ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte) const; + void makeRequestInternalImpl( Aws::Http::HttpRequest & request, std::shared_ptr & response, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; - ConnectionTimeouts getTimeouts(const String & method, bool first_attempt, bool first_byte) const; - static S3LatencyType getFirstByteLatencyType(const String & sdk_attempt, const String & ch_attempt); protected: + virtual void makeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request); void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const; void addLatency(const Aws::Http::HttpRequest & request, S3LatencyType type, LatencyBuckets::Count amount = 1) const; @@ -215,6 +220,35 @@ class PocoHTTPClient : public Aws::Http::HttpClient const HTTPHeaderEntries extra_headers; }; +class PocoHTTPClientGCPOAuth : public PocoHTTPClient +{ +public: + explicit PocoHTTPClientGCPOAuth(const PocoHTTPClientConfiguration & client_configuration); + + std::string getBearerToken() const; +private: + void makeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; + + struct BearerToken + { + String token; + std::chrono::system_clock::time_point is_valid_to; + }; + + const String service_account; + const String metadata_service; + const String request_token_path; + + mutable std::mutex mutex; + mutable std::optional bearer_token TSA_GUARDED_BY(mutex); + + BearerToken requestBearerToken() const TSA_REQUIRES(mutex); +}; + } #endif diff --git a/src/IO/S3/PocoHTTPClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp index f4db72abb8e2..9a99ed9e99fd 100644 --- a/src/IO/S3/PocoHTTPClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -1,5 +1,7 @@ #include "config.h" +#include + #if USE_AWS_S3 #include "PocoHTTPClientFactory.h" @@ -16,7 +18,14 @@ std::shared_ptr PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & client_configuration) const { if (client_configuration.userAgent.starts_with("ClickHouse")) - return std::make_shared(static_cast(client_configuration)); + { + const auto & poco_client_configuration = static_cast(client_configuration); + if (Poco::toLower(poco_client_configuration.http_client) == "gcp_oauth") + return std::make_shared(poco_client_configuration); + + return std::make_shared(poco_client_configuration); + } + /// This client is created inside the AWS SDK with default settings to obtain ECS credentials from localhost. return std::make_shared(client_configuration); } diff --git a/src/IO/S3/tests/TestPocoHTTPServer.h b/src/IO/S3/tests/TestPocoHTTPServer.h index 35ceb5e16327..f81091b75ddc 100644 --- a/src/IO/S3/tests/TestPocoHTTPServer.h +++ b/src/IO/S3/tests/TestPocoHTTPServer.h @@ -13,8 +13,10 @@ #include #include #include +#include #include #include +#include class MockRequestHandler : public Poco::Net::HTTPRequestHandler { @@ -82,3 +84,116 @@ class TestPocoHTTPServer return last_request_header; } }; + +struct StsRequestInfo +{ + Poco::Net::MessageHeader headers; + Poco::URI::QueryParameters query_params; +}; + +class MockStsRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit MockStsRequestHandler(std::optional & last_request_info_, std::string role_access_key_, std::string role_secret_key_) + : last_request_info(last_request_info_) + , role_access_key(std::move(role_access_key_)) + , role_secret_key(std::move(role_secret_key_)) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override + { + last_request_info.emplace(); + last_request_info->headers = request; + + Poco::URI uri(request.getURI()); + last_request_info->query_params = uri.getQueryParameters(); + + response.setStatus(Poco::Net::HTTPResponse::HTTP_OK); + auto & out = response.send(); + + std::string result_xml = fmt::format(R"( + + + + {} + {} + session_token + + +)", role_access_key, role_secret_key); + out << result_xml; + out.flush(); + } +private: + std::optional & last_request_info; + std::string role_access_key; + std::string role_secret_key; +}; + +class StsHTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ + std::optional & last_request_info; + std::string role_access_key; + std::string role_secret_key; + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override + { + return new MockStsRequestHandler(last_request_info, role_access_key, role_secret_key); + } +public: + explicit StsHTTPRequestHandlerFactory(std::optional & last_request_info_, std::string role_access_key_, std::string role_secret_key_) + : last_request_info(last_request_info_) + , role_access_key(std::move(role_access_key_)) + , role_secret_key(std::move(role_secret_key_)) + { + } + + ~StsHTTPRequestHandlerFactory() override = default; +}; + +class TestPocoHTTPStsServer +{ + std::unique_ptr server_socket; + Poco::SharedPtr handler_factory; + Poco::AutoPtr server_params; + std::unique_ptr server; + // Stores the last request header handled. It's obviously not thread-safe to share the same + // reference across request handlers, but it's good enough for this the purposes of this test. + std::optional last_request_info; + +public: + TestPocoHTTPStsServer(std::string role_access_key, std::string role_secret_key): + server_socket(std::make_unique(0)), + handler_factory(new StsHTTPRequestHandlerFactory(last_request_info, std::move(role_access_key), std::move(role_secret_key))), + server_params(new Poco::Net::HTTPServerParams()), + server(std::make_unique(handler_factory, *server_socket, server_params)) + { + server->start(); + } + + std::string getUrl() + { + return "http://" + server_socket->address().toString(); + } + + void resetLastRequest() + { + last_request_info.reset(); + } + + bool hasLastRequest() const + { + return last_request_info.has_value(); + } + + const Poco::Net::MessageHeader & getLastRequestHeader() const + { + return last_request_info->headers; + } + + const auto & getLastQueryParams() const + { + return last_request_info->query_params; + } +}; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 35eb75297f2d..e8c493d55385 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -306,6 +306,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersWrite) "x-amz-server-side-encryption-context: arn:aws:s3:::bucket_ARN\n"); } + TEST(IOTestAwsS3Client, ChecksumHeaderIsPresentForS3Express) { /// See https://github.com/ClickHouse/ClickHouse/pull/19748 @@ -327,4 +328,189 @@ TEST(IOTestAwsS3Client, ChecksumHeaderIsPresentForS3Express) /*is_s3express_bucket=*/true); } +namespace +{ + +void validateCredential(const std::string_view credential_string, const std::string_view service_name, const std::string_view expected_access_key, const std::string_view expected_region) +{ + ASSERT_FALSE(credential_string.empty()); + if (!expected_access_key.empty()) + { + const auto expected_start = fmt::format("Credential={}", expected_access_key); + ASSERT_TRUE(credential_string.starts_with(expected_start)); + } + + if (!expected_region.empty()) + { + const auto expected_end = fmt::format("{}/{}/aws4_request,", expected_region, service_name); + ASSERT_TRUE(credential_string.ends_with(expected_end)); + + } +} + +void validateAssumeRoleQueryParams(const Poco::URI::QueryParameters query_params, const std::string_view expected_role_arn, const std::string_view expected_role_session_name) +{ + for (const auto & [param, value] : query_params) + { + if (param == "Action") + ASSERT_EQ(value, "AssumeRole"); + else if (param == "RoleArn") + ASSERT_EQ(value, expected_role_arn); + else if (param == "RoleSessionName") + ASSERT_EQ(value, expected_role_session_name); + } +} + +} + +TEST(IOTestAwsS3Client, AssumeRole) +{ + const auto get_credential_string = [&](const Poco::Net::MessageHeader & headers) -> std::string + { + for (const auto & [header_name, header_value] : headers) + { + if (header_name == "authorization") + { + std::vector parts; + boost::split(parts, header_value, [](char c){ return c == ' '; }); + for (const auto & part : parts) + { + if (part.starts_with("Credential=")) + { + return part; + } + } + } + } + + return ""; + }; + + TestPocoHTTPServer http; + + static constexpr std::string_view role_access_key = "role_access_key"; + static constexpr std::string_view role_secret_key = "role_secret_key"; + + TestPocoHTTPStsServer sts_http(std::string{role_access_key}, std::string{role_secret_key}); + + DB::RemoteHostFilter remote_host_filter; + unsigned int s3_max_redirects = 100; + unsigned int s3_retry_attempts = 0; + bool s3_slow_all_threads_after_network_error = true; + DB::S3::URI uri(http.getUrl() + "/IOTestAwsS3ClientAppendExtraHeaders/test.txt"); + String access_key_id = "ACCESS_KEY_ID"; + String secret_access_key = "SECRET_ACCESS_KEY"; + String region = "eu-west-1"; + String version_id; + UInt64 max_single_read_retries = 1; + bool enable_s3_requests_logging = false; + + DB::S3::PocoHTTPClientConfiguration client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + region, + remote_host_filter, + s3_max_redirects, + s3_retry_attempts, + s3_slow_all_threads_after_network_error, + enable_s3_requests_logging, + /* for_disk_s3 = */ false, + /* get_request_throttler = */ {}, + /* put_request_throttler = */ {}, + "http" + ); + + client_configuration.endpointOverride = uri.endpoint; + client_configuration.retryStrategy = std::make_shared(); + + DB::HTTPHeaderEntries headers; + bool use_environment_credentials = false; + bool use_insecure_imds_request = false; + + + const auto read_from_s3 = [&](const std::string & role_arn, const std::string & role_session_name) + { + DB::S3::ClientSettings client_settings{ + .use_virtual_addressing = uri.is_virtual_hosted_style, + .disable_checksum = false, + }; + + std::shared_ptr client = DB::S3::ClientFactory::instance().create( + client_configuration, + client_settings, + access_key_id, + secret_access_key, + "", + {}, + headers, + DB::S3::CredentialsConfiguration + { + .use_environment_credentials = use_environment_credentials, + .use_insecure_imds_request = use_insecure_imds_request, + .role_arn = role_arn, + .role_session_name = role_session_name, + .sts_endpoint_override = sts_http.getUrl() + } + ); + + ASSERT_TRUE(client); + + DB::ReadSettings read_settings; + DB::S3::S3RequestSettings request_settings; + request_settings[DB::S3RequestSetting::max_single_read_retries] = max_single_read_retries; + DB::ReadBufferFromS3 read_buffer( + client, + uri.bucket, + uri.key, + version_id, + request_settings, + read_settings + ); + + std::string content; + DB::readStringUntilEOF(content, read_buffer); + + }; + + { + SCOPED_TRACE("With role arn and role session name set"); + + std::string role_arn = "arn::role/my_role"; + std::string role_session_name = "session_name"; + + read_from_s3(role_arn, role_session_name); + + validateCredential(get_credential_string(http.getLastRequestHeader()), "s3", role_access_key, region); + + ASSERT_TRUE(sts_http.hasLastRequest()); + validateCredential(get_credential_string(sts_http.getLastRequestHeader()), "sts", access_key_id, region); + validateAssumeRoleQueryParams(sts_http.getLastQueryParams(), role_arn, role_session_name); + } + + { + SCOPED_TRACE("With no role arn set"); + + sts_http.resetLastRequest(); + + read_from_s3("", ""); + + validateCredential(get_credential_string(http.getLastRequestHeader()), "s3", access_key_id, region); + ASSERT_FALSE(sts_http.hasLastRequest()); + } + + { + SCOPED_TRACE("With role arn set and no role session name"); + + sts_http.resetLastRequest(); + + std::string role_arn = "arn::role/my_role"; + + read_from_s3(role_arn, ""); + + validateCredential(get_credential_string(http.getLastRequestHeader()), "s3", role_access_key, region); + + ASSERT_TRUE(sts_http.hasLastRequest()); + validateCredential(get_credential_string(sts_http.getLastRequestHeader()), "sts", access_key_id, region); + validateAssumeRoleQueryParams(sts_http.getLastQueryParams(), role_arn, "ClickHouseSession"); + } +} + #endif diff --git a/src/IO/S3AuthSettings.cpp b/src/IO/S3AuthSettings.cpp index b87280ef3209..1e100fa1e958 100644 --- a/src/IO/S3AuthSettings.cpp +++ b/src/IO/S3AuthSettings.cpp @@ -36,7 +36,13 @@ namespace ErrorCodes DECLARE(String, secret_access_key, "", "", 0) \ DECLARE(String, session_token, "", "", 0) \ DECLARE(String, region, "", "", 0) \ - DECLARE(String, server_side_encryption_customer_key_base64, "", "", 0) + DECLARE(String, server_side_encryption_customer_key_base64, "", "", 0) \ + DECLARE(String, role_arn, "", "", 0) \ + DECLARE(String, role_session_name, "", "", 0) \ + DECLARE(String, http_client, "", "", 0) \ + DECLARE(String, service_account, "", "", 0) \ + DECLARE(String, metadata_service, "", "", 0) \ + DECLARE(String, request_token_path, "", "", 0) \ #define CLIENT_SETTINGS_LIST(M, ALIAS) \ CLIENT_SETTINGS(M, ALIAS) \ diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index fd9f3b16ee6a..d98475ff7e78 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -387,7 +387,7 @@ void WriteBufferFromS3::allocateBuffer() return; } - memory = Memory(buffer_allocation_policy->getBufferSize()); + memory = Memory<>(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 70f318bed7d7..22224ad12595 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -49,6 +49,11 @@ #include #include +#if CLICKHOUSE_CLOUD +#include +#include +#endif + #include @@ -337,6 +342,9 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf member = createSystemLog(global_context, "system", #member, config, #member, descr); \ LIST_OF_ALL_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) + #if CLICKHOUSE_CLOUD + LIST_OF_CLOUD_SYSTEM_LOGS(CREATE_PUBLIC_MEMBERS) + #endif #undef CREATE_PUBLIC_MEMBERS /// NOLINTEND(bugprone-macro-parentheses) @@ -411,6 +419,9 @@ std::vector SystemLogs::getAllLogs() const std::vector result = { LIST_OF_ALL_SYSTEM_LOGS(GET_RAW_POINTERS) + #if CLICKHOUSE_CLOUD + LIST_OF_CLOUD_SYSTEM_LOGS(GET_RAW_POINTERS) + #endif }; #undef GET_RAW_POINTERS @@ -463,6 +474,9 @@ void SystemLogs::flush(bool should_prepare_tables_anyway, const Strings & names) std::unordered_map logs_map { LIST_OF_ALL_SYSTEM_LOGS(GET_MAP_VALUES) + #if CLICKHOUSE_CLOUD + LIST_OF_CLOUD_SYSTEM_LOGS(GET_MAP_VALUES) + #endif }; #undef GET_MAP_VALUES @@ -802,5 +816,8 @@ ASTPtr SystemLog::getCreateTableQuery() #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) +#if CLICKHOUSE_CLOUD +SYSTEM_LOG_ELEMENTS_CLOUD(INSTANTIATE_SYSTEM_LOG) +#endif } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 696927a61530..3b923bda10f2 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -1,5 +1,7 @@ #pragma once +#include "config.h" + #include #include #include @@ -33,6 +35,10 @@ M(BlobStorageLog, blob_storage_log, "Contains logging entries with information about various blob storage operations such as uploads and deletes.") \ M(QueryMetricLog, query_metric_log, "Contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk.") \ +#define LIST_OF_CLOUD_SYSTEM_LOGS(M) \ + M(DistributedCacheLog, distributed_cache_log, "Contains the history of all interactions with distributed cache.") \ + M(DistributedCacheServerLog, distributed_cache_server_log, "Contains the history of all interactions with distributed cache client.") \ + namespace DB { @@ -67,6 +73,9 @@ namespace DB class log_type; \ LIST_OF_ALL_SYSTEM_LOGS(FORWARD_DECLARATION) +#if CLICKHOUSE_CLOUD + LIST_OF_CLOUD_SYSTEM_LOGS(FORWARD_DECLARATION) +#endif #undef FORWARD_DECLARATION /// NOLINTEND(bugprone-macro-parentheses) @@ -88,6 +97,9 @@ class SystemLogs std::shared_ptr member; \ LIST_OF_ALL_SYSTEM_LOGS(DECLARE_PUBLIC_MEMBERS) + #if CLICKHOUSE_CLOUD + LIST_OF_CLOUD_SYSTEM_LOGS(DECLARE_PUBLIC_MEMBERS) + #endif #undef DECLARE_PUBLIC_MEMBERS private: diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c8e6765ee2f7..002a92e5b967 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1023,7 +1023,7 @@ void AvroRowInputFormat::readPrefix() file_reader_ptr->init(); } -bool AvroRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &ext) +bool AvroRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) { if (file_reader_ptr->hasMore()) { diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index 4aa9b177d8e2..13cec9b867d4 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -93,8 +93,10 @@ ObjectStoragePtr StorageAzureConfiguration::createObjectStorage(ContextPtr conte connection_params.auth_method, std::move(client), std::move(settings), + connection_params, connection_params.getContainer(), - connection_params.getConnectionURL()); + connection_params.getConnectionURL(), + /*common_key_prefix*/ ""); } static AzureBlobStorage::ConnectionParams getConnectionParams( diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 02371dec4211..c7bea03127fe 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -18,6 +19,7 @@ #include #include #include +#include #include #include @@ -48,6 +50,13 @@ namespace S3AuthSetting extern const S3AuthSettingsString secret_access_key; extern const S3AuthSettingsString session_token; extern const S3AuthSettingsBool use_environment_credentials; + + extern const S3AuthSettingsString role_arn; + extern const S3AuthSettingsString role_session_name; + extern const S3AuthSettingsString http_client; + extern const S3AuthSettingsString service_account; + extern const S3AuthSettingsString metadata_service; + extern const S3AuthSettingsString request_token_path; } namespace ErrorCodes @@ -57,11 +66,13 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static const std::unordered_set required_configuration_keys = { +static const std::unordered_set required_configuration_keys = +{ "url", }; -static const std::unordered_set optional_configuration_keys = { +static const std::unordered_set optional_configuration_keys = +{ "format", "compression", "compression_method", @@ -81,6 +92,13 @@ static const std::unordered_set optional_configuration_keys = "no_sign_request", "partition_strategy", "partition_columns_in_data_file" + /// Private configuration options + "role_arn", /// for extra_credentials + "role_session_name", /// for extra_credentials + "http_client", /// For GCP + "metadata_service", /// For GCP + "service_account", /// For GCP + "request_token_path", /// For GCP }; String StorageS3Configuration::getDataSourceDescription() const @@ -176,6 +194,13 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect s3_settings->auth_settings[S3AuthSetting::no_sign_request] = collection.getOrDefault("no_sign_request", false); s3_settings->auth_settings[S3AuthSetting::expiration_window_seconds] = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); s3_settings->auth_settings[S3AuthSetting::session_token] = collection.getOrDefault("session_token", ""); + s3_settings->auth_settings[S3AuthSetting::role_arn] = collection.getOrDefault("role_arn", ""); + s3_settings->auth_settings[S3AuthSetting::role_session_name] = collection.getOrDefault("role_session_name", ""); + + s3_settings->auth_settings[S3AuthSetting::http_client] = collection.getOrDefault("http_client", ""); + s3_settings->auth_settings[S3AuthSetting::service_account] = collection.getOrDefault("service_account", ""); + s3_settings->auth_settings[S3AuthSetting::metadata_service] = collection.getOrDefault("metadata_service", ""); + s3_settings->auth_settings[S3AuthSetting::request_token_path] = collection.getOrDefault("request_token_path", ""); if (collection.has("partition_strategy")) { @@ -206,8 +231,72 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect } +ASTPtr StorageS3Configuration::extractExtraCredentials(ASTs & args) +{ + for (size_t i = 0; i != args.size(); ++i) + { + const auto * ast_function = args[i]->as(); + if (ast_function && ast_function->name == "extra_credentials") + { + auto credentials = args[i]; + args.erase(args.begin() + i); + return credentials; + } + } + return nullptr; +} + +bool StorageS3Configuration::collectCredentials(ASTPtr maybe_credentials, S3::S3AuthSettings & auth_settings_, ContextPtr local_context) +{ + if (!maybe_credentials) + return false; + + const auto * credentials_ast_function = maybe_credentials->as(); + if (!credentials_ast_function || credentials_ast_function->name != "extra_credentials") + return false; + + const auto * credentials_function_args_expr = assert_cast(credentials_ast_function->arguments.get()); + auto credentials_function_args = credentials_function_args_expr->children; + + for (auto & credential_arg : credentials_function_args) + { + const auto * credential_ast = credential_arg->as(); + if (!credential_ast || credential_ast->name != "equals") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Credentials argument is incorrect"); + + auto * credential_args_expr = assert_cast(credential_ast->arguments.get()); + auto & credential_args = credential_args_expr->children; + if (credential_args.size() != 2) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Credentials argument is incorrect: expected 2 arguments, got {}", + credential_args.size()); + + credential_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(credential_args[0], local_context); + auto arg_name_value = credential_args[0]->as()->value; + if (arg_name_value.getType() != Field::Types::Which::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected string as credential name"); + auto arg_name = arg_name_value.safeGet(); + + credential_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(credential_args[1], local_context); + auto arg_value = credential_args[1]->as()->value; + if (arg_value.getType() != Field::Types::Which::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected string as credential value"); + else if (arg_name == "role_arn") + auth_settings_[S3AuthSetting::role_arn] = arg_value.safeGet(); + else if (arg_name == "role_session_name") + auth_settings_[S3AuthSetting::role_session_name] = arg_value.safeGet(); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid credential argument found: {}", arg_name); + } + + return true; +} + void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) { + auto extra_credentials = extractExtraCredentials(args); + size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context); if (count == 0 || count > getMaxNumberOfArguments(with_structure)) @@ -434,6 +523,8 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ s3_settings = std::make_unique(); s3_settings->loadFromConfigForObjectStorage(config, "s3", context->getSettingsRef(), url.uri.getScheme(), context->getSettingsRef()[Setting::s3_validate_request_settings]); + collectCredentials(extra_credentials, s3_settings->auth_settings, context); + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) { s3_settings->auth_settings.updateIfChanged(endpoint_settings->auth_settings); @@ -488,6 +579,9 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ static_configuration = !s3_settings->auth_settings[S3AuthSetting::access_key_id].value.empty() || s3_settings->auth_settings[S3AuthSetting::no_sign_request].changed; s3_settings->auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; + if (extra_credentials) + args.push_back(extra_credentials); + keys = {url.key}; } @@ -513,6 +607,8 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded( } else { + auto extra_credentials = extractExtraCredentials(args); + HTTPHeaderEntries tmp_headers; size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); @@ -679,6 +775,10 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded( if (with_structure && checkAndGetLiteralArgument(args[5], "format") == "auto") args[5] = structure_literal; } + + /// Add extracted extra credentials to the end of the args. + if (extra_credentials) + args.push_back(extra_credentials); } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 170e510aa49b..830d36494d5a 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -101,6 +101,9 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration ContextPtr context, bool with_structure) override; + static ASTPtr extractExtraCredentials(ASTs & args); + static bool collectCredentials(ASTPtr maybe_credentials, S3::S3AuthSettings & auth_settings_, ContextPtr local_context); + private: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b4ae724abd03..f6ce7f6bd6b2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -30,6 +30,11 @@ #include #include #include +#if ENABLE_DISTRIBUTED_CACHE +#include +#include +#include +#endif #include @@ -57,6 +62,8 @@ namespace Setting extern const SettingsString filesystem_cache_name; extern const SettingsUInt64 filesystem_cache_boundary_alignment; extern const SettingsBool use_iceberg_partition_pruning; + extern const SettingsBool cluster_function_process_archive_on_multiple_nodes; + extern const SettingsBool table_engine_read_through_distributed_cache; } namespace ErrorCodes @@ -594,19 +601,35 @@ std::unique_ptr StorageObjectStorageSource::createReadBu const auto & settings = context_->getSettingsRef(); const auto & effective_read_settings = read_settings.has_value() ? read_settings.value() : context_->getReadSettings(); - const auto filesystem_cache_name = settings[Setting::filesystem_cache_name].value; - bool use_cache = effective_read_settings.enable_filesystem_cache - && !filesystem_cache_name.empty() - && (object_storage->getType() == ObjectStorageType::Azure - || object_storage->getType() == ObjectStorageType::S3); + bool use_distributed_cache = false; +#if ENABLE_DISTRIBUTED_CACHE + ObjectStorageConnectionInfoPtr connection_info; + if (settings[Setting::table_engine_read_through_distributed_cache] + && DistributedCache::Registry::instance().isReady( + effective_read_settings.distributed_cache_settings.read_only_from_current_az)) + { + connection_info = object_storage->getConnectionInfo(); + if (connection_info) + use_distributed_cache = true; + } +#endif - if (!object_info.metadata) + bool use_filesystem_cache = false; + std::string filesystem_cache_name; + if (!use_distributed_cache) { - if (!use_cache) - return object_storage->readObject(StoredObject(object_info.getPath()), effective_read_settings); + filesystem_cache_name = settings[Setting::filesystem_cache_name].value; + use_filesystem_cache = effective_read_settings.enable_filesystem_cache + && !filesystem_cache_name.empty() + && (object_storage->getType() == ObjectStorageType::Azure + || object_storage->getType() == ObjectStorageType::S3); + } + /// We need object metadata for two cases: + /// 1. object size suggests whether we need to use prefetch + /// 2. object etag suggests a cache key in case we use filesystem cache + if (!object_info.metadata) object_info.metadata = object_storage->getObjectMetadata(object_info.getPath()); - } const auto & object_size = object_info.metadata->size_bytes; @@ -624,15 +647,46 @@ std::unique_ptr StorageObjectStorageSource::createReadBu && modified_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool && modified_read_settings.remote_fs_prefetch; - /// FIXME: Use async buffer if use_cache, - /// because CachedOnDiskReadBufferFromFile does not work as an independent buffer currently. - const bool use_async_buffer = use_prefetch || use_cache; + bool use_async_buffer = false; + ReadSettings nested_buffer_read_settings = modified_read_settings; + if (use_prefetch || use_filesystem_cache || use_distributed_cache) + { + nested_buffer_read_settings.remote_read_buffer_use_external_buffer = true; + + /// FIXME: Use async buffer if use_cache, + /// because CachedOnDiskReadBufferFromFile does not work as an independent buffer currently. + use_async_buffer = true; + } if (use_async_buffer) modified_read_settings.remote_read_buffer_use_external_buffer = true; std::unique_ptr impl; - if (use_cache) +#if ENABLE_DISTRIBUTED_CACHE + if (use_distributed_cache) + { + const std::string path = object_info.getPath(); + StoredObject object(path, "", object_size); + auto read_buffer_creator = [object, nested_buffer_read_settings, object_storage]() + { + return object_storage->readObject(object, nested_buffer_read_settings); + }; + + impl = std::make_unique( + path, + StoredObjects({object}), + effective_read_settings, + connection_info, + ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context_->getSettingsRef()), + read_buffer_creator, + /*use_external_buffer*/use_async_buffer, + context_->getDistributedCacheLog(), + /* include_credentials_in_cache_key */true); + } + else if (use_filesystem_cache) +#else + if (use_filesystem_cache) +#endif { chassert(object_info.metadata.has_value()); if (object_info.metadata->etag.empty()) @@ -687,10 +741,13 @@ std::unique_ptr StorageObjectStorageSource::createReadBu LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - bool prefer_bigger_buffer_size = effective_read_settings.filesystem_cache_prefer_bigger_buffer_size && impl->isCached(); + bool prefer_bigger_buffer_size = effective_read_settings.filesystem_cache_prefer_bigger_buffer_size + && impl->isCached(); + size_t buffer_size = prefer_bigger_buffer_size ? std::max(effective_read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) : effective_read_settings.remote_fs_buffer_size; + if (object_size) buffer_size = std::min(object_size, buffer_size); From 1d1eca572349356ab24bb1382484ab5af4e4496d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sun, 31 Aug 2025 23:20:28 +0200 Subject: [PATCH 7/9] try to fix build --- .../AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp index 8473de860e0d..0769eeeb13db 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorageConnectionInfo.cpp @@ -121,7 +121,7 @@ std::shared_ptr AzureObjectStorageConnectionI AzureBlobStorage::processURL(info.endpoint, info.container_name, params.endpoint, params.auth_method); auto settings = AzureBlobStorage::getRequestSettings(Context::getGlobalContextInstance()->getSettingsRef()); - params.client_options = AzureBlobStorage::getClientOptions(Context::getGlobalContextInstance(), *settings, /*for_disk*/true); + params.client_options = AzureBlobStorage::getClientOptions(*settings, /*for_disk*/true); params.auth_method = std::make_shared(info.session_token, info.expires_on); return params.createForContainer(); From 30799ca40401fb830a05eac758f11805c0b0ae10 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Jul 2025 14:55:37 +0000 Subject: [PATCH 8/9] Merge pull request #83379 from ClickHouse/fix-no-sign-config Fix no_sign_request config for S3 --- src/Storages/ObjectStorage/S3/Configuration.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index c7bea03127fe..32f8b8906f6a 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -577,7 +577,6 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ s3_settings->auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; static_configuration = !s3_settings->auth_settings[S3AuthSetting::access_key_id].value.empty() || s3_settings->auth_settings[S3AuthSetting::no_sign_request].changed; - s3_settings->auth_settings[S3AuthSetting::no_sign_request] = no_sign_request; if (extra_credentials) args.push_back(extra_credentials); From d242870e1a6d21c0b6e0098d8d730159346b8553 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 3 Sep 2025 00:42:44 +0200 Subject: [PATCH 9/9] fix typo --- src/Storages/ObjectStorage/S3/Configuration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 32f8b8906f6a..4fd306707581 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -91,7 +91,7 @@ static const std::unordered_set optional_configuration_keys = "expiration_window_seconds", "no_sign_request", "partition_strategy", - "partition_columns_in_data_file" + "partition_columns_in_data_file", /// Private configuration options "role_arn", /// for extra_credentials "role_session_name", /// for extra_credentials