diff --git a/src/v/archival/adjacent_segment_merger.cc b/src/v/archival/adjacent_segment_merger.cc index 1859043034663..6426531a36c50 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 e67a7d62c41cb..ffc8a8f29cb98 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 7af73c67059ca..d635f838ba2d8 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/archival_metadata_stm.cc b/src/v/archival/archival_metadata_stm.cc index 65fa71cab5e2b..17d05f9e93fa3 100644 --- a/src/v/archival/archival_metadata_stm.cc +++ b/src/v/archival/archival_metadata_stm.cc @@ -14,7 +14,9 @@ #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" #include "cluster/errc.h" #include "cluster/logger.h" @@ -641,14 +643,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, @@ -1145,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); @@ -1687,10 +1695,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 +1710,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 0f7806df815a1..e0cfb3550bb88 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/ntp_archiver_service.cc b/src/v/archival/ntp_archiver_service.cc index bf379b2f66877..ffd03e523a912 100644 --- a/src/v/archival/ntp_archiver_service.cc +++ b/src/v/archival/ntp_archiver_service.cc @@ -21,7 +21,9 @@ #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" #include "cloud_storage/remote_segment_index.h" #include "cloud_storage/spillover_manifest.h" @@ -514,12 +516,11 @@ 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()); - auto key = tm.get_manifest_path(); + 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( - _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); } @@ -954,14 +955,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( @@ -971,9 +978,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); } /** @@ -1110,15 +1119,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 +1172,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> @@ -1430,7 +1440,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> @@ -2134,9 +2144,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 +2410,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 +2446,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 +2635,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 +2646,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 +2683,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 +2875,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 +3269,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 97fad4d23a73b..e3574d8488b78 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. diff --git a/src/v/archival/purger.cc b/src/v/archival/purger.cc index b1ad8f09b4688..b2843762f4efb 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, @@ -412,6 +415,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) { @@ -435,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 @@ -463,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) { @@ -509,6 +519,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 +555,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); @@ -553,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 3b54e8029c0cf..615a1baeec17e 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, @@ -111,7 +115,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/archival/scrubber.cc b/src/v/archival/scrubber.cc index 2068f09047883..ccd9003cd551c 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/archival/tests/archival_metadata_stm_gtest.cc b/src/v/archival/tests/archival_metadata_stm_gtest.cc index d5dc2c4e6bbcc..4a085c7c543d2 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 490e2307f7809..35bd108830e12 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/archival/tests/ntp_archiver_reupload_test.cc b/src/v/archival/tests/ntp_archiver_reupload_test.cc index f6af3a30cedbf..beea37b2d4174 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 3e0af5a672244..6828f69f35820 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 @@ -92,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) { @@ -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, @@ -629,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}); @@ -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(), @@ -1667,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; } } @@ -1768,7 +1782,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 +1896,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 +2108,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 +2161,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 +2211,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 +2288,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 +2372,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/archival/tests/segment_reupload_test.cc b/src/v/archival/tests/segment_reupload_test.cc index 3f1f57c9a3d85..9fe324f8ee5c3 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); diff --git a/src/v/archival/tests/service_fixture.cc b/src/v/archival/tests/service_fixture.cc index 3e8ff4c4c8a10..66ae3ef470996 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/anomalies_detector.cc b/src/v/cloud_storage/anomalies_detector.cc index 5fb9fcbf1ef8b..5c4a2cac030c7 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) { @@ -176,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)) { @@ -187,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; @@ -213,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; @@ -233,9 +234,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 +247,7 @@ anomalies_detector::check_manifest( vlog( _logger.debug, "Failed to check existence of segment at {}", - segment_path()); + segment_path); _result.status = scrub_status::partial; } @@ -262,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/anomalies_detector.h b/src/v/cloud_storage/anomalies_detector.h index 682da557c08eb..abcb84d5d62b1 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/async_manifest_view.cc b/src/v/cloud_storage/async_manifest_view.cc index 044ca19dee81b..8c06be2c84419 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" @@ -369,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{}; } }); @@ -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()) @@ -792,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) { @@ -1149,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` @@ -1420,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()); @@ -1599,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> diff --git a/src/v/cloud_storage/async_manifest_view.h b/src/v/cloud_storage/async_manifest_view.h index cf3e59930ce39..5023970e9b329 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/base_manifest.h b/src/v/cloud_storage/base_manifest.h index 1951b0964f210..6369a9660db38 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/lifecycle_marker.h b/src/v/cloud_storage/lifecycle_marker.h index 99c39496780d9..f4cdac33fdca0 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)}; } }; diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index 05d984ba48554..080489bc38c19 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" @@ -98,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())); @@ -179,58 +159,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 { @@ -403,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)}; @@ -446,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{}) { @@ -614,7 +532,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 +556,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 +680,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 +849,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 +894,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 +920,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 +935,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 +965,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 +1087,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 +1875,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 cd69c2f9a7ad5..8b6f91641f09a 100644 --- a/src/v/cloud_storage/partition_manifest.h +++ b/src/v/cloud_storage/partition_manifest.h @@ -57,19 +57,9 @@ 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); -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; @@ -117,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(); @@ -200,33 +187,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; @@ -275,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( @@ -615,6 +578,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 b46dda3eaf930..0ff57ad70c07d 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, @@ -363,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, @@ -1006,40 +971,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 5977e0ebda8eb..937d5f152252f 100644 --- a/src/v/cloud_storage/remote.h +++ b/src/v/cloud_storage/remote.h @@ -268,28 +268,12 @@ 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 /// \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, @@ -394,28 +378,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. diff --git a/src/v/cloud_storage/remote_partition.cc b/src/v/cloud_storage/remote_partition.cc index e790cc1d76d6c..997f308eff625 100644 --- a/src/v/cloud_storage/remote_partition.cc +++ b/src/v/cloud_storage/remote_partition.cc @@ -16,6 +16,8 @@ #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_path_provider.h" #include "cloud_storage/remote_segment.h" #include "cloud_storage/tx_range_manifest.h" #include "cloud_storage/types.h" @@ -189,7 +191,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. @@ -198,7 +201,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; @@ -1067,7 +1071,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); @@ -1099,7 +1104,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(); @@ -1107,7 +1112,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; @@ -1306,6 +1312,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 +1329,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 +1415,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()}; @@ -1430,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) { @@ -1441,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); @@ -1452,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 24a19710ad566..7fa7324bf2efc 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&); diff --git a/src/v/cloud_storage/remote_path_provider.cc b/src/v/cloud_storage/remote_path_provider.cc index d222b656d7980..1a14ccc9e4e1f 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 9457ef9323fff..9198224ab4f1f 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/spillover_manifest.h b/src/v/cloud_storage/spillover_manifest.h index 9e0321243ef79..9500f564b4618 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 fab4c956ac84d..7ab9b0c924ca0 100644 --- a/src/v/cloud_storage/tests/anomalies_detector_test.cc +++ b/src/v/cloud_storage/tests/anomalies_detector_test.cc @@ -12,7 +12,9 @@ #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" #include "cloud_storage/types.h" #include "http/tests/http_imposter.h" @@ -42,6 +44,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 +282,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); @@ -352,21 +357,21 @@ 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())); } 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) { @@ -425,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() @@ -455,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) @@ -602,11 +609,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 f82837218379b..9ed17e0684e1b 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); @@ -85,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(); @@ -130,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() @@ -162,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, @@ -208,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 27b2d1a544676..b8fe33f175692 100644 --- a/src/v/cloud_storage/tests/partition_manifest_test.cc +++ b/src/v/cloud_storage/tests/partition_manifest_test.cc @@ -14,6 +14,8 @@ #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/segment_path_utils.h" #include "cloud_storage/spillover_manifest.h" #include "cloud_storage/types.h" #include "model/fundamental.h" @@ -32,6 +34,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", @@ -471,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"), @@ -503,7 +509,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 +518,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 +545,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 +639,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 +708,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 +758,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 +770,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 +2019,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); @@ -2114,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); } { @@ -2126,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); } { @@ -2138,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); } { @@ -2150,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_fuzz_test.cc b/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc index 4d82026c543a1..25dba4b23c525 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 5047e8a95cb19..80db6b97e454d 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" @@ -73,6 +74,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 +148,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 +216,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 +270,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(); }); @@ -416,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)); @@ -1032,7 +1036,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 +1060,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( @@ -1085,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( @@ -1124,7 +1128,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 +1231,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 +1320,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 +1527,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 +2019,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, @@ -2278,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{ @@ -2351,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()), }}); } @@ -2382,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{ @@ -2470,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_segment_test.cc b/src/v/cloud_storage/tests/remote_segment_test.cc index f0083cb9dd2d8..73463cbc58e0e 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 8410ebb136a51..94ad06795a3cd 100644 --- a/src/v/cloud_storage/tests/remote_test.cc +++ b/src/v/cloud_storage/tests/remote_test.cc @@ -16,8 +16,11 @@ #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_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" @@ -88,15 +91,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 +148,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 +164,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); @@ -244,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> { @@ -272,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> { @@ -299,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> { @@ -323,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> { @@ -366,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); @@ -386,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); @@ -452,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> { @@ -483,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 @@ -495,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(); @@ -556,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; @@ -1159,7 +1097,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); @@ -1228,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> { @@ -1317,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> { @@ -1378,8 +1319,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 +1336,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/segment_chunk_hydration_test.cc b/src/v/cloud_storage/tests/segment_chunk_hydration_test.cc index eb4fa5de7e6f0..22ce37d478b4f 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/topic_manifest_test.cc b/src/v/cloud_storage/tests/topic_manifest_test.cc index 80805b8c7a7bc..c2f8ac6b1c418 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,14 +164,13 @@ 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"); } 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); @@ -268,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); @@ -299,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); @@ -373,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(); @@ -391,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)) @@ -420,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{}; @@ -485,17 +483,13 @@ 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()().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/topic_recovery_service_test.cc b/src/v/cloud_storage/tests/topic_recovery_service_test.cc index 636ab13ab2db6..0b62e56c73bb9 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/cloud_storage/tests/util.cc b/src/v/cloud_storage/tests/util.cc index 1618b3e484934..7c4b9556ded8f 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" @@ -25,6 +26,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; @@ -428,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}); } @@ -438,13 +442,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; } @@ -488,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}); } @@ -499,14 +505,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; } @@ -565,13 +572,13 @@ 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); // 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); @@ -619,11 +626,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; } @@ -657,7 +668,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 +756,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 +817,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 +880,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, @@ -952,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; diff --git a/src/v/cloud_storage/topic_manifest.cc b/src/v/cloud_storage/topic_manifest.cc index 710202329f4c7..a98c0d1df6ae0 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" @@ -254,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) {} @@ -319,7 +317,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 +331,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 +345,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 +359,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 +374,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 +407,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 +531,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 b3d6cc73997d1..2bfb8bb6c431c 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(); @@ -56,12 +52,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 +79,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/topic_path_utils.cc b/src/v/cloud_storage/topic_path_utils.cc index e4b7d27db0dc9..25689f6484485 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 f5b4a12987825..5a2cb9c364df4 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 diff --git a/src/v/cloud_storage/tx_range_manifest.h b/src/v/cloud_storage/tx_range_manifest.h index 1e34198ba1e31..699dc6f4e306a 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 6d028cd75e5f0..e2a672cb4a15d 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 6a29e828434e6..e7f042aadaff8 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 d5b630e71b0e6..ea434980e718f 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 1d0a64c806ad9..26fe568562c1a 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/controller.cc b/src/v/cluster/controller.cc index d6cf21da13b91..12cbd994e896e 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/controller_backend.cc b/src/v/cluster/controller_backend.cc index c61eebcd241c4..a171d3888a9e8 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.cc b/src/v/cluster/partition.cc index ec8d998b09e81..b2aff32a334a9 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" @@ -422,7 +423,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( @@ -1171,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 diff --git a/src/v/cluster/partition_manager.cc b/src/v/cluster/partition_manager.cc index d94fe6748103b..b99586a1b7aa8 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 7730cf42f7387..4997491803a02 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 fcc47165985dc..75a192241f649 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 7b57ba88b4837..77f4619c8bd3d 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; diff --git a/src/v/cluster/remote_topic_configuration_source.cc b/src/v/cluster/remote_topic_configuration_source.cc index e8555101fd830..438dae88353b5 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; } diff --git a/src/v/cluster/topic_properties.cc b/src/v/cluster/topic_properties.cc index 0195254f29a2f..4b2d9aba9a9d5 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 824483b8c65d6..b947fde92d704 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&) diff --git a/src/v/cluster/topic_recovery_service.cc b/src/v/cluster/topic_recovery_service.cc index efc69d659d0b2..68aa0694b9dd2 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"); @@ -314,14 +314,14 @@ 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(); 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(); @@ -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( @@ -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(); }); @@ -684,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; diff --git a/src/v/cluster/topic_recovery_service.h b/src/v/cluster/topic_recovery_service.h index 59fa0d3e983b0..21c8f6f8bc009 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; }; diff --git a/src/v/cluster/topic_recovery_validator.cc b/src/v/cluster/topic_recovery_validator.cc index b688e4b095a24..cd928293cd43a 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 3b94ac941cf29..195726ffe7963 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_; diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index f6c85bc870aec..1da0e6d1d9e71 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 22f68faeb8290..1374dfb4abbec 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; diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 92abbc7f69998..a4470f267c91d 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -2161,6 +2161,15 @@ 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}, + false) , 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 92c65d77c7541..08e348fbcc197 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; diff --git a/src/v/features/feature_table.cc b/src/v/features/feature_table.cc index f91d00987c101..1540868e15836 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 89101c808b6df..8191116a816f2 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); diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index d5b3b214b8967..eb82fca57924b 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> diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index ecfdbe0332a60..0bf30e696a7bc 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(); diff --git a/tests/rptest/services/admin.py b/tests/rptest/services/admin.py index 6647024d66630..c7c2de5485c0f 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: diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 2d45ba89663a5..2a0ece6d024a9 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() @@ -3514,7 +3520,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/tests/archival_test.py b/tests/rptest/tests/archival_test.py index d0ca27661ced2..b7bcacef817be 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/remote_label_test.py b/tests/rptest/tests/remote_label_test.py new file mode 100644 index 0000000000000..5148fdf42c818 --- /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}" diff --git a/tests/rptest/tests/topic_recovery_test.py b/tests/rptest/tests/topic_recovery_test.py index cb9fce6ab06ad..fa1743435b597 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}" diff --git a/tests/rptest/utils/si_utils.py b/tests/rptest/utils/si_utils.py index 6d0c54eb2acb7..65c98031a203b 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() diff --git a/tools/offline_log_viewer/controller.py b/tools/offline_log_viewer/controller.py index bfe2d4f6cde07..1498534d6c2d1 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