Skip to content

Commit

Permalink
Merge pull request #22902 from WillemKauf/oracle_cloud_storage_backend
Browse files Browse the repository at this point in the history
`cloud_storage_clients`: add `cloud_storage_backend::oracle` and fix `s3_client::self_configure()`
  • Loading branch information
WillemKauf authored Aug 16, 2024
2 parents 6a68058 + b198688 commit 441d3fe
Show file tree
Hide file tree
Showing 9 changed files with 110 additions and 14 deletions.
3 changes: 3 additions & 0 deletions src/v/cloud_storage/remote.cc
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,9 @@ int remote::delete_objects_max_keys() const {
switch (_cloud_storage_backend) {
case model::cloud_storage_backend::aws:
[[fallthrough]];
case model::cloud_storage_backend::oracle_s3_compat:
// https://docs.oracle.com/en-us/iaas/api/#/en/s3objectstorage/20160918/Object/BulkDelete
[[fallthrough]];
case model::cloud_storage_backend::minio:
// https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
return 1000;
Expand Down
25 changes: 16 additions & 9 deletions src/v/cloud_storage_clients/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,21 @@ operator<<(std::ostream& o, const client_self_configuration_output& r) {
r);
}

model::cloud_storage_backend
infer_backend_from_uri(const access_point_uri& uri) {
auto result
= string_switch<model::cloud_storage_backend>(uri())
.match_expr("google", model::cloud_storage_backend::google_s3_compat)
.match_expr(R"(127\.0\.0\.1)", model::cloud_storage_backend::aws)
.match_expr("localhost", model::cloud_storage_backend::aws)
.match_expr("minio", model::cloud_storage_backend::minio)
.match_expr("amazon", model::cloud_storage_backend::aws)
.match_expr(
"oraclecloud", model::cloud_storage_backend::oracle_s3_compat)
.default_match(model::cloud_storage_backend::unknown);
return result;
}

model::cloud_storage_backend infer_backend_from_configuration(
const client_configuration& client_config,
model::cloud_credentials_source cloud_storage_credentials_source) {
Expand Down Expand Up @@ -374,15 +389,7 @@ model::cloud_storage_backend infer_backend_from_configuration(

auto& s3_config = std::get<s3_configuration>(client_config);
const auto& uri = s3_config.uri;

auto result
= string_switch<model::cloud_storage_backend>(uri())
.match_expr("google", model::cloud_storage_backend::google_s3_compat)
.match_expr(R"(127\.0\.0\.1)", model::cloud_storage_backend::aws)
.match_expr("localhost", model::cloud_storage_backend::aws)
.match_expr("minio", model::cloud_storage_backend::minio)
.match_expr("amazon", model::cloud_storage_backend::aws)
.default_match(model::cloud_storage_backend::unknown);
auto result = infer_backend_from_uri(uri);

vlog(
client_config_log.info,
Expand Down
5 changes: 5 additions & 0 deletions src/v/cloud_storage_clients/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,11 @@ std::ostream& operator<<(std::ostream&, const s3_self_configuration_result&);
std::ostream&
operator<<(std::ostream&, const client_self_configuration_output&);

// In the case of S3-compatible providers, all that is needed to infer the
// backend is the access point/uri.
model::cloud_storage_backend
infer_backend_from_uri(const access_point_uri& uri);

model::cloud_storage_backend infer_backend_from_configuration(
const client_configuration& client_config,
model::cloud_credentials_source cloud_storage_credentials_source);
Expand Down
15 changes: 13 additions & 2 deletions src/v/cloud_storage_clients/s3_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -556,12 +556,23 @@ s3_client::s3_client(

ss::future<result<client_self_configuration_output, error_outcome>>
s3_client::self_configure() {
auto result = s3_self_configuration_result{
.url_style = s3_url_style::virtual_host};
// Oracle cloud storage only supports path-style requests
// (https://www.oracle.com/ca-en/cloud/storage/object-storage/faq/#category-amazon),
// but self-configuration will misconfigure to virtual-host style due to a
// ListObjects request that happens to succeed. Override for this
// specific case.
auto inferred_backend = infer_backend_from_uri(_requestor._ap);
if (inferred_backend == model::cloud_storage_backend::oracle_s3_compat) {
result.url_style = s3_url_style::path;
co_return result;
}

// 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
Expand Down
1 change: 1 addition & 0 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2080,6 +2080,7 @@ configuration::configuration()
model::cloud_storage_backend::google_s3_compat,
model::cloud_storage_backend::azure,
model::cloud_storage_backend::minio,
model::cloud_storage_backend::oracle_s3_compat,
model::cloud_storage_backend::unknown})
, cloud_storage_credentials_host(
*this,
Expand Down
10 changes: 9 additions & 1 deletion src/v/config/convert.h
Original file line number Diff line number Diff line change
Expand Up @@ -368,7 +368,12 @@ struct convert<model::cloud_storage_backend> {
using type = model::cloud_storage_backend;

static constexpr auto acceptable_values = std::to_array(
{"aws", "google", "azure", "minio", "unknown"});
{"aws",
"google_s3_compat",
"azure",
"minio",
"oracle_s3_compat",
"unknown"});

static Node encode(const type& rhs) { return Node(fmt::format("{}", rhs)); }

Expand All @@ -388,6 +393,9 @@ struct convert<model::cloud_storage_backend> {
model::cloud_storage_backend::google_s3_compat)
.match("minio", model::cloud_storage_backend::minio)
.match("azure", model::cloud_storage_backend::azure)
.match(
"oracle_s3_compat",
model::cloud_storage_backend::oracle_s3_compat)
.match("unknown", model::cloud_storage_backend::unknown);

return true;
Expand Down
5 changes: 4 additions & 1 deletion src/v/model/metadata.h
Original file line number Diff line number Diff line change
Expand Up @@ -468,7 +468,8 @@ enum class cloud_storage_backend {
google_s3_compat = 1,
azure = 2,
minio = 3,
unknown = 4,
oracle_s3_compat = 4,
unknown
};

inline std::ostream& operator<<(std::ostream& os, cloud_storage_backend csb) {
Expand All @@ -481,6 +482,8 @@ inline std::ostream& operator<<(std::ostream& os, cloud_storage_backend csb) {
return os << "azure";
case cloud_storage_backend::minio:
return os << "minio";
case cloud_storage_backend::oracle_s3_compat:
return os << "oracle_s3_compat";
case cloud_storage_backend::unknown:
return os << "unknown";
}
Expand Down
6 changes: 6 additions & 0 deletions tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -3870,6 +3870,12 @@ def add_to_started_nodes(self, node):

def clean(self, **kwargs):
super().clean(**kwargs)
# If we bypassed bucket creation, there is no need to try to delete it.
if self._si_settings and self._si_settings.bypass_bucket_creation:
self.logger.info(
f"Skipping deletion of bucket/container: {self.si_settings.cloud_storage_bucket},"
"because its creation was bypassed.")
return
if self._cloud_storage_client:
try:
self.delete_bucket_from_si()
Expand Down
54 changes: 53 additions & 1 deletion tests/rptest/tests/cluster_self_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ def test_s3_self_config(self, cloud_storage_type,
"""
si_settings = SISettings(
self.ctx,
#Force self configuration through setting cloud_storage_url_style to None.
# Force self configuration through setting cloud_storage_url_style to None.
cloud_storage_url_style=None,
cloud_storage_enable_remote_read=cloud_storage_enable_remote_read,
cloud_storage_enable_remote_write=cloud_storage_enable_remote_write
Expand Down Expand Up @@ -95,3 +95,55 @@ def test_s3_self_config(self, cloud_storage_type,
]

assert self_config_result and self_config_result in self_config_expected_results

@cluster(num_nodes=1)
@matrix(cloud_storage_type=get_cloud_storage_type(
applies_only_on=[CloudStorageType.S3]),
cloud_storage_enable_remote_read=[True, False],
cloud_storage_enable_remote_write=[True, False])
def test_s3_oracle_self_config(self, cloud_storage_type,
cloud_storage_enable_remote_read,
cloud_storage_enable_remote_write):
"""
Verify that the cloud_storage_url_style self-configuration for OCI
backend always results in path-style.
"""
si_settings = SISettings(
self.ctx,
# Force self configuration through setting cloud_storage_url_style to None.
cloud_storage_url_style=None,
# Set Oracle endpoint to expected format.
# https://docs.oracle.com/en-us/iaas/Content/Object/Tasks/s3compatibleapi_topic-Amazon_S3_Compatibility_API_Support.htm#s3-api-support
cloud_storage_api_endpoint=
'mynamespace.compat.objectstorage.us-phoenix-1.oraclecloud.com',
cloud_storage_enable_remote_read=cloud_storage_enable_remote_read,
cloud_storage_enable_remote_write=cloud_storage_enable_remote_write,
# Bypass bucket creation, cleanup, and scrubbing, as we won't actually be able to access the endpoint (Self configuration will usi the endpoint to set path-style).
bypass_bucket_creation=True,
use_bucket_cleanup_policy=False,
skip_end_of_test_scrubbing=True)

self.start_redpanda(si_settings=si_settings)
admin = Admin(self.redpanda)
self.log_searcher = LogSearchLocal(self.ctx, [], self.redpanda.logger,
self.redpanda.STDOUT_STDERR_CAPTURE)

config = 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

for node in self.redpanda.nodes:
# Assert that self configuration started.
assert self.self_config_start_in_logs(node)

# Assert that self configuration returned a result.
self_config_result = self.self_config_result_from_logs(node)

# Oracle only supports path-style requests, self-configuration will always succeed.
self_config_expected_results = [
'{s3_self_configuration_result: {s3_url_style: path}}'
]

assert self_config_result and self_config_result in self_config_expected_results

0 comments on commit 441d3fe

Please sign in to comment.