diff --git a/src/v/cloud_storage/remote.cc b/src/v/cloud_storage/remote.cc index 3e29ed61e2c02..ee6ca2a1dfcfa 100644 --- a/src/v/cloud_storage/remote.cc +++ b/src/v/cloud_storage/remote.cc @@ -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; diff --git a/src/v/cloud_storage_clients/configuration.cc b/src/v/cloud_storage_clients/configuration.cc index 02c2e73f39402..7c17707563bbf 100644 --- a/src/v/cloud_storage_clients/configuration.cc +++ b/src/v/cloud_storage_clients/configuration.cc @@ -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(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) { @@ -374,15 +389,7 @@ model::cloud_storage_backend infer_backend_from_configuration( auto& s3_config = std::get(client_config); const auto& uri = s3_config.uri; - - auto result - = string_switch(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, diff --git a/src/v/cloud_storage_clients/configuration.h b/src/v/cloud_storage_clients/configuration.h index 34d009fda2493..088aba47f95bc 100644 --- a/src/v/cloud_storage_clients/configuration.h +++ b/src/v/cloud_storage_clients/configuration.h @@ -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); diff --git a/src/v/cloud_storage_clients/s3_client.cc b/src/v/cloud_storage_clients/s3_client.cc index 03e292dac49fb..1191fafa3c525 100644 --- a/src/v/cloud_storage_clients/s3_client.cc +++ b/src/v/cloud_storage_clients/s3_client.cc @@ -556,12 +556,23 @@ s3_client::s3_client( ss::future> 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 diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index a042d2e3c3e6f..f2bae6bb7a540 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -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, diff --git a/src/v/config/convert.h b/src/v/config/convert.h index d16afab7ec8ca..fa69a28851221 100644 --- a/src/v/config/convert.h +++ b/src/v/config/convert.h @@ -368,7 +368,12 @@ struct convert { 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)); } @@ -388,6 +393,9 @@ struct convert { 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; diff --git a/src/v/model/metadata.h b/src/v/model/metadata.h index 07775df12363f..ea650c8d7251b 100644 --- a/src/v/model/metadata.h +++ b/src/v/model/metadata.h @@ -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) { @@ -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"; } diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 61cc6ae420d4d..b62d633efc317 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -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() diff --git a/tests/rptest/tests/cluster_self_config_test.py b/tests/rptest/tests/cluster_self_config_test.py index 0de06875cc411..1abc3f31868e4 100644 --- a/tests/rptest/tests/cluster_self_config_test.py +++ b/tests/rptest/tests/cluster_self_config_test.py @@ -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 @@ -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