From 82224469e5d215052e4a76a4063ef9c966c59095 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 2 May 2024 10:33:23 -0400 Subject: [PATCH 1/3] config: switch `cloud_storage_url_style` to `std::optional` To allow for self-configuration, the `cloud_storage_url_style` is now an `std::optional`, and can be left unspecified in the `redpanda` cluster config. If it is not set, it will default to `virtual_host`. --- src/v/cloud_storage/types.cc | 4 +++- src/v/cloud_storage_clients/configuration.cc | 5 +++++ src/v/cloud_storage_clients/configuration.h | 1 + .../test_client/s3_test_client_main.cc | 22 ++++++++++++------- src/v/config/configuration.cc | 20 ++++++++++++----- src/v/config/configuration.h | 3 ++- 6 files changed, 39 insertions(+), 16 deletions(-) diff --git a/src/v/cloud_storage/types.cc b/src/v/cloud_storage/types.cc index 7bcb4ded66eb..026cb1cd9124 100644 --- a/src/v/cloud_storage/types.cc +++ b/src/v/cloud_storage/types.cc @@ -28,7 +28,6 @@ cloud_storage_clients::default_overrides get_default_overrides() { optep.has_value()) { overrides.endpoint = cloud_storage_clients::endpoint_url(*optep); } - overrides.url_style = config::shard_local_cfg().cloud_storage_url_style(); overrides.disable_tls = config::shard_local_cfg().cloud_storage_disable_tls; if (auto cert = config::shard_local_cfg().cloud_storage_trust_file.value(); cert.has_value()) { @@ -412,6 +411,8 @@ ss::future configuration::get_s3_config() { auto region = cloud_roles::aws_region_name(get_value_or_throw( config::shard_local_cfg().cloud_storage_region, "cloud_storage_region")); + auto url_style = config::shard_local_cfg().cloud_storage_url_style.value(); + auto disable_metrics = net::metrics_disabled( config::shard_local_cfg().disable_metrics()); auto disable_public_metrics = net::public_metrics_disabled( @@ -422,6 +423,7 @@ ss::future configuration::get_s3_config() { access_key, secret_key, region, + url_style, get_default_overrides(), disable_metrics, disable_public_metrics); diff --git a/src/v/cloud_storage_clients/configuration.cc b/src/v/cloud_storage_clients/configuration.cc index 26dbd6c0f7c9..be1fc115e957 100644 --- a/src/v/cloud_storage_clients/configuration.cc +++ b/src/v/cloud_storage_clients/configuration.cc @@ -74,6 +74,7 @@ ss::future s3_configuration::make_configuration( const std::optional& pkey, const std::optional& skey, const cloud_roles::aws_region_name& region, + const std::optional& url_style, const default_overrides& overrides, net::metrics_disabled disable_metrics, net::public_metrics_disabled disable_public_metrics) { @@ -93,6 +94,10 @@ ss::future s3_configuration::make_configuration( client_cfg.uri = access_point_uri(endpoint_uri); client_cfg.url_style = overrides.url_style; + if (url_style.has_value()) { + client_cfg.url_style = url_style.value(); + } + if (overrides.disable_tls == false) { client_cfg.credentials = co_await build_tls_credentials( "s3", overrides.trust_file, s3_log); diff --git a/src/v/cloud_storage_clients/configuration.h b/src/v/cloud_storage_clients/configuration.h index 6ca9437b0cc7..33aabd2118f7 100644 --- a/src/v/cloud_storage_clients/configuration.h +++ b/src/v/cloud_storage_clients/configuration.h @@ -67,6 +67,7 @@ struct s3_configuration : common_configuration { const std::optional& pkey, const std::optional& skey, const cloud_roles::aws_region_name& region, + const std::optional& url_style, const default_overrides& overrides = {}, net::metrics_disabled disable_metrics = net::metrics_disabled::yes, net::public_metrics_disabled disable_public_metrics diff --git a/src/v/cloud_storage_clients/test_client/s3_test_client_main.cc b/src/v/cloud_storage_clients/test_client/s3_test_client_main.cc index b9be864a9244..796b29d3c011 100644 --- a/src/v/cloud_storage_clients/test_client/s3_test_client_main.cc +++ b/src/v/cloud_storage_clients/test_client/s3_test_client_main.cc @@ -79,7 +79,7 @@ void cli_opts(boost::program_options::options_description_easy_init opt) { opt( "url_style", - po::value()->default_value("virtual_host"), + po::value()->default_value(""), "aws addressing style"); opt( @@ -150,11 +150,24 @@ test_conf cfg_from(boost::program_options::variables_map& m) { auto secret_key = cloud_roles::private_key_str( m["secretkey"].as()); auto region = cloud_roles::aws_region_name(m["region"].as()); + auto url_style = + [&]() -> std::optional { + const auto url_style_str = m["url_style"].as(); + if (url_style_str == "virtual_host") { + return cloud_storage_clients::s3_url_style::virtual_host; + } else if (url_style_str == "path") { + return cloud_storage_clients::s3_url_style::path; + } else { + return std::nullopt; + } + }(); + cloud_storage_clients::s3_configuration client_cfg = cloud_storage_clients::s3_configuration::make_configuration( access_key, secret_key, region, + url_style, cloud_storage_clients::default_overrides{ .endpoint = [&]() -> std::optional { @@ -170,13 +183,6 @@ test_conf cfg_from(boost::program_options::variables_map& m) { } return std::nullopt; }(), - .url_style = [&]() -> cloud_storage_clients::s3_url_style { - if (m["url_style"].as() == "virtual_host") { - return cloud_storage_clients::s3_url_style::virtual_host; - } else { - return cloud_storage_clients::s3_url_style::path; - } - }(), .disable_tls = m.contains("disable-tls") > 0, }) .get0(); diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 4f9b48b61ee9..39809885e551 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -1685,15 +1685,23 @@ configuration::configuration() , cloud_storage_url_style( *this, "cloud_storage_url_style", - "The addressing style to use for S3 requests.", + "Specifies the addressing style to use for Amazon S3 requests. This " + "configuration determines how S3 bucket URLs are formatted. You can " + "choose between: `virtual_host`, (e.g. " + "`.s3.amazonaws.com`), `path`, (e.g. " + "`s3.amazonaws.com/`), and `null`. Path style is supported " + "for backward compatibility with legacy systems. When this property is " + "not set (`null`), the client tries to use `virtual_host` addressing. If " + "the initial request fails, the client automatically tries the `path` " + "style. If neither addressing style works, Redpanda terminates the " + "startup, requiring manual configuration to proceed.", {.needs_restart = needs_restart::yes, .example = "virtual_host", .visibility = visibility::user}, - cloud_storage_clients::s3_url_style::virtual_host, - { - cloud_storage_clients::s3_url_style::virtual_host, - cloud_storage_clients::s3_url_style::path, - }) + std::nullopt, + {cloud_storage_clients::s3_url_style::virtual_host, + cloud_storage_clients::s3_url_style::path, + std::nullopt}) , cloud_storage_credentials_source( *this, "cloud_storage_credentials_source", diff --git a/src/v/config/configuration.h b/src/v/config/configuration.h index bb74a0c29d3b..f9ffa525bcc5 100644 --- a/src/v/config/configuration.h +++ b/src/v/config/configuration.h @@ -309,7 +309,8 @@ struct configuration final : public config_store { property> cloud_storage_region; property> cloud_storage_bucket; property> cloud_storage_api_endpoint; - enum_property cloud_storage_url_style; + enum_property> + cloud_storage_url_style; enum_property cloud_storage_credentials_source; property> From 05b1327ad1f129dd143540eeaa4d22cb4a0c7d79 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 2 May 2024 10:20:52 -0400 Subject: [PATCH 2/3] cloud_storage: use self configuration for url style If the user does not specify `cloud_storage_url_style` in their cluster config file (i.e leaves as `nit`), the `s3_client` will set its url addressing style through self configuration. The default behavior is to attempt `virtual_host` addressing first, falling back to `path` style requests if `virtual_host` does not work. If both are attempted and neither work, the `redpanda` start-up is terminated. --- src/v/cloud_storage_clients/client_pool.cc | 2 +- src/v/cloud_storage_clients/configuration.cc | 19 ++-- src/v/cloud_storage_clients/configuration.h | 7 +- src/v/cloud_storage_clients/s3_client.cc | 101 ++++++++++++++++++- src/v/cloud_storage_clients/s3_client.h | 10 ++ src/v/redpanda/tests/fixture.h | 2 + 6 files changed, 128 insertions(+), 13 deletions(-) diff --git a/src/v/cloud_storage_clients/client_pool.cc b/src/v/cloud_storage_clients/client_pool.cc index 6025f60fa731..2dcd5fa66934 100644 --- a/src/v/cloud_storage_clients/client_pool.cc +++ b/src/v/cloud_storage_clients/client_pool.cc @@ -91,7 +91,7 @@ ss::future<> client_pool::client_self_configure( self_config_output = *result; vlog( pool_log.info, - "Client self configuration completed with result {} ", + "Client self configuration completed with result {}", *self_config_output); } diff --git a/src/v/cloud_storage_clients/configuration.cc b/src/v/cloud_storage_clients/configuration.cc index be1fc115e957..536e04b37098 100644 --- a/src/v/cloud_storage_clients/configuration.cc +++ b/src/v/cloud_storage_clients/configuration.cc @@ -92,10 +92,13 @@ ss::future s3_configuration::make_configuration( client_cfg.secret_key = skey; client_cfg.region = region; client_cfg.uri = access_point_uri(endpoint_uri); - client_cfg.url_style = overrides.url_style; if (url_style.has_value()) { client_cfg.url_style = url_style.value(); + } else { + // If the url style is not specified, it will be determined with + // self configuration. + client_cfg.requires_self_configuration = true; } if (overrides.disable_tls == false) { @@ -124,8 +127,8 @@ std::ostream& operator<<(std::ostream& o, const s3_configuration& c) { o << "{access_key:" << c.access_key.value_or(cloud_roles::public_key_str{""}) << ",region:" << c.region() << ",secret_key:****" - << ",access_point_uri:" << c.uri() << ",server_addr:" << c.server_addr - << ",max_idle_time:" + << ",url_style:" << c.url_style << ",access_point_uri:" << c.uri() + << ",server_addr:" << c.server_addr << ",max_idle_time:" << std::chrono::duration_cast(c.max_idle_time) .count() << "}"; @@ -216,7 +219,10 @@ void apply_self_configuration_result( "result {}", cfg, res); - // No self configuration for S3 at this point + + cfg.url_style + = std::get(res).url_style; + } else if constexpr (std::is_same_v) { vassert( std::holds_alternative(res), @@ -251,8 +257,9 @@ operator<<(std::ostream& o, const abs_self_configuration_result& r) { return o; } -std::ostream& operator<<(std::ostream& o, const s3_self_configuration_result&) { - o << "{}"; +std::ostream& +operator<<(std::ostream& o, const s3_self_configuration_result& r) { + o << "{s3_url_style: " << r.url_style << "}"; return o; } diff --git a/src/v/cloud_storage_clients/configuration.h b/src/v/cloud_storage_clients/configuration.h index 33aabd2118f7..487b62ef2a9b 100644 --- a/src/v/cloud_storage_clients/configuration.h +++ b/src/v/cloud_storage_clients/configuration.h @@ -26,7 +26,6 @@ struct default_overrides { std::optional port = std::nullopt; std::optional trust_file = std::nullopt; std::optional max_idle_time = std::nullopt; - s3_url_style url_style = s3_url_style::virtual_host; bool disable_tls = false; }; @@ -50,7 +49,7 @@ struct s3_configuration : common_configuration { /// AWS secret key, optional if configuration uses temporary credentials std::optional secret_key; /// AWS URL style, either virtual-hosted-style or path-style. - s3_url_style url_style; + s3_url_style url_style = s3_url_style::virtual_host; /// \brief opinionated configuraiton initialization /// Generates uri field from region, initializes credentials for the @@ -111,7 +110,9 @@ struct abs_self_configuration_result { bool is_hns_enabled; }; -struct s3_self_configuration_result {}; +struct s3_self_configuration_result { + s3_url_style url_style; +}; using client_self_configuration_output = std::variant; diff --git a/src/v/cloud_storage_clients/s3_client.cc b/src/v/cloud_storage_clients/s3_client.cc index 420bf532046a..0ea3d8e74f1e 100644 --- a/src/v/cloud_storage_clients/s3_client.cc +++ b/src/v/cloud_storage_clients/s3_client.cc @@ -12,10 +12,12 @@ #include "base/vlog.h" #include "bytes/bytes.h" +#include "bytes/iostream.h" #include "cloud_storage_clients/logger.h" #include "cloud_storage_clients/s3_error.h" #include "cloud_storage_clients/util.h" #include "cloud_storage_clients/xml_sax_parser.h" +#include "config/configuration.h" #include "hashing/secure.h" #include "http/client.h" #include "net/types.h" @@ -27,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -550,11 +553,103 @@ s3_client::s3_client( ss::future> s3_client::self_configure() { + // Test virtual host style addressing, fall back to path if necessary. + // If any configuration options prevent testing, addressing style will + // default to virtual_host. + // If both addressing methods fail, return an error. + auto result = s3_self_configuration_result{ + .url_style = s3_url_style::virtual_host}; + const auto remote_read + = config::shard_local_cfg().cloud_storage_enable_remote_read(); + const auto remote_write + = config::shard_local_cfg().cloud_storage_enable_remote_write(); + if (!remote_read && !remote_write) { + vlog( + s3_log.warn, + "Could not self-configure S3 Client, {}, {} are not enabled. " + "Defaulting to {} ", + config::shard_local_cfg().cloud_storage_enable_remote_read.name(), + config::shard_local_cfg().cloud_storage_enable_remote_write.name(), + result.url_style); + co_return result; + } + + const auto& bucket_config = config::shard_local_cfg().cloud_storage_bucket; + + if (!bucket_config.value().has_value()) { + vlog( + s3_log.warn, + "Could not self-configure S3 Client, {} is not set. Defaulting to {}", + bucket_config.name(), + result.url_style); + co_return result; + } + + const auto bucket = cloud_storage_clients::bucket_name{ + bucket_config.value().value()}; + + // Test virtual_host style. + vassert( + _requestor._ap_style == s3_url_style::virtual_host, + "_ap_style should be virtual host by default before self configuration " + "begins"); + if (co_await self_configure_test(bucket, remote_read, remote_write)) { + // Virtual-host style request succeeded. + co_return result; + } + + // Test path style. + _requestor._ap_style = s3_url_style::path; + result.url_style = _requestor._ap_style; + if (co_await self_configure_test(bucket, remote_read, remote_write)) { + // Path style request succeeded. + co_return result; + } + + // Both addressing styles failed. vlog( s3_log.error, - "Call to self_configure was made, but the S3 client doesn't require self " - "configuration"); - co_return s3_self_configuration_result{}; + "Couldn't reach S3 storage with either path style or virtual_host style " + "requests.", + bucket_config.name()); + co_return error_outcome::fail; +} + +ss::future s3_client::self_configure_test( + const bucket_name& bucket, bool remote_read, bool remote_write) { + if (remote_read) { + // Verify with a list objects request. + auto list_objects_result = co_await list_objects( + bucket, std::nullopt, std::nullopt, 1); + co_return list_objects_result; + } else { + vassert(remote_write, "Remote write is not enabled"); + // Verify with a upload and delete request. + auto now = ss::lowres_clock::now(); + const ss::sstring key_and_payload = fmt::format( + "S3ClientSelfConfigurationKey.{}", now.time_since_epoch().count()); + iobuf payload; + payload.append(key_and_payload.data(), key_and_payload.size()); + auto payload_stream = make_iobuf_input_stream(std::move(payload)); + const ss::lowres_clock::duration timeout = {std::chrono::seconds(30)}; + + auto upload_object_result = co_await put_object( + bucket, + object_key{key_and_payload}, + key_and_payload.size(), + std::move(payload_stream), + timeout); + + if (!upload_object_result) { + // Upload failed, return early. + co_return upload_object_result; + } + + // Clean up uploaded object. + auto delete_object_result = co_await delete_object( + bucket, object_key{key_and_payload}, timeout); + co_return (upload_object_result && delete_object_result); + } } ss::future<> s3_client::stop() { return _client.stop(); } diff --git a/src/v/cloud_storage_clients/s3_client.h b/src/v/cloud_storage_clients/s3_client.h index 2866db6bc681..a536a37f7a51 100644 --- a/src/v/cloud_storage_clients/s3_client.h +++ b/src/v/cloud_storage_clients/s3_client.h @@ -107,6 +107,7 @@ class request_creator { std::optional delimiter = std::nullopt); private: + friend class s3_client; std::string make_host(const bucket_name& name) const; std::string @@ -245,6 +246,15 @@ class s3_client : public client { const bucket_name& bucket, const object_key& key); + // Performs testing as part of the self-configuration step. + // If remote_read is true, the test will use list_objects(). + // If remote_read is false, the test will instead use put_object() and + // delete_object(). If both remote_read and remote_write are false, the test + // will fail a vassert() call. + // Returns true if the test ran was succesfully and false otherwise. + ss::future self_configure_test( + const bucket_name& bucket, bool remote_read, bool remote_write); + private: request_creator _requestor; http::client _client; diff --git a/src/v/redpanda/tests/fixture.h b/src/v/redpanda/tests/fixture.h index 400d8aaa00e8..7193bf20ea3c 100644 --- a/src/v/redpanda/tests/fixture.h +++ b/src/v/redpanda/tests/fixture.h @@ -377,6 +377,8 @@ class redpanda_thread_fixture { .set_value(std::make_optional((*s3_config->access_key)())); config.get("cloud_storage_secret_key") .set_value(std::make_optional((*s3_config->secret_key)())); + config.get("cloud_storage_url_style") + .set_value(std::make_optional((s3_config->url_style))); config.get("cloud_storage_api_endpoint") .set_value(std::make_optional(s3_config->server_addr.host())); config.get("cloud_storage_api_endpoint_port") From f55d01ea5b8d799e7dcb39165823ba52b5b0c706 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 14 May 2024 10:55:45 -0400 Subject: [PATCH 3/3] rptest: add cluster_self_config_test --- .../rptest/tests/cluster_self_config_test.py | 94 +++++++++++++++++++ 1 file changed, 94 insertions(+) create mode 100644 tests/rptest/tests/cluster_self_config_test.py diff --git a/tests/rptest/tests/cluster_self_config_test.py b/tests/rptest/tests/cluster_self_config_test.py new file mode 100644 index 000000000000..722f070ad429 --- /dev/null +++ b/tests/rptest/tests/cluster_self_config_test.py @@ -0,0 +1,94 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 +import re + +from ducktape.mark import parametrize, matrix + +from rptest.services.admin import Admin +from rptest.services.cluster import cluster +from rptest.services.redpanda import CloudStorageType, SISettings +from rptest.tests.redpanda_test import RedpandaTest +from rptest.services.utils import LogSearchLocal + + +class ClusterSelfConfigTest(RedpandaTest): + def __init__(self, ctx): + si_settings = SISettings( + ctx, + #Force self configuration through setting cloud_storage_url_style to None. + cloud_storage_url_style=None, + cloud_storage_enable_remote_read=ctx. + injected_args["cloud_storage_enable_remote_read"], + cloud_storage_enable_remote_write=ctx. + injected_args["cloud_storage_enable_remote_write"]) + + super().__init__(ctx, si_settings=si_settings) + + self.log_searcher = LogSearchLocal(ctx, [], self.redpanda.logger, + self.redpanda.STDOUT_STDERR_CAPTURE) + self.admin = Admin(self.redpanda) + + @cluster(num_nodes=1) + @matrix(cloud_storage_enable_remote_read=[True, False], + cloud_storage_enable_remote_write=[True, False]) + def test_s3_self_config(self, cloud_storage_enable_remote_read, + cloud_storage_enable_remote_write): + """ + Verify that cloud_storage_url_style self configuration occurs for the s3_client + when it is not specified. There aren't any endpoints for testing this, so + it will be manually checked for from the logs. + """ + + config = self.admin.get_cluster_config() + + # Even after self-configuring, the cloud_storage_url_style setting will + # still be left unset at the cluster config level. + assert config['cloud_storage_url_style'] is None + + def str_in_logs(node, s): + return any(s in log.strip() + for log in self.log_searcher._capture_log(node, s)) + + def self_config_start_in_logs(node): + client_self_configuration_start_string = 'Client requires self configuration step' + return str_in_logs(node, client_self_configuration_start_string) + + def self_config_default_in_logs(node): + client_self_configuration_default_string = 'Could not self-configure S3 Client' + return str_in_logs(node, client_self_configuration_default_string) + + def self_config_result_from_logs(node): + client_self_configuration_complete_string = 'Client self configuration completed with result' + for log in self.log_searcher._capture_log( + node, client_self_configuration_complete_string): + m = re.search( + client_self_configuration_complete_string + r' (\{.*\})', + log.strip()) + if m: + return m.group(1) + return None + + for node in self.redpanda.nodes: + #Assert that self configuration started. + assert self_config_start_in_logs(node) + + #If neither remote_read or remote_write are enabled, check for the "defaulting" output + if not cloud_storage_enable_remote_read and not cloud_storage_enable_remote_write: + assert self_config_default_in_logs(node) + + #Assert that self configuration returned a result. + self_config_result = self_config_result_from_logs(node) + + #Currently, virtual_host will succeed in all cases with MinIO. + self_config_expected_results = [ + '{s3_self_configuration_result: {s3_url_style: virtual_host}}', + '{s3_self_configuration_result: {s3_url_style: path}}' + ] + + assert self_config_result and self_config_result in self_config_expected_results