Skip to content

Commit

Permalink
cloud_storage: use self configuration for url style
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
WillemKauf committed May 14, 2024
1 parent 8222446 commit 800d960
Show file tree
Hide file tree
Showing 6 changed files with 128 additions and 13 deletions.
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
19 changes: 13 additions & 6 deletions src/v/cloud_storage_clients/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -92,10 +92,13 @@ 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) {
Expand Down Expand Up @@ -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<std::chrono::milliseconds>(c.max_idle_time)
.count()
<< "}";
Expand Down Expand Up @@ -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<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 @@ -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;
}

Expand Down
7 changes: 4 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 Down Expand Up @@ -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<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,
"Can't 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,
"Can't 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
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

0 comments on commit 800d960

Please sign in to comment.