Skip to content

Commit

Permalink
Merge pull request redpanda-data#22923 from WillemKauf/s3_self_config…
Browse files Browse the repository at this point in the history
…_remove_perms_check

`cloud_storage_clients`: remove remote perm checks from `s3_client::self_configure()`
  • Loading branch information
WillemKauf authored Aug 17, 2024
2 parents 4a779bd + 011e56b commit 5e3d995
Show file tree
Hide file tree
Showing 5 changed files with 33 additions and 98 deletions.
60 changes: 8 additions & 52 deletions src/v/cloud_storage_clients/s3_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -573,21 +573,6 @@ s3_client::self_configure() {
// If any configuration options prevent testing, addressing style will
// default to virtual_host.
// If both addressing methods fail, return an error.
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()) {
Expand All @@ -607,7 +592,7 @@ s3_client::self_configure() {
_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)) {
if (co_await self_configure_test(bucket)) {
// Virtual-host style request succeeded.
co_return result;
}
Expand All @@ -622,7 +607,7 @@ s3_client::self_configure() {
// 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)) {
if (co_await self_configure_test(bucket)) {
// Path style request succeeded.
co_return result;
}
Expand All @@ -636,41 +621,12 @@ s3_client::self_configure() {
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<bool> s3_client::self_configure_test(const bucket_name& bucket) {
// Check that the current addressing-style works by issuing a ListObjects
// request.
auto list_objects_result = co_await list_objects(
bucket, std::nullopt, std::nullopt, 1);
co_return list_objects_result;
}

ss::future<> s3_client::stop() { return _client.stop(); }
Expand Down
13 changes: 5 additions & 8 deletions src/v/cloud_storage_clients/s3_client.h
Original file line number Diff line number Diff line change
Expand Up @@ -248,14 +248,11 @@ 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);
// Performs testing as part of the self-configuration step. Returns true if
// the test was successful (indicating that the current addressing style is
// compatible with the configured cloud storage provider), and false
// otherwise.
ss::future<bool> self_configure_test(const bucket_name& bucket);

private:
request_creator _requestor;
Expand Down
15 changes: 10 additions & 5 deletions tests/rptest/tests/cluster_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -1455,27 +1455,31 @@ def test_cloud_validation(self):
'cloud_storage_enabled': True,
'cloud_storage_credentials_source': 'aws_instance_metadata',
'cloud_storage_region': 'us-east-1',
'cloud_storage_bucket': 'dearliza'
'cloud_storage_bucket': 'dearliza',
'cloud_storage_url_style': 'virtual_host'
},
{
'cloud_storage_enabled': True,
'cloud_storage_credentials_source': 'gcp_instance_metadata',
'cloud_storage_region': 'us-east-1',
'cloud_storage_bucket': 'dearliza'
'cloud_storage_bucket': 'dearliza',
'cloud_storage_url_style': 'virtual_host'
},
{
'cloud_storage_enabled': True,
'cloud_storage_credentials_source': 'sts',
'cloud_storage_region': 'us-east-1',
'cloud_storage_bucket': 'dearliza'
'cloud_storage_bucket': 'dearliza',
'cloud_storage_url_style': 'virtual_host'
},
{
'cloud_storage_enabled': True,
'cloud_storage_secret_key': 'open',
'cloud_storage_access_key': 'sesame',
'cloud_storage_credentials_source': 'config_file',
'cloud_storage_region': 'us-east-1',
'cloud_storage_bucket': 'dearliza'
'cloud_storage_bucket': 'dearliza',
'cloud_storage_url_style': 'virtual_host'
},
]
for payload in valid_updates:
Expand Down Expand Up @@ -1504,7 +1508,8 @@ def test_cloud_validation(self):
'cloud_storage_secret_key': 'open',
'cloud_storage_access_key': 'sesame',
'cloud_storage_region': 'us-east-1',
'cloud_storage_bucket': 'dearliza'
'cloud_storage_bucket': 'dearliza',
'cloud_storage_url_style': 'virtual_host'
}
patch_result = self.admin.patch_cluster_config(upsert=static_config,
remove=[])
Expand Down
27 changes: 5 additions & 22 deletions tests/rptest/tests/cluster_self_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,8 @@ def self_config_result_from_logs(self, node):

@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_self_config(self, cloud_storage_type,
cloud_storage_enable_remote_read,
cloud_storage_enable_remote_write):
applies_only_on=[CloudStorageType.S3]))
def test_s3_self_config(self, cloud_storage_type):
"""
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
Expand All @@ -61,10 +57,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.
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
)
cloud_storage_url_style=None)

self.start_redpanda(si_settings=si_settings)
admin = Admin(self.redpanda)
Expand All @@ -81,10 +74,6 @@ def test_s3_self_config(self, cloud_storage_type,
# Assert that self configuration started.
assert self.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.self_config_default_in_logs(node)

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

Expand All @@ -98,12 +87,8 @@ def test_s3_self_config(self, cloud_storage_type,

@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):
applies_only_on=[CloudStorageType.S3]))
def test_s3_oracle_self_config(self, cloud_storage_type):
"""
Verify that the cloud_storage_url_style self-configuration for OCI
backend always results in path-style.
Expand All @@ -116,8 +101,6 @@ def test_s3_oracle_self_config(self, cloud_storage_type,
# 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,
Expand Down
16 changes: 5 additions & 11 deletions tests/rptest/tests/self_test_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -49,16 +49,12 @@ def all_idle():
return wait_until_result(all_idle, timeout_sec=90, backoff_sec=1)

@cluster(num_nodes=3)
@matrix(remote_read=[True, False], remote_write=[True, False])
def test_self_test(self, remote_read, remote_write):
def test_self_test(self):
"""Assert the self test starts/completes with success."""
num_nodes = 3
self.start_redpanda(
num_nodes=num_nodes,
si_settings=SISettings(
test_context=self.test_context,
cloud_storage_enable_remote_read=remote_read,
cloud_storage_enable_remote_write=remote_write))
si_settings=SISettings(test_context=self.test_context))
self.rpk_client().self_test_start(2000, 2000, 5000, 100)

# Wait for completion
Expand Down Expand Up @@ -218,11 +214,9 @@ def test_self_test_unknown_test_type(self):
with mixed versions of Redpanda."""
num_nodes = 3

self.start_redpanda(num_nodes=num_nodes,
si_settings=SISettings(
test_context=self.test_context,
cloud_storage_enable_remote_read=True,
cloud_storage_enable_remote_write=True))
self.start_redpanda(
num_nodes=num_nodes,
si_settings=SISettings(test_context=self.test_context))

# Attempt to run with an unknown test type "pandatest"
# and possibly unknown "cloud" test.
Expand Down

0 comments on commit 5e3d995

Please sign in to comment.