From 44978ef7387f702d21f4b1d5b87955aee479d8c7 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 20 Jun 2024 16:25:35 -0700 Subject: [PATCH 01/34] offline_log_viewer: support remote_labels in topic properties This is a preemptive commit for the upcoming addition of remote labels to the topic properties. --- tools/offline_log_viewer/controller.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tools/offline_log_viewer/controller.py b/tools/offline_log_viewer/controller.py index bfe2d4f6cde0..1498534d6c2d 100644 --- a/tools/offline_log_viewer/controller.py +++ b/tools/offline_log_viewer/controller.py @@ -19,6 +19,10 @@ def read_remote_topic_properties_serde(rdr: Reader): }) +def read_remote_label_serde(rdr: Reader): + return rdr.read_envelope(lambda rdr, _: {"cluster_uuid": rdr.read_uuid()}) + + def read_topic_properties_serde(rdr: Reader, version): topic_properties = { @@ -113,6 +117,10 @@ def read_topic_properties_serde(rdr: Reader, version): 'flush_ms': rdr.read_optional(Reader.read_int64), 'flush_bytes': rdr.read_optional(Reader.read_int64) } + if version >= 9: + topic_properties |= { + 'remote_labels': rdr.read_optional(read_remote_label_serde) + } return topic_properties @@ -128,7 +136,7 @@ def read_topic_config(rdr: Reader, version): 'replication_factor': rdr.read_int16(), 'properties': - rdr.read_envelope(read_topic_properties_serde, max_version=8), + rdr.read_envelope(read_topic_properties_serde, max_version=9), } if version < 1: # see https://github.com/redpanda-data/redpanda/pull/6613 From 7d38dbaf55e5990fe319f5694bb9bf5c03149464 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 12 Jun 2024 15:24:35 -0700 Subject: [PATCH 02/34] cluster: add remote label as topic property --- src/v/cluster/topic_properties.cc | 8 +++++--- src/v/cluster/topic_properties.h | 19 +++++++++++++++++-- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/v/cluster/topic_properties.cc b/src/v/cluster/topic_properties.cc index 0195254f29a2..4b2d9aba9a9d 100644 --- a/src/v/cluster/topic_properties.cc +++ b/src/v/cluster/topic_properties.cc @@ -36,7 +36,8 @@ std::ostream& operator<<(std::ostream& o, const topic_properties& properties) { "mpx_virtual_cluster_id: {}, " "write_caching: {}, " "flush_ms: {}, " - "flush_bytes: {}}}", + "flush_bytes: {}, " + "remote_label: {}}}", properties.compression, properties.cleanup_policy_bitflags, properties.compaction_strategy, @@ -67,7 +68,8 @@ std::ostream& operator<<(std::ostream& o, const topic_properties& properties) { properties.mpx_virtual_cluster_id, properties.write_caching, properties.flush_ms, - properties.flush_bytes); + properties.flush_bytes, + properties.remote_label); return o; } @@ -101,7 +103,7 @@ bool topic_properties::has_overrides() const { || initial_retention_local_target_bytes.is_engaged() || initial_retention_local_target_ms.is_engaged() || write_caching.has_value() || flush_ms.has_value() - || flush_bytes.has_value(); + || flush_bytes.has_value() || remote_label.has_value(); } bool topic_properties::requires_remote_erase() const { diff --git a/src/v/cluster/topic_properties.h b/src/v/cluster/topic_properties.h index 824483b8c65d..b947fde92d70 100644 --- a/src/v/cluster/topic_properties.h +++ b/src/v/cluster/topic_properties.h @@ -9,6 +9,7 @@ #pragma once +#include "cloud_storage/remote_label.h" #include "cluster/remote_topic_properties.h" #include "model/compression.h" #include "model/fundamental.h" @@ -28,7 +29,7 @@ namespace cluster { */ struct topic_properties : serde:: - envelope, serde::compat_version<0>> { + envelope, serde::compat_version<0>> { topic_properties() noexcept = default; topic_properties( std::optional compression, @@ -152,6 +153,19 @@ struct topic_properties std::optional flush_ms; std::optional flush_bytes; + // Label to be used when generating paths of remote objects (manifests, + // segments, etc) of this topic. + // + // The topic's data is associated with exactly one label: as a topic is + // removed and recovered across different clusters, its label will be the + // same, even though the clusters' UUIDs hosting it will be different. This + // allows recovered topics and read replica topics to download with just + // one label in mind. + // + // std::nullopt indicates this topic was created before labels were + // supported, in which case objects will use a legacy naming scheme. + std::optional remote_label; + bool is_compacted() const; bool has_overrides() const; bool requires_remote_erase() const; @@ -191,7 +205,8 @@ struct topic_properties mpx_virtual_cluster_id, write_caching, flush_ms, - flush_bytes); + flush_bytes, + remote_label); } friend bool operator==(const topic_properties&, const topic_properties&) From fab93f121b215579d72dfe69b2453a8bbf1db390 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 28 Jun 2024 18:30:42 -0700 Subject: [PATCH 03/34] archival_stm: add remote path provider to archival stm This plumbs the remote label (cluster uuid) associated with a topic from the topics table into the archival meta stm, in the same fashion that vcluster-id is plumbed into the rm_stm. Conceptually, the archival STM will be the owner of the root remote path provider used throughout the cloud_storage and archival subsystems. --- src/v/archival/archival_metadata_stm.cc | 25 +++++++++++++++---- src/v/archival/archival_metadata_stm.h | 14 +++++++++-- .../tests/archival_metadata_stm_gtest.cc | 3 ++- .../tests/archival_metadata_stm_test.cc | 24 +++++++++++++++--- src/v/redpanda/application.cc | 3 ++- 5 files changed, 56 insertions(+), 13 deletions(-) diff --git a/src/v/archival/archival_metadata_stm.cc b/src/v/archival/archival_metadata_stm.cc index 65fa71cab5e2..ef419e3484ee 100644 --- a/src/v/archival/archival_metadata_stm.cc +++ b/src/v/archival/archival_metadata_stm.cc @@ -15,6 +15,7 @@ #include "bytes/iostream.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/types.h" #include "cluster/errc.h" #include "cluster/logger.h" @@ -641,14 +642,16 @@ archival_metadata_stm::archival_metadata_stm( raft::consensus* raft, cloud_storage::remote& remote, features::feature_table& ft, - ss::logger& logger) + ss::logger& logger, + std::optional remote_label) : raft::persisted_stm<>(archival_stm_snapshot, logger, raft) , _logger(logger, ssx::sformat("ntp: {}", raft->ntp())) , _mem_tracker(ss::make_shared(raft->ntp().path())) , _manifest(ss::make_shared( raft->ntp(), raft->log_config().get_initial_revision(), _mem_tracker)) , _cloud_storage_api(remote) - , _feature_table(ft) {} + , _feature_table(ft) + , _remote_path_provider({remote_label}) {} ss::future archival_metadata_stm::truncate( model::offset start_rp_offset, @@ -1687,10 +1690,12 @@ archival_metadata_stm::state_dirty archival_metadata_stm::get_dirty( archival_metadata_stm_factory::archival_metadata_stm_factory( bool cloud_storage_enabled, ss::sharded& cloud_storage_api, - ss::sharded& feature_table) + ss::sharded& feature_table, + ss::sharded& topics) : _cloud_storage_enabled(cloud_storage_enabled) , _cloud_storage_api(cloud_storage_api) - , _feature_table(feature_table) {} + , _feature_table(feature_table) + , _topics(topics) {} bool archival_metadata_stm_factory::is_applicable_for( const storage::ntp_config& ntp_cfg) const { @@ -1700,8 +1705,18 @@ bool archival_metadata_stm_factory::is_applicable_for( void archival_metadata_stm_factory::create( raft::state_machine_manager_builder& builder, raft::consensus* raft) { + auto topic_md = _topics.local().get_topic_metadata_ref( + model::topic_namespace_view(raft->ntp())); + auto remote_label + = topic_md.has_value() + ? topic_md->get().get_configuration().properties.remote_label + : std::nullopt; auto stm = builder.create_stm( - raft, _cloud_storage_api.local(), _feature_table.local(), clusterlog); + raft, + _cloud_storage_api.local(), + _feature_table.local(), + clusterlog, + remote_label); raft->log()->stm_manager()->add_stm(stm); } diff --git a/src/v/archival/archival_metadata_stm.h b/src/v/archival/archival_metadata_stm.h index 0f7806df815a..e0cfb3550bb8 100644 --- a/src/v/archival/archival_metadata_stm.h +++ b/src/v/archival/archival_metadata_stm.h @@ -13,9 +13,11 @@ #include "cloud_storage/fwd.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/types.h" #include "cluster/errc.h" #include "cluster/state_machine_registry.h" +#include "cluster/topic_table.h" #include "features/fwd.h" #include "model/fundamental.h" #include "model/record.h" @@ -133,7 +135,8 @@ class archival_metadata_stm final : public raft::persisted_stm<> { raft::consensus*, cloud_storage::remote& remote, features::feature_table&, - ss::logger& logger); + ss::logger& logger, + std::optional); /// Add segments to the raft log, replicate them and /// wait until it is applied to the STM. @@ -277,6 +280,10 @@ class archival_metadata_stm final : public raft::persisted_stm<> { return _compacted_replaced_bytes; } + const cloud_storage::remote_path_provider& path_provider() const { + return _remote_path_provider; + } + private: ss::future do_sync(model::timeout_clock::duration timeout, ss::abort_source* as); @@ -374,6 +381,7 @@ class archival_metadata_stm final : public raft::persisted_stm<> { cloud_storage::remote& _cloud_storage_api; features::feature_table& _feature_table; + const cloud_storage::remote_path_provider _remote_path_provider; ss::abort_source _download_as; // for observability: keep track of the number of cloud bytes "removed" by @@ -389,7 +397,8 @@ class archival_metadata_stm_factory : public state_machine_factory { archival_metadata_stm_factory( bool cloud_storage_enabled, ss::sharded&, - ss::sharded&); + ss::sharded&, + ss::sharded&); bool is_applicable_for(const storage::ntp_config&) const final; void create(raft::state_machine_manager_builder&, raft::consensus*) final; @@ -398,6 +407,7 @@ class archival_metadata_stm_factory : public state_machine_factory { bool _cloud_storage_enabled; ss::sharded& _cloud_storage_api; ss::sharded& _feature_table; + ss::sharded& _topics; }; } // namespace cluster diff --git a/src/v/archival/tests/archival_metadata_stm_gtest.cc b/src/v/archival/tests/archival_metadata_stm_gtest.cc index d5dc2c4e6bbc..4a085c7c543d 100644 --- a/src/v/archival/tests/archival_metadata_stm_gtest.cc +++ b/src/v/archival/tests/archival_metadata_stm_gtest.cc @@ -102,7 +102,8 @@ class archival_metadata_stm_gtest_fixture : public raft::raft_fixture { node->raft().get(), stm_node.remote.local(), node->get_feature_table().local(), - fixture_logger); + fixture_logger, + std::nullopt); stm_node.archival_stm = std::move(stm); diff --git a/src/v/archival/tests/archival_metadata_stm_test.cc b/src/v/archival/tests/archival_metadata_stm_test.cc index 490e2307f780..35bd108830e1 100644 --- a/src/v/archival/tests/archival_metadata_stm_test.cc +++ b/src/v/archival/tests/archival_metadata_stm_test.cc @@ -140,7 +140,11 @@ struct archival_metadata_stm_fixture : archival_metadata_stm_base_fixture { create_raft(); raft::state_machine_manager_builder builder; archival_stm = builder.create_stm( - _raft.get(), cloud_api.local(), feature_table.local(), logger); + _raft.get(), + cloud_api.local(), + feature_table.local(), + logger, + std::nullopt); _raft->start(std::move(builder)).get(); _started = true; @@ -351,7 +355,11 @@ FIXTURE_TEST(test_snapshot_loading, archival_metadata_stm_base_fixture) { raft::state_machine_manager_builder builder; auto archival_stm = builder.create_stm( - _raft.get(), cloud_api.local(), feature_table.local(), logger); + _raft.get(), + cloud_api.local(), + feature_table.local(), + logger, + std::nullopt); _raft->start(std::move(builder)).get(); _started = true; wait_for_confirmed_leader(); @@ -446,7 +454,11 @@ FIXTURE_TEST(test_sname_derivation, archival_metadata_stm_base_fixture) { raft::state_machine_manager_builder builder; auto archival_stm = builder.create_stm( - _raft.get(), cloud_api.local(), feature_table.local(), logger); + _raft.get(), + cloud_api.local(), + feature_table.local(), + logger, + std::nullopt); _raft->start(std::move(builder)).get(); _started = true; @@ -655,7 +667,11 @@ FIXTURE_TEST( raft::state_machine_manager_builder builder; auto archival_stm = builder.create_stm( - _raft.get(), cloud_api.local(), feature_table.local(), logger); + _raft.get(), + cloud_api.local(), + feature_table.local(), + logger, + std::nullopt); _raft->start(std::move(builder)).get(); _started = true; wait_for_confirmed_leader(); diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index ecfdbe0332a6..0bf30e696a7b 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -2685,7 +2685,8 @@ void application::start_runtime_services( pm.register_factory( config::shard_local_cfg().cloud_storage_enabled(), cloud_storage_api, - feature_table); + feature_table, + controller->get_topics_state()); pm.register_factory(); }) .get(); From cda86a5ca20c1f200b1269e24b5da6200d4a743a Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 19 Jun 2024 18:26:03 -0700 Subject: [PATCH 04/34] ntp_archiver: use path provider for naming This replaces path generation in ntp_archiver with variants that use the path provider. Namely: - topic manifest uploads - partition manifest uploads - segment uploads - generating spillover manifests --- src/v/archival/ntp_archiver_service.cc | 41 ++++++++++++++++---------- src/v/archival/ntp_archiver_service.h | 2 ++ 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/src/v/archival/ntp_archiver_service.cc b/src/v/archival/ntp_archiver_service.cc index bf379b2f6687..7ac46586a2ce 100644 --- a/src/v/archival/ntp_archiver_service.cc +++ b/src/v/archival/ntp_archiver_service.cc @@ -22,6 +22,7 @@ #include "cloud_storage/async_manifest_view.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/remote_segment_index.h" #include "cloud_storage/spillover_manifest.h" @@ -516,10 +517,10 @@ ss::future<> ntp_archiver::upload_topic_manifest() { cfg_copy.replication_factor = replication_factor; cloud_storage::topic_manifest tm( cfg_copy, _rev, _feature_table.local()); - auto key = tm.get_manifest_path(); + auto key = tm.get_manifest_path(remote_path_provider()); vlog(ctxlog.debug, "Topic manifest object key is '{}'", key); auto res = co_await _remote.upload_manifest( - _conf->bucket_name, tm, fib); + _conf->bucket_name, tm, key, fib); if (res != cloud_storage::upload_result::success) { vlog(ctxlog.warn, "Topic manifest upload failed: {}", key); } else { @@ -818,12 +819,12 @@ ss::future<> ntp_archiver::sync_manifest_until_term_change() { vlog( _rtclog.error, "Failed to download manifest {}", - manifest().get_manifest_path()); + manifest().get_manifest_path(remote_path_provider())); } else { vlog( _rtclog.debug, "Successfuly downloaded manifest {}", - manifest().get_manifest_path()); + manifest().get_manifest_path(remote_path_provider())); } co_await ss::sleep_abortable(_sync_manifest_timeout(), _as); } @@ -1110,15 +1111,16 @@ ss::future ntp_archiver::upload_manifest( auto upload_insync_offset = manifest().get_insync_offset(); + auto path = manifest().get_manifest_path(remote_path_provider()); vlog( _rtclog.debug, "[{}] Uploading partition manifest, insync_offset={}, path={}", upload_ctx, upload_insync_offset, - manifest().get_manifest_path()); + path()); auto result = co_await _remote.upload_manifest( - get_bucket_name(), manifest(), fib); + get_bucket_name(), manifest(), path, fib); // now that manifest() is updated in cloud, updated the // compacted_away_cloud_bytes metric @@ -1162,7 +1164,7 @@ remote_segment_path ntp_archiver::segment_path_for_candidate( .sname_format = cloud_storage::segment_name_format::v3, }; - return manifest().generate_segment_path(val); + return manifest().generate_segment_path(val, remote_path_provider()); } static std::pair, ss::input_stream> @@ -2134,9 +2136,7 @@ ntp_archiver::maybe_truncate_manifest() { _conf->manifest_upload_timeout(), _conf->upload_loop_initial_backoff(), &rtc); - auto sname = cloud_storage::generate_local_segment_name( - meta.base_offset, meta.segment_term); - auto spath = m.generate_segment_path(meta); + auto spath = m.generate_segment_path(meta, remote_path_provider()); auto result = co_await _remote.segment_exists( get_bucket_name(), spath, fib); if (result == cloud_storage::download_result::notfound) { @@ -2402,7 +2402,8 @@ ss::future<> ntp_archiver::garbage_collect_archive() { continue; } if (meta.committed_offset < start_offset) { - const auto path = manifest.generate_segment_path(meta); + const auto path = manifest.generate_segment_path( + meta, remote_path_provider()); vlog( _rtclog.info, "Enqueuing spillover segment delete from cloud " @@ -2437,7 +2438,8 @@ ss::future<> ntp_archiver::garbage_collect_archive() { if (stop) { break; } - const auto path = cursor->manifest()->get_manifest_path(); + const auto path = cursor->manifest()->get_manifest_path( + remote_path_provider()); vlog( _rtclog.info, "Enqueuing spillover manifest delete from cloud " @@ -2625,8 +2627,9 @@ ss::future<> ntp_archiver::apply_spillover() { retry_chain_node upload_rtc( manifest_upload_timeout, manifest_upload_backoff, &_rtcnode); + const auto path = tail.get_manifest_path(remote_path_provider()); auto res = co_await _remote.upload_manifest( - get_bucket_name(), tail, upload_rtc); + get_bucket_name(), tail, path, upload_rtc); if (res != cloud_storage::upload_result::success) { vlog(_rtclog.error, "Failed to upload spillover manifest {}", res); co_return; @@ -2635,7 +2638,7 @@ ss::future<> ntp_archiver::apply_spillover() { // Put manifest into cache to avoid roundtrip to the cloud storage auto reservation = co_await _cache.reserve_space(len, 1); co_await _cache.put( - tail.get_manifest_path()(), + tail.get_manifest_path(remote_path_provider())(), str, reservation, _conf->upload_io_priority); @@ -2672,7 +2675,7 @@ ss::future<> ntp_archiver::apply_spillover() { vlog( _rtclog.info, "Uploaded spillover manifest: {}", - tail.get_manifest_path()); + tail.get_manifest_path(remote_path_provider())); } } } @@ -2864,7 +2867,8 @@ ss::future<> ntp_archiver::garbage_collect() { std::deque objects_to_remove; for (const auto& meta : to_remove) { - const auto path = manifest().generate_segment_path(meta); + const auto path = manifest().generate_segment_path( + meta, remote_path_provider()); vlog(_rtclog.info, "Deleting segment from cloud storage: {}", path); objects_to_remove.emplace_back(path); @@ -3257,6 +3261,11 @@ const storage::ntp_config& ntp_archiver::ntp_config() const { return _parent.log()->config(); } +const cloud_storage::remote_path_provider& +ntp_archiver::remote_path_provider() const { + return _parent.archival_meta_stm()->path_provider(); +} + void ntp_archiver::complete_transfer_leadership() { vlog( _rtclog.trace, diff --git a/src/v/archival/ntp_archiver_service.h b/src/v/archival/ntp_archiver_service.h index 97fad4d23a73..e3574d8488b7 100644 --- a/src/v/archival/ntp_archiver_service.h +++ b/src/v/archival/ntp_archiver_service.h @@ -17,6 +17,7 @@ #include "cloud_storage/fwd.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment_index.h" #include "cloud_storage/types.h" #include "cluster/fwd.h" @@ -371,6 +372,7 @@ class ntp_archiver { void complete_transfer_leadership(); const storage::ntp_config& ntp_config() const; + const cloud_storage::remote_path_provider& remote_path_provider() const; /// If we have a projected manifest clean offset, then flush it to /// the persistent stm clean offset. From 160a40e3c8f64161603e4366c27fe7ca6f77480a Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 19 Jun 2024 17:28:26 -0700 Subject: [PATCH 05/34] cloud_storage: plumb path provider into async manifest view This plumbs the archival stm's path provider into the manifest view. The manifest view serves as the underlying abstraction to the remote_partition class, so this will be necessary for getting the path provider to be used on the remote read path. --- .../tests/ntp_archiver_reupload_test.cc | 8 ++- src/v/archival/tests/ntp_archiver_test.cc | 50 +++++++++++++------ src/v/cloud_storage/async_manifest_view.cc | 5 +- src/v/cloud_storage/async_manifest_view.h | 9 +++- .../tests/async_manifest_view_test.cc | 3 +- .../tests/remote_partition_fuzz_test.cc | 6 ++- .../tests/remote_partition_test.cc | 26 ++++++---- src/v/cloud_storage/tests/util.cc | 26 ++++++++-- src/v/cluster/partition.cc | 3 +- 9 files changed, 100 insertions(+), 36 deletions(-) diff --git a/src/v/archival/tests/ntp_archiver_reupload_test.cc b/src/v/archival/tests/ntp_archiver_reupload_test.cc index f6af3a30cedb..beea37b2d417 100644 --- a/src/v/archival/tests/ntp_archiver_reupload_test.cc +++ b/src/v/archival/tests/ntp_archiver_reupload_test.cc @@ -12,6 +12,7 @@ #include "archival/tests/service_fixture.h" #include "cloud_storage/async_manifest_view.h" #include "cloud_storage/read_path_probes.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage_clients/client_pool.h" #include "config/configuration.h" #include "storage/ntp_config.h" @@ -24,6 +25,10 @@ using namespace archival; inline ss::logger test_log("test"); +namespace { +cloud_storage::remote_path_provider path_provider(std::nullopt); +} // anonymous namespace + static const auto manifest_namespace = model::ns("kafka"); static const auto manifest_topic = model::topic("test-topic"); static const auto manifest_partition = model::partition_id(42); @@ -208,7 +213,8 @@ struct reupload_fixture : public archiver_fixture { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archiver.emplace( get_ntp_conf(), diff --git a/src/v/archival/tests/ntp_archiver_test.cc b/src/v/archival/tests/ntp_archiver_test.cc index 3e0af5a67224..2a66bc0ecff1 100644 --- a/src/v/archival/tests/ntp_archiver_test.cc +++ b/src/v/archival/tests/ntp_archiver_test.cc @@ -19,6 +19,7 @@ #include "cloud_storage/fwd.h" #include "cloud_storage/read_path_probes.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/tests/manual_fixture.h" #include "cloud_storage/types.h" #include "cloud_storage_clients/client_pool.h" @@ -50,6 +51,10 @@ using namespace archival; inline ss::logger test_log("test"); // NOLINT +namespace { +cloud_storage::remote_path_provider path_provider(std::nullopt); +} // anonymous namespace + static ss::abort_source never_abort; static const auto manifest_namespace = model::ns("kafka"); // NOLINT @@ -156,7 +161,8 @@ FIXTURE_TEST(test_upload_segments, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -287,7 +293,8 @@ FIXTURE_TEST(test_upload_after_failure, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -381,7 +388,8 @@ FIXTURE_TEST( remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -465,7 +473,8 @@ FIXTURE_TEST(test_retention, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -586,7 +595,8 @@ FIXTURE_TEST(test_archive_retention, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -765,7 +775,8 @@ FIXTURE_TEST(test_segments_pending_deletion_limit, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -1170,7 +1181,8 @@ FIXTURE_TEST(test_upload_segments_leadership_transfer, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -1397,7 +1409,8 @@ static void test_partial_upload_impl( test.remote, test.app.shadow_index_cache, part->archival_meta_stm()->manifest(), - aconf->bucket_name); + aconf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), @@ -1768,7 +1781,8 @@ static void test_manifest_spillover_impl( test.remote, test.app.shadow_index_cache, part->archival_meta_stm()->manifest(), - aconf->bucket_name); + aconf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), @@ -1881,7 +1895,8 @@ FIXTURE_TEST(test_upload_with_gap_blocked, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), @@ -2092,7 +2107,8 @@ FIXTURE_TEST(test_flush_wait_out_of_bounds, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -2144,7 +2160,8 @@ FIXTURE_TEST(test_flush_wait_with_no_flush, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -2193,7 +2210,8 @@ FIXTURE_TEST(test_flush_wait_with_flush, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -2269,7 +2287,8 @@ FIXTURE_TEST(test_flush_wait_with_flush_multiple_waiters, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, @@ -2352,7 +2371,8 @@ FIXTURE_TEST(test_flush_with_leadership_change, archiver_fixture) { remote, app.shadow_index_cache, part->archival_meta_stm()->manifest(), - arch_conf->bucket_name); + arch_conf->bucket_name, + path_provider); archival::ntp_archiver archiver( get_ntp_conf(), arch_conf, diff --git a/src/v/cloud_storage/async_manifest_view.cc b/src/v/cloud_storage/async_manifest_view.cc index 044ca19dee81..f1a63c9aa6c8 100644 --- a/src/v/cloud_storage/async_manifest_view.cc +++ b/src/v/cloud_storage/async_manifest_view.cc @@ -16,6 +16,7 @@ #include "cloud_storage/partition_manifest.h" #include "cloud_storage/read_path_probes.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "cloud_storage_clients/types.h" @@ -391,8 +392,10 @@ async_manifest_view::async_manifest_view( ss::sharded& remote, ss::sharded& cache, const partition_manifest& stm_manifest, - cloud_storage_clients::bucket_name bucket) + cloud_storage_clients::bucket_name bucket, + const remote_path_provider& path_provider) : _bucket(bucket) + , _remote_path_provider(path_provider) , _remote(remote) , _cache(cache) , _ts_probe(remote.local().materialized().get_read_path_probe()) diff --git a/src/v/cloud_storage/async_manifest_view.h b/src/v/cloud_storage/async_manifest_view.h index cf3e59930ce3..5023970e9b32 100644 --- a/src/v/cloud_storage/async_manifest_view.h +++ b/src/v/cloud_storage/async_manifest_view.h @@ -14,6 +14,7 @@ #include "cloud_storage/fwd.h" #include "cloud_storage/materialized_manifest_cache.h" #include "cloud_storage/read_path_probes.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_probe.h" #include "cloud_storage/types.h" #include "cloud_storage_clients/types.h" @@ -100,7 +101,8 @@ class async_manifest_view { ss::sharded& remote, ss::sharded& cache, const partition_manifest& stm_manifest, - cloud_storage_clients::bucket_name bucket); + cloud_storage_clients::bucket_name bucket, + const remote_path_provider& path_provider); ss::future<> start(); ss::future<> stop(); @@ -158,6 +160,10 @@ class async_manifest_view { std::optional size_limit, std::optional time_limit) noexcept; + const remote_path_provider& path_provider() const { + return _remote_path_provider; + } + private: ss::future> time_based_retention(std::chrono::milliseconds time_limit) noexcept; @@ -222,6 +228,7 @@ class async_manifest_view { mutable ss::gate _gate; ss::abort_source _as; cloud_storage_clients::bucket_name _bucket; + const remote_path_provider& _remote_path_provider; ss::sharded& _remote; ss::sharded& _cache; ts_read_path_probe& _ts_probe; diff --git a/src/v/cloud_storage/tests/async_manifest_view_test.cc b/src/v/cloud_storage/tests/async_manifest_view_test.cc index f82837218379..b1c93d86b500 100644 --- a/src/v/cloud_storage/tests/async_manifest_view_test.cc +++ b/src/v/cloud_storage/tests/async_manifest_view_test.cc @@ -43,6 +43,7 @@ using eof = async_manifest_view_cursor::eof; static ss::logger test_log("async_manifest_view_log"); static const model::initial_revision_id manifest_rev(111); +static const remote_path_provider path_provider(std::nullopt); class set_config_mixin { public: @@ -71,7 +72,7 @@ class async_manifest_view_fixture , rtc(as) , ctxlog(test_log, rtc) , probe(manifest_ntp) - , view(api, cache, stm_manifest, bucket) { + , view(api, cache, stm_manifest, bucket, path_provider) { stm_manifest.set_archive_start_offset( model::offset{0}, model::offset_delta{0}); stm_manifest.set_archive_clean_offset(model::offset{0}, 0); diff --git a/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc b/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc index 4d82026c543a..25dba4b23c52 100644 --- a/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc +++ b/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc @@ -24,6 +24,8 @@ using namespace cloud_storage; +static const remote_path_provider path_provider(std::nullopt); + inline ss::logger test_log("test"); // NOLINT static std::vector @@ -52,7 +54,7 @@ scan_remote_partition_incrementally_with_reuploads( auto manifest = hydrate_manifest(fixt.api.local(), fixt.bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - fixt.api, fixt.cache, manifest, fixt.bucket_name); + fixt.api, fixt.cache, manifest, fixt.bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, fixt.api.local(), @@ -463,7 +465,7 @@ FIXTURE_TEST(test_scan_while_shutting_down, cloud_storage_fixture) { auto manifest = hydrate_manifest(api.local(), bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), this->cache.local(), bucket_name, probe); partition->start().get(); diff --git a/src/v/cloud_storage/tests/remote_partition_test.cc b/src/v/cloud_storage/tests/remote_partition_test.cc index 5047e8a95cb1..5f11aefb0039 100644 --- a/src/v/cloud_storage/tests/remote_partition_test.cc +++ b/src/v/cloud_storage/tests/remote_partition_test.cc @@ -73,6 +73,8 @@ static void print_segments(const std::vector& segments) { } } +static const remote_path_provider path_provider(std::nullopt); + /// Return vector which have a value for every recrod_batch_header in /// 'segments' If i'th value is true then the value are present in both /// 'headers' and 'segments' Otherwise the i'th value will be false. @@ -145,7 +147,7 @@ static model::record_batch_header read_single_batch_from_remote_partition( auto manifest = hydrate_manifest(fixture.api.local(), fixture.bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - fixture.api, fixture.cache, manifest, fixture.bucket_name); + fixture.api, fixture.cache, manifest, fixture.bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, fixture.api.local(), @@ -213,7 +215,7 @@ FIXTURE_TEST( auto manifest = hydrate_manifest(api.local(), bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -267,7 +269,7 @@ test_remote_partition_cache_size_estimate_materialized_segments_args( auto manifest = hydrate_manifest(api.local(), context.bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, context.bucket_name); + api, cache, manifest, context.bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), context.bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -1032,7 +1034,7 @@ FIXTURE_TEST(test_remote_partition_read_cached_index, cloud_storage_fixture) { { partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer( @@ -1056,7 +1058,7 @@ FIXTURE_TEST(test_remote_partition_read_cached_index, cloud_storage_fixture) { { partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer( @@ -1124,7 +1126,7 @@ FIXTURE_TEST(test_remote_partition_concurrent_truncate, cloud_storage_fixture) { partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -1227,7 +1229,7 @@ FIXTURE_TEST( partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -1316,7 +1318,7 @@ FIXTURE_TEST( partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -1523,7 +1525,7 @@ FIXTURE_TEST(test_remote_partition_abort_eos_race, cloud_storage_fixture) { auto manifest = hydrate_manifest(api.local(), bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - api, cache, manifest, bucket_name); + api, cache, manifest, bucket_name, path_provider); auto partition = ss::make_shared( manifest_view, api.local(), cache.local(), bucket_name, probe); auto partition_stop = ss::defer([&partition] { partition->stop().get(); }); @@ -2015,7 +2017,11 @@ std::vector scan_remote_partition_with_replacements( partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - imposter.api, imposter.cache, manifest, imposter.bucket_name); + imposter.api, + imposter.cache, + manifest, + imposter.bucket_name, + path_provider); auto partition = ss::make_shared( manifest_view, diff --git a/src/v/cloud_storage/tests/util.cc b/src/v/cloud_storage/tests/util.cc index 1618b3e48493..5b5530d933f1 100644 --- a/src/v/cloud_storage/tests/util.cc +++ b/src/v/cloud_storage/tests/util.cc @@ -25,6 +25,8 @@ namespace cloud_storage { +static const remote_path_provider path_provider(std::nullopt); + segment_layout generate_segment_layout(int num_segments, int seed, bool exclude_tx_fence) { static constexpr size_t max_segment_size = 20; @@ -657,7 +659,11 @@ std::vector scan_remote_partition_incrementally( imposter.api.local(), imposter.bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - imposter.api, imposter.cache, manifest, imposter.bucket_name); + imposter.api, + imposter.cache, + manifest, + imposter.bucket_name, + path_provider); auto partition = ss::make_shared( manifest_view, imposter.api.local(), @@ -741,7 +747,11 @@ std::vector scan_remote_partition( imposter.api.local(), imposter.bucket_name); partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - imposter.api, imposter.cache, manifest, imposter.bucket_name); + imposter.api, + imposter.cache, + manifest, + imposter.bucket_name, + path_provider); auto manifest_view_stop = ss::defer( [&manifest_view] { manifest_view->stop().get(); }); manifest_view->start().get(); @@ -798,7 +808,11 @@ scan_result scan_remote_partition( partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - imposter.api, imposter.cache, manifest, imposter.bucket_name); + imposter.api, + imposter.cache, + manifest, + imposter.bucket_name, + path_provider); auto manifest_view_stop = ss::defer( [&manifest_view] { manifest_view->stop().get(); }); @@ -857,7 +871,11 @@ scan_remote_partition_incrementally_with_closest_lso( partition_probe probe(manifest.get_ntp()); auto manifest_view = ss::make_shared( - imposter.api, imposter.cache, manifest, imposter.bucket_name); + imposter.api, + imposter.cache, + manifest, + imposter.bucket_name, + path_provider); auto partition = ss::make_shared( manifest_view, diff --git a/src/v/cluster/partition.cc b/src/v/cluster/partition.cc index ec8d998b09e8..b56f071bfd10 100644 --- a/src/v/cluster/partition.cc +++ b/src/v/cluster/partition.cc @@ -422,7 +422,8 @@ ss::future<> partition::start(state_machine_registry& stm_registry) { _cloud_storage_api, _cloud_storage_cache, _archival_meta_stm->manifest(), - cloud_storage_clients::bucket_name{*bucket}); + cloud_storage_clients::bucket_name{*bucket}, + _archival_meta_stm->path_provider()); _cloud_storage_partition = ss::make_shared( From 291ed3f36aa84d4778a03a127addec165b73b44b Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 14:20:08 -0700 Subject: [PATCH 06/34] cluster: use topic_manifest_downloader in topic recovery Topic recovery need to be able to find a topic manifest whose format (serde or json) is not yet known. This is handled transparently by the topic manifest downloader, while also taking into account remote labels. This commit updates the create topic path to use the downloader. --- .../remote_topic_configuration_source.cc | 90 ++++++------------- 1 file changed, 25 insertions(+), 65 deletions(-) diff --git a/src/v/cluster/remote_topic_configuration_source.cc b/src/v/cluster/remote_topic_configuration_source.cc index e8555101fd83..438dae88353b 100644 --- a/src/v/cluster/remote_topic_configuration_source.cc +++ b/src/v/cluster/remote_topic_configuration_source.cc @@ -13,11 +13,13 @@ #include "cloud_storage/remote.h" #include "cloud_storage/topic_manifest.h" +#include "cloud_storage/topic_manifest_downloader.h" #include "cloud_storage/types.h" #include "cloud_storage_clients/configuration.h" #include "cluster/logger.h" #include "cluster/types.h" #include "config/configuration.h" +#include "model/timeout_clock.h" namespace cluster { @@ -25,8 +27,7 @@ remote_topic_configuration_source::remote_topic_configuration_source( cloud_storage::remote& remote) : _remote(remote) {} -static ss::future> -download_topic_manifest( +static ss::future download_topic_manifest( cloud_storage::remote& remote, custom_assignable_topic_configuration& cfg, cloud_storage::topic_manifest& manifest, @@ -35,59 +36,24 @@ download_topic_manifest( auto timeout = config::shard_local_cfg().cloud_storage_manifest_upload_timeout_ms(); auto backoff = config::shard_local_cfg().cloud_storage_initial_backoff_ms(); - retry_chain_node rc_node(as, timeout, backoff); - - model::ns ns = cfg.cfg.tp_ns.ns; - model::topic topic = cfg.cfg.tp_ns.tp; - auto serde_path = std::pair{ - cloud_storage::manifest_format::serde, - cloud_storage::topic_manifest::get_topic_manifest_path( - ns, topic, cloud_storage::manifest_format::serde), - }; - // try serde first - auto res = co_await remote.download_manifest( - bucket, serde_path, manifest, rc_node); - - if (res == cloud_storage::download_result::success) { - co_return std::make_tuple(errc::success, serde_path.second); - } - - if (res != cloud_storage::download_result::notfound) { - vlog( - clusterlog.warn, - "Could not download topic manifest {} from bucket {}: {}", - serde_path.second, - bucket, - res); - co_return std::make_tuple( - errc::topic_operation_error, serde_path.second); - } - - vlog( - clusterlog.debug, - "Could not find serde manifest from bucket {}: {}. trying json", + retry_chain_node retry_node(as); + cloud_storage::topic_manifest_downloader dl( bucket, - serde_path.second); - - // no serde manifest and no generic error. try to fallback to json - auto json_path = std::pair{ - cloud_storage::manifest_format::json, - cloud_storage::topic_manifest::get_topic_manifest_path( - ns, topic, cloud_storage::manifest_format::json)}; - res = co_await remote.download_manifest_json( - bucket, json_path.second, manifest, rc_node); - - if (res == cloud_storage::download_result::success) { - co_return std::make_tuple(errc::success, json_path.second); + /*remote_label=*/std::nullopt, + cfg.cfg.tp_ns, + remote); + auto deadline = model::timeout_clock::now() + timeout; + auto download_res = co_await dl.download_manifest( + retry_node, deadline, backoff, &manifest); + if (download_res.has_error()) { + co_return errc::topic_operation_error; } - - vlog( - clusterlog.warn, - "Could not download topic manifest {} from bucket {}: {}", - json_path.second, - bucket, - res); - co_return std::make_tuple(errc::topic_operation_error, json_path.second); + if ( + download_res.value() + != cloud_storage::find_topic_manifest_outcome::success) { + co_return errc::topic_operation_error; + } + co_return errc::success; } ss::future @@ -97,22 +63,19 @@ remote_topic_configuration_source::set_remote_properties_in_config( ss::abort_source& as) { cloud_storage::topic_manifest manifest; - auto [res, key] = co_await download_topic_manifest( + auto res = co_await download_topic_manifest( _remote, cfg, manifest, bucket, as); if (res != errc::success) { co_return res; } if (!manifest.get_topic_config()) { - vlog( - clusterlog.warn, - "Topic manifest {} doesn't contain topic config", - key); co_return errc::topic_operation_error; } else { + const auto& dl_cfg = manifest.get_topic_config(); cfg.cfg.properties.remote_topic_properties = remote_topic_properties( - manifest.get_revision(), - manifest.get_topic_config()->partition_count); + manifest.get_revision(), dl_cfg->partition_count); + cfg.cfg.properties.remote_label = dl_cfg->properties.remote_label; } co_return errc::success; } @@ -140,17 +103,13 @@ remote_topic_configuration_source::set_recovered_topic_properties( ss::abort_source& as) { cloud_storage::topic_manifest manifest; - auto [res, key] = co_await download_topic_manifest( + auto res = co_await download_topic_manifest( _remote, cfg, manifest, bucket, as); if (res != errc::success) { co_return res; } if (!manifest.get_topic_config()) { - vlog( - clusterlog.warn, - "Topic manifest {} doesn't contain topic config", - key); co_return errc::topic_operation_error; } else { // Update all topic properties @@ -163,6 +122,7 @@ remote_topic_configuration_source::set_recovered_topic_properties( cfg.cfg.properties.remote_topic_properties = remote_topic_properties( manifest.get_revision(), manifest.get_topic_config()->partition_count); + cfg.cfg.properties.remote_label = rc.value().properties.remote_label; } co_return errc::success; } From b6d1e162ead195831204b214eb21bca17bbec76d Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 24 Jun 2024 18:22:39 -0700 Subject: [PATCH 07/34] cluster: use topic_manifest_downloader in list-based recovery Automated topic recovery performs a list bucket operation to discover what topics can be restored. This commit updates this to account for topic manifests labeled with the cluster UUID. --- .../tests/topic_recovery_service_test.cc | 38 ++-- src/v/cluster/topic_recovery_service.cc | 214 +++++------------- src/v/cluster/topic_recovery_service.h | 8 +- 3 files changed, 81 insertions(+), 179 deletions(-) diff --git a/src/v/cloud_storage/tests/topic_recovery_service_test.cc b/src/v/cloud_storage/tests/topic_recovery_service_test.cc index 636ab13ab2db..0b62e56c73bb 100644 --- a/src/v/cloud_storage/tests/topic_recovery_service_test.cc +++ b/src/v/cloud_storage/tests/topic_recovery_service_test.cc @@ -112,6 +112,15 @@ generate_no_manifests_expectations( .body = no_manifests, }); } + expectations.emplace_back(s3_imposter_fixture::expectation{ + .url = fmt::format( + "?list-type=2&prefix=meta/{}/{}/", tp_ns.ns(), tp_ns.tp()), + .body = no_manifests, + }); + expectations.emplace_back(s3_imposter_fixture::expectation{ + .url = "?list-type=2&prefix=meta/", + .body = no_manifests, + }); for (auto& e : additional_expectations) { expectations.emplace_back(std::move(e)); } @@ -228,7 +237,7 @@ FIXTURE_TEST(recovery_with_no_topics_exits_early, fixture) { const auto& list_topics_req = get_requests()[0]; BOOST_REQUIRE_EQUAL( - list_topics_req.url, "/" + url_base() + "?list-type=2&prefix=00000000/"); + list_topics_req.url, "/" + url_base() + "?list-type=2&prefix=meta/"); // Wait until recovery exits after finding no topics to create tests::cooperative_spin_wait_with_timeout(10s, [&service] { @@ -236,7 +245,7 @@ FIXTURE_TEST(recovery_with_no_topics_exits_early, fixture) { }).get(); // No other calls were made - BOOST_REQUIRE_EQUAL(get_requests().size(), 16); + BOOST_REQUIRE_EQUAL(get_requests().size(), 17); } void do_test(fixture& f) { @@ -250,20 +259,20 @@ void do_test(fixture& f) { BOOST_REQUIRE_EQUAL(result, expected); // Wait until three requests are received: - // 1..16. to list bucket for topic meta prefixes - // 17. to download manifest - f.wait_for_n_requests(17, fixture::equals::yes); + // 1. meta/kafka for labeled topic manifests + // 2..17. to list bucket for topic meta prefixes + // 18..20. to download manifest, which now takes three requests + f.wait_for_n_requests(20, fixture::equals::yes); - const auto& get_manifest_req = f.get_requests()[16]; + const auto& get_manifest_req = f.get_requests()[19]; BOOST_REQUIRE_EQUAL( get_manifest_req.url, "/" + f.url_base() + manifest.url); // Wait until recovery exits after finding no topics to create - tests::cooperative_spin_wait_with_timeout(10s, [&service] { - return service.local().is_active() == false; - }).get(); + RPTEST_REQUIRE_EVENTUALLY( + 10s, [&service] { return service.local().is_active() == false; }); - BOOST_REQUIRE_EQUAL(f.get_requests().size(), 17); + BOOST_REQUIRE_EQUAL(f.get_requests().size(), 20); } FIXTURE_TEST(recovery_with_unparseable_topic_manifest, fixture) { @@ -360,9 +369,10 @@ FIXTURE_TEST(recovery_result_clear_before_start, fixture) { start_recovery(); wait_for_n_requests(22); - // 16 to check each manifest prefix, 1 to download the topic manifest, 1 to - // check recovery results, 1 to delete. - const auto& delete_request = get_requests()[18]; + // 1 to check the labeled root, 16 to check each manifest prefix, 3 to + // download the JSON topic manifest, 1 to check recovery results, 1 to + // delete. + const auto& delete_request = get_requests()[21]; BOOST_REQUIRE_EQUAL(delete_request.url, "/" + url_base() + "?delete"); BOOST_REQUIRE_EQUAL(delete_request.method, "POST"); } @@ -404,7 +414,7 @@ FIXTURE_TEST(recovery_with_topic_name_pattern_without_match, fixture) { return !service.local().is_active(); }).get(); - BOOST_REQUIRE_EQUAL(get_requests().size(), 16); + BOOST_REQUIRE_EQUAL(get_requests().size(), 17); } FIXTURE_TEST(recovery_with_topic_name_pattern_with_match, fixture) { diff --git a/src/v/cluster/topic_recovery_service.cc b/src/v/cluster/topic_recovery_service.cc index efc69d659d0b..80321aa99343 100644 --- a/src/v/cluster/topic_recovery_service.cc +++ b/src/v/cluster/topic_recovery_service.cc @@ -14,10 +14,13 @@ #include "cloud_storage/recovery_request.h" #include "cloud_storage/recovery_utils.h" #include "cloud_storage/topic_manifest.h" +#include "cloud_storage/topic_manifest_downloader.h" #include "cluster/topic_recovery_status_frontend.h" #include "cluster/topics_frontend.h" #include "cluster/types.h" +#include +#include #include #include @@ -31,12 +34,6 @@ namespace { -const std::regex manifest_path_expr{ - R"REGEX(\w+/meta/(.*?)/(.*?)/topic_manifest\.(json|bin))REGEX"}; - -// Possible prefix for a path which contains a topic manifest file -const std::regex prefix_expr{"[a-fA-F0-9]0000000/"}; - constexpr size_t list_api_timeout_multiplier{10}; constexpr ss::lowres_clock::duration downloads_check_interval{60s}; @@ -226,50 +223,6 @@ topic_recovery_service::recovery_status_log() const { return {_status_log.begin(), _status_log.end()}; } -// NOTE rewritten as continuations to address arm64 miscompilation of coroutines -// under clang-14 -static ss::future> collect_manifest_paths( - remote& remote, ss::abort_source& as, const recovery_task_config& cfg) { - // Look under each manifest prefix for topic manifests. - constexpr static auto hex_chars = std::string_view{"0123456789abcdef"}; - return ss::do_with(std::vector{}, [&](auto& paths) { - return ss::do_for_each( - hex_chars, - [&](char hex_ch) { - return ss::do_with( - std::make_unique( - as, cfg.operation_timeout_ms, cfg.backoff_ms), - fmt::format("{}0000000/", hex_ch), - [&](auto& rtc, auto& prefix) { - return remote - .list_objects( - cfg.bucket, - *rtc, - cloud_storage_clients::object_key{prefix}) - .then([&](auto meta) { - if (meta.has_error()) { - vlog( - cst_log.error, - "Failed to list meta items: {}", - meta.error()); - return; - } - - for (auto&& item : meta.value().contents) { - vlog( - cst_log.trace, - "adding path {} for {}", - item.key, - prefix); - paths.emplace_back(item.key); - } - }); - }); - }) - .then([&] { return std::move(paths); }); - }); -} - ss::future> topic_recovery_service::start_bg_recovery_task(recovery_request request) { vlog(cst_log.info, "Starting recovery task with request: {}", request); @@ -298,12 +251,59 @@ topic_recovery_service::start_bg_recovery_task(recovery_request request) { _recovery_request.emplace(request); set_state(state::scanning_bucket); + vlog(cst_log.debug, "scanning bucket {}", _config.bucket); - auto bucket_contents = co_await collect_manifest_paths( - _remote.local(), _as, _config); + auto fib = make_rtc(_as, _config); + std::optional requested_pattern = std::nullopt; + if (request.topic_names_pattern().has_value()) { + requested_pattern.emplace( + request.topic_names_pattern().value().data(), + request.topic_names_pattern().value().size()); + } + const auto requested_topic = + [&requested_pattern](const model::topic_namespace& topic) { + if (!requested_pattern) { + return true; + } + return std::regex_search(topic.tp().c_str(), *requested_pattern); + }; + + absl::flat_hash_set existing_topics; + for (auto topic : _topic_state.local().all_topics()) { + if (requested_topic(topic)) { + existing_topics.emplace(std::move(topic)); + } + } - auto manifests = co_await filter_existing_topics( - bucket_contents, request, model::ns{"kafka"}); + auto should_create = [&requested_topic, &existing_topics]( + const model::topic_namespace& topic) { + return requested_topic(topic) && !existing_topics.contains(topic); + }; + + chunked_vector manifests; + auto res + = co_await cloud_storage::topic_manifest_downloader::find_manifests( + _remote.local(), + _config.bucket, + fib, + ss::lowres_clock::now() + _config.operation_timeout_ms, + 10ms, + std::move(should_create), + &manifests); + if (res.has_error()) { + _recovery_request = std::nullopt; + set_state(state::inactive); + co_return recovery_error_ctx::make( + fmt::format("failed to create topics: {}", res.error()), + recovery_error_code::error_creating_topics); + } + if (res.value() != find_topic_manifest_outcome::success) { + _recovery_request = std::nullopt; + set_state(state::inactive); + co_return recovery_error_ctx::make( + "failed to create topics", + recovery_error_code::error_creating_topics); + } if (manifests.empty()) { vlog(cst_log.info, "exiting recovery, no topics to create"); @@ -321,7 +321,7 @@ topic_recovery_service::start_bg_recovery_task(recovery_request request) { auto clear_fib = make_rtc(_as, _config); co_await clear_recovery_results( _remote.local(), _config.bucket, clear_fib, std::nullopt); - _downloaded_manifests.emplace(manifests); + _downloaded_manifests = std::move(manifests); populate_recovery_status(); @@ -426,110 +426,6 @@ topic_recovery_service::create_topics(const recovery_request& request) { config::shard_local_cfg().create_topic_timeout_ms()); } -ss::future> -topic_recovery_service::filter_existing_topics( - std::vector items, - const recovery_request& request, - std::optional) { - absl::flat_hash_map> - topic_index; - - for (const auto& topic : _topic_state.local().all_topics()) { - topic_index.try_emplace(topic.ns, absl::flat_hash_set{}); - topic_index[topic.ns].insert(topic.tp); - } - - std::vector manifests; - manifests.reserve(items.size()); - - std::optional requested_pattern = std::nullopt; - if (request.topic_names_pattern().has_value()) { - requested_pattern.emplace( - request.topic_names_pattern().value().data(), - request.topic_names_pattern().value().size()); - } - - for (const auto& item : items) { - // Although we filter for topic manifest pattern earlier, we still use - // this regex match here to extract the namespace and topic from the - // pattern. - std::smatch matches; - const auto& path = item().string(); - const auto is_topic_manifest = std::regex_match( - path.cbegin(), path.cend(), matches, manifest_path_expr); - if (!is_topic_manifest) { - continue; - } - - const auto& ns = matches[1].str(); - const auto& tp = matches[2].str(); - - if ( - requested_pattern.has_value() - && !std::regex_search(tp, requested_pattern.value())) { - vlog( - cst_log.debug, - "will skip topic {}, it does not match pattern {}", - tp, - request.topic_names_pattern().value()); - continue; - } - - if (topic_index.contains(ns) && topic_index[ns].contains(tp)) { - vlog( - cst_log.debug, - "will skip creating {}:{}, topic already exists", - ns, - tp); - continue; - } - - if (auto download_r = co_await download_manifest(path); - download_r.has_value()) { - manifests.push_back(std::move(download_r.value())); - } - } - co_return manifests; -} - -ss::future> -topic_recovery_service::download_manifest(ss::sstring path) { - cloud_storage::topic_manifest m; - auto fib = make_rtc(_as, _config); - auto expected_format = path.ends_with("json") ? manifest_format::json - : manifest_format::serde; - try { - auto download_r = co_await _remote.local().download_manifest( - _config.bucket, - {expected_format, remote_manifest_path{path}}, - m, - fib); - if (download_r != download_result::success) { - auto error = recovery_error_ctx::make( - fmt::format( - "failed to download manifest from {} format {}: {}", - path, - expected_format, - download_r), - recovery_error_code::error_downloading_manifest); - vlog(cst_log.error, "{}", error.context); - co_return error; - } - co_return m; - - } catch (const std::exception& ex) { - auto error = recovery_error_ctx::make( - fmt::format( - "failed to download manifest from {} format {}: {}", - path, - expected_format, - ex.what()), - recovery_error_code::error_downloading_manifest); - vlog(cst_log.error, "{}", error.context); - co_return error; - } -} - void topic_recovery_service::start_download_bg_tracker() { _pending_status_timer.set_callback([this] { ssx::spawn_with_gate(_gate, [this] { return check_for_downloads(); }); diff --git a/src/v/cluster/topic_recovery_service.h b/src/v/cluster/topic_recovery_service.h index 59fa0d3e983b..21c8f6f8bc00 100644 --- a/src/v/cluster/topic_recovery_service.h +++ b/src/v/cluster/topic_recovery_service.h @@ -15,6 +15,7 @@ #include "cloud_storage/remote.h" #include "cloud_storage/topic_manifest.h" #include "cluster/types.h" +#include "container/fragmented_vector.h" #include "model/fundamental.h" #include @@ -127,11 +128,6 @@ struct topic_recovery_service const recovery_request& request, std::optional filter_ns); - /// \brief Try to download a manifest JSON file, parse it and return the - /// parsed manifest - ss::future> - download_manifest(ss::sstring path); - ss::future> create_topics(const recovery_request& request); @@ -192,7 +188,7 @@ struct topic_recovery_service // once the recovery has ended. One example is the topic retention which // could be set to some small value during recovery and restored back to // original value from manifest once recovery has ended. - std::optional> _downloaded_manifests; + std::optional> _downloaded_manifests; boost::circular_buffer _status_log; }; From 41df5633e0b22c9b8c9337f30000a0392651c971 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 22:38:53 -0700 Subject: [PATCH 08/34] archival_stm: use partition_manifest_downloader for snapshot recovery Snapshot recovery needs to be able to find a partition manifest whose format (serde or json) is not yet known. This is handled transparently by the partition manifest downloader, while also taking remote labels into account. This commit updates the STM to use the downloader for recovery. --- src/v/archival/archival_metadata_stm.cc | 31 ++++++++++++++----------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/src/v/archival/archival_metadata_stm.cc b/src/v/archival/archival_metadata_stm.cc index ef419e3484ee..17d05f9e93fa 100644 --- a/src/v/archival/archival_metadata_stm.cc +++ b/src/v/archival/archival_metadata_stm.cc @@ -14,6 +14,7 @@ #include "bytes/iobuf.h" #include "bytes/iostream.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_path_provider.h" #include "cloud_storage/types.h" @@ -1148,22 +1149,26 @@ ss::future<> archival_metadata_stm::apply_raft_snapshot(const iobuf&) { auto backoff = config::shard_local_cfg().cloud_storage_initial_backoff_ms(); retry_chain_node rc_node(_download_as, timeout, backoff); - auto [res, res_fmt] - = co_await _cloud_storage_api.try_download_partition_manifest( - cloud_storage_clients::bucket_name{*bucket}, new_manifest, rc_node); - - if (res == cloud_storage::download_result::notfound) { - set_next(_raft->start_offset()); - vlog(_logger.info, "handled log eviction, the manifest is absent"); - co_return; - } else if (res != cloud_storage::download_result::success) { + cloud_storage::partition_manifest_downloader dl( + cloud_storage_clients::bucket_name{*bucket}, + _remote_path_provider, + _manifest->get_ntp(), + _manifest->get_revision_id(), + _cloud_storage_api); + auto res = co_await dl.download_manifest(rc_node, &new_manifest); + if (res.has_error()) { // sleep to the end of timeout to avoid calling handle_eviction in a // busy loop. co_await ss::sleep_abortable(rc_node.get_timeout(), _download_as); - throw std::runtime_error{fmt::format( - "couldn't download manifest {}: {}", - new_manifest.get_manifest_path(res_fmt), - res)}; + throw std::runtime_error{ + fmt::format("couldn't download manifest: {}", res.error())}; + } + if ( + res.value() + == cloud_storage::find_partition_manifest_outcome::no_matching_manifest) { + set_next(_raft->start_offset()); + vlog(_logger.info, "handled log eviction, the manifest is absent"); + co_return; } *_manifest = std::move(new_manifest); From e16c2477df1c4cc2e9db3e94e5f179ef0a52d28a Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 14:33:58 -0700 Subject: [PATCH 09/34] ntp_archiver: use partition_manifest_downlaoder for read replicas Read replicas need to be able to find a partition manifest whose format (serde or json) is not yet known. This is handled transparently by the partition manifest downloader, while also taking remote labels into account. This commit updates the archiver to use the downloader for the read replica loop. --- src/v/archival/ntp_archiver_service.cc | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/v/archival/ntp_archiver_service.cc b/src/v/archival/ntp_archiver_service.cc index 7ac46586a2ce..a8e8b249bb44 100644 --- a/src/v/archival/ntp_archiver_service.cc +++ b/src/v/archival/ntp_archiver_service.cc @@ -21,6 +21,7 @@ #include "base/vlog.h" #include "cloud_storage/async_manifest_view.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" @@ -955,14 +956,20 @@ ntp_archiver::download_manifest() { &_rtcnode); cloud_storage::partition_manifest tmp(_ntp, _rev); vlog(_rtclog.debug, "Downloading manifest"); - auto [result, _] = co_await _remote.try_download_partition_manifest( - get_bucket_name(), tmp, fib); + cloud_storage::partition_manifest_downloader dl( + get_bucket_name(), remote_path_provider(), _ntp, _rev, _remote); + auto result = co_await dl.download_manifest(fib, &tmp); + if (result.has_error()) { + co_return std::make_pair( + std::move(tmp), cloud_storage::download_result::failed); + } // It's OK if the manifest is not found for a newly created topic. The // condition in if statement is not guaranteed to cover all cases for new // topics, so false positives may happen for this warn. if ( - result == cloud_storage::download_result::notfound + result.value() + == cloud_storage::find_partition_manifest_outcome::no_matching_manifest && _parent.high_watermark() != model::offset(0) && _parent.term() != model::term_id(1)) { vlog( @@ -972,9 +979,11 @@ ntp_archiver::download_manifest() { _ntp, _parent.high_watermark(), _parent.term()); + co_return std::make_pair( + std::move(tmp), cloud_storage::download_result::notfound); } - - co_return std::make_pair(std::move(tmp), result); + co_return std::make_pair( + std::move(tmp), cloud_storage::download_result::success); } /** From 13a1567442ae03f9b33964a3ec0628a9e2ae55da Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 14:14:06 -0700 Subject: [PATCH 10/34] cluster: use partition_manifest_downloader in partition recovery The partition recovery manager downloads the partition manifest and remote segments to be able to create a "deep" copy of the remote replica that is served by a local Raft log. This recovery needs to be able to find a partition manifest whose format (serde or json) is not yet known. This is handled transparently by the partition manifest downloader, while also taking remote labels into account. On top of this, with the naming format of all objects (including segments) changing, the recovery needs to be able to know what remote label to use, if any. To that end, this commit updates the partition recovery path to use a path provider and partition manifest downloader. Note, while the source cluster uuid of the path providers will ultimately be the same, this is a different path provider than what will be used at runtime by the partition. The latter is owned by the archival metadata STM and its lifecycle is therefore tied to the partition. The former is transient and only lives for as long as the recovery is happening (it is constructed slightly before the partition is created). --- src/v/cluster/controller_backend.cc | 7 ++- src/v/cluster/partition_manager.cc | 17 ++++++-- src/v/cluster/partition_manager.h | 8 +++- src/v/cluster/partition_recovery_manager.cc | 48 +++++++++++++-------- src/v/cluster/partition_recovery_manager.h | 7 ++- 5 files changed, 62 insertions(+), 25 deletions(-) diff --git a/src/v/cluster/controller_backend.cc b/src/v/cluster/controller_backend.cc index c61eebcd241c..a171d3888a9e 100644 --- a/src/v/cluster/controller_backend.cc +++ b/src/v/cluster/controller_backend.cc @@ -12,6 +12,7 @@ #include "archival/archival_metadata_stm.h" #include "base/outcome.h" #include "base/vassert.h" +#include "cloud_storage/remote_path_provider.h" #include "cluster/cluster_utils.h" #include "cluster/errc.h" #include "cluster/fwd.h" @@ -36,6 +37,7 @@ #include "ssx/event.h" #include "ssx/future-util.h" #include "storage/offset_translator.h" +#include "types.h" #include #include @@ -1402,7 +1404,10 @@ ss::future controller_backend::create_partition( group_id, std::move(initial_brokers), cfg->properties.remote_topic_properties, - read_replica_bucket); + read_replica_bucket, + raft::with_learner_recovery_throttle::yes, + raft::keep_snapshotted_log::no, + cfg->properties.remote_label); co_await add_to_shard_table( ntp, group_id, ss::this_shard_id(), log_revision); diff --git a/src/v/cluster/partition_manager.cc b/src/v/cluster/partition_manager.cc index d94fe6748103..b99586a1b7aa 100644 --- a/src/v/cluster/partition_manager.cc +++ b/src/v/cluster/partition_manager.cc @@ -16,7 +16,9 @@ #include "cloud_storage/cache_service.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_label.h" #include "cloud_storage/remote_partition.h" +#include "cloud_storage/remote_path_provider.h" #include "cluster/fwd.h" #include "cluster/logger.h" #include "cluster/partition.h" @@ -117,9 +119,11 @@ ss::future partition_manager::manage( std::optional rtp, std::optional read_replica_bucket, raft::with_learner_recovery_throttle enable_learner_recovery_throttle, - raft::keep_snapshotted_log keep_snapshotted_log) { + raft::keep_snapshotted_log keep_snapshotted_log, + std::optional remote_label) { auto guard = _gate.hold(); - auto dl_result = co_await maybe_download_log(ntp_cfg, rtp); + cloud_storage::remote_path_provider path_provider(remote_label); + auto dl_result = co_await maybe_download_log(ntp_cfg, rtp, path_provider); auto& [logs_recovered, clean_download, min_offset, max_offset, manifest, ot_state] = dl_result; if (logs_recovered) { @@ -266,10 +270,15 @@ ss::future partition_manager::manage( ss::future partition_manager::maybe_download_log( - storage::ntp_config& ntp_cfg, std::optional rtp) { + storage::ntp_config& ntp_cfg, + std::optional rtp, + cloud_storage::remote_path_provider& path_provider) { if (rtp.has_value() && _partition_recovery_mgr.local_is_initialized()) { auto res = co_await _partition_recovery_mgr.local().download_log( - ntp_cfg, rtp->remote_revision, rtp->remote_partition_count); + ntp_cfg, + rtp->remote_revision, + rtp->remote_partition_count, + path_provider); co_return res; } vlog( diff --git a/src/v/cluster/partition_manager.h b/src/v/cluster/partition_manager.h index 7730cf42f738..4997491803a0 100644 --- a/src/v/cluster/partition_manager.h +++ b/src/v/cluster/partition_manager.h @@ -13,6 +13,7 @@ #include "archival/fwd.h" #include "cloud_storage/fwd.h" +#include "cloud_storage/remote_path_provider.h" #include "cluster/fwd.h" #include "cluster/ntp_callbacks.h" #include "cluster/partition.h" @@ -94,7 +95,8 @@ class partition_manager std::optional = std::nullopt, raft::with_learner_recovery_throttle = raft::with_learner_recovery_throttle::yes, - raft::keep_snapshotted_log = raft::keep_snapshotted_log::no); + raft::keep_snapshotted_log = raft::keep_snapshotted_log::no, + std::optional = std::nullopt); ss::future<> shutdown(const model::ntp& ntp); @@ -247,7 +249,9 @@ class partition_manager /// \param ntp_cfg is an ntp_config instance to recover /// \return true if the recovery was invoked, false otherwise ss::future maybe_download_log( - storage::ntp_config& ntp_cfg, std::optional rtp); + storage::ntp_config& ntp_cfg, + std::optional rtp, + cloud_storage::remote_path_provider& path_provider); ss::future<> do_shutdown(ss::lw_shared_ptr); diff --git a/src/v/cluster/partition_recovery_manager.cc b/src/v/cluster/partition_recovery_manager.cc index fcc47165985d..75a192241f64 100644 --- a/src/v/cluster/partition_recovery_manager.cc +++ b/src/v/cluster/partition_recovery_manager.cc @@ -12,7 +12,10 @@ #include "bytes/streambuf.h" #include "cloud_storage/logger.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/recovery_utils.h" +#include "cloud_storage/remote_label.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/topic_manifest.h" #include "cloud_storage/types.h" #include "cluster/topic_recovery_status_frontend.h" @@ -85,7 +88,8 @@ ss::future<> partition_recovery_manager::stop() { ss::future partition_recovery_manager::download_log( const storage::ntp_config& ntp_cfg, model::initial_revision_id remote_revision, - int32_t remote_partition_count) { + int32_t remote_partition_count, + cloud_storage::remote_path_provider& path_provider) { if (!ntp_cfg.has_overrides()) { vlog( cst_log.debug, "No overrides for {} found, skipping", ntp_cfg.ntp()); @@ -101,6 +105,7 @@ ss::future partition_recovery_manager::download_log( } partition_downloader downloader( ntp_cfg, + path_provider, &_remote.local(), remote_revision, remote_partition_count, @@ -115,7 +120,7 @@ ss::future partition_recovery_manager::download_log( cst_log.debug, "topic recovery service is active, uploading result: {} for {}", result.logs_recovered, - result.manifest.get_manifest_path()); + result.manifest.get_manifest_path(path_provider)); co_await cloud_storage::place_download_result( _remote.local(), _bucket, ntp_cfg, result.logs_recovered, fib); } @@ -149,6 +154,7 @@ void partition_recovery_manager::set_topic_recovery_components( partition_downloader::partition_downloader( const storage::ntp_config& ntpc, + const cloud_storage::remote_path_provider& path_provider, remote* remote, model::initial_revision_id remote_rev_id, int32_t remote_partition_count, @@ -157,6 +163,7 @@ partition_downloader::partition_downloader( retry_chain_node& parent, storage::opt_abort_source_t as) : _ntpc(ntpc) + , _remote_path_provider(path_provider) , _bucket(std::move(bucket)) , _remote(remote) , _remote_revision_id(remote_rev_id) @@ -604,20 +611,27 @@ partition_downloader::find_recovery_material() { vlog( _ctxlog.info, "Downloading partition manifest {}", - tmp.get_manifest_path()); - auto [res, res_fmt] = co_await _remote->try_download_partition_manifest( - _bucket, tmp, _rtcnode); - if (res == download_result::success) { - recovery_mat.partition_manifest = std::move(tmp); - co_return recovery_mat; + tmp.get_manifest_path(_remote_path_provider)); + cloud_storage::partition_manifest_downloader dl( + _bucket, + _remote_path_provider, + _ntpc.ntp(), + _remote_revision_id, + *_remote); + auto download_res = co_await dl.download_manifest(_rtcnode, &tmp); + if (download_res.has_error()) { + throw std::runtime_error(fmt_with_ctx( + fmt::format, "Can't download manifest: {}", download_res.error())); } - if (res == download_result::notfound) { - // Manifest is not available in the cloud - throw missing_partition_exception(tmp.get_manifest_path(res_fmt)); + if ( + download_res.value() + == find_partition_manifest_outcome::no_matching_manifest) { + throw missing_partition_exception( + remote_manifest_path{_remote_path_provider.partition_manifest_path( + _ntpc.ntp(), _remote_revision_id)}); } - // Some other, possibly transient error - throw std::runtime_error( - fmt_with_ctx(fmt::format, "Can't download manifest: {}", res)); + recovery_mat.partition_manifest = std::move(tmp); + co_return recovery_mat; } static ss::future> @@ -669,9 +683,6 @@ partition_downloader::download_segment_file( const segment_meta& segm, const download_part& part) { auto name = generate_local_segment_name( segm.base_offset, segm.segment_term); - auto remote_path = partition_manifest::generate_remote_segment_path( - _ntpc.ntp(), segm); - auto localpath = part.part_prefix / std::filesystem::path(name()); vlog( @@ -696,6 +707,9 @@ partition_downloader::download_segment_file( } auto stream_stats = cloud_storage::stream_stats{}; + auto remote_path = cloud_storage::remote_segment_path( + _remote_path_provider.segment_path( + _ntpc.ntp(), _ntpc.get_initial_revision(), segm)); auto stream = [this, &stream_stats, diff --git a/src/v/cluster/partition_recovery_manager.h b/src/v/cluster/partition_recovery_manager.h index 7b57ba88b483..77f4619c8bd3 100644 --- a/src/v/cluster/partition_recovery_manager.h +++ b/src/v/cluster/partition_recovery_manager.h @@ -12,6 +12,8 @@ #include "cloud_storage/offset_translation_layer.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_label.h" +#include "cloud_storage/remote_path_provider.h" #include "model/metadata.h" #include "model/record.h" #include "storage/ntp_config.h" @@ -77,7 +79,8 @@ class partition_recovery_manager { ss::future download_log( const storage::ntp_config& ntp_cfg, model::initial_revision_id remote_revsion, - int32_t remote_partition_count); + int32_t remote_partition_count, + cloud_storage::remote_path_provider& path_provider); void set_topic_recovery_components( ss::sharded& @@ -110,6 +113,7 @@ class partition_downloader { public: partition_downloader( const storage::ntp_config& ntpc, + const cloud_storage::remote_path_provider& path_provider, remote* remote, model::initial_revision_id remote_revision_id, int32_t remote_partition_count, @@ -216,6 +220,7 @@ class partition_downloader { read_first_record_header(const std::filesystem::path& path); const storage::ntp_config& _ntpc; + const cloud_storage::remote_path_provider& _remote_path_provider; cloud_storage_clients::bucket_name _bucket; remote* _remote; model::initial_revision_id _remote_revision_id; From fe640c9ef1502db0e21a740c03b8daacae064593 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 19 Jun 2024 19:15:05 -0700 Subject: [PATCH 11/34] cluster: use partition_manifest_downloader in topic recovery validation Before allowing a recovery topic creation, the controller leader will perform some validation on the partitions and segments. During this process, Redpanda may not know the format of the partition manifests (serde or json), and segment paths may include a cluster uuid moving forward. The former is handled transparently by the partition manifest downloader, and the latter can be addressed by using a path provider directly. --- src/v/cluster/topic_recovery_validator.cc | 64 ++++++++++++++--------- src/v/cluster/topic_recovery_validator.h | 7 +-- 2 files changed, 42 insertions(+), 29 deletions(-) diff --git a/src/v/cluster/topic_recovery_validator.cc b/src/v/cluster/topic_recovery_validator.cc index b688e4b095a2..cd928293cd43 100644 --- a/src/v/cluster/topic_recovery_validator.cc +++ b/src/v/cluster/topic_recovery_validator.cc @@ -10,6 +10,9 @@ #include "topic_recovery_validator.h" #include "cloud_storage/anomalies_detector.h" +#include "cloud_storage/partition_manifest_downloader.h" +#include "cloud_storage/remote_path_provider.h" +#include "cloud_storage/types.h" #include "cluster/logger.h" #include @@ -28,12 +31,14 @@ partition_validator::partition_validator( ss::abort_source& as, model::ntp ntp, model::initial_revision_id rev_id, + const cloud_storage::remote_path_provider& path_provider, recovery_checks checks) : remote_{&remote} , bucket_{&bucket} , as_{&as} , ntp_{std::move(ntp)} , rev_id_{rev_id} + , remote_path_provider_(path_provider) , op_rtc_{retry_chain_node{ as, 300s, @@ -66,31 +71,28 @@ ss::future partition_validator::run() { ss::future partition_validator::do_validate_manifest_existence() { - auto [download_res, manifest_format] - = co_await remote_->partition_manifest_exists( - *bucket_, ntp_, rev_id_, op_rtc_); - // only res==success (manifest found) or res==notfound (manifest NOT - // found) make sense. warn for timedout and failed - if (download_res == cloud_storage::download_result::success) { - op_logger_.info("manifest found, validation ok"); - co_return validation_result::passed; + auto dl = cloud_storage::partition_manifest_downloader( + *bucket_, remote_path_provider_, ntp_, rev_id_, *remote_); + auto download_res = co_await dl.manifest_exists(op_rtc_); + if (download_res.has_error()) { + // Abnormal failure mode: could be a configuration issue or an external + // service issue. + op_logger_.error( + "manifest download error: download_result: {}, validation not ok", + download_res.error()); + co_return validation_result::download_issue; } - - if (download_res == cloud_storage::download_result::notfound) { + // Note that missing manifests is okay -- it may mean that the partition + // didn't live long enough to upload a manifest. In that case, recovery can + // proceed with an empty partition. + switch (download_res.value()) { + case cloud_storage::find_partition_manifest_outcome::no_matching_manifest: op_logger_.info("no manifest, validation ok"); co_return validation_result::missing_manifest; + case cloud_storage::find_partition_manifest_outcome::success: + op_logger_.info("manifest found, validation ok"); + co_return validation_result::passed; } - - // abnormal failure mode: could be a configuration issue or an - // external service issue (note that the manifest path will end in - // .bin but the value is just a hint of the HEAD request that - // generated the abnormal result) - op_logger_.error( - "manifest {} download error: download_result: {}, validation not ok", - get_path(manifest_format), - download_res); - - co_return validation_result::download_issue; } ss::future @@ -103,6 +105,7 @@ partition_validator::do_validate_manifest_metadata() { *bucket_, ntp_, rev_id_, + remote_path_provider_, *remote_, op_logger_, *as_, @@ -206,10 +209,9 @@ partition_validator::do_validate_manifest_metadata() { } } -cloud_storage::remote_manifest_path -partition_validator::get_path(cloud_storage::manifest_format format) { - return cloud_storage::generate_partition_manifest_path( - ntp_, rev_id_, format); +cloud_storage::remote_manifest_path partition_validator::get_path() { + return cloud_storage::remote_manifest_path{ + remote_path_provider_.partition_manifest_path(ntp_, rev_id_)}; } // wrap allocation and execution of partition_validation, @@ -219,10 +221,17 @@ ss::future do_validate_recovery_partition( ss::abort_source& as, model::ntp ntp, model::initial_revision_id rev_id, + const cloud_storage::remote_path_provider& path_provider, model::recovery_validation_mode mode, size_t max_segment_depth) { auto p_validator = partition_validator{ - remote, bucket, as, std::move(ntp), rev_id, {mode, max_segment_depth}}; + remote, + bucket, + as, + std::move(ntp), + rev_id, + path_provider, + {mode, max_segment_depth}}; co_return co_await p_validator.run(); } @@ -291,6 +300,8 @@ maybe_validate_recovery_topic( // start validation for each partition, collect the results and return // them + const cloud_storage::remote_path_provider path_provider( + assignable_config.cfg.properties.remote_label); co_await ss::max_concurrent_for_each( enumerate_partitions, concurrency, [&](model::partition_id p) { @@ -300,6 +311,7 @@ maybe_validate_recovery_topic( as, model::ntp{ns, topic, p}, initial_rev_id, + path_provider, checks_mode, checks_depth) .then([&results, p](validation_result res) { results[p] = res; }); diff --git a/src/v/cluster/topic_recovery_validator.h b/src/v/cluster/topic_recovery_validator.h index 3b94ac941cf2..195726ffe796 100644 --- a/src/v/cluster/topic_recovery_validator.h +++ b/src/v/cluster/topic_recovery_validator.h @@ -10,6 +10,7 @@ #pragma once #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cluster/types.h" #include @@ -63,6 +64,7 @@ class partition_validator { ss::abort_source& as, model::ntp ntp, model::initial_revision_id rev_id, + const cloud_storage::remote_path_provider&, recovery_checks checks); /// Perform validation on the ntp as specified with checks @@ -81,15 +83,14 @@ class partition_validator { ss::future do_validate_manifest_metadata(); // utility method for logging - cloud_storage::remote_manifest_path get_path( - cloud_storage::manifest_format format - = cloud_storage::manifest_format::serde); + cloud_storage::remote_manifest_path get_path(); cloud_storage::remote* remote_; cloud_storage_clients::bucket_name const* bucket_; ss::abort_source* as_; model::ntp ntp_; model::initial_revision_id rev_id_; + const cloud_storage::remote_path_provider& remote_path_provider_; retry_chain_node op_rtc_; retry_chain_logger op_logger_; recovery_checks checks_; From 9d97ff6ae3c884b173f8bf50b5d8a4302480d4dc Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 14:41:54 -0700 Subject: [PATCH 12/34] cluster: use partition_manifest_downloader for unsafe reset from cloud The unsafe reset method needs to be able to find a partition manifest whose format (serde or json) is not yet known. This is handled transparently by the partition manifest downlaoder, while also taking remote labels into account. This commit updates the method to use the downloader. --- src/v/cluster/partition.cc | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/src/v/cluster/partition.cc b/src/v/cluster/partition.cc index b56f071bfd10..b2aff32a334a 100644 --- a/src/v/cluster/partition.cc +++ b/src/v/cluster/partition.cc @@ -13,6 +13,7 @@ #include "archival/ntp_archiver_service.h" #include "archival/upload_housekeeping_service.h" #include "cloud_storage/async_manifest_view.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/read_path_probes.h" #include "cloud_storage/remote_partition.h" #include "cluster/id_allocator_stm.h" @@ -1172,13 +1173,22 @@ partition::do_unsafe_reset_remote_partition_manifest_from_cloud(bool force) { auto backoff = config::shard_local_cfg().cloud_storage_initial_backoff_ms(); retry_chain_node rtc(_as, timeout, backoff); - auto [res, res_fmt] - = co_await _cloud_storage_api.local().try_download_partition_manifest( - bucket, new_manifest, rtc); - - if (res != cloud_storage::download_result::success) { + cloud_storage::partition_manifest_downloader dl( + bucket, + _archival_meta_stm->path_provider(), + ntp(), + initial_rev, + _cloud_storage_api.local()); + auto res = co_await dl.download_manifest(rtc, &new_manifest); + if (res.has_error()) { + throw std::runtime_error(ssx::sformat( + "Failed to download partition manifest with error: {}", res.error())); + } + if ( + res.value() + == cloud_storage::find_partition_manifest_outcome::no_matching_manifest) { throw std::runtime_error(ssx::sformat( - "Failed to download partition manifest with error: {}", res)); + "No matching manifest for {} rev {}", ntp(), initial_rev)); } const auto max_collectible From 8cb5db2a06b60d9979125917d9caf92e239e7eac Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 19 Jun 2024 19:16:03 -0700 Subject: [PATCH 13/34] cloud_storage: use partition_manifest_downloader in scrubber/anomaly detector The anomaly detector (and by extension, the scubber) needs to be able to find a partition manifest whose format (serde or json) is not yet known. This is handled transparently by the partition manifest downlaoder, while also taking remote labels into account. This commit updates anomaly detector to use the downloader, and plugs in the archiver's path provider from the scrubber. As a part of this move, there's an anomalous case that is removed: when we detect a JSON manifest that's a spillover manifest. Redpanda never published JSON spillover manifests, so indeed this would be anomalous. In practice this should never be the case, and this commit just removes it, since it depends on knowing the downloaded format, which is no longer the case when using the downloader. --- src/v/archival/scrubber.cc | 2 +- src/v/cloud_storage/anomalies_detector.cc | 47 +++++++++---------- src/v/cloud_storage/anomalies_detector.h | 3 ++ .../tests/anomalies_detector_test.cc | 4 ++ 4 files changed, 31 insertions(+), 25 deletions(-) diff --git a/src/v/archival/scrubber.cc b/src/v/archival/scrubber.cc index 2068f0904788..ccd9003cd551 100644 --- a/src/v/archival/scrubber.cc +++ b/src/v/archival/scrubber.cc @@ -30,7 +30,7 @@ scrubber::scrubber( , _archiver(archiver) , _remote(remote) , _feature_table(feature_table) - , _detector{_archiver.get_bucket_name(), _archiver.get_ntp(), _archiver.get_revision_id(), _remote, _logger, _as} + , _detector{_archiver.get_bucket_name(), _archiver.get_ntp(), _archiver.get_revision_id(), _archiver.remote_path_provider(), _remote, _logger, _as} , _scheduler( [this] { const auto at = _archiver.manifest().last_partition_scrub(); diff --git a/src/v/cloud_storage/anomalies_detector.cc b/src/v/cloud_storage/anomalies_detector.cc index 5fb9fcbf1ef8..7f3a5498274d 100644 --- a/src/v/cloud_storage/anomalies_detector.cc +++ b/src/v/cloud_storage/anomalies_detector.cc @@ -12,6 +12,7 @@ #include "cloud_storage/base_manifest.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote.h" namespace cloud_storage { @@ -20,12 +21,14 @@ anomalies_detector::anomalies_detector( cloud_storage_clients::bucket_name bucket, model::ntp ntp, model::initial_revision_id initial_rev, + const remote_path_provider& path_provider, remote& remote, retry_chain_logger& logger, ss::abort_source& as) : _bucket(std::move(bucket)) , _ntp(std::move(ntp)) , _initial_rev(initial_rev) + , _remote_path_provider(path_provider) , _remote(remote) , _logger(logger) , _as(as) {} @@ -39,19 +42,22 @@ ss::future anomalies_detector::run( vlog(_logger.debug, "Downloading partition manifest ..."); + partition_manifest_downloader dl( + _bucket, _remote_path_provider, _ntp, _initial_rev, _remote); partition_manifest manifest(_ntp, _initial_rev); - auto [dl_result, format] = co_await _remote.try_download_partition_manifest( - _bucket, manifest, rtc_node); + auto dl_result = co_await dl.download_manifest(rtc_node, &manifest); ++_result.ops; - - if (dl_result == download_result::notfound) { - _result.detected.missing_partition_manifest = true; - co_return _result; - } else if (dl_result != download_result::success) { + if (dl_result.has_error()) { vlog(_logger.debug, "Failed downloading partition manifest ..."); _result.status = scrub_status::failed; co_return _result; } + if ( + dl_result.value() + == find_partition_manifest_outcome::no_matching_manifest) { + _result.detected.missing_partition_manifest = true; + co_return _result; + } std::deque spill_manifest_paths; const auto& spillovers = manifest.get_spillover_map(); @@ -64,32 +70,24 @@ ss::future anomalies_detector::run( .base_ts = iter->base_timestamp, .last_ts = iter->max_timestamp, }; - - auto spill_path = generate_spillover_manifest_path( - _ntp, _initial_rev, comp); + auto spill_path = _remote_path_provider.spillover_manifest_path( + manifest, comp); auto exists_result = co_await _remote.segment_exists( - _bucket, remote_segment_path{spill_path()}, rtc_node); + _bucket, remote_segment_path{spill_path}, rtc_node); ++_result.ops; if (exists_result == download_result::notfound) { _result.detected.missing_spillover_manifests.emplace(comp); - } else if (dl_result != download_result::success) { + } else if (exists_result != download_result::success) { vlog( _logger.debug, "Failed to check existence of spillover manifest {}", - spill_path()); + spill_path); _result.status = scrub_status::partial; } else { - spill_manifest_paths.emplace_front(spill_path()); + spill_manifest_paths.emplace_front(spill_path); } } - // Binary manifest encoding and spillover manifests were both added - // in the same release. Hence, it's an anomaly to have a JSON - // encoded manifest and spillover manifests. - if (format == manifest_format::json && spill_manifest_paths.size() > 0) { - _result.detected.missing_partition_manifest = true; - } - const auto stop_at_stm = co_await check_manifest( manifest, scrub_from, rtc_node); if (stop_at_stm == stop_detector::yes) { @@ -233,9 +231,10 @@ anomalies_detector::check_manifest( const auto seg_meta = *seg_iter; - const auto segment_path = manifest.generate_segment_path(seg_meta); + const auto segment_path = _remote_path_provider.segment_path( + manifest, seg_meta); const auto exists_result = co_await _remote.segment_exists( - _bucket, segment_path, rtc_node); + _bucket, remote_segment_path{segment_path}, rtc_node); _result.ops += 1; _result.segments_visited += 1; @@ -245,7 +244,7 @@ anomalies_detector::check_manifest( vlog( _logger.debug, "Failed to check existence of segment at {}", - segment_path()); + segment_path); _result.status = scrub_status::partial; } diff --git a/src/v/cloud_storage/anomalies_detector.h b/src/v/cloud_storage/anomalies_detector.h index 682da557c08e..abcb84d5d62b 100644 --- a/src/v/cloud_storage/anomalies_detector.h +++ b/src/v/cloud_storage/anomalies_detector.h @@ -13,6 +13,7 @@ #include "archival/types.h" #include "base/seastarx.h" #include "cloud_storage/fwd.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "model/fundamental.h" @@ -40,6 +41,7 @@ class anomalies_detector { cloud_storage_clients::bucket_name bucket, model::ntp ntp, model::initial_revision_id initial_rev, + const remote_path_provider&, remote& remote, retry_chain_logger& logger, ss::abort_source& as); @@ -102,6 +104,7 @@ class anomalies_detector { cloud_storage_clients::bucket_name _bucket; model::ntp _ntp; model::initial_revision_id _initial_rev; + const remote_path_provider& _remote_path_provider; remote& _remote; retry_chain_logger& _logger; diff --git a/src/v/cloud_storage/tests/anomalies_detector_test.cc b/src/v/cloud_storage/tests/anomalies_detector_test.cc index fab4c956ac84..dbffa02230e3 100644 --- a/src/v/cloud_storage/tests/anomalies_detector_test.cc +++ b/src/v/cloud_storage/tests/anomalies_detector_test.cc @@ -13,6 +13,7 @@ #include "cloud_storage/base_manifest.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "http/tests/http_imposter.h" @@ -42,6 +43,8 @@ bool operator==(const anomalies& lhs, const anomalies& rhs) { namespace { +cloud_storage::remote_path_provider path_provider(std::nullopt); + ss::logger test_logger{"anomaly_detection_test"}; constexpr std::string_view stm_manifest = R"json( @@ -278,6 +281,7 @@ class bucket_view_fixture : http_imposter_fixture { cloud_storage_clients::bucket_name{"test-bucket"}, _stm_manifest.get_ntp(), _stm_manifest.get_revision_id(), + path_provider, _remote.local(), _rtc_logger, _as); From ebf2fa1ca65aa9c4f527245b49cd78f0d3fbf7c9 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 15:39:25 -0700 Subject: [PATCH 14/34] remote_partition: use partition_manifest_downloader for manifest finalization --- src/v/cloud_storage/remote_partition.cc | 29 ++++++++++++++++++------- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/src/v/cloud_storage/remote_partition.cc b/src/v/cloud_storage/remote_partition.cc index e790cc1d76d6..1e3df5b42f24 100644 --- a/src/v/cloud_storage/remote_partition.cc +++ b/src/v/cloud_storage/remote_partition.cc @@ -16,6 +16,7 @@ #include "cloud_storage/materialized_resources.h" #include "cloud_storage/offset_translation_layer.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tx_range_manifest.h" #include "cloud_storage/types.h" @@ -1306,6 +1307,7 @@ 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; @@ -1322,16 +1324,25 @@ ss::future<> finalize_background(remote& api, finalize_data data) { partition_manifest remote_manifest(data.ntp, data.revision); - auto [manifest_get_result, result_fmt] - = co_await api.try_download_partition_manifest( - data.bucket, remote_manifest, local_rtc); - - if (manifest_get_result != download_result::success) { + partition_manifest_downloader dl( + data.bucket, data.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()) { vlog( cst_log.error, "[{}] Failed to fetch manifest during finalize(). Error: {}", data.ntp, - manifest_get_result); + manifest_get_result.error()); + co_return; + } + if ( + manifest_get_result.value() + == find_partition_manifest_outcome::no_matching_manifest) { + vlog( + cst_log.error, + "[{}] Failed to fetch manifest during finalize(). Not found", + data.ntp); co_return; } @@ -1399,12 +1410,14 @@ 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()()}, + .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()}; From 653c9ec9914202347ded4103647f71fe3b9cf4ad Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 14:49:59 -0700 Subject: [PATCH 15/34] cloud_storage: remove try_download_partition_manifest This has been subsumed by the partition manifest downloader. Swaps out the remaining (test-only) usages of the method for calls into the partition_manifest_downloader. --- src/v/cloud_storage/remote.cc | 27 -------- src/v/cloud_storage/remote.h | 12 ---- src/v/cloud_storage/tests/remote_test.cc | 82 ++++-------------------- src/v/cloud_storage/tests/util.cc | 11 +++- 4 files changed, 21 insertions(+), 111 deletions(-) diff --git a/src/v/cloud_storage/remote.cc b/src/v/cloud_storage/remote.cc index b46dda3eaf93..b3c6c09de4c0 100644 --- a/src/v/cloud_storage/remote.cc +++ b/src/v/cloud_storage/remote.cc @@ -231,33 +231,6 @@ ss::future remote::maybe_download_manifest( co_return co_await do_download_manifest(bucket, fk, manifest, parent, true); } -ss::future> -remote::try_download_partition_manifest( - const cloud_storage_clients::bucket_name& bucket, - partition_manifest& manifest, - retry_chain_node& parent, - bool expect_missing) { - vassert( - manifest.get_ntp() != model::ntp{} - && manifest.get_revision_id() != model::initial_revision_id{}, - "partition manifest must have ntp"); - - // first try to download the serde format - auto format_path = manifest.get_manifest_format_and_path(); - auto serde_result = co_await do_download_manifest( - bucket, format_path, manifest, parent, expect_missing); - if (serde_result != download_result::notfound) { - // propagate success, timedout and failed to caller - co_return std::pair{serde_result, manifest_format::serde}; - } - // fallback to json format - format_path = manifest.get_legacy_manifest_format_and_path(); - co_return std::pair{ - co_await do_download_manifest( - bucket, format_path, manifest, parent, expect_missing), - manifest_format::json}; -} - ss::future remote::do_download_manifest( const cloud_storage_clients::bucket_name& bucket, const std::pair& format_key, diff --git a/src/v/cloud_storage/remote.h b/src/v/cloud_storage/remote.h index 5977e0ebda8e..c4ebb59c39df 100644 --- a/src/v/cloud_storage/remote.h +++ b/src/v/cloud_storage/remote.h @@ -268,18 +268,6 @@ class remote base_manifest& manifest, retry_chain_node& parent); - /// \brief Try downloading partition_manifest. the function tries first the - /// manifest_format::serde path, and then manifest_format::json path. it's - /// expected that manifest is constructed with the approprieate npt and - /// revision_id, as it will be used to generate the paths return type is - /// download_result and index of path that generated the result - ss::future> - try_download_partition_manifest( - const cloud_storage_clients::bucket_name& bucket, - partition_manifest& manifest, - retry_chain_node& parent, - bool expect_missing = false); - /// \brief Upload manifest to the pre-defined S3 location /// /// \param bucket is a bucket name diff --git a/src/v/cloud_storage/tests/remote_test.cc b/src/v/cloud_storage/tests/remote_test.cc index 8410ebb136a5..8d0d82a3b50e 100644 --- a/src/v/cloud_storage/tests/remote_test.cc +++ b/src/v/cloud_storage/tests/remote_test.cc @@ -16,6 +16,7 @@ #include "cloud_storage/materialized_resources.h" #include "cloud_storage/offset_translation_layer.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tests/common_def.h" @@ -88,15 +89,6 @@ static cloud_storage::lazy_abort_source always_continue{ static constexpr model::cloud_credentials_source config_file{ model::cloud_credentials_source::config_file}; -static partition_manifest load_manifest_from_str(std::string_view v) { - partition_manifest m; - iobuf i; - i.append(v.data(), v.size()); - auto s = make_iobuf_input_stream(std::move(i)); - m.update(manifest_format::json, std::move(s)).get(); - return m; -} - static remote::event_filter allow_all; static iobuf make_iobuf_from_string(std::string_view s) { @@ -154,24 +146,6 @@ using remote_fixture = remote_fixture_base; using gcs_remote_fixture = remote_fixture_base< backend_override_mixin_t>; -static auto run_manifest_download_and_check( - auto& remote, - const cloud_storage_clients::bucket_name& bucket_name, - partition_manifest expected_manifest, - manifest_format expected_download_format, - std::string_view test_context) { - partition_manifest actual(manifest_ntp, manifest_revision); - retry_chain_node fib(never_abort, 100ms, 20ms); - auto [res, fmt] = remote.local() - .try_download_partition_manifest( - bucket_name, actual, fib) - .get(); - - EXPECT_TRUE(res == download_result::success); - EXPECT_TRUE(fmt == expected_download_format); - EXPECT_TRUE(expected_manifest == actual); -} - class all_types_remote_fixture : public remote_fixture , public testing::TestWithParam { @@ -188,44 +162,6 @@ class all_types_gcs_remote_fixture : gcs_remote_fixture(GetParam().url_style) {} }; -TEST_P(all_types_remote_fixture, test_download_manifest_json) { - set_expectations_and_listen({expectation{ - .url = manifest_url, .body = ss::sstring(manifest_payload)}}); - auto subscription = remote.local().subscribe(allow_all); - run_manifest_download_and_check( - remote, - bucket_name, - load_manifest_from_str(manifest_payload), - manifest_format::json, - "manifest load from json"); - EXPECT_TRUE(subscription.available()); - EXPECT_TRUE( - subscription.get().type == api_activity_type::manifest_download); -} - -TEST_P(all_types_remote_fixture, test_download_manifest_serde) { - auto translator = load_manifest_from_str(manifest_payload); - auto serialized = translator.serialize().get(); - auto manifest_binary - = serialized.stream.read_exactly(serialized.size_bytes).get(); - - set_expectations_and_listen({expectation{ - .url = manifest_serde_url, - .body = ss::sstring{manifest_binary.begin(), manifest_binary.end()}}}); - - auto subscription = remote.local().subscribe(allow_all); - run_manifest_download_and_check( - remote, - bucket_name, - std::move(translator), - manifest_format::serde, - "manifest load from serde"); - - EXPECT_TRUE(subscription.available()); - EXPECT_TRUE( - subscription.get().type == api_activity_type::manifest_download); -} - TEST_P(all_types_remote_fixture, test_download_manifest_timeout) { // NOLINT partition_manifest actual(manifest_ntp, manifest_revision); auto subscription = remote.local().subscribe(allow_all); @@ -1378,8 +1314,15 @@ TEST_P(all_types_remote_fixture, test_notification_retry_meta) { partition_manifest actual(manifest_ntp, manifest_revision); auto filter = remote::event_filter{}; - auto fut = remote.local().try_download_partition_manifest( - bucket_name, actual, fib); + remote_path_provider path_provider(std::nullopt); + partition_manifest_downloader dl( + bucket_name, + path_provider, + manifest_ntp, + manifest_revision, + remote.local()); + + auto fut = dl.download_manifest(fib, &actual); RPTEST_REQUIRE_EVENTUALLY(2s, [&] { auto sub = remote.local().subscribe(filter); @@ -1388,8 +1331,9 @@ TEST_P(all_types_remote_fixture, test_notification_retry_meta) { }); }); - auto [res, fmt] = fut.get(); - EXPECT_TRUE(res == download_result::timedout); + auto res = fut.get(); + EXPECT_TRUE(res.has_error()); + EXPECT_TRUE(res.error() == error_outcome::manifest_download_error); } TEST_P(all_types_remote_fixture, test_get_object) { diff --git a/src/v/cloud_storage/tests/util.cc b/src/v/cloud_storage/tests/util.cc index 5b5530d933f1..8351e9d3f791 100644 --- a/src/v/cloud_storage/tests/util.cc +++ b/src/v/cloud_storage/tests/util.cc @@ -10,6 +10,7 @@ */ #include "cloud_storage/tests/util.h" +#include "cloud_storage/partition_manifest_downloader.h" #include "model/record.h" #include "model/record_batch_types.h" @@ -621,11 +622,15 @@ partition_manifest hydrate_manifest( remote& api, const cloud_storage_clients::bucket_name& bucket) { static ss::abort_source never_abort; + remote_path_provider path_provider(std::nullopt); + partition_manifest_downloader dl( + bucket, path_provider, manifest_ntp, manifest_revision, api); partition_manifest m(manifest_ntp, manifest_revision); - ss::lowres_clock::update(); retry_chain_node rtc(never_abort, 300s, 200ms); - auto [res, _] = api.try_download_partition_manifest(bucket, m, rtc).get(); - BOOST_REQUIRE(res == cloud_storage::download_result::success); + ss::lowres_clock::update(); + auto res = dl.download_manifest(rtc, &m).get(); + BOOST_REQUIRE(res.has_value()); + BOOST_REQUIRE(res.value() == find_partition_manifest_outcome::success); return m; } From eaa7b9597b0f561f6e392a44a7f7a345d028e28e Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 20 Jun 2024 00:10:42 -0700 Subject: [PATCH 16/34] cloud_storage: remove remote::partition_manifest_exists() Callers should use the partition_manifest_downloader instead. --- src/v/cloud_storage/remote.cc | 34 ---------------------------------- src/v/cloud_storage/remote.h | 22 ---------------------- 2 files changed, 56 deletions(-) diff --git a/src/v/cloud_storage/remote.cc b/src/v/cloud_storage/remote.cc index b3c6c09de4c0..db9fb52829fa 100644 --- a/src/v/cloud_storage/remote.cc +++ b/src/v/cloud_storage/remote.cc @@ -979,40 +979,6 @@ ss::future remote::segment_exists( existence_check_type::segment); } -ss::future -remote::partition_manifest_exists( - const cloud_storage_clients::bucket_name& bucket, - model::ntp ntp, - model::initial_revision_id rev_id, - retry_chain_node& parent) { - // first check serde and exit early if it exists - auto serde_res = co_await object_exists( - bucket, - cloud_storage_clients::object_key{generate_partition_manifest_path( - ntp, rev_id, manifest_format::serde)()}, - parent, - existence_check_type::manifest); - - switch (serde_res) { - case download_result::success: - co_return partition_manifest_existence{ - download_result::success, manifest_format::serde}; - case download_result::notfound: { - auto json_res = co_await object_exists( - bucket, - cloud_storage_clients::object_key{generate_partition_manifest_path( - ntp, rev_id, manifest_format::json)()}, - parent, - existence_check_type::manifest); - co_return partition_manifest_existence{json_res, manifest_format::json}; - } - case download_result::failed: - case download_result::timedout: - // do not try to check for json in case of failures - co_return partition_manifest_existence{serde_res, {}}; - } -} - ss::future remote::delete_object( const cloud_storage_clients::bucket_name& bucket, const cloud_storage_clients::object_key& path, diff --git a/src/v/cloud_storage/remote.h b/src/v/cloud_storage/remote.h index c4ebb59c39df..80392112f7d6 100644 --- a/src/v/cloud_storage/remote.h +++ b/src/v/cloud_storage/remote.h @@ -382,28 +382,6 @@ class remote const remote_segment_path& path, retry_chain_node& parent); - struct partition_manifest_existence { - download_result download_result; - manifest_format manifest_format; - }; - - /// \brief Specialization of object_exists for partition_manifest. - /// Checks the existence of either serde or json format. - /// This a chained operation, possible results are - /// , , - /// the serde format is checked - /// first, if found the function will not check the existence of the json - /// format. - /// \param ntp ntp to query. - /// \param rev_id initial revision id of the remote partition. - /// \return partition_manifest_exists_result that - /// contains the result of the download and the format of the manifest. - ss::future partition_manifest_exists( - const cloud_storage_clients::bucket_name& bucket, - model::ntp npt, - model::initial_revision_id rev_id, - retry_chain_node& parent); - /// \brief Delete object from S3 /// /// The method deletes the object. It can retry after some errors. From 849aef7aa7a10e8c3bb12feae66b1ee2e85e8056 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Tue, 18 Jun 2024 18:48:32 -0700 Subject: [PATCH 17/34] cloud_storage: use path provider for spillovers in manifest view The manifest view transparently downloads and caches spillover manifests when Redpanda reads in the spillover region. This commit updates the relevant path generation code to use the path provider. The cursor doesn't need the path provider for anything but logging, so instead of logging the full paths, this just has the cursor log the filenames. --- src/v/cloud_storage/async_manifest_view.cc | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/v/cloud_storage/async_manifest_view.cc b/src/v/cloud_storage/async_manifest_view.cc index f1a63c9aa6c8..8c06be2c8441 100644 --- a/src/v/cloud_storage/async_manifest_view.cc +++ b/src/v/cloud_storage/async_manifest_view.cc @@ -370,14 +370,14 @@ void async_manifest_view_cursor::on_timeout() { vlog( _view._ctxlog.debug, "Spillover manifest {} is being evicted, last offset: {}", - m->manifest.get_manifest_path(), + m->manifest.get_manifest_filename(), m->manifest.get_last_offset()); return model::next_offset(m->manifest.get_last_offset()); } else { vlog( _view._ctxlog.debug, "Spillover manifest {} is not evicted, rearming", - m->manifest.get_manifest_path()); + m->manifest.get_manifest_filename()); return model::offset{}; } }); @@ -795,7 +795,7 @@ async_manifest_view::get_term_last_offset(model::term_id term) noexcept { vlog( _ctxlog.debug, "Scanning manifest {} for term {}", - manifest.get_manifest_path(), + manifest.get_manifest_path(path_provider()), term); for (auto meta : manifest) { if (meta.segment_term > term) { @@ -1152,7 +1152,7 @@ async_manifest_view::time_based_retention( "Failed to find the retention boundary, the manifest {} " "doesn't " "have any matching segment", - cursor->manifest()->get_manifest_path()); + cursor->manifest()->get_manifest_path(path_provider())); } } catch (const std::system_error& err) { // Thrown by `async_manifest_view::maybe_sync_manifest` @@ -1423,7 +1423,7 @@ async_manifest_view::hydrate_manifest( auto [str, len] = co_await manifest.serialize(); auto reservation = co_await _cache.local().reserve_space(len, 1); co_await _cache.local().put( - manifest.get_manifest_path()(), + manifest.get_manifest_path(path_provider())(), str, reservation, priority_manager::local().shadow_indexing_priority()); @@ -1602,8 +1602,8 @@ remote_manifest_path async_manifest_view::get_spillover_manifest_path( .base_ts = meta.base_timestamp, .last_ts = meta.max_timestamp, }; - return generate_spillover_manifest_path( - get_ntp(), _stm_manifest.get_revision_id(), comp); + return remote_manifest_path{ + path_provider().spillover_manifest_path(_stm_manifest, comp)}; } ss::future> From 5248547e99cd09d03bacded40c52c76a4580b94c Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 17 Jun 2024 23:49:01 -0700 Subject: [PATCH 18/34] remote_partition: use path provider for segment paths --- src/v/cloud_storage/remote_partition.cc | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/v/cloud_storage/remote_partition.cc b/src/v/cloud_storage/remote_partition.cc index 1e3df5b42f24..d3ddfc5e1e59 100644 --- a/src/v/cloud_storage/remote_partition.cc +++ b/src/v/cloud_storage/remote_partition.cc @@ -190,7 +190,8 @@ remote_partition::borrow_result_t remote_partition::borrow_next_segment_reader( if (iter != _segments.end()) { if ( iter->second->segment->get_segment_path() - != manifest.generate_segment_path(*mit)) { + != manifest.generate_segment_path( + *mit, _manifest_view->path_provider())) { // The segment was replaced and doesn't match metadata anymore. We // want to avoid picking it up because otherwise we won't be able to // make any progress. @@ -199,7 +200,8 @@ remote_partition::borrow_result_t remote_partition::borrow_next_segment_reader( } } if (iter == _segments.end()) { - auto path = manifest.generate_segment_path(*mit); + auto path = manifest.generate_segment_path( + *mit, _manifest_view->path_provider()); iter = get_or_materialize_segment(path, *mit, std::move(segment_unit)); } auto mit_committed_offset = mit->committed_offset; @@ -1068,7 +1070,8 @@ remote_partition::aborted_transactions(offset_range offsets) { // up front at the start of the function. auto segment_unit = co_await materialized().get_segment_units( std::nullopt); - auto path = stm_manifest.generate_segment_path(*it); + auto path = stm_manifest.generate_segment_path( + *it, _manifest_view->path_provider()); auto m = get_or_materialize_segment( path, *it, std::move(segment_unit)); remote_segs.emplace_back(m->second->segment); @@ -1100,7 +1103,7 @@ remote_partition::aborted_transactions(offset_range offsets) { auto cursor = std::move(cur_res.value()); co_await for_each_manifest( std::move(cursor), - [&offsets, &meta_to_materialize]( + [&offsets, &meta_to_materialize, this]( ssx::task_local_ptr manifest) { for (auto it = manifest->segment_containing(offsets.begin); it != manifest->end(); @@ -1108,7 +1111,8 @@ remote_partition::aborted_transactions(offset_range offsets) { if (it->base_offset > offsets.end_rp) { return ss::stop_iteration::yes; } - auto path = manifest->generate_segment_path(*it); + auto path = manifest->generate_segment_path( + *it, _manifest_view->path_provider()); meta_to_materialize.emplace_back(*it, path); } return ss::stop_iteration::no; From 3937454f712793c831f3e320583fdebcd0e9caec Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 28 Jun 2024 23:53:52 -0700 Subject: [PATCH 19/34] cloud_storage: add utils for lifecycle marker paths --- src/v/cloud_storage/remote_path_provider.cc | 9 +++++++++ src/v/cloud_storage/remote_path_provider.h | 5 +++++ src/v/cloud_storage/topic_path_utils.cc | 16 ++++++++++++++++ src/v/cloud_storage/topic_path_utils.h | 10 ++++++++++ 4 files changed, 40 insertions(+) diff --git a/src/v/cloud_storage/remote_path_provider.cc b/src/v/cloud_storage/remote_path_provider.cc index d222b656d798..1a14ccc9e4e1 100644 --- a/src/v/cloud_storage/remote_path_provider.cc +++ b/src/v/cloud_storage/remote_path_provider.cc @@ -15,6 +15,7 @@ #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/topic_path_utils.h" #include "cloud_storage/types.h" +#include "model/fundamental.h" namespace cloud_storage { @@ -106,4 +107,12 @@ ss::sstring remote_path_provider::segment_path( manifest.get_ntp(), manifest.get_revision_id(), segment); } +ss::sstring remote_path_provider::topic_lifecycle_marker_path( + const model::topic_namespace& topic, model::initial_revision_id rev) const { + if (label_.has_value()) { + return labeled_topic_lifecycle_marker_path(*label_, topic, rev); + } + return prefixed_topic_lifecycle_marker_path(topic, rev); +} + } // namespace cloud_storage diff --git a/src/v/cloud_storage/remote_path_provider.h b/src/v/cloud_storage/remote_path_provider.h index 9457ef9323ff..9198224ab4f1 100644 --- a/src/v/cloud_storage/remote_path_provider.h +++ b/src/v/cloud_storage/remote_path_provider.h @@ -70,6 +70,11 @@ class remote_path_provider { model::initial_revision_id rev, const segment_meta& segment) const; + // Topic lifecycle marker path. + ss::sstring topic_lifecycle_marker_path( + const model::topic_namespace& topic, + model::initial_revision_id rev) const; + private: std::optional label_; }; diff --git a/src/v/cloud_storage/topic_path_utils.cc b/src/v/cloud_storage/topic_path_utils.cc index e4b7d27db0dc..25689f648448 100644 --- a/src/v/cloud_storage/topic_path_utils.cc +++ b/src/v/cloud_storage/topic_path_utils.cc @@ -57,6 +57,16 @@ ss::sstring labeled_topic_manifest_path( rev()); } +ss::sstring labeled_topic_lifecycle_marker_path( + const remote_label& label, + const model::topic_namespace& topic, + model::initial_revision_id rev) { + return fmt::format( + "{}/{}_lifecycle.bin", + labeled_topic_manifest_prefix(label, topic), + rev()); +} + ss::sstring prefixed_topic_manifest_prefix(const model::topic_namespace& topic) { constexpr uint32_t bitmask = 0xF0000000; @@ -77,6 +87,12 @@ prefixed_topic_manifest_json_path(const model::topic_namespace& topic) { "{}/topic_manifest.json", prefixed_topic_manifest_prefix(topic)); } +ss::sstring prefixed_topic_lifecycle_marker_path( + const model::topic_namespace& topic, model::initial_revision_id rev) { + return fmt::format( + "{}/{}_lifecycle.bin", prefixed_topic_manifest_prefix(topic), rev()); +} + std::optional tp_ns_from_labeled_path(const std::string& path) { std::smatch matches; diff --git a/src/v/cloud_storage/topic_path_utils.h b/src/v/cloud_storage/topic_path_utils.h index f5b4a1298782..5a2cb9c364df 100644 --- a/src/v/cloud_storage/topic_path_utils.h +++ b/src/v/cloud_storage/topic_path_utils.h @@ -44,6 +44,12 @@ ss::sstring labeled_topic_manifest_path( const model::topic_namespace& topic, model::initial_revision_id rev); +// meta/kafka/panda-topic/806a0f4a-e691-4a2b-9352-ec4b769a5e6e/0_lifecycle.bin +ss::sstring labeled_topic_lifecycle_marker_path( + const remote_label& label, + const model::topic_namespace& topic, + model::initial_revision_id rev); + //[0-9a-f]0000000 chunked_vector prefixed_topic_manifests_roots(); @@ -58,6 +64,10 @@ prefixed_topic_manifest_bin_path(const model::topic_namespace& topic); ss::sstring prefixed_topic_manifest_json_path(const model::topic_namespace& topic); +// a0000000/meta/kafka/panda-topic/0_lifecycle.bin +ss::sstring prefixed_topic_lifecycle_marker_path( + const model::topic_namespace& topic, model::initial_revision_id rev); + // Returns the topic_namespace of the given labeled manifest path, or // std::nullopt if the input is not a labeled manifest path. std::optional From d10a60d31d1f4dfd0d7617f86e81eaac53bca765 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 28 Jun 2024 23:55:49 -0700 Subject: [PATCH 20/34] archival: use path provider to generate lifecycle marker paths --- src/v/archival/purger.cc | 5 ++++- src/v/archival/purger.h | 3 ++- src/v/cloud_storage/lifecycle_marker.h | 20 ++++++-------------- 3 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/v/archival/purger.cc b/src/v/archival/purger.cc index b1ad8f09b468..66bc7b5aca3e 100644 --- a/src/v/archival/purger.cc +++ b/src/v/archival/purger.cc @@ -412,6 +412,7 @@ ss::future purger::run(run_quota_t quota) { auto marker_r = co_await write_remote_lifecycle_marker( nt_revision, bucket, + path_provider, cloud_storage::lifecycle_status::purging, pre_purge_marker_rtc); if (marker_r != cloud_storage::upload_result::success) { @@ -509,6 +510,7 @@ ss::future purger::run(run_quota_t quota) { marker_r = co_await write_remote_lifecycle_marker( nt_revision, bucket, + path_provider, cloud_storage::lifecycle_status::purged, post_purge_marker_rtc); if (marker_r != cloud_storage::upload_result::success) { @@ -544,7 +546,8 @@ ss::future purger::run(run_quota_t quota) { ss::future purger::write_remote_lifecycle_marker( const cluster::nt_revision& nt_revision, - cloud_storage_clients::bucket_name& bucket, + const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, cloud_storage::lifecycle_status status, retry_chain_node& parent_rtc) { retry_chain_node marker_rtc(5s, 1s, &parent_rtc); diff --git a/src/v/archival/purger.h b/src/v/archival/purger.h index 3b54e8029c0c..aaea4b6acb04 100644 --- a/src/v/archival/purger.h +++ b/src/v/archival/purger.h @@ -111,7 +111,8 @@ class purger : public housekeeping_job { ss::future write_remote_lifecycle_marker( const cluster::nt_revision&, - cloud_storage_clients::bucket_name& bucket, + const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, cloud_storage::lifecycle_status status, retry_chain_node& parent_rtc); diff --git a/src/v/cloud_storage/lifecycle_marker.h b/src/v/cloud_storage/lifecycle_marker.h index 99c39496780d..f4cdac33fdca 100644 --- a/src/v/cloud_storage/lifecycle_marker.h +++ b/src/v/cloud_storage/lifecycle_marker.h @@ -10,6 +10,7 @@ #pragma once +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage_clients/types.h" #include "cluster/types.h" #include "hashing/xx.h" @@ -93,20 +94,11 @@ struct remote_nt_lifecycle_marker lifecycle_status status; - cloud_storage_clients::object_key get_key() { - return generate_key( - topic.nt.ns, topic.nt.tp, topic.initial_revision_id); - } - - static cloud_storage_clients::object_key generate_key( - const model::ns& ns, - const model::topic& tp, - model::initial_revision_id rev) { - constexpr uint32_t bitmask = 0xF0000000; - auto path = fmt::format("{}/{}", ns(), tp()); - uint32_t hash = bitmask & xxhash_32(path.data(), path.size()); - return cloud_storage_clients::object_key( - fmt::format("{:08x}/meta/{}/{}_lifecycle.bin", hash, path, rev())); + cloud_storage_clients::object_key + get_key(const remote_path_provider& path_provider) { + return cloud_storage_clients::object_key{ + path_provider.topic_lifecycle_marker_path( + topic.nt, topic.initial_revision_id)}; } }; From 5d30320613ba4c31aba66dcf399e34d1a5fa4758 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Tue, 18 Jun 2024 13:33:41 -0700 Subject: [PATCH 21/34] archival: use the path provider throughout the purger By extension, this also plugs the path provider into remote_partition::erase(), which is only used by the purger. --- src/v/archival/purger.cc | 77 ++++++++++++++----------- src/v/archival/purger.h | 4 ++ src/v/cloud_storage/remote_partition.cc | 7 ++- src/v/cloud_storage/remote_partition.h | 1 + 4 files changed, 53 insertions(+), 36 deletions(-) diff --git a/src/v/archival/purger.cc b/src/v/archival/purger.cc index 66bc7b5aca3e..b2843762f4ef 100644 --- a/src/v/archival/purger.cc +++ b/src/v/archival/purger.cc @@ -16,8 +16,10 @@ #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_partition.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/topic_manifest.h" #include "cloud_storage/tx_range_manifest.h" +#include "cloud_storage_clients/types.h" #include "cluster/members_table.h" #include "cluster/topic_table.h" #include "cluster/topics_frontend.h" @@ -26,7 +28,6 @@ #include "hashing/xx.h" namespace { -static constexpr std::string_view serde_extension = ".bin"; static constexpr std::string_view json_extension = ".json"; static constexpr auto partition_purge_timeout = 20s; @@ -51,6 +52,7 @@ purger::purger( ss::future purger::purge_partition( const cluster::nt_lifecycle_marker& lifecycle_marker, const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, model::ntp ntp, model::initial_revision_id remote_revision, retry_chain_node& parent_rtc) { @@ -82,7 +84,7 @@ ss::future purger::purge_partition( } auto collected = co_await collect_manifest_paths( - bucket, ntp, remote_revision, partition_purge_rtc); + bucket, path_provider, ntp, remote_revision, partition_purge_rtc); if (!collected) { co_return purge_result{ .status = purge_status::retryable_failure, .ops = 0}; @@ -106,6 +108,7 @@ ss::future purger::purge_partition( const auto local_res = co_await purge_manifest( bucket, + path_provider, ntp, remote_revision, remote_manifest_path{*rit}, @@ -184,24 +187,16 @@ ss::future purger::purge_partition( ss::future> purger::collect_manifest_paths( const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, model::ntp ntp, model::initial_revision_id remote_revision, retry_chain_node& parent_rtc) { retry_chain_node collection_rtc(&parent_rtc); retry_chain_logger ctxlog(archival_log, collection_rtc); - cloud_storage::partition_manifest manifest(ntp, remote_revision); - auto path = manifest.get_manifest_path( - cloud_storage::manifest_format::serde); - - std::string_view base_path{path().native()}; - - vassert( - base_path.ends_with(serde_extension) - && base_path.length() > serde_extension.length(), - "Generated manifest path should end in .bin"); + auto base_path = path_provider.partition_manifest_prefix( + ntp, remote_revision); - base_path.remove_suffix(serde_extension.length()); auto list_result = co_await _api.list_objects( bucket, collection_rtc, @@ -238,6 +233,7 @@ purger::collect_manifest_paths( ss::future purger::purge_manifest( const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, model::ntp ntp, model::initial_revision_id remote_revision, remote_manifest_path manifest_key, @@ -281,7 +277,12 @@ ss::future purger::purge_manifest( static_cast(manifest.size() / 1000), size_t{1}); const auto erase_result = co_await cloud_storage::remote_partition::erase( - _api, bucket, std::move(manifest), manifest_key, manifest_purge_rtc); + _api, + bucket, + path_provider, + std::move(manifest), + manifest_key, + manifest_purge_rtc); result.ops += estimate_delete_ops; @@ -397,6 +398,8 @@ ss::future purger::run(run_quota_t quota) { inc_hash.update(nt_revision.initial_revision_id); uint32_t hash = static_cast(inc_hash.digest() & 0xffffffff); + cloud_storage::remote_path_provider path_provider( + marker.config.properties.remote_label); if (my_global_position.self == hash % my_global_position.total) { vlog( archival_log.info, @@ -436,7 +439,12 @@ ss::future purger::run(run_quota_t quota) { } auto purge_r = co_await purge_partition( - marker, bucket, ntp, marker.initial_revision_id, _root_rtc); + marker, + bucket, + path_provider, + ntp, + marker.initial_revision_id, + _root_rtc); result.consumed += run_quota_t(purge_r.ops); result.remaining @@ -464,32 +472,33 @@ ss::future purger::run(run_quota_t quota) { // At this point, all partition deletions either succeeded or // permanently failed: clean up the topic manifest and erase // the controller tombstone. - auto topic_manifest_path_serde - = cloud_storage::topic_manifest::get_topic_manifest_path( - topic_config.tp_ns.ns, - topic_config.tp_ns.tp, - cloud_storage::manifest_format::serde); - auto topic_manifest_path_json - = cloud_storage::topic_manifest::get_topic_manifest_path( - topic_config.tp_ns.ns, - topic_config.tp_ns.tp, - cloud_storage::manifest_format::json); + const auto& tp_ns = topic_config.tp_ns; + auto topic_manifest_path_serde = cloud_storage_clients::object_key{ + path_provider.topic_manifest_path( + tp_ns, marker.initial_revision_id)}; vlog( archival_log.debug, "Erasing topic manifest {}", topic_manifest_path_serde); retry_chain_node topic_manifest_rtc(5s, 1s, &_root_rtc); + ss::future delete_result = _api.delete_object( + bucket, topic_manifest_path_serde, topic_manifest_rtc); + + auto topic_manifest_path_json_opt + = path_provider.topic_manifest_path_json(tp_ns); + ss::future delete_result_json + = ss::make_ready_future(upload_result::success); + cloud_storage_clients::object_key topic_manifest_path_json{}; + if (topic_manifest_path_json_opt.has_value()) { + topic_manifest_path_json = cloud_storage_clients::object_key{ + *topic_manifest_path_json_opt}; + delete_result_json = _api.delete_object( + bucket, topic_manifest_path_json, topic_manifest_rtc); + } auto [manifest_delete_result_serde, manifest_delete_result_json] = co_await ss::when_all_succeed( - _api.delete_object( - bucket, - cloud_storage_clients::object_key(topic_manifest_path_serde), - topic_manifest_rtc), - _api.delete_object( - bucket, - cloud_storage_clients::object_key(topic_manifest_path_json), - topic_manifest_rtc)); + std::move(delete_result), std::move(delete_result_json)); if ( manifest_delete_result_serde != upload_result::success || manifest_delete_result_json != upload_result::success) { @@ -556,7 +565,7 @@ ss::future purger::write_remote_lifecycle_marker( .topic = nt_revision, .status = status, }; - auto marker_key = remote_marker.get_key(); + auto marker_key = remote_marker.get_key(path_provider); co_return co_await _api.upload_object({ .transfer_details = {.bucket = bucket, .key = marker_key, .parent_rtc = marker_rtc}, diff --git a/src/v/archival/purger.h b/src/v/archival/purger.h index aaea4b6acb04..615a1baeec17 100644 --- a/src/v/archival/purger.h +++ b/src/v/archival/purger.h @@ -12,6 +12,7 @@ #include "cloud_storage/base_manifest.h" #include "cloud_storage/fwd.h" #include "cloud_storage/lifecycle_marker.h" +#include "cloud_storage/remote_path_provider.h" #include "cluster/fwd.h" #include "cluster/types.h" @@ -74,6 +75,7 @@ class purger : public housekeeping_job { ss::future purge_partition( const cluster::nt_lifecycle_marker&, const cloud_storage_clients::bucket_name& bucket, + const cloud_storage::remote_path_provider& path_provider, model::ntp, model::initial_revision_id, retry_chain_node& rtc); @@ -92,12 +94,14 @@ class purger : public housekeeping_job { ss::future> collect_manifest_paths( const cloud_storage_clients::bucket_name&, + const cloud_storage::remote_path_provider&, model::ntp, model::initial_revision_id, retry_chain_node&); ss::future purge_manifest( const cloud_storage_clients::bucket_name&, + const cloud_storage::remote_path_provider&, model::ntp, model::initial_revision_id, remote_manifest_path, diff --git a/src/v/cloud_storage/remote_partition.cc b/src/v/cloud_storage/remote_partition.cc index d3ddfc5e1e59..997f308eff62 100644 --- a/src/v/cloud_storage/remote_partition.cc +++ b/src/v/cloud_storage/remote_partition.cc @@ -17,6 +17,7 @@ #include "cloud_storage/offset_translation_layer.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/partition_manifest_downloader.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tx_range_manifest.h" #include "cloud_storage/types.h" @@ -1447,6 +1448,7 @@ void remote_partition::finalize() { ss::future remote_partition::erase( cloud_storage::remote& api, cloud_storage_clients::bucket_name bucket, + const remote_path_provider& path_provider, partition_manifest manifest, remote_manifest_path manifest_path, retry_chain_node& parent_rtc) { @@ -1458,7 +1460,8 @@ ss::future remote_partition::erase( auto replaced_segments = manifest.lw_replaced_segments(); for (const auto& lw_meta : replaced_segments) { - const auto path = manifest.generate_segment_path(lw_meta); + const auto path = manifest.generate_segment_path( + lw_meta, path_provider); ++segments_to_remove_count; objects_to_remove.emplace_back(path); @@ -1469,7 +1472,7 @@ ss::future remote_partition::erase( } for (const auto& meta : manifest) { - const auto path = manifest.generate_segment_path(meta); + const auto path = manifest.generate_segment_path(meta, path_provider); ++segments_to_remove_count; objects_to_remove.emplace_back(path); diff --git a/src/v/cloud_storage/remote_partition.h b/src/v/cloud_storage/remote_partition.h index 24a19710ad56..7fa7324bf2ef 100644 --- a/src/v/cloud_storage/remote_partition.h +++ b/src/v/cloud_storage/remote_partition.h @@ -132,6 +132,7 @@ class remote_partition static ss::future erase( cloud_storage::remote&, cloud_storage_clients::bucket_name, + const remote_path_provider& path_provider, partition_manifest, remote_manifest_path, retry_chain_node&); From a0be06a463a92bf9468967c465f4acc62862a1e8 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Tue, 18 Jun 2024 16:56:02 -0700 Subject: [PATCH 22/34] archival/segment_merger: use path provider for remote runs --- src/v/archival/adjacent_segment_merger.cc | 6 +++++- src/v/archival/adjacent_segment_run.cc | 16 +++++++++------- src/v/archival/adjacent_segment_run.h | 8 ++++++-- src/v/archival/tests/ntp_archiver_test.cc | 3 ++- src/v/archival/tests/segment_reupload_test.cc | 17 ++++++++++++++--- 5 files changed, 36 insertions(+), 14 deletions(-) diff --git a/src/v/archival/adjacent_segment_merger.cc b/src/v/archival/adjacent_segment_merger.cc index 185904303466..6426531a36c5 100644 --- a/src/v/archival/adjacent_segment_merger.cc +++ b/src/v/archival/adjacent_segment_merger.cc @@ -107,7 +107,11 @@ std::optional adjacent_segment_merger::scan_manifest( // We're looking for the remote segment break; } - if (run.maybe_add_segment(*it, max_segment_size)) { + if (run.maybe_add_segment( + manifest, + *it, + max_segment_size, + _archiver.remote_path_provider())) { // We have found a run with the size close to max_segment_size // and can proceed early. break; diff --git a/src/v/archival/adjacent_segment_run.cc b/src/v/archival/adjacent_segment_run.cc index e67a7d62c41c..ffc8a8f29cb9 100644 --- a/src/v/archival/adjacent_segment_run.cc +++ b/src/v/archival/adjacent_segment_run.cc @@ -11,7 +11,9 @@ #include "archival/logger.h" #include "base/vlog.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/types.h" +#include "model/fundamental.h" #include "model/metadata.h" #include @@ -19,7 +21,10 @@ namespace archival { bool adjacent_segment_run::maybe_add_segment( - const cloud_storage::segment_meta& s, size_t max_size) { + const cloud_storage::partition_manifest& manifest, + const cloud_storage::segment_meta& s, + size_t max_size, + const cloud_storage::remote_path_provider& path_provider) { vlog( archival_log.debug, "{} Segments collected, looking at segment meta: {}, current run meta: " @@ -27,6 +32,7 @@ bool adjacent_segment_run::maybe_add_segment( num_segments, s, meta); + auto remote_path = manifest.generate_segment_path(s, path_provider); if (num_segments == 1 && meta.size_bytes + s.size_bytes > max_size) { // Corner case, we hit a segment which is smaller than the max_size // but it's larger than max_size when combined with its neighbor. In @@ -41,9 +47,7 @@ bool adjacent_segment_run::maybe_add_segment( if (s.size_bytes < max_size) { meta = s; num_segments = 1; - segments.push_back( - cloud_storage::partition_manifest::generate_remote_segment_path( - ntp, s)); + segments.push_back(remote_path); } } else { if ( @@ -71,9 +75,7 @@ bool adjacent_segment_run::maybe_add_segment( meta.max_timestamp = s.max_timestamp; num_segments++; meta.size_bytes += s.size_bytes; - segments.push_back( - cloud_storage::partition_manifest::generate_remote_segment_path( - ntp, s)); + segments.push_back(remote_path); } else { return num_segments > 1; } diff --git a/src/v/archival/adjacent_segment_run.h b/src/v/archival/adjacent_segment_run.h index 7af73c67059c..d635f838ba2d 100644 --- a/src/v/archival/adjacent_segment_run.h +++ b/src/v/archival/adjacent_segment_run.h @@ -8,6 +8,7 @@ // by the Apache License, Version 2.0 #pragma once +#include "cloud_storage/fwd.h" #include "cloud_storage/types.h" #include "model/metadata.h" @@ -37,8 +38,11 @@ struct adjacent_segment_run { /// /// \return true if the run is assembled, false if more segments can be /// added to the run - bool - maybe_add_segment(const cloud_storage::segment_meta& s, size_t max_size); + bool maybe_add_segment( + const cloud_storage::partition_manifest& manifest, + const cloud_storage::segment_meta& s, + size_t max_size, + const cloud_storage::remote_path_provider& path_provider); }; std::ostream& operator<<(std::ostream& o, const adjacent_segment_run& run); diff --git a/src/v/archival/tests/ntp_archiver_test.cc b/src/v/archival/tests/ntp_archiver_test.cc index 2a66bc0ecff1..da4b7d85184b 100644 --- a/src/v/archival/tests/ntp_archiver_test.cc +++ b/src/v/archival/tests/ntp_archiver_test.cc @@ -1680,9 +1680,10 @@ SEASTAR_THREAD_TEST_CASE(small_segment_run_test) { .segment_term = model::term_id{4}, .sname_format = cloud_storage::segment_name_format::v2}, }; + cloud_storage::partition_manifest pm(manifest_ntp, manifest_revision); archival::adjacent_segment_run run(manifest_ntp); for (const auto& s : segments) { - if (run.maybe_add_segment(s, high_watermark)) { + if (run.maybe_add_segment(pm, s, high_watermark, path_provider)) { break; } } diff --git a/src/v/archival/tests/segment_reupload_test.cc b/src/v/archival/tests/segment_reupload_test.cc index 3f1f57c9a3d8..9fe324f8ee5c 100644 --- a/src/v/archival/tests/segment_reupload_test.cc +++ b/src/v/archival/tests/segment_reupload_test.cc @@ -12,6 +12,7 @@ #include "archival/segment_reupload.h" #include "archival/tests/service_fixture.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/types.h" #include "model/metadata.h" #include "storage/log_manager.h" @@ -26,6 +27,10 @@ using namespace archival; +namespace { +static const cloud_storage::remote_path_provider path_provider(std::nullopt); +} // anonymous namespace + inline ss::logger test_log("test"); static constexpr std::string_view manifest = R"json({ @@ -1350,6 +1355,7 @@ SEASTAR_THREAD_TEST_CASE(test_adjacent_segment_collection_x_term) { // This covers three segments with total size of 3000 BOOST_REQUIRE(!run.maybe_add_segment( + m, cloud_storage::segment_meta{ .is_compacted = false, .size_bytes = 100, @@ -1364,9 +1370,11 @@ SEASTAR_THREAD_TEST_CASE(test_adjacent_segment_collection_x_term) { .delta_offset_end = model::offset_delta(0), .sname_format = cloud_storage::segment_name_format::v3, }, - 5000)); + 5000, + path_provider)); BOOST_REQUIRE(!run.maybe_add_segment( + m, cloud_storage::segment_meta{ .is_compacted = false, .size_bytes = 100, @@ -1381,13 +1389,15 @@ SEASTAR_THREAD_TEST_CASE(test_adjacent_segment_collection_x_term) { .delta_offset_end = model::offset_delta(0), .sname_format = cloud_storage::segment_name_format::v3, }, - 5000)); + 5000, + path_provider)); // The extra segment fits in by size but can't be added because it // has different term. The method should return 'true' because // we were able to add a segment to the run and we can't extend it // further. BOOST_REQUIRE(run.maybe_add_segment( + m, cloud_storage::segment_meta{ .is_compacted = false, .size_bytes = 100, @@ -1402,7 +1412,8 @@ SEASTAR_THREAD_TEST_CASE(test_adjacent_segment_collection_x_term) { .delta_offset_end = model::offset_delta(0), .sname_format = cloud_storage::segment_name_format::v3, }, - 5000)); + 5000, + path_provider)); BOOST_REQUIRE_EQUAL(run.num_segments, 2); BOOST_REQUIRE_EQUAL(run.meta.base_offset(), 0); From d2681ba5d4c19a0c49fe8a6ae07130a4b9d893ba Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Tue, 18 Jun 2024 17:49:22 -0700 Subject: [PATCH 23/34] admin: use path provider for anomalies report --- src/v/redpanda/admin/server.cc | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index d5b3b214b896..eb82fca57924 100644 --- a/src/v/redpanda/admin/server.cc +++ b/src/v/redpanda/admin/server.cc @@ -15,6 +15,7 @@ #include "base/vlog.h" #include "cloud_storage/cache_service.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" #include "cluster/cluster_recovery_manager.h" #include "cluster/cluster_recovery_table.h" @@ -3969,6 +3970,7 @@ map_metadata_anomaly_to_json(const cloud_storage::anomaly_meta& meta) { ss::httpd::shadow_indexing_json::cloud_storage_partition_anomalies map_anomalies_to_json( + const cloud_storage::remote_path_provider& path_provider, const model::ntp& ntp, const model::initial_revision_id& initial_rev, const cloud_storage::anomalies& detected) { @@ -4001,24 +4003,21 @@ map_anomalies_to_json( json.missing_partition_manifest = true; } + cloud_storage::partition_manifest tmp{ntp, initial_rev}; if (detected.missing_spillover_manifests.size() > 0) { const auto& missing_spills = detected.missing_spillover_manifests; for (auto iter = missing_spills.begin(); iter != missing_spills.end(); ++iter) { json.missing_spillover_manifests.push( - cloud_storage::generate_spillover_manifest_path( - ntp, initial_rev, *iter)() - .string()); + path_provider.spillover_manifest_path(tmp, *iter)); } } if (detected.missing_segments.size() > 0) { - cloud_storage::partition_manifest tmp{ntp, initial_rev}; const auto& missing_segs = detected.missing_segments; for (auto iter = missing_segs.begin(); iter != missing_segs.end(); ++iter) { - json.missing_segments.push( - tmp.generate_segment_path(*iter)().string()); + json.missing_segments.push(path_provider.segment_path(tmp, *iter)); } } @@ -4218,11 +4217,12 @@ admin_server::get_cloud_storage_anomalies( const auto& topic_table = _controller->get_topics_state().local(); const auto initial_rev = topic_table.get_initial_revision(ntp); - if (!initial_rev) { + const auto& tp = topic_table.get_topic_cfg( + model::topic_namespace{ntp.ns, ntp.tp.topic}); + if (!initial_rev.has_value() || !tp.has_value()) { throw ss::httpd::not_found_exception( fmt::format("topic {} not found", ntp.tp)); } - const auto shard = _shard_table.local().shard_for(ntp); if (!shard) { throw ss::httpd::not_found_exception(fmt::format( @@ -4231,6 +4231,8 @@ admin_server::get_cloud_storage_anomalies( ntp)); } + cloud_storage::remote_path_provider path_provider( + tp->properties.remote_label); auto status = co_await _partition_manager.invoke_on( *shard, [&ntp](const auto& pm) -> std::optional { @@ -4249,7 +4251,7 @@ admin_server::get_cloud_storage_anomalies( "Cloud partition {} could not be found on shard {}.", ntp, *shard)); } - co_return map_anomalies_to_json(ntp, *initial_rev, *status); + co_return map_anomalies_to_json(path_provider, ntp, *initial_rev, *status); } ss::future> From 1066344eb0fc9e16d9eaa64b3cd1c349d1642c74 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 3 Jul 2024 22:28:11 -0700 Subject: [PATCH 24/34] features: add flag for remote labels --- src/v/features/feature_table.cc | 2 ++ src/v/features/feature_table.h | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/src/v/features/feature_table.cc b/src/v/features/feature_table.cc index f91d00987c10..1540868e1583 100644 --- a/src/v/features/feature_table.cc +++ b/src/v/features/feature_table.cc @@ -109,6 +109,8 @@ std::string_view to_string_view(feature f) { return "group_tx_fence_dedicated_batch_type"; case feature::transforms_specify_offset: return "transforms_specify_offset"; + case feature::remote_labels: + return "remote_labels"; /* * testing features diff --git a/src/v/features/feature_table.h b/src/v/features/feature_table.h index 89101c808b6d..8191116a816f 100644 --- a/src/v/features/feature_table.h +++ b/src/v/features/feature_table.h @@ -79,6 +79,7 @@ enum class feature : std::uint64_t { data_migrations = 1ULL << 48U, group_tx_fence_dedicated_batch_type = 1ULL << 49U, transforms_specify_offset = 1ULL << 50U, + remote_labels = 1ULL << 51U, // Dummy features for testing only test_alpha = 1ULL << 61U, @@ -409,6 +410,12 @@ constexpr static std::array feature_schema{ feature::transforms_specify_offset, feature_spec::available_policy::always, feature_spec::prepare_policy::always}, + feature_spec{ + cluster::cluster_version{13}, + "remote_labels", + feature::remote_labels, + feature_spec::available_policy::always, + feature_spec::prepare_policy::always}, }; std::string_view to_string_view(feature); From 2d7e19a14f4a3c891070313eb507492a492cdfa5 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 20 Jun 2024 01:28:45 -0700 Subject: [PATCH 25/34] config: property for disabling remote labels for tests --- src/v/config/configuration.cc | 10 ++++++++++ src/v/config/configuration.h | 1 + 2 files changed, 11 insertions(+) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 92abbc7f6999..0a60ef306895 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -2161,6 +2161,16 @@ configuration::configuration() "timeout error.", {.needs_restart = needs_restart::no, .visibility = visibility::tunable}, 600s) + , cloud_storage_disable_remote_labels_for_tests( + *this, + "cloud_storage_disable_remote_labels_for_tests", + "If 'true', Redpanda disables remote labels and falls back on the " + "hash-based object naming scheme for new topics. This property exists to " + "simplify testing " + "and shouldn't be set in production.", + {.needs_restart = needs_restart::no, .visibility = visibility::tunable}, + // TODO(awong): enable once stable + true) , cloud_storage_azure_storage_account( *this, "cloud_storage_azure_storage_account", diff --git a/src/v/config/configuration.h b/src/v/config/configuration.h index 92c65d77c754..08e348fbcc19 100644 --- a/src/v/config/configuration.h +++ b/src/v/config/configuration.h @@ -394,6 +394,7 @@ struct configuration final : public config_store { property cloud_storage_disable_upload_consistency_checks; property cloud_storage_disable_metadata_consistency_checks; property cloud_storage_hydration_timeout_ms; + property cloud_storage_disable_remote_labels_for_tests; // Azure Blob Storage property> cloud_storage_azure_storage_account; From e7669a9a855f48791156c7030ae2f4cb6c69557a Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 19 Jun 2024 23:24:55 -0700 Subject: [PATCH 26/34] cloud_storage: clean up manifest path generation from manifests Topic/partition manifests are by far the biggest offender of things that generate paths without the path provider that really should be using the path provider (because their paths naturally collide across clusters in the same bucket). Thus far, we've been switching the important usages of path generation to use the path provider, which helps avoid collisions. This commit tightens things up a bit, cleaning up the remaining usages of manifest-generated paths. This means removal of: - static manifest path generation methods - format-focused interfaces, which were generally not using the format for anything important (mostly logging) - methods in the manifest mentioning "legacy" formats (now subsumed by path utils and the path provider) - importantly, remote::upload_manifest(), the most important offender of using paths generated by the manifest without consulting the path provider Several tests and log lines are updated to accomodate this cleanup. --- src/v/archival/ntp_archiver_service.cc | 2 +- src/v/archival/tests/ntp_archiver_test.cc | 22 +++--- src/v/cloud_storage/anomalies_detector.cc | 11 +-- src/v/cloud_storage/base_manifest.h | 3 - src/v/cloud_storage/partition_manifest.cc | 70 ++++--------------- src/v/cloud_storage/partition_manifest.h | 25 +------ src/v/cloud_storage/remote.cc | 8 --- src/v/cloud_storage/remote.h | 6 +- src/v/cloud_storage/spillover_manifest.h | 37 ---------- .../tests/anomalies_detector_test.cc | 28 ++++---- .../tests/async_manifest_view_test.cc | 8 +-- .../tests/partition_manifest_test.cc | 23 +++--- .../tests/remote_partition_test.cc | 18 ++--- src/v/cloud_storage/tests/remote_test.cc | 5 +- .../tests/topic_manifest_test.cc | 17 +++-- src/v/cloud_storage/tests/util.cc | 12 ++-- src/v/cloud_storage/topic_manifest.cc | 34 +++------ src/v/cloud_storage/topic_manifest.h | 8 +-- src/v/cloud_storage/tx_range_manifest.h | 2 +- .../cluster/cloud_metadata/cluster_manifest.h | 2 +- .../tests/manifest_downloads_test.cc | 11 ++- .../cloud_metadata/tests/uploader_test.cc | 8 ++- src/v/cluster/cloud_metadata/uploader.cc | 2 +- src/v/cluster/topic_recovery_service.cc | 6 +- 24 files changed, 134 insertions(+), 234 deletions(-) diff --git a/src/v/archival/ntp_archiver_service.cc b/src/v/archival/ntp_archiver_service.cc index a8e8b249bb44..d1296859c49c 100644 --- a/src/v/archival/ntp_archiver_service.cc +++ b/src/v/archival/ntp_archiver_service.cc @@ -1441,7 +1441,7 @@ ss::future ntp_archiver::upload_tx( cloud_storage::tx_range_manifest manifest(path, std::move(tx_range)); co_return co_await _remote.upload_manifest( - get_bucket_name(), manifest, fib); + get_bucket_name(), manifest, manifest.get_manifest_path(), fib); } ss::future> diff --git a/src/v/archival/tests/ntp_archiver_test.cc b/src/v/archival/tests/ntp_archiver_test.cc index da4b7d85184b..6828f69f3582 100644 --- a/src/v/archival/tests/ntp_archiver_test.cc +++ b/src/v/archival/tests/ntp_archiver_test.cc @@ -97,18 +97,18 @@ static void log_segment_set(storage::log_manager& lm) { } static remote_manifest_path generate_spill_manifest_path( - model::ntp ntp, - model::initial_revision_id rev_id, - const cloud_storage::segment_meta& meta) { + const cloud_storage::partition_manifest& stm_manifest, + const cloud_storage::segment_meta& spillover_manifest) { cloud_storage::spillover_manifest_path_components comp{ - .base = meta.base_offset, - .last = meta.committed_offset, - .base_kafka = meta.base_kafka_offset(), - .next_kafka = meta.next_kafka_offset(), - .base_ts = meta.base_timestamp, - .last_ts = meta.max_timestamp, + .base = spillover_manifest.base_offset, + .last = spillover_manifest.committed_offset, + .base_kafka = spillover_manifest.base_kafka_offset(), + .next_kafka = spillover_manifest.next_kafka_offset(), + .base_ts = spillover_manifest.base_timestamp, + .last_ts = spillover_manifest.max_timestamp, }; - return cloud_storage::generate_spillover_manifest_path(ntp, rev_id, comp); + return remote_manifest_path{ + path_provider.spillover_manifest_path(stm_manifest, comp)}; } void log_upload_candidate(const archival::upload_candidate& up) { @@ -639,7 +639,7 @@ FIXTURE_TEST(test_archive_retention, archiver_fixture) { BOOST_REQUIRE_EQUAL(spills.begin()->base_offset, model::offset{0}); BOOST_REQUIRE_EQUAL(spills.begin()->committed_offset, model::offset{1999}); auto spill_path = generate_spill_manifest_path( - manifest_ntp, manifest_revision, *(spills.begin())); + part->archival_meta_stm()->manifest(), *(spills.begin())); config::shard_local_cfg().log_retention_ms.set_value( std::chrono::milliseconds{5min}); diff --git a/src/v/cloud_storage/anomalies_detector.cc b/src/v/cloud_storage/anomalies_detector.cc index 7f3a5498274d..5c4a2cac030c 100644 --- a/src/v/cloud_storage/anomalies_detector.cc +++ b/src/v/cloud_storage/anomalies_detector.cc @@ -174,7 +174,10 @@ anomalies_detector::check_manifest( const partition_manifest& manifest, std::optional scrub_from, retry_chain_node& rtc_node) { - vlog(_logger.debug, "Checking manifest {}", manifest.get_manifest_path()); + vlog( + _logger.debug, + "Checking manifest {}", + manifest.get_manifest_path(_remote_path_provider)); if ( scrub_from && (manifest.get_start_offset() > *scrub_from || manifest.get_last_offset() == scrub_from)) { @@ -185,7 +188,7 @@ anomalies_detector::check_manifest( "Skipping ...", manifest.get_start_offset(), manifest.get_last_offset(), - manifest.get_manifest_path(), + manifest.get_manifest_path(_remote_path_provider), scrub_from); co_return stop_detector::no; @@ -211,7 +214,7 @@ anomalies_detector::check_manifest( manifest.get_start_offset(), manifest.get_last_offset(), manifest.size(), - manifest.get_manifest_path()); + manifest.get_manifest_path(_remote_path_provider)); co_return stop_detector::no; } std::optional previous_seg_meta; @@ -261,7 +264,7 @@ anomalies_detector::check_manifest( vlog( _logger.debug, "Finished checking manifest {}", - manifest.get_manifest_path()); + manifest.get_manifest_path(_remote_path_provider)); co_return stop_detector::no; } diff --git a/src/v/cloud_storage/base_manifest.h b/src/v/cloud_storage/base_manifest.h index 1951b0964f21..6369a9660db3 100644 --- a/src/v/cloud_storage/base_manifest.h +++ b/src/v/cloud_storage/base_manifest.h @@ -57,9 +57,6 @@ class base_manifest { /// \return asynchronous input_stream with the serialized json virtual ss::future serialize() const = 0; - /// Manifest object format and name in S3 - virtual remote_manifest_path get_manifest_path() const = 0; - /// Get manifest type virtual manifest_type get_manifest_type() const = 0; }; diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index 05d984ba4855..1970519fecda 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -16,6 +16,7 @@ #include "bytes/streambuf.h" #include "cloud_storage/base_manifest.h" #include "cloud_storage/logger.h" +#include "cloud_storage/partition_path_utils.h" #include "cloud_storage/remote_path_provider.h" #include "cloud_storage/segment_meta_cstore.h" #include "cloud_storage/types.h" @@ -179,58 +180,15 @@ partition_manifest::partition_manifest( , _segments() , _last_offset(0) {} -// NOTE: the methods that generate remote paths use the xxhash function -// to randomize the prefix. S3 groups the objects into chunks based on -// these prefixes. It also applies rate limit to chunks so if all segments -// and manifests will have the same prefix we will be able to do around -// 3000-5000 req/sec. AWS doc mentions that having only two prefix -// characters should be enough for most workloads -// (https://aws.amazon.com/blogs/aws/amazon-s3-performance-tips-tricks-seattle-hiring-event/) -// We're using eight because it's free and because AWS S3 is not the only -// backend and other S3 API implementations might benefit from that. - -remote_manifest_path generate_partition_manifest_path( - const model::ntp& ntp, - model::initial_revision_id rev, - manifest_format format) { - // NOTE: the idea here is to split all possible hash values into - // 16 bins. Every bin should have lowest 28-bits set to 0. - // As result, for segment names all prefixes are possible, but - // for manifests, only 0x00000000, 0x10000000, ... 0xf0000000 - // are used. This will allow us to quickly find all manifests - // that S3 bucket contains. - constexpr uint32_t bitmask = 0xF0000000; - auto path = ssx::sformat("{}_{}", ntp.path(), rev()); - uint32_t hash = bitmask & xxhash_32(path.data(), path.size()); - return remote_manifest_path(fmt::format( - "{:08x}/meta/{}_{}/manifest.{}", hash, ntp.path(), rev(), [&] { - switch (format) { - case manifest_format::json: - return "json"; - case manifest_format::serde: - return "bin"; - } - }())); -} - -std::pair -partition_manifest::get_manifest_format_and_path() const { - return { - manifest_format::serde, - generate_partition_manifest_path(_ntp, _rev, manifest_format::serde)}; -} - remote_manifest_path partition_manifest::get_manifest_path( const remote_path_provider& path_provider) const { return remote_manifest_path{path_provider.partition_manifest_path(*this)}; } -std::pair -partition_manifest::get_legacy_manifest_format_and_path() const { - return { - manifest_format::json, - generate_partition_manifest_path(_ntp, _rev, manifest_format::json)}; +ss::sstring partition_manifest::display_name() const { + return fmt::format("{}_{}", get_ntp().path(), _rev()); } + const model::ntp& partition_manifest::get_ntp() const { return _ntp; } model::offset partition_manifest::get_last_offset() const { @@ -614,7 +572,7 @@ void partition_manifest::set_archive_clean_offset( "{} Requested to advance archive_clean_offset to {} which is greater " "than the current archive_start_offset {}. The offset won't be " "changed. Archive size won't be changed by {} bytes.", - get_manifest_path(), + display_name(), start_rp_offset, _archive_start_offset, size_bytes); @@ -638,7 +596,7 @@ void partition_manifest::set_archive_clean_offset( "{} archive clean offset moved to {} but the archive size can't " "be updated because current size {} is smaller than the update " "{}. This needs to be reported and investigated.", - get_manifest_path(), + display_name(), _archive_clean_offset, _archive_size_bytes, size_bytes); @@ -762,7 +720,7 @@ bool partition_manifest::advance_start_offset(model::offset new_start_offset) { cst_log.error, "Previous start offset is not within segment in " "manifest for {}: previous_start_offset={}", - get_manifest_path(), + display_name(), previous_start_offset); previous_head_segment = _segments.begin(); } @@ -931,7 +889,7 @@ size_t partition_manifest::safe_segment_meta_to_add( "[{}] New segment does not line up with last offset of empty " "log: " "last_offset: {}, new_segment: {}", - get_manifest_path(), + display_name(), subst.last_offset, m); break; @@ -976,7 +934,7 @@ size_t partition_manifest::safe_segment_meta_to_add( cst_log.error, "[{}] New segment does not line up with previous " "segment: {}", - get_manifest_path(), + display_name(), format_seg_meta_anomalies(anomalies)); break; } @@ -1002,7 +960,7 @@ size_t partition_manifest::safe_segment_meta_to_add( "[{}] New replacement segment does not line up with " "previous " "segment: {}", - get_manifest_path(), + display_name(), format_seg_meta_anomalies(anomalies)); break; } @@ -1017,7 +975,7 @@ size_t partition_manifest::safe_segment_meta_to_add( "[{}] New replacement segment has the same size as " "replaced " "segment: new_segment: {}, replaced_segment: {}", - get_manifest_path(), + display_name(), m, *it); break; @@ -1047,7 +1005,7 @@ size_t partition_manifest::safe_segment_meta_to_add( "committed " "offset of " "any previous segment: new_segment: {}", - get_manifest_path(), + display_name(), m); break; } @@ -1169,7 +1127,7 @@ void partition_manifest::spillover(const segment_meta& spillover_meta) { cst_log.error, "[{}] Expected spillover metadata {} doesn't match actual spillover " "metadata {}", - get_manifest_path(), + display_name(), expected_meta, spillover_meta); } else { @@ -1957,7 +1915,7 @@ void partition_manifest::update_with_json(iobuf buf) { throw std::runtime_error(fmt_with_ctx( fmt::format, "Failed to parse partition manifest {}: {} at offset {}", - get_legacy_manifest_format_and_path().second, + prefixed_partition_manifest_json_path(get_ntp(), get_revision_id()), rapidjson::GetParseError_En(e), o)); } diff --git a/src/v/cloud_storage/partition_manifest.h b/src/v/cloud_storage/partition_manifest.h index cd69c2f9a7ad..eb5804b244a9 100644 --- a/src/v/cloud_storage/partition_manifest.h +++ b/src/v/cloud_storage/partition_manifest.h @@ -67,9 +67,6 @@ remote_segment_path generate_remote_segment_path( /// Generate correct S3 segment name based on term and base offset segment_name generate_local_segment_name(model::offset o, model::term_id t); -remote_manifest_path generate_partition_manifest_path( - const model::ntp&, model::initial_revision_id, manifest_format); - // This structure can be impelenented // to allow access to private fields of the manifest. struct partition_manifest_accessor; @@ -200,33 +197,12 @@ class partition_manifest : public base_manifest { } } - /// Manifest object name in S3 - std::pair - get_manifest_format_and_path() const; - - remote_manifest_path get_manifest_path(manifest_format fmt) const { - switch (fmt) { - case manifest_format::json: - return get_legacy_manifest_format_and_path().second; - case manifest_format::serde: - return get_manifest_format_and_path().second; - } - } - virtual remote_manifest_path get_manifest_path(const remote_path_provider&) const; static ss::sstring filename() { return "manifest.bin"; } virtual ss::sstring get_manifest_filename() const { return filename(); } - remote_manifest_path get_manifest_path() const override { - return get_manifest_format_and_path().second; - } - - /// Manifest object name before feature::cloud_storage_manifest_format_v2 - std::pair - get_legacy_manifest_format_and_path() const; - /// Get NTP const model::ntp& get_ntp() const; @@ -615,6 +591,7 @@ class partition_manifest : public base_manifest { anomalies detected); private: + ss::sstring display_name() const; std::optional compute_start_kafka_offset_local() const; void set_start_offset(model::offset start_offset); diff --git a/src/v/cloud_storage/remote.cc b/src/v/cloud_storage/remote.cc index db9fb52829fa..0ff57ad70c07 100644 --- a/src/v/cloud_storage/remote.cc +++ b/src/v/cloud_storage/remote.cc @@ -336,14 +336,6 @@ ss::future remote::do_download_manifest( co_return *result; } -ss::future remote::upload_manifest( - const cloud_storage_clients::bucket_name& bucket, - const base_manifest& manifest, - retry_chain_node& parent) { - auto key = manifest.get_manifest_path(); - co_return co_await upload_manifest(bucket, manifest, key, parent); -} - ss::future remote::upload_manifest( const cloud_storage_clients::bucket_name& bucket, const base_manifest& manifest, diff --git a/src/v/cloud_storage/remote.h b/src/v/cloud_storage/remote.h index 80392112f7d6..937d5f152252 100644 --- a/src/v/cloud_storage/remote.h +++ b/src/v/cloud_storage/remote.h @@ -272,12 +272,8 @@ class remote /// /// \param bucket is a bucket name /// \param manifest is a manifest to upload + /// \param key is the remote object name /// \return future that returns success code - ss::future upload_manifest( - const cloud_storage_clients::bucket_name& bucket, - const base_manifest& manifest, - retry_chain_node& parent); - ss::future upload_manifest( const cloud_storage_clients::bucket_name& bucket, const base_manifest& manifest, diff --git a/src/v/cloud_storage/spillover_manifest.h b/src/v/cloud_storage/spillover_manifest.h index 9e0321243ef7..9500f564b461 100644 --- a/src/v/cloud_storage/spillover_manifest.h +++ b/src/v/cloud_storage/spillover_manifest.h @@ -19,28 +19,6 @@ namespace cloud_storage { -namespace { - -remote_manifest_path generate_spillover_manifest_path( - const model::ntp& ntp, - model::initial_revision_id rev, - const spillover_manifest_path_components& c) { - auto path = generate_partition_manifest_path( - ntp, rev, manifest_format::serde); - // Given the topic name size limit the name should fit into - // the AWS S3 size limit. - return remote_manifest_path(fmt::format( - "{}.{}.{}.{}.{}.{}.{}", - path().string(), - c.base(), - c.last(), - c.base_kafka(), - c.next_kafka(), - c.base_ts.value(), - c.last_ts.value())); -} -} // namespace - /// The section of the partition manifest /// /// The only purpose of this class is to provide different implementation of the @@ -84,21 +62,6 @@ class spillover_manifest final : public partition_manifest { }; return filename(c); } - remote_manifest_path get_manifest_path() const override { - const auto ls = last_segment(); - vassert(ls.has_value(), "Spillover manifest can't be empty"); - const auto fs = *begin(); - spillover_manifest_path_components smc{ - .base = fs.base_offset, - .last = ls->committed_offset, - .base_kafka = fs.base_kafka_offset(), - .next_kafka = ls->next_kafka_offset(), - .base_ts = fs.base_timestamp, - .last_ts = ls->max_timestamp, - }; - return generate_spillover_manifest_path( - get_ntp(), get_revision_id(), smc); - } manifest_type get_manifest_type() const override { return manifest_type::spillover; diff --git a/src/v/cloud_storage/tests/anomalies_detector_test.cc b/src/v/cloud_storage/tests/anomalies_detector_test.cc index dbffa02230e3..aff1c17f4de7 100644 --- a/src/v/cloud_storage/tests/anomalies_detector_test.cc +++ b/src/v/cloud_storage/tests/anomalies_detector_test.cc @@ -12,6 +12,7 @@ #include "cloud_storage/anomalies_detector.h" #include "cloud_storage/base_manifest.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_path_utils.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" @@ -361,16 +362,16 @@ class bucket_view_fixture : http_imposter_fixture { } void remove_manifest(const cloud_storage::partition_manifest& manifest) { - auto path = manifest.get_manifest_path(); + auto path = manifest.get_manifest_path(path_provider); remove_object(ssx::sformat("/{}", path().string())); } private: void remove_json_stm_manifest( const cloud_storage::partition_manifest& manifest) { - auto path = manifest.get_manifest_path( - cloud_storage::manifest_format::json); - remove_object(ssx::sformat("/{}", path().string())); + auto path = cloud_storage::prefixed_partition_manifest_json_path( + manifest.get_ntp(), manifest.get_revision_id()); + remove_object(ssx::sformat("/{}", path)); } void remove_object(ss::sstring full_path) { @@ -429,15 +430,16 @@ class bucket_view_fixture : http_imposter_fixture { .last_ts = iter->max_timestamp, }; - auto spill_path = cloud_storage::generate_spillover_manifest_path( - _stm_manifest.get_ntp(), _stm_manifest.get_revision_id(), comp); - BOOST_REQUIRE_EQUAL(spill_path, spill.get_manifest_path()); + auto spill_path = cloud_storage::remote_manifest_path{ + path_provider.spillover_manifest_path(_stm_manifest, comp)}; + BOOST_REQUIRE_EQUAL( + spill_path, spill.get_manifest_path(path_provider)); } } void set_expectations_for_manifest( const cloud_storage::partition_manifest& manifest) { - const auto path = manifest.get_manifest_path()().string(); + const auto path = manifest.get_manifest_path(path_provider)().string(); const auto reply_body = iobuf_to_string(manifest.to_iobuf()); when() @@ -606,11 +608,11 @@ FIXTURE_TEST(test_missing_spillover_manifest, bucket_view_fixture) { const auto& missing_spills = result.detected.missing_spillover_manifests; BOOST_REQUIRE_EQUAL(missing_spills.size(), 1); - const auto expected_path = cloud_storage::generate_spillover_manifest_path( - first_spill.get_ntp(), - first_spill.get_revision_id(), - *missing_spills.begin()); - BOOST_REQUIRE_EQUAL(first_spill.get_manifest_path(), expected_path); + auto expected_path = cloud_storage::remote_manifest_path{ + path_provider.spillover_manifest_path( + first_spill, *missing_spills.begin())}; + BOOST_REQUIRE_EQUAL( + first_spill.get_manifest_path(path_provider), expected_path); auto partial_results = run_detector_until_log_end(archival::run_quota_t{6}); diff --git a/src/v/cloud_storage/tests/async_manifest_view_test.cc b/src/v/cloud_storage/tests/async_manifest_view_test.cc index b1c93d86b500..9ed17e0684e1 100644 --- a/src/v/cloud_storage/tests/async_manifest_view_test.cc +++ b/src/v/cloud_storage/tests/async_manifest_view_test.cc @@ -86,7 +86,7 @@ class async_manifest_view_fixture expectation spill_manifest(const spillover_manifest& spm, bool hydrate) { stm_manifest.spillover(spm.make_manifest_metadata()); // update cache - auto path = spm.get_manifest_path(); + auto path = spm.get_manifest_path(path_provider); if (hydrate) { auto stream = spm.serialize().get(); auto reservation = cache.local().reserve_space(123, 1).get(); @@ -131,7 +131,7 @@ class async_manifest_view_fixture } void put_spill_to_cache(const spillover_manifest& spm) { - auto path = spm.get_manifest_path(); + auto path = spm.get_manifest_path(path_provider); auto stream = spm.serialize().get(); auto reservation = cache.local().reserve_space(123, 1).get(); cache.local() @@ -163,7 +163,7 @@ class async_manifest_view_fixture in_stream.close().get(); out_stream.close().get(); ss::sstring body = linearize_iobuf(std::move(tmp_buf)); - auto path = pm.get_manifest_path(); + auto path = pm.get_manifest_path(path_provider); _expectations.push_back({ .url = path().string(), .body = body, @@ -209,7 +209,7 @@ class async_manifest_view_fixture stm_manifest.spillover(spm.make_manifest_metadata()); // update cache - auto path = spm.get_manifest_path(); + auto path = spm.get_manifest_path(path_provider); if (hydrate) { auto stream = spm.serialize().get(); auto reservation = cache.local().reserve_space(123, 1).get(); diff --git a/src/v/cloud_storage/tests/partition_manifest_test.cc b/src/v/cloud_storage/tests/partition_manifest_test.cc index 27b2d1a54467..07a556d4f897 100644 --- a/src/v/cloud_storage/tests/partition_manifest_test.cc +++ b/src/v/cloud_storage/tests/partition_manifest_test.cc @@ -14,6 +14,7 @@ #include "bytes/iostream.h" #include "cloud_storage/base_manifest.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "model/fundamental.h" @@ -32,6 +33,10 @@ using namespace cloud_storage; +namespace { +const remote_path_provider path_provider(std::nullopt); +} // anonymous namespace + static constexpr std::string_view empty_manifest_json = R"json({ "version": 1, "namespace": "test-ns", @@ -503,7 +508,7 @@ SEASTAR_THREAD_TEST_CASE(test_segment_name_parsing_failure_2) { SEASTAR_THREAD_TEST_CASE(test_manifest_path) { partition_manifest m(manifest_ntp, model::initial_revision_id(0)); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "20000000/meta/test-ns/test-topic/42_0/manifest.bin"); } @@ -512,7 +517,7 @@ SEASTAR_THREAD_TEST_CASE(test_empty_manifest_update) { partition_manifest m; m.update(manifest_format::json, make_manifest_stream(empty_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "20000000/meta/test-ns/test-topic/42_0/manifest.bin"); } @@ -539,7 +544,7 @@ SEASTAR_THREAD_TEST_CASE(test_complete_manifest_update) { m.update( manifest_format::json, make_manifest_stream(complete_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "60000000/meta/test-ns/test-topic/42_1/manifest.bin"); BOOST_REQUIRE_EQUAL(m.size(), 5); @@ -633,7 +638,7 @@ SEASTAR_THREAD_TEST_CASE(test_max_segment_meta_update) { manifest_format::json, make_manifest_stream(max_segment_meta_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "60000000/meta/test-ns/test-topic/42_1/manifest.bin"); BOOST_REQUIRE_EQUAL(m.size(), 1); @@ -702,7 +707,7 @@ SEASTAR_THREAD_TEST_CASE(test_metas_get_smaller) { manifest_format::json, make_manifest_stream(segment_meta_gets_smaller_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "60000000/meta/test-ns/test-topic/42_1/manifest.bin"); BOOST_REQUIRE_EQUAL(m.size(), 2); @@ -752,8 +757,8 @@ SEASTAR_THREAD_TEST_CASE(test_no_closing_bracket_meta) { [](std::runtime_error ex) { return std::string(ex.what()).find( "Failed to parse partition manifest " - "\"b0000000/meta///-2147483648_-9223372036854775808/" - "manifest.json\": Missing a comma or '}' after an object " + "b0000000/meta///-2147483648_-9223372036854775808/" + "manifest.json: Missing a comma or '}' after an object " "member. at offset 325") != std::string::npos; }); @@ -764,7 +769,7 @@ SEASTAR_THREAD_TEST_CASE(test_fields_after_segments) { m.update( manifest_format::json, make_manifest_stream(fields_after_segments_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "60000000/meta/test-ns/test-topic/42_1/manifest.bin"); BOOST_REQUIRE_EQUAL(m.size(), 1); @@ -2013,7 +2018,7 @@ SEASTAR_THREAD_TEST_CASE(test_reset_manifest) { m.update( manifest_format::json, make_manifest_stream(complete_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "60000000/meta/test-ns/test-topic/42_1/manifest.bin"); BOOST_REQUIRE_EQUAL(m.size(), 5); diff --git a/src/v/cloud_storage/tests/remote_partition_test.cc b/src/v/cloud_storage/tests/remote_partition_test.cc index 5f11aefb0039..a872634360ab 100644 --- a/src/v/cloud_storage/tests/remote_partition_test.cc +++ b/src/v/cloud_storage/tests/remote_partition_test.cc @@ -16,6 +16,7 @@ #include "cloud_storage/download_exception.h" #include "cloud_storage/offset_translation_layer.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/partition_path_utils.h" #include "cloud_storage/remote.h" #include "cloud_storage/remote_partition.h" #include "cloud_storage/remote_segment.h" @@ -418,7 +419,8 @@ FIXTURE_TEST(test_overlapping_segments, cloud_storage_fixture) { body.find(to_replace), to_replace.size(), "\"committed_offset\":6"); // overwrite uploaded manifest with a json version expectations.back() = { - .url = manifest.get_legacy_manifest_format_and_path().second().string(), + .url = prefixed_partition_manifest_json_path( + manifest.get_ntp(), manifest.get_revision_id()), .body = body}; set_expectations_and_listen(expectations); BOOST_REQUIRE(check_scan(*this, kafka::offset(0), 9)); @@ -2284,10 +2286,10 @@ FIXTURE_TEST(test_out_of_range_query, cloud_storage_fixture) { vlog( test_util_log.info, "Rewriting manifest at {}:\n{}", - manifest.get_manifest_path(), + manifest.get_manifest_path(path_provider), ostr.str()); - auto manifest_url = manifest.get_manifest_path()().string(); + auto manifest_url = manifest.get_manifest_path(path_provider)().string(); remove_expectations({manifest_url}); add_expectations({ cloud_storage_fixture::expectation{ @@ -2357,13 +2359,13 @@ FIXTURE_TEST(test_out_of_range_spillover_query, cloud_storage_fixture) { vlog( test_util_log.info, "Uploading spillover manifest at {}:\n{}", - spm.get_manifest_path(), + spm.get_manifest_path(path_provider), ostr.str()); auto s_data = spm.serialize().get(); auto buf = s_data.stream.read_exactly(s_data.size_bytes).get(); add_expectations({cloud_storage_fixture::expectation{ - .url = spm.get_manifest_path()().string(), + .url = spm.get_manifest_path(path_provider)().string(), .body = ss::sstring(buf.begin(), buf.end()), }}); } @@ -2388,10 +2390,10 @@ FIXTURE_TEST(test_out_of_range_spillover_query, cloud_storage_fixture) { vlog( test_util_log.info, "Rewriting manifest at {}:\n{}", - manifest.get_manifest_path(), + manifest.get_manifest_path(path_provider), ostr.str()); - auto manifest_url = manifest.get_manifest_path()().string(); + auto manifest_url = manifest.get_manifest_path(path_provider)().string(); remove_expectations({manifest_url}); add_expectations({ cloud_storage_fixture::expectation{ @@ -2476,7 +2478,7 @@ FIXTURE_TEST(test_out_of_range_spillover_query, cloud_storage_fixture) { vlog( test_util_log.info, "Rewriting manifest at {}:\n{}", - manifest.get_manifest_path(), + manifest.get_manifest_path(path_provider), ostr.str()); remove_expectations({manifest_url}); diff --git a/src/v/cloud_storage/tests/remote_test.cc b/src/v/cloud_storage/tests/remote_test.cc index 8d0d82a3b50e..906c2ef66a47 100644 --- a/src/v/cloud_storage/tests/remote_test.cc +++ b/src/v/cloud_storage/tests/remote_test.cc @@ -1095,7 +1095,10 @@ TEST_P(all_types_remote_fixture, test_filter_by_type) { // NOLINT subscription2.get().type == api_activity_type::manifest_download); auto upl_res - = remote.local().upload_manifest(bucket_name, actual, root_rtc).get(); + = remote.local() + .upload_manifest( + bucket_name, actual, json_manifest_format_path.second, root_rtc) + .get(); ASSERT_TRUE(upl_res == upload_result::success); ASSERT_TRUE(subscription1.available()); ASSERT_TRUE(subscription1.get().type == api_activity_type::manifest_upload); diff --git a/src/v/cloud_storage/tests/topic_manifest_test.cc b/src/v/cloud_storage/tests/topic_manifest_test.cc index 80805b8c7a7b..eaed2de603ab 100644 --- a/src/v/cloud_storage/tests/topic_manifest_test.cc +++ b/src/v/cloud_storage/tests/topic_manifest_test.cc @@ -13,7 +13,9 @@ #include "bytes/iobuf_parser.h" #include "bytes/iostream.h" #include "bytes/streambuf.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/topic_manifest.h" +#include "cloud_storage/topic_path_utils.h" #include "cloud_storage/types.h" #include "cluster/types.h" #include "model/compression.h" @@ -38,6 +40,10 @@ using namespace cloud_storage; +namespace { +const remote_path_provider path_provider(std::nullopt); +} // anonymous namespace + // update manifest, serialize, compare jsons static const cluster::topic_configuration cfg{ @@ -143,13 +149,11 @@ SEASTAR_THREAD_TEST_CASE(manifest_type_topic) { } SEASTAR_THREAD_TEST_CASE(create_topic_manifest_correct_path) { - auto path = topic_manifest::get_topic_manifest_path( - cfg.tp_ns.ns, cfg.tp_ns.tp, manifest_format::json); + auto path = prefixed_topic_manifest_json_path(cfg.tp_ns); BOOST_REQUIRE_EQUAL( path, "50000000/meta/cfg-test-namespace/cfg-test-topic/topic_manifest.json"); - auto serde_path = topic_manifest::get_topic_manifest_path( - cfg.tp_ns.ns, cfg.tp_ns.tp, manifest_format::serde); + auto serde_path = prefixed_topic_manifest_bin_path(cfg.tp_ns); BOOST_REQUIRE_EQUAL( serde_path, "50000000/meta/cfg-test-namespace/cfg-test-topic/topic_manifest.bin"); @@ -160,7 +164,7 @@ SEASTAR_THREAD_TEST_CASE(update_topic_manifest_correct_path) { m.update( manifest_format::json, make_manifest_stream(min_topic_manifest_json)) .get(); - auto path = m.get_manifest_path(); + auto path = m.get_manifest_path(path_provider); BOOST_REQUIRE_EQUAL( path, "70000000/meta/test-namespace/test-topic/topic_manifest.bin"); } @@ -495,7 +499,8 @@ SEASTAR_THREAD_TEST_CASE(test_topic_manifest_serde_feature_table) { auto manifest = topic_manifest{ random_topic_configuration, random_initial_revision_id, local_ft}; BOOST_CHECK(manifest.get_revision() == random_initial_revision_id); - BOOST_CHECK(manifest.get_manifest_path()().extension() == ".bin"); + BOOST_CHECK( + manifest.get_manifest_path(path_provider)().extension() == ".bin"); auto serialized_manifest = manifest.serialize().get().stream; diff --git a/src/v/cloud_storage/tests/util.cc b/src/v/cloud_storage/tests/util.cc index 8351e9d3f791..25d05be0de05 100644 --- a/src/v/cloud_storage/tests/util.cc +++ b/src/v/cloud_storage/tests/util.cc @@ -441,13 +441,14 @@ std::vector make_imposter_expectations( return ss::sstring(buf.begin(), buf.end()); }; results.push_back(cloud_storage_fixture::expectation{ - .url = m.get_manifest_path()().string(), .body = serialized()}); + .url = m.get_manifest_path(path_provider)().string(), + .body = serialized()}); std::stringstream ostr; m.serialize_json(ostr); vlog( test_util_log.info, "Uploaded manifest at {}:\n{}", - m.get_manifest_path(), + m.get_manifest_path(path_provider), ostr.str()); return results; } @@ -502,14 +503,15 @@ std::vector make_imposter_expectations( return ss::sstring(buf.begin(), buf.end()); }; results.push_back(cloud_storage_fixture::expectation{ - .url = m.get_manifest_path()().string(), .body = serialized()}); + .url = m.get_manifest_path(path_provider)().string(), + .body = serialized()}); std::ostringstream ostr; m.serialize_json(ostr); vlog( test_util_log.info, "Uploaded manifest at {}:\n{}", - m.get_manifest_path(), + m.get_manifest_path(path_provider), ostr.str()); return results; } @@ -574,7 +576,7 @@ std::vector replace_segments( fixture.remove_expectations(segments_to_remove); // remove manifest from the list - auto manifest_url = manifest.get_manifest_path()().string(); + auto manifest_url = manifest.get_manifest_path(path_provider)().string(); auto expectations = make_imposter_expectations( manifest, segments, false, base_delta); diff --git a/src/v/cloud_storage/topic_manifest.cc b/src/v/cloud_storage/topic_manifest.cc index 710202329f4c..3890db81bd7d 100644 --- a/src/v/cloud_storage/topic_manifest.cc +++ b/src/v/cloud_storage/topic_manifest.cc @@ -14,6 +14,7 @@ #include "bytes/streambuf.h" #include "cloud_storage/logger.h" #include "cloud_storage/remote_path_provider.h" +#include "cloud_storage/topic_path_utils.h" #include "cloud_storage/types.h" #include "cluster/types.h" #include "hashing/xx.h" @@ -319,7 +320,7 @@ void topic_manifest::do_update(const topic_manifest_handler& handler) { fmt::format, "Failed to parse topic manifest {}: Invalid compaction_strategy: " "{}", - get_manifest_path(), + display_name(), handler.compaction_strategy_sv.value())); } } @@ -333,7 +334,7 @@ void topic_manifest::do_update(const topic_manifest_handler& handler) { fmt::format, "Failed to parse topic manifest {}: Invalid timestamp_type " "value: {}", - get_manifest_path(), + display_name(), handler.timestamp_type_sv.value())); } } @@ -347,7 +348,7 @@ void topic_manifest::do_update(const topic_manifest_handler& handler) { fmt::format, "Failed to parse topic manifest {}: Invalid compression value: " "{}", - get_manifest_path(), + display_name(), handler.compression_sv.value())); } } @@ -361,7 +362,7 @@ void topic_manifest::do_update(const topic_manifest_handler& handler) { fmt::format, "Failed to parse topic manifest {}: Invalid " "cleanup_policy_bitflags value: {}", - get_manifest_path(), + display_name(), handler.cleanup_policy_bitflags_sv.value())); } } @@ -376,7 +377,7 @@ void topic_manifest::do_update(const topic_manifest_handler& handler) { fmt::format, "Failed to parse topic manifest {}: Invalid " "virtual_cluster_id_sv value: {}", - get_manifest_path(), + display_name(), handler.virtual_cluster_id_sv.value())); } } @@ -409,7 +410,7 @@ topic_manifest::update(manifest_format format, ss::input_stream is) { throw std::runtime_error(fmt_with_ctx( fmt::format, "Failed to parse topic manifest {}: {} at offset {}", - get_manifest_path(), + display_name(), rapidjson::GetParseError_En(e), o)); } else { @@ -533,27 +534,10 @@ void topic_manifest::serialize_v1_json(std::ostream& out) const { } w.EndObject(); } - -remote_manifest_path topic_manifest::get_topic_manifest_path( - model::ns ns, model::topic topic, manifest_format format) { - // The path is /meta///topic_manifest.json or - // topic_manifest.bin depending on format - constexpr uint32_t bitmask = 0xF0000000; - auto path = fmt::format("{}/{}", ns(), topic()); - uint32_t hash = bitmask & xxhash_32(path.data(), path.size()); - // use format to decide if the path is json or bin - return remote_manifest_path(fmt::format( - "{:08x}/meta/{}/topic_manifest.{}", - hash, - path, - format == manifest_format::json ? "json" : "bin")); -} - -remote_manifest_path topic_manifest::get_manifest_path() const { +ss::sstring topic_manifest::display_name() const { // The path is /meta///topic_manifest.json vassert(_topic_config, "Topic config is not set"); - return get_topic_manifest_path( - _topic_config->tp_ns.ns, _topic_config->tp_ns.tp, manifest_format::serde); + return fmt::format("tp_ns: {}, rev: {}", _topic_config->tp_ns, _rev); } remote_manifest_path topic_manifest::get_manifest_path( diff --git a/src/v/cloud_storage/topic_manifest.h b/src/v/cloud_storage/topic_manifest.h index b3d6cc73997d..5d710911ed6b 100644 --- a/src/v/cloud_storage/topic_manifest.h +++ b/src/v/cloud_storage/topic_manifest.h @@ -56,12 +56,8 @@ class topic_manifest final : public base_manifest { ss::future serialize() const override; /// Manifest object name in S3 - remote_manifest_path get_manifest_path() const override; remote_manifest_path get_manifest_path(const remote_path_provider&) const; - static remote_manifest_path - get_topic_manifest_path(model::ns ns, model::topic topic, manifest_format); - /// Serialize manifest object in json format. only fields up to /// first_version are serialized /// @@ -87,6 +83,10 @@ class topic_manifest final : public base_manifest { == std::tie(other._topic_config, other._rev); }; + /// Name to address this manifest by. Note that the exact path is not + /// tracked by the manifest. + ss::sstring display_name() const; + private: /// Update manifest content from json document that supposed to be generated /// from manifest.json file diff --git a/src/v/cloud_storage/tx_range_manifest.h b/src/v/cloud_storage/tx_range_manifest.h index 1e34198ba1e3..699dc6f4e306 100644 --- a/src/v/cloud_storage/tx_range_manifest.h +++ b/src/v/cloud_storage/tx_range_manifest.h @@ -49,7 +49,7 @@ class tx_range_manifest final : public base_manifest { ss::future serialize() const override; /// Manifest object name in S3 - remote_manifest_path get_manifest_path() const override; + remote_manifest_path get_manifest_path() const; /// Serialize manifest object /// diff --git a/src/v/cluster/cloud_metadata/cluster_manifest.h b/src/v/cluster/cloud_metadata/cluster_manifest.h index 6d028cd75e5f..e2a672cb4a15 100644 --- a/src/v/cluster/cloud_metadata/cluster_manifest.h +++ b/src/v/cluster/cloud_metadata/cluster_manifest.h @@ -75,7 +75,7 @@ struct cluster_metadata_manifest ss::future<> update(ss::input_stream is) override; ss::future serialize() const override; - cloud_storage::remote_manifest_path get_manifest_path() const override; + cloud_storage::remote_manifest_path get_manifest_path() const; cloud_storage::manifest_type get_manifest_type() const override { return cloud_storage::manifest_type::cluster_metadata; } diff --git a/src/v/cluster/cloud_metadata/tests/manifest_downloads_test.cc b/src/v/cluster/cloud_metadata/tests/manifest_downloads_test.cc index 6a29e828434e..e7f042aadaff 100644 --- a/src/v/cluster/cloud_metadata/tests/manifest_downloads_test.cc +++ b/src/v/cluster/cloud_metadata/tests/manifest_downloads_test.cc @@ -76,6 +76,7 @@ FIXTURE_TEST(test_download_manifest, cluster_metadata_fixture) { .upload_manifest( cloud_storage_clients::bucket_name("test-bucket"), manifest, + manifest.get_manifest_path(), retry_node) .get(); @@ -112,7 +113,10 @@ FIXTURE_TEST( manifest.metadata_id = cluster_metadata_id(10); remote .upload_manifest( - cloud_storage_clients::bucket_name("test-bucket"), manifest, retry_node) + cloud_storage_clients::bucket_name("test-bucket"), + manifest, + manifest.get_manifest_path(), + retry_node) .get(); m_res @@ -128,7 +132,10 @@ FIXTURE_TEST( // Upload a new manifest with a higher metadata ID for a new cluster. remote .upload_manifest( - cloud_storage_clients::bucket_name("test-bucket"), manifest, retry_node) + cloud_storage_clients::bucket_name("test-bucket"), + manifest, + manifest.get_manifest_path(), + retry_node) .get(); m_res = download_highest_manifest_in_bucket(remote, bucket, retry_node).get(); diff --git a/src/v/cluster/cloud_metadata/tests/uploader_test.cc b/src/v/cluster/cloud_metadata/tests/uploader_test.cc index d5b630e71b0e..ea434980e718 100644 --- a/src/v/cluster/cloud_metadata/tests/uploader_test.cc +++ b/src/v/cluster/cloud_metadata/tests/uploader_test.cc @@ -154,7 +154,9 @@ FIXTURE_TEST( m.metadata_id = cluster_metadata_id(10); // Upload a manifest and check that we download it. - auto up_res = remote.upload_manifest(bucket, m, retry_node).get(); + auto up_res + = remote.upload_manifest(bucket, m, m.get_manifest_path(), retry_node) + .get(); BOOST_REQUIRE_EQUAL(up_res, cloud_storage::upload_result::success); down_res = uploader.download_highest_manifest_or_create(retry_node).get(); BOOST_REQUIRE(down_res.has_value()); @@ -163,7 +165,9 @@ FIXTURE_TEST( // If we upload a manifest with a lower metadata ID, the higher one should // be downloaded. m.metadata_id = cluster_metadata_id(9); - up_res = remote.upload_manifest(bucket, m, retry_node).get(); + up_res = remote + .upload_manifest(bucket, m, m.get_manifest_path(), retry_node) + .get(); m.metadata_id = cluster_metadata_id(10); BOOST_REQUIRE_EQUAL(up_res, cloud_storage::upload_result::success); down_res = uploader.download_highest_manifest_or_create(retry_node).get(); diff --git a/src/v/cluster/cloud_metadata/uploader.cc b/src/v/cluster/cloud_metadata/uploader.cc index 1d0a64c806ad..26fe568562c1 100644 --- a/src/v/cluster/cloud_metadata/uploader.cc +++ b/src/v/cluster/cloud_metadata/uploader.cc @@ -197,7 +197,7 @@ ss::future uploader::upload_next_metadata( manifest.get_manifest_path(), manifest); auto upload_result = co_await _remote.upload_manifest( - _bucket, manifest, retry_node); + _bucket, manifest, manifest.get_manifest_path(), retry_node); if (upload_result != cloud_storage::upload_result::success) { vlog( clusterlog.warn, diff --git a/src/v/cluster/topic_recovery_service.cc b/src/v/cluster/topic_recovery_service.cc index 80321aa99343..68aa0694b9dd 100644 --- a/src/v/cluster/topic_recovery_service.cc +++ b/src/v/cluster/topic_recovery_service.cc @@ -314,7 +314,7 @@ topic_recovery_service::start_bg_recovery_task(recovery_request request) { vlog(cst_log.info, "found {} topics to create", manifests.size()); for (const auto& manifest : manifests) { - vlog(cst_log.debug, "topic manifest: {}", manifest.get_manifest_path()); + vlog(cst_log.debug, "topic manifest: {}", manifest.display_name()); } _download_counts.clear(); @@ -366,7 +366,7 @@ static cluster::topic_configuration make_topic_config( vlog( cst_log.warn, "skipping topic creation for {}, missing values in manifest", - tm.get_manifest_path()); + tm.display_name()); } cluster::topic_configuration topic_to_create_cfg( @@ -580,7 +580,7 @@ void topic_recovery_service::populate_recovery_status() { vlog( cst_log.warn, "skipping {}, missing ntp config in manifest", - m.get_manifest_path()); + m.display_name()); continue; } auto topic = ntp_cfg->tp_ns.tp; From 46ed45351d133007211b7545137b4b184424c3ce Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 28 Jun 2024 18:15:02 -0700 Subject: [PATCH 27/34] cloud_storage: clean up remote segment name generation This cleans up the remaining test-only usages of methods in the partition manifest that would create remote segment paths without the path provider. --- src/v/archival/tests/service_fixture.cc | 7 ++- src/v/cloud_storage/partition_manifest.cc | 40 --------------- src/v/cloud_storage/partition_manifest.h | 13 ----- .../tests/anomalies_detector_test.cc | 5 +- .../tests/partition_manifest_test.cc | 31 ++++-------- .../tests/remote_partition_test.cc | 2 +- .../tests/remote_segment_test.cc | 23 +++++---- src/v/cloud_storage/tests/remote_test.cc | 50 ++++++++++--------- .../tests/segment_chunk_hydration_test.cc | 8 +-- src/v/cloud_storage/tests/util.cc | 11 ++-- 10 files changed, 72 insertions(+), 118 deletions(-) diff --git a/src/v/archival/tests/service_fixture.cc b/src/v/archival/tests/service_fixture.cc index 3e8ff4c4c8a1..66ae3ef47099 100644 --- a/src/v/archival/tests/service_fixture.cc +++ b/src/v/archival/tests/service_fixture.cc @@ -16,6 +16,7 @@ #include "bytes/iobuf.h" #include "bytes/iobuf_parser.h" #include "cloud_storage/base_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage_clients/configuration.h" #include "cluster/members_table.h" @@ -47,6 +48,10 @@ namespace archival { +namespace { +cloud_storage::remote_path_provider path_provider(std::nullopt); +} // namespace + using namespace std::chrono_literals; inline ss::logger fixt_log("archiver_fixture"); // NOLINT @@ -509,7 +514,7 @@ archival::remote_segment_path get_segment_path( BOOST_REQUIRE(meta); auto key = cloud_storage::parse_segment_name(name); BOOST_REQUIRE(key); - return manifest.generate_segment_path(*meta); + return manifest.generate_segment_path(*meta, path_provider); } archival::remote_segment_path get_segment_index_path( diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index 1970519fecda..080489bc38c1 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -99,27 +99,6 @@ parse_segment_name(const segment_name& name) { }; } -remote_segment_path generate_remote_segment_path( - const model::ntp& ntp, - model::initial_revision_id rev_id, - const segment_name& name, - model::term_id archiver_term) { - vassert( - rev_id != model::initial_revision_id(), - "ntp {}: ntp revision must be known for segment {}", - ntp, - name); - - auto path = ssx::sformat("{}_{}/{}", ntp.path(), rev_id(), name()); - uint32_t hash = xxhash_32(path.data(), path.size()); - if (archiver_term != model::term_id{}) { - return remote_segment_path( - fmt::format("{:08x}/{}.{}", hash, path, archiver_term())); - } else { - return remote_segment_path(fmt::format("{:08x}/{}", hash, path)); - } -} - segment_name generate_local_segment_name(model::offset o, model::term_id t) { vassert(t != model::term_id{}, "Invalid term id"); return segment_name(ssx::sformat("{}-{}-v1.log", o(), t())); @@ -361,18 +340,6 @@ model::initial_revision_id partition_manifest::get_revision_id() const { return _rev; } -remote_segment_path -partition_manifest::generate_segment_path(const segment_meta& meta) const { - auto name = generate_remote_segment_name(meta); - return cloud_storage::generate_remote_segment_path( - _ntp, meta.ntp_revision, name, meta.archiver_term); -} - -remote_segment_path -partition_manifest::generate_segment_path(const lw_segment_meta& meta) const { - return generate_segment_path(lw_segment_meta::convert(meta)); -} - remote_segment_path partition_manifest::generate_segment_path( const segment_meta& meta, const remote_path_provider& path_provider) const { return remote_segment_path{path_provider.segment_path(*this, meta)}; @@ -404,13 +371,6 @@ segment_name partition_manifest::generate_remote_segment_name( __builtin_unreachable(); } -remote_segment_path partition_manifest::generate_remote_segment_path( - const model::ntp& ntp, const partition_manifest::value& val) { - auto name = generate_remote_segment_name(val); - return cloud_storage::generate_remote_segment_path( - ntp, val.ntp_revision, name, val.archiver_term); -} - partition_manifest::const_iterator partition_manifest::first_addressable_segment() const { if (_start_offset == model::offset{}) { diff --git a/src/v/cloud_storage/partition_manifest.h b/src/v/cloud_storage/partition_manifest.h index eb5804b244a9..8b6f91641f09 100644 --- a/src/v/cloud_storage/partition_manifest.h +++ b/src/v/cloud_storage/partition_manifest.h @@ -57,13 +57,6 @@ struct segment_name_components { std::optional parse_segment_name(const segment_name& name); -/// Segment file name in S3 -remote_segment_path generate_remote_segment_path( - const model::ntp&, - model::initial_revision_id, - const segment_name&, - model::term_id archiver_term); - /// Generate correct S3 segment name based on term and base offset segment_name generate_local_segment_name(model::offset o, model::term_id t); @@ -114,9 +107,6 @@ class partition_manifest : public base_manifest { /// Generate segment name to use in the cloud static segment_name generate_remote_segment_name(const value& val); - /// Generate segment path to use in the cloud - static remote_segment_path - generate_remote_segment_path(const model::ntp& ntp, const value& val); /// Create empty manifest that supposed to be updated later partition_manifest(); @@ -251,9 +241,6 @@ class partition_manifest : public base_manifest { /// Find the earliest segment that has max timestamp >= t std::optional timequery(model::timestamp t) const; - remote_segment_path generate_segment_path(const segment_meta&) const; - remote_segment_path generate_segment_path(const lw_segment_meta&) const; - remote_segment_path generate_segment_path( const segment_meta&, const remote_path_provider&) const; remote_segment_path generate_segment_path( diff --git a/src/v/cloud_storage/tests/anomalies_detector_test.cc b/src/v/cloud_storage/tests/anomalies_detector_test.cc index aff1c17f4de7..7ab9b0c924ca 100644 --- a/src/v/cloud_storage/tests/anomalies_detector_test.cc +++ b/src/v/cloud_storage/tests/anomalies_detector_test.cc @@ -357,7 +357,7 @@ class bucket_view_fixture : http_imposter_fixture { void remove_segment( const cloud_storage::partition_manifest& manifest, const cloud_storage::segment_meta& meta) { - auto path = manifest.generate_segment_path(meta); + auto path = manifest.generate_segment_path(meta, path_provider); remove_object(ssx::sformat("/{}", path().string())); } @@ -461,7 +461,8 @@ class bucket_view_fixture : http_imposter_fixture { void set_expectations_for_segments( const cloud_storage::partition_manifest& manifest) { for (const auto& seg : manifest) { - auto path = manifest.generate_segment_path(seg)().string(); + auto path + = manifest.generate_segment_path(seg, path_provider)().string(); when() .request(fmt::format("/{}", path)) .with_method(ss::httpd::operation_type::HEAD) diff --git a/src/v/cloud_storage/tests/partition_manifest_test.cc b/src/v/cloud_storage/tests/partition_manifest_test.cc index 07a556d4f897..b8fe33f17569 100644 --- a/src/v/cloud_storage/tests/partition_manifest_test.cc +++ b/src/v/cloud_storage/tests/partition_manifest_test.cc @@ -15,6 +15,7 @@ #include "cloud_storage/base_manifest.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote_path_provider.h" +#include "cloud_storage/segment_path_utils.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "model/fundamental.h" @@ -476,7 +477,7 @@ SEASTAR_THREAD_TEST_CASE(test_manifest_type) { } SEASTAR_THREAD_TEST_CASE(test_segment_path) { - auto path = generate_remote_segment_path( + auto path = prefixed_segment_path( manifest_ntp, model::initial_revision_id(0), segment_name("22-11-v1.log"), @@ -2119,11 +2120,8 @@ SEASTAR_THREAD_TEST_CASE(test_generate_segment_name_format) { auto s = m.find(model::offset(10)); auto expected = remote_segment_path( "9b367cb7/test-ns/test-topic/42_1/10-1-v1.log.1"); - auto actual1 = partition_manifest::generate_remote_segment_path( - m.get_ntp(), *s); - auto actual2 = m.generate_segment_path(*s); - BOOST_REQUIRE_EQUAL(expected, actual1); - BOOST_REQUIRE_EQUAL(expected, actual2); + auto actual = m.generate_segment_path(*s, path_provider); + BOOST_REQUIRE_EQUAL(expected, actual); } { @@ -2131,11 +2129,8 @@ SEASTAR_THREAD_TEST_CASE(test_generate_segment_name_format) { auto s = m.find(model::offset(20)); auto expected = remote_segment_path( "96c6b7a9/test-ns/test-topic/42_1/20-29-2048-1-v1.log.2"); - auto actual1 = partition_manifest::generate_remote_segment_path( - m.get_ntp(), *s); - auto actual2 = m.generate_segment_path(*s); - BOOST_REQUIRE_EQUAL(expected, actual1); - BOOST_REQUIRE_EQUAL(expected, actual2); + auto actual = m.generate_segment_path(*s, path_provider); + BOOST_REQUIRE_EQUAL(expected, actual); } { @@ -2143,11 +2138,8 @@ SEASTAR_THREAD_TEST_CASE(test_generate_segment_name_format) { auto s = m.find(model::offset(30)); auto expected = remote_segment_path( "df1262f5/test-ns/test-topic/42_1/30-1-v1.log"); - auto actual1 = partition_manifest::generate_remote_segment_path( - m.get_ntp(), *s); - auto actual2 = m.generate_segment_path(*s); - BOOST_REQUIRE_EQUAL(expected, actual1); - BOOST_REQUIRE_EQUAL(expected, actual2); + auto actual = m.generate_segment_path(*s, path_provider); + BOOST_REQUIRE_EQUAL(expected, actual); } { @@ -2155,11 +2147,8 @@ SEASTAR_THREAD_TEST_CASE(test_generate_segment_name_format) { auto s = m.find(model::offset(40)); auto expected = remote_segment_path( "e44e8104/test-ns/test-topic/42_1/40-42-4096-2-v1.log"); - auto actual1 = partition_manifest::generate_remote_segment_path( - m.get_ntp(), *s); - auto actual2 = m.generate_segment_path(*s); - BOOST_REQUIRE_EQUAL(expected, actual1); - BOOST_REQUIRE_EQUAL(expected, actual2); + auto actual = m.generate_segment_path(*s, path_provider); + BOOST_REQUIRE_EQUAL(expected, actual); } } diff --git a/src/v/cloud_storage/tests/remote_partition_test.cc b/src/v/cloud_storage/tests/remote_partition_test.cc index a872634360ab..80db6b97e454 100644 --- a/src/v/cloud_storage/tests/remote_partition_test.cc +++ b/src/v/cloud_storage/tests/remote_partition_test.cc @@ -1089,7 +1089,7 @@ static void remove_segment_from_s3( auto meta = m.get(o); BOOST_REQUIRE(meta.has_value()); - auto path = m.generate_segment_path(*meta); + auto path = m.generate_segment_path(*meta, path_provider); retry_chain_node fib(never_abort, 60s, 1s); auto res = api .delete_object( diff --git a/src/v/cloud_storage/tests/remote_segment_test.cc b/src/v/cloud_storage/tests/remote_segment_test.cc index f0083cb9dd2d..73463cbc58e0 100644 --- a/src/v/cloud_storage/tests/remote_segment_test.cc +++ b/src/v/cloud_storage/tests/remote_segment_test.cc @@ -15,6 +15,7 @@ #include "cloud_storage/materialized_resources.h" #include "cloud_storage/partition_manifest.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tests/cloud_storage_fixture.h" #include "cloud_storage/tests/common_def.h" @@ -46,6 +47,10 @@ inline ss::logger test_log("test"); // NOLINT static ss::abort_source never_abort; +namespace { +remote_path_provider path_provider(std::nullopt); +} // namespace + static cloud_storage::lazy_abort_source always_continue([]() { return std::nullopt; }); @@ -82,7 +87,7 @@ FIXTURE_TEST( .delta_offset = model::offset_delta(0), .ntp_revision = segment_ntp_revision, .sname_format = segment_name_format::v2}; - auto path = m.generate_segment_path(meta); + auto path = m.generate_segment_path(meta, path_provider); set_expectations_and_listen({}); auto upl_res = api.local() @@ -99,7 +104,7 @@ FIXTURE_TEST( api.local(), cache.local(), bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, @@ -144,7 +149,7 @@ FIXTURE_TEST(test_remote_segment_timeout, cloud_storage_fixture) { // NOLINT api.local(), cache.local(), bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, @@ -208,7 +213,7 @@ FIXTURE_TEST( .delta_offset = model::offset_delta(0), .ntp_revision = manifest_revision, .sname_format = segment_name_format::v3}; - auto path = m.generate_segment_path(meta); + auto path = m.generate_segment_path(meta, path_provider); uint64_t clen = segment_bytes.size_bytes(); auto reset_stream = make_reset_fn(segment_bytes); retry_chain_node fib(never_abort, 10000ms, 200ms); @@ -232,7 +237,7 @@ FIXTURE_TEST( api.local(), cache.local(), bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, @@ -308,7 +313,7 @@ void test_remote_segment_batch_reader( .ntp_revision = manifest_revision, .sname_format = segment_name_format::v3}; - auto path = m.generate_segment_path(meta); + auto path = m.generate_segment_path(meta, path_provider); retry_chain_node fib(never_abort, 10000ms, 200ms); upload_index(fixture, meta, segment_bytes, path, fib); @@ -340,7 +345,7 @@ void test_remote_segment_batch_reader( fixture.api.local(), fixture.cache.local(), cloud_storage_clients::bucket_name{fixture.bucket_name}, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, @@ -435,7 +440,7 @@ FIXTURE_TEST( .max_timestamp = {}, .delta_offset = model::offset_delta(0), .ntp_revision = manifest_revision}; - auto path = m.generate_segment_path(meta); + auto path = m.generate_segment_path(meta, path_provider); auto reset_stream = make_reset_fn(segment_bytes); retry_chain_node fib(never_abort, 1000ms, 200ms); auto upl_res @@ -452,7 +457,7 @@ FIXTURE_TEST( api.local(), cache.local(), bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, diff --git a/src/v/cloud_storage/tests/remote_test.cc b/src/v/cloud_storage/tests/remote_test.cc index 906c2ef66a47..94ad06795a3c 100644 --- a/src/v/cloud_storage/tests/remote_test.cc +++ b/src/v/cloud_storage/tests/remote_test.cc @@ -18,7 +18,9 @@ #include "cloud_storage/partition_manifest.h" #include "cloud_storage/partition_manifest_downloader.h" #include "cloud_storage/remote.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" +#include "cloud_storage/segment_path_utils.h" #include "cloud_storage/tests/common_def.h" #include "cloud_storage/tests/s3_imposter.h" #include "cloud_storage/types.h" @@ -180,8 +182,8 @@ TEST_P(all_types_remote_fixture, test_upload_segment) { // NOLINT set_expectations_and_listen({}); auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -208,8 +210,8 @@ TEST_P( set_expectations_and_listen({}); auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -235,8 +237,8 @@ TEST_P( TEST_P(all_types_remote_fixture, test_upload_segment_timeout) { // NOLINT auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -259,8 +261,8 @@ TEST_P(all_types_remote_fixture, test_download_segment) { // NOLINT set_expectations_and_listen({}); auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -302,8 +304,8 @@ TEST_P(all_types_remote_fixture, test_download_segment) { // NOLINT TEST_P(all_types_remote_fixture, test_download_segment_timeout) { // NOLINT auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; auto try_consume = [](uint64_t, ss::input_stream) { return ss::make_ready_future(0); @@ -322,11 +324,11 @@ TEST_P(all_types_remote_fixture, test_download_segment_timeout) { // NOLINT TEST_P(all_types_remote_fixture, test_download_segment_range) { auto subscription = remote.local().subscribe(allow_all); - auto path = generate_remote_segment_path( + auto path = remote_segment_path{prefixed_segment_path( manifest_ntp, manifest_revision, segment_name("1-2-v1.log"), - model::term_id{123}); + model::term_id{123})}; retry_chain_node fib(never_abort, 100ms, 20ms); @@ -388,8 +390,8 @@ TEST_P(all_types_remote_fixture, test_download_segment_range) { TEST_P(all_types_remote_fixture, test_segment_exists) { // NOLINT set_expectations_and_listen({}); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -419,8 +421,8 @@ TEST_P(all_types_remote_fixture, test_segment_exists) { // NOLINT TEST_P(all_types_remote_fixture, test_segment_exists_timeout) { // NOLINT auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; retry_chain_node fib(never_abort, 100ms, 20ms); auto expect_timeout @@ -431,8 +433,8 @@ TEST_P(all_types_remote_fixture, test_segment_exists_timeout) { // NOLINT TEST_P(all_types_remote_fixture, test_segment_delete) { // NOLINT set_expectations_and_listen({}); auto name = segment_name("0-1-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{1}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{1})}; retry_chain_node fib(never_abort, 100ms, 20ms); uint64_t clen = manifest_payload.size(); @@ -492,11 +494,11 @@ TEST_P(all_types_remote_fixture, test_concat_segment_upload) { start_offset = segment.offsets().get_dirty_offset() + model::offset{1}; } - auto path = generate_remote_segment_path( + auto path = remote_segment_path{prefixed_segment_path( test_ntp, manifest_revision, segment_name("1-2-v1.log"), - model::term_id{123}); + model::term_id{123})}; auto start_pos = 20; auto end_pos = b.get_log_segments().back()->file_size() - 20; @@ -1167,8 +1169,8 @@ TEST_P( set_expectations_and_listen({}); auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { @@ -1256,8 +1258,8 @@ TEST_P( set_expectations_and_listen({}); auto subscription = remote.local().subscribe(allow_all); auto name = segment_name("1-2-v1.log"); - auto path = generate_remote_segment_path( - manifest_ntp, manifest_revision, name, model::term_id{123}); + auto path = remote_segment_path{prefixed_segment_path( + manifest_ntp, manifest_revision, name, model::term_id{123})}; uint64_t clen = manifest_payload.size(); auto reset_stream = []() -> ss::future> { diff --git a/src/v/cloud_storage/tests/segment_chunk_hydration_test.cc b/src/v/cloud_storage/tests/segment_chunk_hydration_test.cc index eb4fa5de7e6f..22ce37d478b4 100644 --- a/src/v/cloud_storage/tests/segment_chunk_hydration_test.cc +++ b/src/v/cloud_storage/tests/segment_chunk_hydration_test.cc @@ -12,6 +12,7 @@ #include "bytes/iostream.h" #include "cloud_storage/materialized_resources.h" #include "cloud_storage/partition_manifest.h" +#include "cloud_storage/remote_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tests/cloud_storage_fixture.h" #include "test_utils/async.h" @@ -21,6 +22,7 @@ inline ss::logger test_log("test"); // NOLINT namespace cloud_storage { +remote_path_provider path_provider(std::nullopt); class remote_segment_test_helper { public: explicit remote_segment_test_helper(remote_segment& r) @@ -108,7 +110,7 @@ partition_manifest chunk_read_baseline( .ntp_revision = segment_ntp_revision, .sname_format = segment_name_format::v3}; - auto path = m.generate_segment_path(meta); + auto path = m.generate_segment_path(meta, path_provider); f.set_expectations_and_listen({}, {{"Range"}}); if (index_upload) { @@ -149,7 +151,7 @@ void test_wrapper( f.api.local(), f.cache.local(), f.bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, @@ -415,7 +417,7 @@ FIXTURE_TEST(test_chunk_multiple_readers, cloud_storage_fixture) { api.local(), cache.local(), bucket_name, - m.generate_segment_path(meta), + m.generate_segment_path(meta, path_provider), m.get_ntp(), meta, fib, diff --git a/src/v/cloud_storage/tests/util.cc b/src/v/cloud_storage/tests/util.cc index 25d05be0de05..7c4b9556ded8 100644 --- a/src/v/cloud_storage/tests/util.cc +++ b/src/v/cloud_storage/tests/util.cc @@ -431,7 +431,8 @@ std::vector make_imposter_expectations( const std::vector& segments) { std::vector results; for (const auto& s : segments) { - auto url = m.generate_segment_path(*m.get(s.base_offset)); + auto url = m.generate_segment_path( + *m.get(s.base_offset), path_provider); results.push_back(cloud_storage_fixture::expectation{ .url = url().string(), .body = s.bytes}); } @@ -492,7 +493,8 @@ std::vector make_imposter_expectations( m.add(s.sname, meta); delta = delta + model::offset(s.num_config_records - s.delta_offset_overlap); - auto url = m.generate_segment_path(*m.get(meta.base_offset)); + auto url = m.generate_segment_path( + *m.get(meta.base_offset), path_provider); results.push_back(cloud_storage_fixture::expectation{ .url = url().string(), .body = body}); } @@ -570,7 +572,7 @@ std::vector replace_segments( auto bo = s.base_offset; auto it = manifest.find(bo); BOOST_REQUIRE(it != manifest.end()); - auto path = manifest.generate_segment_path(*it); + auto path = manifest.generate_segment_path(*it, path_provider); segments_to_remove.push_back(path().native()); } fixture.remove_expectations(segments_to_remove); @@ -977,7 +979,8 @@ void reupload_compacted_segments( // and object store state getting out of sync. m.add(s.sname, meta); - auto url = m.generate_segment_path(*m.get(meta.base_offset)); + auto url = m.generate_segment_path( + *m.get(meta.base_offset), path_provider); vlog(test_util_log.debug, "reuploading segment {}", url); retry_chain_node rtc(never_abort, 60s, 1s); bytes bb; From 459106c21a8ee8e302ae97a5b309bb408d09533c Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 21 Jun 2024 19:37:13 -0700 Subject: [PATCH 28/34] topic_manifest: remove feature table This is no longer used, now that the topic_manifest class isn't in charge of naming or deciding serialization format. --- src/v/archival/ntp_archiver_service.cc | 3 +-- .../tests/topic_manifest_test.cc | 25 ++++++------------- src/v/cloud_storage/topic_manifest.cc | 5 +--- src/v/cloud_storage/topic_manifest.h | 8 ++---- 4 files changed, 11 insertions(+), 30 deletions(-) diff --git a/src/v/archival/ntp_archiver_service.cc b/src/v/archival/ntp_archiver_service.cc index d1296859c49c..ffd03e523a91 100644 --- a/src/v/archival/ntp_archiver_service.cc +++ b/src/v/archival/ntp_archiver_service.cc @@ -516,8 +516,7 @@ ss::future<> ntp_archiver::upload_topic_manifest() { vlog(ctxlog.info, "Uploading topic manifest {}", _parent.ntp()); auto cfg_copy = topic_cfg.get(); cfg_copy.replication_factor = replication_factor; - cloud_storage::topic_manifest tm( - cfg_copy, _rev, _feature_table.local()); + cloud_storage::topic_manifest tm(cfg_copy, _rev); auto key = tm.get_manifest_path(remote_path_provider()); vlog(ctxlog.debug, "Topic manifest object key is '{}'", key); auto res = co_await _remote.upload_manifest( diff --git a/src/v/cloud_storage/tests/topic_manifest_test.cc b/src/v/cloud_storage/tests/topic_manifest_test.cc index eaed2de603ab..c2f8ac6b1c41 100644 --- a/src/v/cloud_storage/tests/topic_manifest_test.cc +++ b/src/v/cloud_storage/tests/topic_manifest_test.cc @@ -170,8 +170,7 @@ SEASTAR_THREAD_TEST_CASE(update_topic_manifest_correct_path) { } SEASTAR_THREAD_TEST_CASE(construct_serialize_update_same_object) { - auto local_ft = features::feature_table{}; - topic_manifest m(cfg, model::initial_revision_id(0), local_ft); + topic_manifest m(cfg, model::initial_revision_id(0)); auto [is, size] = m.serialize().get(); iobuf buf; auto os = make_iobuf_ref_output_stream(buf); @@ -272,8 +271,7 @@ SEASTAR_THREAD_TEST_CASE(topic_manifest_min_serialization) { min_cfg.properties.segment_size = std::make_optional( std::numeric_limits::min()); - features::feature_table local_ft; - topic_manifest m(min_cfg, model::initial_revision_id{0}, local_ft); + topic_manifest m(min_cfg, model::initial_revision_id{0}); iobuf buf; iobuf_ostreambuf obuf(buf); std::ostream os(&obuf); @@ -303,8 +301,7 @@ SEASTAR_THREAD_TEST_CASE(topic_manifest_max_serialization) { std::chrono::milliseconds::max()); max_cfg.properties.segment_size = std::make_optional( std::numeric_limits::max()); - auto local_ft = features::feature_table{}; - topic_manifest m(max_cfg, model::initial_revision_id{0}, local_ft); + topic_manifest m(max_cfg, model::initial_revision_id{0}); iobuf buf; iobuf_ostreambuf obuf(buf); std::ostream os(&obuf); @@ -377,8 +374,7 @@ SEASTAR_THREAD_TEST_CASE(full_update_serialize_update_same_object) { } SEASTAR_THREAD_TEST_CASE(update_non_empty_manifest) { - auto local_ft = features::feature_table{}; - topic_manifest m(cfg, model::initial_revision_id(0), local_ft); + topic_manifest m(cfg, model::initial_revision_id(0)); m.update( manifest_format::json, make_manifest_stream(full_topic_manifest_json)) .get(); @@ -395,8 +391,7 @@ SEASTAR_THREAD_TEST_CASE(update_non_empty_manifest) { } SEASTAR_THREAD_TEST_CASE(test_negative_property_manifest) { - auto local_ft = features::feature_table{}; - topic_manifest m(cfg, model::initial_revision_id(0), local_ft); + topic_manifest m(cfg, model::initial_revision_id(0)); m.update( manifest_format::json, make_manifest_stream(negative_properties_manifest)) @@ -424,8 +419,7 @@ SEASTAR_THREAD_TEST_CASE(test_retention_ms_bytes_manifest) { test_cfg.properties.retention_duration = tristate{disable_tristate}; - auto local_ft = features::feature_table{}; - auto m = topic_manifest{test_cfg, model::initial_revision_id{0}, local_ft}; + auto m = topic_manifest{test_cfg, model::initial_revision_id{0}}; auto serialized = m.serialize().get().stream; auto buf = iobuf{}; @@ -489,15 +483,10 @@ SEASTAR_THREAD_TEST_CASE(test_topic_manifest_serde_feature_table) { auto random_initial_revision_id = tests::random_named_int(); - // activate the new format via feature_table and test that the - // serialization is in serde format - auto local_ft = features::feature_table{}; - local_ft.testing_activate_all(); - // create serde image of random_topic_configuration, deserialize it through // topic_manifest and check that the result is equal auto manifest = topic_manifest{ - random_topic_configuration, random_initial_revision_id, local_ft}; + random_topic_configuration, random_initial_revision_id}; BOOST_CHECK(manifest.get_revision() == random_initial_revision_id); BOOST_CHECK( manifest.get_manifest_path(path_provider)().extension() == ".bin"); diff --git a/src/v/cloud_storage/topic_manifest.cc b/src/v/cloud_storage/topic_manifest.cc index 3890db81bd7d..a98c0d1df6ae 100644 --- a/src/v/cloud_storage/topic_manifest.cc +++ b/src/v/cloud_storage/topic_manifest.cc @@ -255,11 +255,8 @@ struct topic_manifest_handler }; }; -// use feature table to decide if to encode with serde topic_manifest::topic_manifest( - const cluster::topic_configuration& cfg, - model::initial_revision_id rev, - const features::feature_table&) + const cluster::topic_configuration& cfg, model::initial_revision_id rev) : _topic_config(cfg) , _rev(rev) {} diff --git a/src/v/cloud_storage/topic_manifest.h b/src/v/cloud_storage/topic_manifest.h index 5d710911ed6b..2bfb8bb6c431 100644 --- a/src/v/cloud_storage/topic_manifest.h +++ b/src/v/cloud_storage/topic_manifest.h @@ -14,7 +14,6 @@ #include "cloud_storage/fwd.h" #include "cloud_storage/types.h" #include "cluster/types.h" -#include "features/feature_table.h" #include @@ -32,12 +31,9 @@ class topic_manifest final : public base_manifest { constexpr static auto current_version = serde_version; - /// Create manifest for specific ntp. feature_table is used to decide which - /// encoding format to use + /// Create manifest for specific topic. explicit topic_manifest( - const cluster::topic_configuration& cfg, - model::initial_revision_id rev, - const features::feature_table& ft); + const cluster::topic_configuration& cfg, model::initial_revision_id rev); /// Create empty manifest that supposed to be updated later topic_manifest(); From 902f5224c64a66938c25d4b55c5bbd523636ce7b Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 24 Jun 2024 00:06:53 -0700 Subject: [PATCH 29/34] rptest/services: make get_cluster_uuid() node optional It's generally unimportant, and most admin endpoints don't require thie field (they select one for the caller). --- tests/rptest/services/admin.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/rptest/services/admin.py b/tests/rptest/services/admin.py index 6647024d6663..c7c2de5485c0 100644 --- a/tests/rptest/services/admin.py +++ b/tests/rptest/services/admin.py @@ -1343,7 +1343,7 @@ def get_controller_status(self, node): return self._request("GET", f"debug/controller_status", node=node).json() - def get_cluster_uuid(self, node): + def get_cluster_uuid(self, node=None): try: r = self._request("GET", "cluster/uuid", node=node) except HTTPError as ex: From 245b4f21ec9bbc5de21ccd2ce99f3adb0c3e22ad Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 24 Jun 2024 00:07:43 -0700 Subject: [PATCH 30/34] rptest: support remote labels in BucketView BucketViews are typically used to examine the state of a bucket in the context of a test. With objects proceeding to be include a given cluster's UUID, this updates the view to know how to interpret the UUID. This means most metadata scructures in the BucketView now account for a label. I tried to not be too disruptive to existing tests, so for calls that get various remote objects, we'll return the only one that exists. If multiple exist (under multiple labels), an exception is thrown. This is not expected to be the case until we begin sharing buckets in CI. --- tests/rptest/services/redpanda.py | 2 +- tests/rptest/utils/si_utils.py | 426 +++++++++++++++++++++--------- 2 files changed, 304 insertions(+), 124 deletions(-) diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 2d45ba89663a..08f5dbced86a 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -3514,7 +3514,7 @@ def _cloud_storage_diagnostics(self): # Decode binary manifests for convenience, but don't give up # if we fail - if ".bin" in m: + if "/manifest.bin" in m: try: decoded = RpStorageTool( self.logger).decode_partition_manifest(body) diff --git a/tests/rptest/utils/si_utils.py b/tests/rptest/utils/si_utils.py index 6d0c54eb2acb..65c98031a203 100644 --- a/tests/rptest/utils/si_utils.py +++ b/tests/rptest/utils/si_utils.py @@ -264,9 +264,44 @@ def make_segment_summary(ntpr: NTPR, reader: SegmentReader) -> SegmentSummary: size_bytes=size_bytes) +def parse_s3_topic_label(path: str) -> str: + """ + Parse S3 manifest path. Return the label, or an empty string if not + labeled with the cluster uuid. + + Sample name: 50000000/meta/kafka/panda-topic/topic_manifest.json + Output: "" + Sample name: meta/kafka/panda-topic/6e94ccdc-443a-4807-b105-0bb86e8f97f7/0/topic_manifest.json + Output: "6e94ccdc-443a-4807-b105-0bb86e8f97f7" + """ + items = path.split('/') + if len(items[0]) == 8 and items[0].endswith('0000000'): + return "" + return items[3] + + +def parse_s3_partition_path_label(path: str) -> str: + """ + Parse S3 manifest path. Return the label, or an empty string if not + labeled with the cluster uuid. + + Sample name: 50000000/meta/kafka/panda-topic/0_19/manifest.json + Output: "" + Sample name: 6e94ccdc-443a-4807-b105-0bb86e8f97f7/meta/kafka/panda-topic/0_18/manifest.bin + Output: "6e94ccdc-443a-4807-b105-0bb86e8f97f7" + Sample name: 6e94ccdc-443a-4807-b105-0bb86e8f97f7/meta/kafka/panda-topic/0_18/manifest.bin.0.21.0.20.1719867209267.1719867209268 + Output: "6e94ccdc-443a-4807-b105-0bb86e8f97f7" + """ + items = path.split('/') + if len(items[0]) == 8 and items[0].endswith('0000000'): + return "" + return items[0] + + def parse_s3_manifest_path(path: str) -> NTPR: """Parse S3 manifest path. Return ntp and revision. Sample name: 50000000/meta/kafka/panda-topic/0_19/manifest.json + Sample name: 6e94ccdc-443a-4807-b105-0bb86e8f97f7/meta/kafka/panda-topic/0_18/manifest.bin """ items = path.split('/') ns = items[2] @@ -427,21 +462,29 @@ def get_ntp_sizes(fdata_per_host, hosts_can_vary=True): def gen_topic_manifest_path(topic: NT, - manifest_format: Literal['json', 'bin'] = 'bin'): + manifest_format: Literal['json', 'bin'] = 'bin', + remote_label: str = "", + rev: int = 0): assert manifest_format in ['json', 'bin'] - x = xxhash.xxh32() path = f"{topic.ns}/{topic.topic}" - x.update(path.encode('ascii')) - hash = x.hexdigest()[0] + '0000000' - return f"{hash}/meta/{path}/topic_manifest.{manifest_format}" + if len(remote_label) == 0: + x = xxhash.xxh32() + x.update(path.encode('ascii')) + hash = x.hexdigest()[0] + '0000000' + return f"{hash}/meta/{path}/topic_manifest.{manifest_format}" + return f"meta/{path}/{remote_label}/{rev}/topic_manifest.{manifest_format}" -def gen_topic_lifecycle_marker_path(topic: NT): - x = xxhash.xxh32() +def gen_topic_lifecycle_marker_path(topic: NT, + rev: int, + remote_label: str = ""): path = f"{topic.ns}/{topic.topic}" - x.update(path.encode('ascii')) - hash = x.hexdigest()[0] + '0000000' - return f"{hash}/meta/{path}/topic_manifest.json" + if len(remote_label) == 0: + x = xxhash.xxh32() + x.update(path.encode('ascii')) + hash = x.hexdigest()[0] + '0000000' + return f"{hash}/meta/{path}/{rev}_lifecycle.bin" + return f"meta/{path}/{remote_label}/{rev}_lifecycle.bin" def gen_segment_name_from_meta(meta: dict, key: str) -> str: @@ -607,10 +650,17 @@ def _match_partition_manifest(self, key): return any(tn in key for tn in self.topic_names) def _match_topic_manifest(self, key): - if self.topic_manifest_paths is None: + if self.topic_names is None: return True else: - return any(key.endswith(t) for t in self.topic_manifest_paths) + for t in self.topic_names: + if not key.endswith( + "/topic_manifest.bin") and not key.endswith( + "/topic_manifest.json"): + continue + if t in key: + return True + return False def is_cluster_metadata_manifest(self, o: ObjectMetadata) -> bool: return o.key.endswith('/cluster_manifest.json') @@ -675,7 +725,10 @@ def path_matches_any_topic(self, path: str) -> bool: return self._match_partition_manifest(path) -def quiesce_uploads(redpanda, topic_names: list[str], timeout_sec): +def quiesce_uploads(redpanda, + topic_names: list[str], + timeout_sec, + target_label: Optional[str] = None): """ Wait until all local data for all topics in `topic_names` has been uploaded to remote storage. This function expects that no new data is being produced: @@ -692,7 +745,7 @@ def remote_has_reached_hwm(ntp: NTP, hwm: int): nonlocal last_msg view = BucketView(redpanda) try: - manifest = view.get_partition_manifest(ntp) + manifest = view.get_partition_manifest(ntp, target_label) except Exception as e: last_msg = f"Partition {ntp} doesn't have a manifest yet ({e})" redpanda.logger.debug(last_msg) @@ -770,7 +823,8 @@ def _parse_path(ntpr: NTPR, path: str) -> list[str]: {base}.{base_rp_offset}.{last_rp_offest}.{base_kafka_offset}.{last_kafka_offset}.{first_ts}.{last_ts} where base = {hash}/meta/{ntpr.ns}/{ntpr.topic}/{ntpr.partition}_{ntpr.revision}/manifest" """ - base = BucketView.gen_manifest_path(ntpr) + label = parse_s3_partition_path_label(path) + base = BucketView.gen_manifest_path(ntpr, remote_label=label) suffix = path.removeprefix(f"{base}.") split = suffix.split(".") @@ -793,12 +847,13 @@ def __init__(self): self.ignored_objects: int = 0 self.tx_manifests: int = 0 self.segment_indexes: int = 0 - self.topic_manifests: dict[NT, dict] = {} - self.partition_manifests: dict[NTP, dict] = {} - self.spillover_manifests: dict[NTP, dict[SpillMeta, dict]] = {} + self.topic_manifests: dict[str, dict[NT, dict]] = {} + self.partition_manifests: dict[str, dict[NTP, dict]] = {} + self.spillover_manifests: dict[str, dict[NTP, dict[SpillMeta, + dict]]] = {} # List of summaries for all segments. These summaries refer # to data in the bucket and not segments in the manifests. - self.segment_summaries: dict[NTP, list[SegmentSummary]] = {} + self.segment_summaries: dict[str, dict[NTP, list[SegmentSummary]]] = {} self.cluster_metadata: dict[str, ClusterMetadata] = {} @@ -904,7 +959,11 @@ def latest_cluster_metadata_manifest(self) -> dict: @property def partition_manifests(self) -> dict[NTP, dict]: self._ensure_listing() - return self._state.partition_manifests + if len(self._state.partition_manifests) != 1: + raise Exception( + f"Bucket doesn't have exactly one cluster's data: {self._state.partition_manifests.keys()}" + ) + return next(iter(self._state.partition_manifests.values())) @staticmethod def kafka_start_offset(manifest) -> Optional[int]: @@ -955,9 +1014,15 @@ def cloud_log_sizes_sum(self) -> CloudLogSize: Returns the cloud log size summed over all ntps. """ self._do_listing() + if len(self._state.partition_manifests) != 1: + raise Exception( + f"Bucket doesn't have exactly one cluster's data: {self._state.partition_manifests.keys()}" + ) total = CloudLogSize.make_empty() - for ns, topic, partition in self._state.partition_manifests.keys(): + partition_manifests = next( + iter(self._state.partition_manifests.values())) + for ns, topic, partition in partition_manifests.keys(): val = self.cloud_log_size_for_ntp(topic, partition, ns) self.logger.debug(f"{topic}/{partition} log_size={val}") total += val @@ -1002,52 +1067,36 @@ def _do_listing(self): def _sort_segment_summaries(self): """Sort segment summary lists by base offset""" - res = {} - for ntp, lst in self._state.segment_summaries.items(): - self.logger.debug(f"Sorting segment summaries for {ntp}") - res[ntp] = sorted(lst, key=lambda x: x.base_offset) - self._state.segment_summaries = res - - def _get_manifest(self, ntpr: NTPR, path: Optional[str] = None) -> dict: + for label, summaries in self._state.segment_summaries.items(): + res = {} + for ntp, lst in summaries.items(): + self.logger.debug( + f"Sorting segment summaries for {label}/{ntp}") + res[ntp] = sorted(lst, key=lambda x: x.base_offset) + self._state.segment_summaries[label] = res + + def _get_manifest(self, ntpr: NTPR, path: str) -> dict: """ Having composed the path for a manifest, download it and return the manifest dict Raises KeyError if the object is not found. """ - if path is None: - # implicit path, try .bin and fall back to .json - path = BucketView.gen_manifest_path(ntpr, "bin") + # explicit path, only try loading that and fail if it fails + if ".bin" in path: format = ManifestFormat.BINARY - try: - data = self.client.get_object_data(self.bucket, path) - except Exception as e: - self.logger.debug(f"Exception loading {path}: {e}") - try: - path = BucketView.gen_manifest_path(ntpr, "json") - format = ManifestFormat.JSON - data = self.client.get_object_data(self.bucket, path) - except Exception as e: - # Very generic exception handling because the storage client - # may be one of several classes with their own exceptions - self.logger.debug(f"Exception loading {path}: {e}") - raise KeyError(f"Manifest for ntp {ntpr} not found") + elif ".json" in path: + format = ManifestFormat.JSON else: - # explicit path, only try loading that and fail if it fails - if ".bin" in path: - format = ManifestFormat.BINARY - elif ".json" in path: - format = ManifestFormat.JSON - else: - raise RuntimeError(f"Unknown manifest key format: '{path}'") + raise RuntimeError(f"Unknown manifest key format: '{path}'") - try: - data = self.client.get_object_data(self.bucket, path) - except Exception as e: - # Very generic exception handling because the storage client - # may be one of several classes with their own exceptions - self.logger.debug(f"Exception loading {path}: {e}") - raise KeyError(f"Manifest for ntp {ntpr} not found") + try: + data = self.client.get_object_data(self.bucket, path) + except Exception as e: + # Very generic exception handling because the storage client + # may be one of several classes with their own exceptions + self.logger.debug(f"Exception loading {path}: {e}") + raise KeyError(f"Manifest for ntp {ntpr} not found") if format == ManifestFormat.BINARY: manifest = RpStorageTool( @@ -1057,12 +1106,16 @@ def _get_manifest(self, ntpr: NTPR, path: Optional[str] = None) -> dict: return manifest - def _load_manifest(self, ntpr: NTPR, path: Optional[str] = None) -> dict: + def _load_manifest(self, ntpr: NTPR, path: str) -> dict: manifest = self._get_manifest(ntpr, path) - self._state.partition_manifests[ntpr.to_ntp()] = manifest + + label = parse_s3_partition_path_label(path) + if label not in self._state.partition_manifests: + self._state.partition_manifests[label] = {} + self._state.partition_manifests[label][ntpr.to_ntp()] = manifest self.logger.debug( - f"Loaded manifest for {ntpr}: {pprint.pformat(manifest, indent=2)}" + f"Loaded manifest for {ntpr} at {path}: {pprint.pformat(manifest, indent=2)}" ) return manifest @@ -1071,15 +1124,18 @@ def _load_spillover_manifest(self, ntpr: NTPR, path: str) -> tuple[SpillMeta, dict]: manifest = self._get_manifest(ntpr, path) ntp = ntpr.to_ntp() + label = parse_s3_partition_path_label(path) - if ntp not in self._state.spillover_manifests: - self._state.spillover_manifests[ntp] = {} + if label not in self._state.spillover_manifests: + self._state.spillover_manifests[label] = {} + if ntp not in self._state.spillover_manifests[label]: + self._state.spillover_manifests[label][ntp] = {} meta = SpillMeta.make(ntpr, path) - self._state.spillover_manifests[ntp][meta] = manifest + self._state.spillover_manifests[label][ntp][meta] = manifest self.logger.debug( - f"Loaded spillover manifest for {ntpr}: {pprint.pformat(manifest, indent=2)}" + f"Loaded spillover manifest for {ntpr} at {path}: {pprint.pformat(manifest, indent=2)}" ) return meta, manifest @@ -1090,17 +1146,23 @@ def _add_segment_metadata(self, path, spc: SegmentPathComponents): if path.endswith(".tx"): return self.logger.debug(f"Parsing segment {spc} at {path}") + label = parse_s3_partition_path_label(path) ntp = spc.ntpr.to_ntp() - if ntp not in self._state.segment_summaries: - self._state.segment_summaries[ntp] = [] + if label not in self._state.segment_summaries: + self._state.segment_summaries[label] = {} + if ntp not in self._state.segment_summaries[label]: + self._state.segment_summaries[label][ntp] = [] payload = self.client.get_object_data(self.bucket, path) reader = SegmentReader(io.BytesIO(payload)) summary = make_segment_summary(spc.ntpr, reader) - self._state.segment_summaries[ntp].append(summary) + self._state.segment_summaries[label][ntp].append(summary) - def _discover_spillover_manifests(self, ntpr: NTPR) -> list[SpillMeta]: + def _discover_spillover_manifests(self, + ntpr: NTPR, + label: str = "") -> list[SpillMeta]: list_res = self.client.list_objects( - bucket=self.bucket, prefix=BucketView.gen_manifest_path(ntpr)) + bucket=self.bucket, + prefix=BucketView.gen_manifest_path(ntpr, remote_label=label)) def is_spillover_manifest_path(path: str) -> bool: return not (path.endswith(".json") or path.endswith(".bin")) @@ -1145,29 +1207,74 @@ def _load_controller_snapshot_size(self, path: str) -> int: return meta.content_length @staticmethod - def gen_manifest_path(ntpr: NTPR, extension: str = "bin"): - x = xxhash.xxh32() + def gen_manifest_path(ntpr: NTPR, + extension: str = "bin", + remote_label: str = ""): path = f"{ntpr.ns}/{ntpr.topic}/{ntpr.partition}_{ntpr.revision}" - x.update(path.encode('ascii')) - hash = x.hexdigest()[0] + '0000000' - return f"{hash}/meta/{path}/manifest.{extension}" - - def get_partition_manifest(self, ntp: NTP | NTPR) -> dict: + if len(remote_label) == 0: + x = xxhash.xxh32() + x.update(path.encode('ascii')) + hash = x.hexdigest()[0] + '0000000' + return f"{hash}/meta/{path}/manifest.{extension}" + return f"{remote_label}/meta/{path}/manifest.{extension}" + + def get_partition_manifest(self, + ntp: NTP | NTPR, + target_label: Optional[str] = None) -> dict: """ Fetch a manifest, looking up revision as needed. + + If a specific remote label is not being targeted, expects there to be + at most one matching manifest in the bucket. """ ntpr = None if isinstance(ntp, NTPR): ntpr = ntp ntp = ntpr.to_ntp() - if ntp in self._state.partition_manifests: - return self._state.partition_manifests[ntp] + matching_labels = [] + for label, pms in self._state.partition_manifests.items(): + if target_label is not None and target_label != label: + continue + if ntp in pms: + matching_labels.append(label) + if len(matching_labels) > 1: + raise Exception( + f"Multiple labels contain {ntp}: {matching_labels}") + + if len(matching_labels) == 1: + return self._state.partition_manifests[matching_labels[0]][ntp] if not ntpr: ntpr = self.ntp_to_ntpr(ntp) - return self._load_manifest(ntpr) + # If we need to look for the partition manifest, look for topic + # manifests first to see what cluster uuid labels to expect (note, + # based on the naming scheme, it's easier to find topic manifests + # without the cluster uuid in hand than it is to find partition + # manifests) + topic = NT(ntpr.ns, ntpr.topic) + topic_manifest_paths = self._find_topic_manifest_paths(topic) + + for tm_path in topic_manifest_paths: + label = parse_s3_topic_label(tm_path) + if target_label is not None and target_label != label: + continue + paths = [BucketView.gen_manifest_path(ntpr, "bin", label)] + if len(label) == 0: + # Versions of Redpanda below 24.2 don't have labels. Farther + # back, we also supported JSON manifests. As a crude heuristic + # assume we may need to look for JSON if we don't have a label. + paths.append(BucketView.gen_manifest_path(ntpr, "json", label)) + + for path in paths: + m: dict = {} + try: + m = self._load_manifest(ntpr, path) + except KeyError: + continue + return m + raise KeyError(f"Manifest for ntp {ntpr} not found") def get_spillover_metadata(self, ntp: NTP | NTPR) -> list[SpillMeta]: """ @@ -1182,7 +1289,16 @@ def get_spillover_metadata(self, ntp: NTP | NTPR) -> list[SpillMeta]: if not ntpr: ntpr = self.ntp_to_ntpr(ntp) - return self._discover_spillover_manifests(ntpr) + topic = NT(ntpr.ns, ntpr.topic) + topic_manifest_paths = self._find_topic_manifest_paths(topic) + + for tm_path in topic_manifest_paths: + label = parse_s3_topic_label(tm_path) + spills = self._discover_spillover_manifests(ntpr, label) + if len(spills) == 0: + continue + return spills + return [] def get_spillover_manifests( self, ntp: NTP | NTPR) -> Optional[dict[SpillMeta, dict]]: @@ -1196,20 +1312,37 @@ def get_spillover_manifests( ntpr = ntp ntp = ntpr.to_ntp() - if ntp in self._state.spillover_manifests: - return self._state.spillover_manifests[ntp] + matching_labels = [] + for label, pms in self._state.spillover_manifests.items(): + if ntp in pms: + matching_labels.append(label) + if len(matching_labels) > 1: + raise Exception( + f"Multiple labels contain {ntp}: {matching_labels}") + + if len(matching_labels) == 1: + return self._state.spillover_manifests[matching_labels[0]][ntp] if not ntpr: ntpr = self.ntp_to_ntpr(ntp) - spills = self._discover_spillover_manifests(ntpr) - for spill in spills: - self._load_spillover_manifest(spill.ntpr, spill.path) - - if ntp in self._state.spillover_manifests: - return self._state.spillover_manifests[ntp] - else: - return None + topic = NT(ntpr.ns, ntpr.topic) + topic_manifest_paths = self._find_topic_manifest_paths(topic) + + # If we need to look for the spillover manifests, look for topic + # manifests first to see what cluster uuid labels to expect (note, + # based on the naming scheme, it's easier to find topic manifests + # without the cluster uuid in hand than it is to find partition + # manifests) + for tm_path in topic_manifest_paths: + label = parse_s3_topic_label(tm_path) + spills = self._discover_spillover_manifests(ntpr, label) + if len(spills) == 0: + continue + for spill in spills: + self._load_spillover_manifest(spill.ntpr, spill.path) + return self._state.spillover_manifests[label][ntp] + return None def _load_manifest_v1_from_data( self, data, manifest_format: Literal['json', 'bin']) -> dict: @@ -1235,9 +1368,13 @@ def _load_topic_manifest(self, topic: NT, path: str, data, manifest_format=manifest_format) self.logger.debug( - f"Loaded topic manifest {topic}: {pprint.pformat(manifest)}") + f"Loaded topic manifest from {path} {topic}: {pprint.pformat(manifest)}" + ) - self._state.topic_manifests[topic] = manifest + label = parse_s3_topic_label(path) + if label not in self._state.topic_manifests: + self._state.topic_manifests[label] = {} + self._state.topic_manifests[label][topic] = manifest return manifest def get_topic_manifest_from_path(self, path: str) -> dict: @@ -1258,43 +1395,78 @@ def get_topic_manifest_from_path(self, path: str) -> dict: ) return manifest - def get_topic_manifest(self, topic: NT) -> dict: + def _find_topic_metas(self, topic: NT) -> list: + path = f"{topic.ns}/{topic.topic}" + list_prefixes = [] + + # First, look for newer, labeled manifests. + list_prefixes.append(f"meta/{path}/") + + # If none, we'll fall back on legacy, hash-prefixed manifests. + x = xxhash.xxh32() + x.update(path.encode('ascii')) + hash = x.hexdigest()[0] + '0000000' + list_prefixes.append(f"{hash}/meta/{path}/") + + ret = [] + for prefix in list_prefixes: + for obj_meta in self.client.list_objects(self.bucket, + prefix=prefix): + ret.append(obj_meta) + return ret + + def _find_topic_manifest_paths(self, topic: NT) -> list: + return [ + meta.key for meta in self._find_topic_metas(topic) + if meta.key.endswith("topic_manifest.bin") + or meta.key.endswith("topic_manifest.json") + ] + + def get_topic_manifest(self, + topic: NT, + target_label: Optional[str] = None) -> dict: """ try to download a topic_manifest.bin for topic. if no object is found, fallback to topic_manifest.json """ - if topic in self._state.topic_manifests: - return self._state.topic_manifests[topic] - - try: - path = gen_topic_manifest_path(topic, manifest_format='bin') + matching_labels = [] + for label, tms in self._state.topic_manifests.items(): + if target_label is not None and target_label != label: + continue + if topic in tms: + matching_labels.append(label) + if len(matching_labels) > 1: + raise Exception( + f"Multiple labels contain {topic}: {matching_labels}") + + if len(matching_labels) == 1: + return self._state.topic_manifests[matching_labels[0]][topic] + + topic_manifest_paths = self._find_topic_manifest_paths(topic) + for path in topic_manifest_paths: + label = parse_s3_topic_label(path) + if target_label is not None and target_label != label: + continue + format = "bin" if path.endswith(".bin") else "json" return self._load_topic_manifest(topic, path, - manifest_format='bin') - except KeyError: - path = gen_topic_manifest_path(topic, manifest_format='json') - return self._load_topic_manifest(topic, - path, - manifest_format='json') + manifest_format=format) - def get_lifecycle_marker_objects(self, topic: NT) -> list[ObjectMetadata]: + raise KeyError(f"Topic manifest not found for {topic}") + + def get_lifecycle_marker_objects( + self, + topic: NT, + ) -> list[ObjectMetadata]: """ Topic manifests are identified by namespace-topic, whereas lifecycle markers are identified by namespace-topic-revision. It is convenient in tests to retrieve by NT though. """ - - x = xxhash.xxh32() - path = f"{topic.ns}/{topic.topic}" - x.update(path.encode('ascii')) - hash = x.hexdigest()[0] + '0000000' - prefix = f"{hash}/meta/{path}/" - results = [] - for obj_meta in self.client.list_objects(self.bucket, prefix=prefix): - if obj_meta.key.endswith("lifecycle.bin"): - results.append(obj_meta) - - return results + return [ + meta for meta in self._find_topic_metas(topic) + if meta.key.endswith("lifecycle.bin") + ] def get_lifecycle_marker(self, topic: NT) -> dict: """ @@ -1492,12 +1664,20 @@ def assert_segments_deleted(self, topic: str, partition: int): key=lambda seg: seg['base_offset']) assert first_segment['base_offset'] > 0 - def segment_summaries(self, ntp: NTP): + def segment_summaries(self, ntp: NTP) -> list[SegmentSummary]: self._ensure_listing() - if ntp in self._state.segment_summaries: - return self._state.segment_summaries[ntp] - else: - return dict() + matching_labels = [] + for label, summaries in self._state.segment_summaries.items(): + if ntp in summaries: + matching_labels.append(label) + if len(matching_labels) > 1: + raise Exception( + f"Multiple labels contain {ntp}: {matching_labels}") + + if len(matching_labels) == 1: + return self._state.segment_summaries[matching_labels[0]][ntp] + + return [] def is_archive_cleanup_complete(self, ntp: NTP): self._ensure_listing() From df36562bf2a02522b34e410df89d488ede5265fd Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 1 Jul 2024 00:45:42 -0700 Subject: [PATCH 31/34] rptest: option to change si_settings bucket This is going to be useful in testing multiple clusters in a single bucket. --- tests/rptest/services/redpanda.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 08f5dbced86a..2a0ece6d024a 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -709,6 +709,12 @@ def cloud_storage_bucket(self) -> str: return bucket + def reset_cloud_storage_bucket(self, new_bucket_name: str) -> None: + if self.cloud_storage_type == CloudStorageType.S3: + self._cloud_storage_bucket = new_bucket_name + elif self.cloud_storage_type == CloudStorageType.ABS: + self._cloud_storage_azure_container = new_bucket_name + def gcp_iam_token(self, logger): logger.info('Getting gcp iam token') s = requests.Session() From 45c62a420a8967bc74ef96ced15deaf8a2f483da Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Fri, 28 Jun 2024 18:48:22 -0700 Subject: [PATCH 32/34] cluster: plug cluster uuid into new topics This only takes effect for topics that aren't being read or restored from the cloud. --- src/v/cluster/controller.cc | 1 + src/v/cluster/topics_frontend.cc | 21 +++++++++++++++++++++ src/v/cluster/topics_frontend.h | 3 +++ 3 files changed, 25 insertions(+) diff --git a/src/v/cluster/controller.cc b/src/v/cluster/controller.cc index d6cf21da13b9..12cbd994e896 100644 --- a/src/v/cluster/controller.cc +++ b/src/v/cluster/controller.cc @@ -386,6 +386,7 @@ ss::future<> controller::start( std::ref(_partition_manager), std::ref(_shard_table), std::ref(_shard_balancer), + std::ref(_storage), ss::sharded_parameter( [this] { return std::ref(_data_migrated_resources.local()); }), ss::sharded_parameter([this] { return std::ref(_plugin_table.local()); }), diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index f6c85bc870ae..1da0e6d1d9e7 100644 --- a/src/v/cluster/topics_frontend.cc +++ b/src/v/cluster/topics_frontend.cc @@ -78,6 +78,7 @@ topics_frontend::topics_frontend( ss::sharded& pm, ss::sharded& shard_table, ss::sharded& sb, + ss::sharded& storage, data_migrations::migrated_resources& migrated_resources, plugin_table& plugin_table, metadata_cache& metadata_cache, @@ -95,6 +96,7 @@ topics_frontend::topics_frontend( , _cloud_storage_api(cloud_storage_api) , _features(features) , _shard_balancer(sb) + , _storage(storage) , _plugin_table(plugin_table) , _metadata_cache(metadata_cache) , _members_table(members_table) @@ -555,6 +557,25 @@ ss::future topics_frontend::do_create_topic( assignable_config.cfg.tp_ns, assignable_config.cfg); } + bool configured_label_from_manifest + = assignable_config.is_read_replica() + || assignable_config.is_recovery_enabled(); + if ( + !configured_label_from_manifest + && !assignable_config.cfg.properties.remote_label.has_value() + && _storage.local().get_cluster_uuid().has_value() + && _features.local().is_active(features::feature::remote_labels) + && !config::shard_local_cfg() + .cloud_storage_disable_remote_labels_for_tests.value()) { + auto remote_label = std::make_optional( + _storage.local().get_cluster_uuid().value()); + assignable_config.cfg.properties.remote_label = remote_label; + vlog( + clusterlog.debug, + "Configuring topic {} with remote label {}", + assignable_config.cfg.tp_ns, + remote_label); + } auto units = co_await _allocator.invoke_on( partition_allocator::shard, diff --git a/src/v/cluster/topics_frontend.h b/src/v/cluster/topics_frontend.h index 22f68faeb829..1374dfb4abbe 100644 --- a/src/v/cluster/topics_frontend.h +++ b/src/v/cluster/topics_frontend.h @@ -23,6 +23,7 @@ #include "model/timeout_clock.h" #include "partition_balancer_types.h" #include "rpc/fwd.h" +#include "storage/api.h" #include #include @@ -67,6 +68,7 @@ class topics_frontend { ss::sharded&, ss::sharded&, ss::sharded&, + ss::sharded&, data_migrations::migrated_resources&, plugin_table&, metadata_cache&, @@ -302,6 +304,7 @@ class topics_frontend { ss::sharded& _cloud_storage_api; ss::sharded& _features; ss::sharded& _shard_balancer; + ss::sharded& _storage; plugin_table& _plugin_table; metadata_cache& _metadata_cache; From b6ae520278a650b48f606da173e934d35d99bf5d Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Tue, 2 Jul 2024 18:33:57 -0700 Subject: [PATCH 33/34] config: enable remote labels by default With this activated in tests, updates several tests that now need to account for the new paths. --- src/v/config/configuration.cc | 3 +-- tests/rptest/tests/archival_test.py | 4 ++-- tests/rptest/tests/topic_recovery_test.py | 4 +++- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 0a60ef306895..a4470f267c91 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -2169,8 +2169,7 @@ configuration::configuration() "simplify testing " "and shouldn't be set in production.", {.needs_restart = needs_restart::no, .visibility = visibility::tunable}, - // TODO(awong): enable once stable - true) + false) , cloud_storage_azure_storage_account( *this, "cloud_storage_azure_storage_account", diff --git a/tests/rptest/tests/archival_test.py b/tests/rptest/tests/archival_test.py index d0ca27661ced..b7bcacef817b 100644 --- a/tests/rptest/tests/archival_test.py +++ b/tests/rptest/tests/archival_test.py @@ -814,8 +814,8 @@ def _get_redpanda_s3_checksums(self): """Get MD5 checksums of log segments stored in S3 (minio). The paths are normalized (//_/...).""" def normalize(path): - # strip archiver term id from the segment path - path = path[9:] + # strip archiver cluster UUID prefix from the segment path + path = path[37:] match = LOG_EXPRESSION.match(path) if match: return match[1] diff --git a/tests/rptest/tests/topic_recovery_test.py b/tests/rptest/tests/topic_recovery_test.py index cb9fce6ab06a..fa1743435b59 100644 --- a/tests/rptest/tests/topic_recovery_test.py +++ b/tests/rptest/tests/topic_recovery_test.py @@ -456,6 +456,7 @@ def __init__(self, redpanda, s3_client, kafka_tools, rpk_client, s3_bucket, def generate_baseline(self): """Produce enough data to trigger uploads to S3/minio""" + self.original_cluster_uuid = self._redpanda._admin.get_cluster_uuid() for topic in self.topics: producer = self._rpk_producer_maker(topic=topic.name, msg_count=10000, @@ -486,7 +487,8 @@ def _find_and_remove_partition_manifest(self): max([seg_meta['delta_offset_end'] for _, seg_meta in manifest_1['segments'].items()]) manifest_0_path = view.gen_manifest_path( - ntp_0.to_ntpr(manifest_0['revision'])) + ntp_0.to_ntpr(manifest_0['revision']), + remote_label=self.original_cluster_uuid) self._delete(manifest_0_path) self.logger.info( f"manifest {manifest_0_path} is removed, partition-1 last offset is {self._part1_offset}" From 263d4b38b6f9ace52a0654b24159d62f1360446b Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Mon, 1 Jul 2024 00:47:01 -0700 Subject: [PATCH 34/34] rptest: add remote_label_test Adds a basic test for remote labeling. --- tests/rptest/tests/remote_label_test.py | 148 ++++++++++++++++++++++++ 1 file changed, 148 insertions(+) create mode 100644 tests/rptest/tests/remote_label_test.py diff --git a/tests/rptest/tests/remote_label_test.py b/tests/rptest/tests/remote_label_test.py new file mode 100644 index 000000000000..5148fdf42c81 --- /dev/null +++ b/tests/rptest/tests/remote_label_test.py @@ -0,0 +1,148 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 + +from rptest.clients.default import DefaultClient +from rptest.clients.types import TopicSpec +from rptest.services.cluster import cluster +from rptest.clients.rpk import RpkTool +from rptest.services.redpanda import RedpandaService, SISettings, make_redpanda_service +from rptest.services.kgo_verifier_services import KgoVerifierProducer +from rptest.tests.redpanda_test import RedpandaTest +from rptest.tests.read_replica_e2e_test import hwms_are_identical, create_read_replica_topic +from rptest.util import wait_until +from ducktape.tests.test import TestContext +from rptest.utils.si_utils import BucketView, NT, quiesce_uploads + + +class RemoteLabelsTest(RedpandaTest): + """ + Tests that exercise multiple clusters sharing a single bucket. + """ + def __init__(self, test_context: TestContext): + extra_rp_conf = dict(cloud_storage_spillover_manifest_size=None, + cloud_storage_topic_purge_grace_period_ms=1000) + super(RemoteLabelsTest, self).__init__( + num_brokers=1, + test_context=test_context, + extra_rp_conf=extra_rp_conf, + si_settings=SISettings( + test_context, + log_segment_size=1024, + fast_uploads=True, + cloud_storage_housekeeping_interval_ms=1000, + cloud_storage_spillover_manifest_max_segments=10)) + + # Set up si_settings so new clusters to reuse the same bucket. + self.new_cluster_si_settings = SISettings( + test_context, + log_segment_size=1024, + fast_uploads=True, + cloud_storage_housekeeping_interval_ms=1000, + cloud_storage_spillover_manifest_max_segments=10) + self.new_cluster_si_settings.bypass_bucket_creation = True + self.new_cluster_si_settings.reset_cloud_storage_bucket( + self.si_settings.cloud_storage_bucket) + self.partition_count = 5 + + self.extra_clusters: list[RedpandaService] = [] + + def start_new_cluster(self) -> RedpandaService: + new_cluster = make_redpanda_service( + self.test_context, + num_brokers=1, + si_settings=self.new_cluster_si_settings) + new_cluster.start() + self.extra_clusters.append(new_cluster) + return new_cluster + + def create_topic(self, cluster: RedpandaService, topic_name: str) -> None: + spec = TopicSpec(name=topic_name, + partition_count=self.partition_count, + replication_factor=1) + DefaultClient(cluster).create_topic(spec) + + def create_read_replica_topic(self, cluster: RedpandaService, + topic_name: str) -> None: + rpk = RpkTool(cluster) + conf = { + 'redpanda.remote.readreplica': + self.si_settings.cloud_storage_bucket, + } + rpk.create_topic(topic_name, config=conf) + + def produce(self, cluster: RedpandaService, topic_name: str, + num_records: int) -> None: + producer = KgoVerifierProducer(self.test_context, + cluster, + topic_name, + msg_size=2056, + msg_count=num_records, + debug_logs=True, + trace_logs=True) + producer.start() + producer.wait(timeout_sec=60) + producer.free() + + @cluster(num_nodes=3) + def test_clusters_share_bucket(self) -> None: + """ + cluster 1 creates topic_a + cluster 2 creates topic_a + cluster 1 deletes topic_a + cluster 1 creates RRR to cluster 2 topic_a + """ + topic_name = "topic-a" + new_cluster = self.start_new_cluster() + self.create_topic(self.redpanda, topic_name) + + # Produce some to the first cluster. + num_records = 200 + self.produce(self.redpanda, topic_name, num_records) + first_cluster_uuid = self.redpanda._admin.get_cluster_uuid() + quiesce_uploads(self.redpanda, [topic_name], 30, first_cluster_uuid) + + # Produce some to the next cluster. + new_cluster_uuid = new_cluster._admin.get_cluster_uuid() + self.create_topic(new_cluster, topic_name) + self.produce(new_cluster, topic_name, num_records) + quiesce_uploads(new_cluster, [topic_name], 30, new_cluster_uuid) + + # Delete the topic on the first cluster. This shouldn't affect the + # second cluster. + DefaultClient(self.redpanda).delete_topic(topic_name) + + def topic_manifest_deleted(): + try: + BucketView(self.redpanda).get_topic_manifest( + NT("kafka", topic_name), first_cluster_uuid) + except KeyError: + return True + return False + + wait_until(topic_manifest_deleted, backoff_sec=1, timeout_sec=30) + + # Point the first cluster at the second cluster's data. + create_read_replica_topic(self.redpanda, topic_name, + self.si_settings.cloud_storage_bucket) + + # We should see that the clusters match, and that we can consume the + # right number of records from the first cluster. + def clusters_report_identical_hwms(): + return hwms_are_identical(self.logger, self.redpanda, new_cluster, + topic_name, self.partition_count) + + wait_until(clusters_report_identical_hwms, + timeout_sec=30, + backoff_sec=1) + rpk = RpkTool(self.redpanda) + out = rpk.consume(topic_name, format="%p,%o\n", n=num_records) + out_lines = out.splitlines() + assert len( + out_lines + ) == num_records, f"output has {len(out_lines)} lines: {out}"