Skip to content

Commit

Permalink
cloud_storage: make remote_path_provider harder to copy around
Browse files Browse the repository at this point in the history
The fact that we can copy around path providers is potentially
error-prone and confusing. The primary path provider meant to be used by
most subsystems (i.e. the archival metadata STM) may not be clear if
callers end up making copies.

This removes most special constructors, and exposes an explicit copy()
method to force callers to think twice before copying.

The only existing callsite where such a copy happens has been updated to
use this method.

This is review follow-up to
redpanda-data#20778
  • Loading branch information
andrwng committed Jul 9, 2024
1 parent 87deb18 commit fe0ec50
Show file tree
Hide file tree
Showing 3 changed files with 34 additions and 11 deletions.
22 changes: 11 additions & 11 deletions src/v/cloud_storage/remote_partition.cc
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#include "cloud_storage/remote_segment.h"
#include "cloud_storage/tx_range_manifest.h"
#include "cloud_storage/types.h"
#include "cloud_storage_clients/types.h"
#include "model/fundamental.h"
#include "model/timestamp.h"
#include "net/connection.h"
Expand Down Expand Up @@ -1312,14 +1313,13 @@ static constexpr ss::lowres_clock::duration finalize_backoff = 1s;
struct finalize_data {
model::ntp ntp;
model::initial_revision_id revision;
remote_path_provider path_provider;
cloud_storage_clients::bucket_name bucket;
cloud_storage_clients::object_key key;
iobuf serialized_manifest;
model::offset insync_offset;
};

ss::future<> finalize_background(remote& api, finalize_data data) {
ss::future<> finalize_background(
remote& api, finalize_data data, remote_path_provider path_provider) {
// This function runs as a detached background fiber, so has no shutdown
// logic of its own: our remote operations will be shut down when the
// `remote` object is shut down.
Expand All @@ -1330,7 +1330,7 @@ ss::future<> finalize_background(remote& api, finalize_data data) {
partition_manifest remote_manifest(data.ntp, data.revision);

partition_manifest_downloader dl(
data.bucket, data.path_provider, data.ntp, data.revision, api);
data.bucket, path_provider, data.ntp, data.revision, api);
auto manifest_get_result = co_await dl.download_manifest(
local_rtc, &remote_manifest);
if (manifest_get_result.has_error()) {
Expand Down Expand Up @@ -1373,9 +1373,11 @@ ss::future<> finalize_background(remote& api, finalize_data data) {
remote_manifest.get_insync_offset(),
data.insync_offset);

const auto key = cloud_storage_clients::object_key{
path_provider.partition_manifest_path(data.ntp, data.revision)};
auto manifest_put_result = co_await api.upload_object(
{.transfer_details
= {.bucket = data.bucket, .key = data.key, .parent_rtc = local_rtc},
= {.bucket = data.bucket, .key = key, .parent_rtc = local_rtc},
.type = upload_type::manifest,
.payload = std::move(data.serialized_manifest)});

Expand Down Expand Up @@ -1415,21 +1417,19 @@ void remote_partition::finalize() {
const auto& stm_manifest = _manifest_view->stm_manifest();
auto serialized_manifest = stm_manifest.to_iobuf();

const auto& path_provider = _manifest_view->path_provider();
finalize_data data{
.ntp = get_ntp(),
.revision = stm_manifest.get_revision_id(),
.path_provider = path_provider,
.bucket = _bucket,
.key = cloud_storage_clients::object_key{stm_manifest.get_manifest_path(
path_provider)()},
.serialized_manifest = std::move(serialized_manifest),
.insync_offset = stm_manifest.get_insync_offset()};

ssx::spawn_with_gate(
_api.gate(),
[&api = _api, data = std::move(data)]() mutable -> ss::future<> {
return finalize_background(api, std::move(data));
[&api = _api,
data = std::move(data),
pp = _manifest_view->path_provider().copy()]() mutable -> ss::future<> {
return finalize_background(api, std::move(data), pp.copy());
});
}

Expand Down
5 changes: 5 additions & 0 deletions src/v/cloud_storage/remote_path_provider.cc
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,11 @@ namespace cloud_storage {
remote_path_provider::remote_path_provider(std::optional<remote_label> label)
: label_(label) {}

remote_path_provider remote_path_provider::copy() const {
remote_path_provider ret(label_);
return ret;
}

ss::sstring remote_path_provider::topic_manifest_prefix(
const model::topic_namespace& topic) const {
if (label_.has_value()) {
Expand Down
18 changes: 18 additions & 0 deletions src/v/cloud_storage/remote_path_provider.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,26 @@ namespace cloud_storage {

class remote_path_provider {
public:
// Discourage accidental copies to encourage referencing of a single path
// provider (e.g. the one owned by the archival STM).
remote_path_provider(const remote_path_provider&) = delete;
remote_path_provider& operator=(const remote_path_provider&) = delete;
remote_path_provider& operator=(remote_path_provider&&) = delete;
~remote_path_provider() = default;

explicit remote_path_provider(std::optional<remote_label> label);

// An explicit copy method. Callers should think twice about using this and
// instead consider if there is an existing path provider that makes sense
// to reference instead (e.g. the one owned by the archival STM).
//
// One may not exist e.g. when doing background finalization after the
// partition and STM is destructed.
remote_path_provider copy() const;

// For use in copy() and in coroutines.
remote_path_provider(remote_path_provider&&) = default;

// Prefix of the topic manifest path. This can be used to filter objects to
// find topic manifests.
ss::sstring
Expand Down

0 comments on commit fe0ec50

Please sign in to comment.