Skip to content

Commit

Permalink
Merge pull request #18107 from WillemKauf/path_style_url_auto_config
Browse files Browse the repository at this point in the history
CORE-2417: cloud_storage_clients: self configure `s3_url_style`
  • Loading branch information
piyushredpanda authored May 16, 2024
2 parents 916ec2d + f55d01e commit 40c7cc3
Show file tree
Hide file tree
Showing 11 changed files with 261 additions and 29 deletions.
4 changes: 3 additions & 1 deletion src/v/cloud_storage/types.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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()) {
Expand Down Expand Up @@ -412,6 +411,8 @@ ss::future<configuration> 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(
Expand All @@ -422,6 +423,7 @@ ss::future<configuration> configuration::get_s3_config() {
access_key,
secret_key,
region,
url_style,
get_default_overrides(),
disable_metrics,
disable_public_metrics);
Expand Down
2 changes: 1 addition & 1 deletion src/v/cloud_storage_clients/client_pool.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}

Expand Down
24 changes: 18 additions & 6 deletions src/v/cloud_storage_clients/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ ss::future<s3_configuration> s3_configuration::make_configuration(
const std::optional<cloud_roles::public_key_str>& pkey,
const std::optional<cloud_roles::private_key_str>& skey,
const cloud_roles::aws_region_name& region,
const std::optional<cloud_storage_clients::s3_url_style>& url_style,
const default_overrides& overrides,
net::metrics_disabled disable_metrics,
net::public_metrics_disabled disable_public_metrics) {
Expand All @@ -91,7 +92,14 @@ ss::future<s3_configuration> 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) {
client_cfg.credentials = co_await build_tls_credentials(
Expand Down Expand Up @@ -119,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<std::chrono::milliseconds>(c.max_idle_time)
.count()
<< "}";
Expand Down Expand Up @@ -211,7 +219,10 @@ void apply_self_configuration_result(
"result {}",
cfg,
res);
// No self configuration for S3 at this point

cfg.url_style
= std::get<s3_self_configuration_result>(res).url_style;

} else if constexpr (std::is_same_v<abs_configuration, cfg_type>) {
vassert(
std::holds_alternative<abs_self_configuration_result>(res),
Expand Down Expand Up @@ -246,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;
}

Expand Down
8 changes: 5 additions & 3 deletions src/v/cloud_storage_clients/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ struct default_overrides {
std::optional<uint16_t> port = std::nullopt;
std::optional<ca_trust_file> trust_file = std::nullopt;
std::optional<ss::lowres_clock::duration> max_idle_time = std::nullopt;
s3_url_style url_style = s3_url_style::virtual_host;
bool disable_tls = false;
};

Expand All @@ -50,7 +49,7 @@ struct s3_configuration : common_configuration {
/// AWS secret key, optional if configuration uses temporary credentials
std::optional<cloud_roles::private_key_str> 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
Expand All @@ -67,6 +66,7 @@ struct s3_configuration : common_configuration {
const std::optional<cloud_roles::public_key_str>& pkey,
const std::optional<cloud_roles::private_key_str>& skey,
const cloud_roles::aws_region_name& region,
const std::optional<cloud_storage_clients::s3_url_style>& url_style,
const default_overrides& overrides = {},
net::metrics_disabled disable_metrics = net::metrics_disabled::yes,
net::public_metrics_disabled disable_public_metrics
Expand Down Expand Up @@ -110,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<abs_self_configuration_result, s3_self_configuration_result>;
Expand Down
101 changes: 98 additions & 3 deletions src/v/cloud_storage_clients/s3_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -27,6 +29,7 @@
#include <seastar/core/gate.hh>
#include <seastar/core/iostream.hh>
#include <seastar/core/loop.hh>
#include <seastar/core/lowres_clock.hh>
#include <seastar/core/shared_ptr.hh>
#include <seastar/core/temporary_buffer.hh>
#include <seastar/net/inet_address.hh>
Expand Down Expand Up @@ -550,11 +553,103 @@ s3_client::s3_client(

ss::future<result<client_self_configuration_output, error_outcome>>
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<bool> 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(); }
Expand Down
10 changes: 10 additions & 0 deletions src/v/cloud_storage_clients/s3_client.h
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ class request_creator {
std::optional<char> delimiter = std::nullopt);

private:
friend class s3_client;
std::string make_host(const bucket_name& name) const;

std::string
Expand Down Expand Up @@ -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<bool> self_configure_test(
const bucket_name& bucket, bool remote_read, bool remote_write);

private:
request_creator _requestor;
http::client _client;
Expand Down
22 changes: 14 additions & 8 deletions src/v/cloud_storage_clients/test_client/s3_test_client_main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ void cli_opts(boost::program_options::options_description_easy_init opt) {

opt(
"url_style",
po::value<std::string>()->default_value("virtual_host"),
po::value<std::string>()->default_value(""),
"aws addressing style");

opt(
Expand Down Expand Up @@ -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<std::string>());
auto region = cloud_roles::aws_region_name(m["region"].as<std::string>());
auto url_style =
[&]() -> std::optional<cloud_storage_clients::s3_url_style> {
const auto url_style_str = m["url_style"].as<std::string>();
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<cloud_storage_clients::endpoint_url> {
Expand All @@ -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<std::string>() == "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();
Expand Down
20 changes: 14 additions & 6 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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. "
"`<bucket-name>.s3.amazonaws.com`), `path`, (e.g. "
"`s3.amazonaws.com/<bucket-name>`), 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",
Expand Down
3 changes: 2 additions & 1 deletion src/v/config/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,8 @@ struct configuration final : public config_store {
property<std::optional<ss::sstring>> cloud_storage_region;
property<std::optional<ss::sstring>> cloud_storage_bucket;
property<std::optional<ss::sstring>> cloud_storage_api_endpoint;
enum_property<cloud_storage_clients::s3_url_style> cloud_storage_url_style;
enum_property<std::optional<cloud_storage_clients::s3_url_style>>
cloud_storage_url_style;
enum_property<model::cloud_credentials_source>
cloud_storage_credentials_source;
property<std::optional<ss::sstring>>
Expand Down
2 changes: 2 additions & 0 deletions src/v/redpanda/tests/fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
Loading

0 comments on commit 40c7cc3

Please sign in to comment.