From 5c8bfeb140c4fdbb70a92e590d338d3259a92663 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 13 Nov 2024 18:45:02 +0100 Subject: [PATCH 01/19] cluster: add a set of iceberg topic tombstones to topic_table --- src/v/cluster/controller_snapshot.cc | 7 +++++++ src/v/cluster/controller_snapshot.h | 9 ++++++++- src/v/cluster/topic_table.cc | 3 +++ src/v/cluster/topic_table.h | 14 +++++++++++++- src/v/cluster/types.h | 18 ++++++++++++++++++ 5 files changed, 49 insertions(+), 2 deletions(-) diff --git a/src/v/cluster/controller_snapshot.cc b/src/v/cluster/controller_snapshot.cc index 7a80ea9ed04e..8caabed0a993 100644 --- a/src/v/cluster/controller_snapshot.cc +++ b/src/v/cluster/controller_snapshot.cc @@ -134,6 +134,7 @@ ss::future<> topics_t::serde_async_write(iobuf& out) { serde::write(out, highest_group_id); co_await write_map_async(out, std::move(lifecycle_markers)); co_await write_map_async(out, partitions_to_force_recover); + co_await write_map_async(out, std::move(iceberg_tombstones)); } ss::future<> @@ -152,6 +153,12 @@ topics_t::serde_async_read(iobuf_parser& in, const serde::header h) { in, h._bytes_left_limit); } + if (h._version >= 2) { + iceberg_tombstones + = co_await read_map_async_nested( + in, h._bytes_left_limit); + } + if (in.bytes_left() > h._bytes_left_limit) { in.skip(in.bytes_left() - h._bytes_left_limit); } diff --git a/src/v/cluster/controller_snapshot.h b/src/v/cluster/controller_snapshot.h index 40665e47c570..7afa506b497e 100644 --- a/src/v/cluster/controller_snapshot.h +++ b/src/v/cluster/controller_snapshot.h @@ -119,7 +119,7 @@ struct config_t struct topics_t : public serde:: - envelope, serde::compat_version<0>> { + envelope, serde::compat_version<0>> { // NOTE: layout here is a bit different than in the topic table because it // allows more compact storage and more convenient generation of controller // backend deltas when applying the snapshot. @@ -193,6 +193,13 @@ struct topics_t force_recoverable_partitions_t partitions_to_force_recover; + chunked_hash_map< + model::topic_namespace, + nt_iceberg_tombstone, + model::topic_namespace_hash, + model::topic_namespace_eq> + iceberg_tombstones; + friend bool operator==(const topics_t&, const topics_t&) = default; ss::future<> serde_async_write(iobuf&); diff --git a/src/v/cluster/topic_table.cc b/src/v/cluster/topic_table.cc index 3ad69bdad6d1..e7253bff896c 100644 --- a/src/v/cluster/topic_table.cc +++ b/src/v/cluster/topic_table.cc @@ -1533,6 +1533,9 @@ ss::future<> topic_table::apply_snapshot( reset_partitions_to_force_reconfigure( controller_snap.topics.partitions_to_force_recover); + _iceberg_tombstones.replace( + controller_snap.topics.iceberg_tombstones.values().copy()); + // 2. re-calculate derived state _partition_count = 0; diff --git a/src/v/cluster/topic_table.h b/src/v/cluster/topic_table.h index 7a7febeba7e8..ded86b4d2e9f 100644 --- a/src/v/cluster/topic_table.h +++ b/src/v/cluster/topic_table.h @@ -263,6 +263,12 @@ class topic_table { model::topic_namespace_hash, model::topic_namespace_eq>; + using iceberg_tombstones_t = chunked_hash_map< + model::topic_namespace, + nt_iceberg_tombstone, + model::topic_namespace_hash, + model::topic_namespace_eq>; + using topic_delta = topic_table_topic_delta; using topic_delta_cb_t @@ -559,7 +565,7 @@ class topic_table { size_t get_node_partition_count(model::node_id) const; /** - * See which topics have pending deletion work + * See which topics have pending cloud storage deletion work */ const lifecycle_markers_t& get_lifecycle_markers() const { return _lifecycle_markers; @@ -607,6 +613,11 @@ class topic_table { return is_disabled(model::topic_namespace_view{ntp}, ntp.tp.partition); } + // Get a set of topics with pending iceberg deletion work. + const iceberg_tombstones_t& get_iceberg_tombstones() const { + return _iceberg_tombstones; + } + auto topics_iterator_begin() const { return stable_iterator< underlying_t::const_iterator, @@ -700,6 +711,7 @@ class topic_table { underlying_t _topics; lifecycle_markers_t _lifecycle_markers; disabled_partitions_t _disabled_partitions; + iceberg_tombstones_t _iceberg_tombstones; size_t _partition_count{0}; updates_t _updates_in_progress; diff --git a/src/v/cluster/types.h b/src/v/cluster/types.h index c5dfd11f7e25..3ffa40e502e5 100644 --- a/src/v/cluster/types.h +++ b/src/v/cluster/types.h @@ -931,6 +931,24 @@ struct nt_lifecycle_marker auto serde_fields() { return std::tie(config, initial_revision_id); } }; +// A record in the topic table for a deleted topic that is pending iceberg table +// deletion. +struct nt_iceberg_tombstone + : serde::envelope< + nt_iceberg_tombstone, + serde::version<0>, + serde::compat_version<0>> { + // The topic revision of a last deleted topic for which the corresponding + // iceberg table has to be deleted. It is used to avoid deleting iceberg + // data from a topic with the same name that was created later. If several + // iceberg-enabled topics with the same name are created and deleted in a + // rapid succession, we just update this revision (the corresponding table + // only has to be deleted once). + model::revision_id last_deleted_revision; + + auto serde_fields() { return std::tie(last_deleted_revision); } +}; + struct topic_lifecycle_transition : serde::envelope< topic_lifecycle_transition, From 64c4341490e35e9d364fd3d675986c6bf30b2dec Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 13 Nov 2024 18:48:18 +0100 Subject: [PATCH 02/19] cluster: introduce topic purge domains GC process after deleting a topic may involve cleanup in several different places. Track this by introducing a "purge domain" and plumbing it through all lifecycle management methods. --- src/v/cluster/archival/purger.cc | 9 ++-- src/v/cluster/service.cc | 4 +- src/v/cluster/topic_table.cc | 80 +++++++++++++++++++++++--------- src/v/cluster/topics_frontend.cc | 43 +++++++++++++---- src/v/cluster/topics_frontend.h | 13 ++++-- src/v/cluster/types.cc | 12 +++++ src/v/cluster/types.h | 28 ++++++++--- src/v/redpanda/admin/server.cc | 3 +- 8 files changed, 143 insertions(+), 49 deletions(-) diff --git a/src/v/cluster/archival/purger.cc b/src/v/cluster/archival/purger.cc index 063b35bcdc65..ab53635a7a9a 100644 --- a/src/v/cluster/archival/purger.cc +++ b/src/v/cluster/archival/purger.cc @@ -368,7 +368,8 @@ ss::future purger::run(run_quota_t quota) { "Dropping lifecycle marker {}, is not suitable for remote purge", marker.config.tp_ns); - co_await _topics_frontend.local().purged_topic(nt_revision, 5s); + co_await _topics_frontend.local().purged_topic( + nt_revision, cluster::topic_purge_domain::cloud_storage, 5s); continue; } @@ -548,10 +549,10 @@ ss::future purger::run(run_quota_t quota) { co_return result; } - // All topic-specific bucket contents are gone, we may erase - // our controller tombstone. + // All topic-specific bucket contents are gone, we may mark the + // topic as purged in the topic table. auto purge_result = co_await _topics_frontend.local().purged_topic( - nt_revision, 5s); + nt_revision, cluster::topic_purge_domain::cloud_storage, 5s); if (purge_result.ec != cluster::errc::success) { auto errc = cluster::make_error_code(purge_result.ec); // Just log: this will get retried next time the scrubber runs diff --git a/src/v/cluster/service.cc b/src/v/cluster/service.cc index 41ce17b4bd2c..343bdb8c93e2 100644 --- a/src/v/cluster/service.cc +++ b/src/v/cluster/service.cc @@ -159,7 +159,9 @@ service::purged_topic(purged_topic_request r, rpc::streaming_context&) { get_scheduling_group(), [this, r = std::move(r)]() mutable { return _topics_frontend.local().do_purged_topic( - std::move(r.topic), model::timeout_clock::now() + r.timeout); + std::move(r.topic), + r.domain, + model::timeout_clock::now() + r.timeout); }) .then( [](topic_result res) { return purged_topic_reply(std::move(res)); }); diff --git a/src/v/cluster/topic_table.cc b/src/v/cluster/topic_table.cc index e7253bff896c..aa75b06519c8 100644 --- a/src/v/cluster/topic_table.cc +++ b/src/v/cluster/topic_table.cc @@ -170,7 +170,8 @@ ss::future topic_table::apply(topic_lifecycle_transition soft_del, model::offset offset) { _last_applied_revision_id = model::revision_id(offset); - if (soft_del.mode == topic_lifecycle_transition_mode::pending_gc) { + switch (soft_del.mode) { + case topic_lifecycle_transition_mode::pending_gc: { // Create a lifecycle marker auto tp = _topics.find(soft_del.topic.nt); if (tp == _topics.end()) { @@ -190,35 +191,72 @@ topic_table::apply(topic_lifecycle_transition soft_del, model::offset offset) { "Created lifecycle marker for topic {} {}", soft_del.topic.nt, soft_del.topic.initial_revision_id); - } else if (soft_del.mode == topic_lifecycle_transition_mode::drop) { - if (_lifecycle_markers.contains(soft_del.topic)) { + [[fallthrough]]; // proceed to local deletion + } + case topic_lifecycle_transition_mode::oneshot_delete: + case topic_lifecycle_transition_mode::delete_migrated: + return ssx::now(do_local_delete( + soft_del.topic.nt, + offset, + soft_del.mode == topic_lifecycle_transition_mode::delete_migrated)); + case topic_lifecycle_transition_mode::purged: + switch (soft_del.domain) { + case topic_purge_domain::cloud_storage: { + if (_lifecycle_markers.contains(soft_del.topic)) { + vlog( + clusterlog.debug, + "Purged cloud storage lifecycle marker for {} {}", + soft_del.topic.nt, + soft_del.topic.initial_revision_id); + _lifecycle_markers.erase(soft_del.topic); + return ss::make_ready_future(errc::success); + } else { + vlog( + clusterlog.info, + "Unexpected record at offset {} to drop non-existent " + "lifecycle marker {} {}", + offset, + soft_del.topic.nt, + soft_del.topic.initial_revision_id); + return ss::make_ready_future( + errc::topic_not_exists); + } + } + case topic_purge_domain::iceberg: { + auto tombstone_it = _iceberg_tombstones.find(soft_del.topic.nt); + if (tombstone_it == _iceberg_tombstones.end()) { + return ss::make_ready_future( + errc::topic_not_exists); + } + + model::revision_id purged_revision{ + soft_del.topic.initial_revision_id}; + if (tombstone_it->second.last_deleted_revision > purged_revision) { + return ss::make_ready_future( + errc::concurrent_modification_error); + } + vlog( clusterlog.debug, - "Purged lifecycle marker for {} {}", - soft_del.topic.nt, - soft_del.topic.initial_revision_id); - _lifecycle_markers.erase(soft_del.topic); + "Purged iceberg tombstone for {} {}", + tombstone_it->first, + tombstone_it->second.last_deleted_revision); + + _iceberg_tombstones.erase(tombstone_it); return ss::make_ready_future(errc::success); - } else { + } + default: vlog( - clusterlog.info, - "Unexpected record at offset {} to drop non-existent lifecycle " - "marker {} {}", - offset, + clusterlog.error, + "Unknown purge domain {} for topic {} (initial rev: {}). " + "This is a bug.", + static_cast(soft_del.domain), soft_del.topic.nt, soft_del.topic.initial_revision_id); return ss::make_ready_future( - errc::topic_not_exists); + errc::invalid_request); } } - - if (soft_del.mode == topic_lifecycle_transition_mode::drop) { - return ssx::now(errc::success); - } - return ssx::now(do_local_delete( - soft_del.topic.nt, - offset, - soft_del.mode == topic_lifecycle_transition_mode::delete_migrated)); } ss::future diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index e4a2ac59e7c4..62b22a3081c9 100644 --- a/src/v/cluster/topics_frontend.cc +++ b/src/v/cluster/topics_frontend.cc @@ -1044,7 +1044,9 @@ ss::future topics_frontend::do_delete_topic( } ss::future topics_frontend::purged_topic( - nt_revision topic, model::timeout_clock::duration timeout) { + nt_revision topic, + topic_purge_domain domain, + model::timeout_clock::duration timeout) { auto leader = _leaders.local().get_leader(model::controller_ntp); // no leader available @@ -1055,26 +1057,43 @@ ss::future topics_frontend::purged_topic( // current node is a leader controller if (leader == _self) { return do_purged_topic( - std::move(topic), model::timeout_clock::now() + timeout); + std::move(topic), domain, model::timeout_clock::now() + timeout); } else { return dispatch_purged_topic_to_leader( - leader.value(), std::move(topic), timeout); + leader.value(), std::move(topic), domain, timeout); } } ss::future topics_frontend::do_purged_topic( - nt_revision topic, model::timeout_clock::time_point deadline) { + nt_revision topic, + topic_purge_domain domain, + model::timeout_clock::time_point deadline) { topic_lifecycle_transition_cmd cmd( topic.nt, topic_lifecycle_transition{ - .topic = topic, .mode = topic_lifecycle_transition_mode::drop}); + .topic = topic, + .mode = topic_lifecycle_transition_mode::purged, + .domain = domain}); + + bool marker_exists = false; + switch (domain) { + case topic_purge_domain::cloud_storage: + marker_exists = _topics.local().get_lifecycle_markers().contains(topic); + break; + case topic_purge_domain::iceberg: + marker_exists = _topics.local().get_iceberg_tombstones().contains( + topic.nt); + break; + } - if (!_topics.local().get_lifecycle_markers().contains(topic)) { + if (!marker_exists) { // Do not write to log if the marker is already gone vlog( clusterlog.info, - "Dropping duplicate purge request for lifecycle marker {}", - topic.nt); + "Dropping duplicate purge request for lifecycle marker {} in domain " + "{}", + topic.nt, + domain); co_return topic_result(std::move(topic.nt), errc::success); } @@ -1150,6 +1169,7 @@ topics_frontend::dispatch_create_to_leader( ss::future topics_frontend::dispatch_purged_topic_to_leader( model::node_id leader, nt_revision topic, + topic_purge_domain domain, model::timeout_clock::duration timeout) { vlog( clusterlog.trace, @@ -1163,10 +1183,13 @@ ss::future topics_frontend::dispatch_purged_topic_to_leader( ss::this_shard_id(), leader, timeout, - [topic, timeout](controller_client_protocol cp) mutable { + [topic, timeout, domain]( + controller_client_protocol cp) mutable { return cp.purged_topic( purged_topic_request{ - .topic = std::move(topic), .timeout = timeout}, + .topic = std::move(topic), + .timeout = timeout, + .domain = domain}, rpc::client_opts(model::timeout_clock::now() + timeout)); }) .then(&rpc::get_ctx_data); diff --git a/src/v/cluster/topics_frontend.h b/src/v/cluster/topics_frontend.h index fa43e49d7667..97f3db52673e 100644 --- a/src/v/cluster/topics_frontend.h +++ b/src/v/cluster/topics_frontend.h @@ -92,12 +92,12 @@ class topics_frontend { ss::future> dispatch_delete_topics( std::vector, std::chrono::milliseconds); // May be called on any node - ss::future - purged_topic(nt_revision, model::timeout_clock::duration); + ss::future purged_topic( + nt_revision, topic_purge_domain, model::timeout_clock::duration); // May only be called on leader - ss::future - do_purged_topic(nt_revision, model::timeout_clock::time_point); + ss::future do_purged_topic( + nt_revision, topic_purge_domain, model::timeout_clock::time_point); ss::future> autocreate_topics( topic_configuration_vector, model::timeout_clock::duration); @@ -250,7 +250,10 @@ class topics_frontend { model::timeout_clock::duration); ss::future dispatch_purged_topic_to_leader( - model::node_id, nt_revision, model::timeout_clock::duration); + model::node_id, + nt_revision, + topic_purge_domain, + model::timeout_clock::duration); ss::future do_update_replication_factor( topic_properties_update&, model::timeout_clock::time_point); diff --git a/src/v/cluster/types.cc b/src/v/cluster/types.cc index 34e2546ed16a..f99c0df16126 100644 --- a/src/v/cluster/types.cc +++ b/src/v/cluster/types.cc @@ -140,6 +140,18 @@ std::ostream& operator<<(std::ostream& o, const topic_properties_update& tpu) { return o; } +std::ostream& operator<<(std::ostream& o, const topic_purge_domain& d) { + switch (d) { + case topic_purge_domain::cloud_storage: + return o << "cloud_storage"; + case topic_purge_domain::iceberg: + return o << "iceberg"; + default: + fmt::print(o, "unknown({})", static_cast(d)); + return o; + } +} + std::ostream& operator<<(std::ostream& o, const topic_result& r) { fmt::print(o, "topic: {}, result: {}", r.tp_ns, r.ec); return o; diff --git a/src/v/cluster/types.h b/src/v/cluster/types.h index 3ffa40e502e5..5df74f14b77c 100644 --- a/src/v/cluster/types.h +++ b/src/v/cluster/types.h @@ -889,6 +889,15 @@ struct configuration_with_assignment using create_partitions_configuration_assignment = configuration_with_assignment; +// GC process consists of deleting topic data in several places, and these +// deletions have to be tracked separately. +enum class topic_purge_domain { + cloud_storage = 0, + iceberg = 1, +}; + +std::ostream& operator<<(std::ostream&, const topic_purge_domain&); + /** * Soft-deleting a topic may put it into different modes: initially this is * just a two stage thing: create a marker that acts as a tombstone, later @@ -899,9 +908,8 @@ using create_partitions_configuration_assignment * no intention of deletion. */ enum class topic_lifecycle_transition_mode : uint8_t { - // Drop the lifecycle marker: we do this after we're done with any - // garbage collection. - drop = 0, + // Purging of topic-related data in some topic_purge_domain is complete. + purged = 0, // Enter garbage collection phase: the topic appears deleted externally, // while internally we are garbage collecting any data that belonged @@ -952,13 +960,18 @@ struct nt_iceberg_tombstone struct topic_lifecycle_transition : serde::envelope< topic_lifecycle_transition, - serde::version<0>, + serde::version<1>, serde::compat_version<0>> { nt_revision topic; topic_lifecycle_transition_mode mode; - auto serde_fields() { return std::tie(topic, mode); } + // Used together with mode=purged. Default is cloud_storage for + // backwards compat (legacy lifecycle transitions were always cloud-storage + // related). + topic_purge_domain domain = topic_purge_domain::cloud_storage; + + auto serde_fields() { return std::tie(topic, mode, domain); } }; using topic_configuration_assignment @@ -1046,12 +1059,13 @@ struct create_topics_reply struct purged_topic_request : serde::envelope< purged_topic_request, - serde::version<0>, + serde::version<1>, serde::compat_version<0>> { using rpc_adl_exempt = std::true_type; nt_revision topic; model::timeout_clock::duration timeout; + topic_purge_domain domain = topic_purge_domain::cloud_storage; friend bool operator==(const purged_topic_request&, const purged_topic_request&) @@ -1059,7 +1073,7 @@ struct purged_topic_request friend std::ostream& operator<<(std::ostream&, const purged_topic_request&); - auto serde_fields() { return std::tie(topic, timeout); } + auto serde_fields() { return std::tie(topic, timeout, domain); } }; struct purged_topic_reply diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index 9f64c09e718e..28dbe42647fa 100644 --- a/src/v/redpanda/admin/server.cc +++ b/src/v/redpanda/admin/server.cc @@ -4329,7 +4329,8 @@ admin_server::delete_cloud_storage_lifecycle( cluster::nt_revision ntr{ .nt = model::topic_namespace(model::kafka_namespace, model::topic{topic}), .initial_revision_id = revision}; - auto r = co_await tp_frontend.local().purged_topic(ntr, 5s); + auto r = co_await tp_frontend.local().purged_topic( + ntr, cluster::topic_purge_domain::cloud_storage, 5s); co_await throw_on_error(*req, r.ec, model::controller_ntp); co_return ss::json::json_return_type(ss::json::json_void()); From af39c6b7a81c6be8f8178ba7e82a1bb955b05d49 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 13 Nov 2024 19:48:56 +0100 Subject: [PATCH 03/19] c/topic_table: create iceberg tombstones based on topic config --- src/v/cluster/topic_table.cc | 52 ++++++++++++++++++++++++-------- src/v/cluster/topics_frontend.cc | 6 +--- 2 files changed, 41 insertions(+), 17 deletions(-) diff --git a/src/v/cluster/topic_table.cc b/src/v/cluster/topic_table.cc index aa75b06519c8..d441700855d3 100644 --- a/src/v/cluster/topic_table.cc +++ b/src/v/cluster/topic_table.cc @@ -172,25 +172,53 @@ topic_table::apply(topic_lifecycle_transition soft_del, model::offset offset) { switch (soft_del.mode) { case topic_lifecycle_transition_mode::pending_gc: { - // Create a lifecycle marker auto tp = _topics.find(soft_del.topic.nt); if (tp == _topics.end()) { return ss::make_ready_future( errc::topic_not_exists); } - auto tombstone = nt_lifecycle_marker{ - .config = tp->second.get_configuration(), - .initial_revision_id = tp->second.get_remote_revision().value_or( - model::initial_revision_id(tp->second.get_revision())), - .timestamp = ss::lowres_system_clock::now()}; + // Create lifecycle markers + + const auto& topic_properties + = tp->second.get_configuration().properties; + + if (topic_properties.requires_remote_erase()) { + auto tombstone = nt_lifecycle_marker{ + .config = tp->second.get_configuration(), + .initial_revision_id = tp->second.get_remote_revision().value_or( + model::initial_revision_id(tp->second.get_revision())), + .timestamp = ss::lowres_system_clock::now()}; + + _lifecycle_markers.emplace(soft_del.topic, tombstone); + vlog( + clusterlog.debug, + "Created lifecycle marker for topic {} {}", + soft_del.topic.nt, + soft_del.topic.initial_revision_id); + } + + if ( + topic_properties.iceberg_mode != model::iceberg_mode::disabled + && topic_properties.iceberg_delete.value_or( + config::shard_local_cfg().iceberg_delete())) { + // Note that for iceberg tombstones we use topic.get_revision() + // (i.e. revision that got assigned to the topic at creation time) + // and not topic.get_remote_revision() (which may be an earlier + // revision if the topic was recovered from cloud storage). + auto tombstone = nt_iceberg_tombstone{ + .last_deleted_revision = tp->second.get_revision()}; + auto it = _iceberg_tombstones.emplace(tp->first, tombstone).first; + it->second.last_deleted_revision = std::max( + it->second.last_deleted_revision, tp->second.get_revision()); + + vlog( + clusterlog.debug, + "created iceberg tombstone for topic {} (revision: {})", + it->first, + it->second.last_deleted_revision); + } - _lifecycle_markers.emplace(soft_del.topic, tombstone); - vlog( - clusterlog.debug, - "Created lifecycle marker for topic {} {}", - soft_del.topic.nt, - soft_del.topic.initial_revision_id); [[fallthrough]]; // proceed to local deletion } case topic_lifecycle_transition_mode::oneshot_delete: diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index 62b22a3081c9..7e2b4f761e97 100644 --- a/src/v/cluster/topics_frontend.cc +++ b/src/v/cluster/topics_frontend.cc @@ -1007,12 +1007,8 @@ ss::future topics_frontend::do_delete_topic( if (migrated_away) { mode = topic_lifecycle_transition_mode::delete_migrated; vlog(clusterlog.info, "Deleting migrated topic {}", tp_ns); - } else if (topic_meta.get_configuration() - .properties.requires_remote_erase()) { - mode = topic_lifecycle_transition_mode::pending_gc; - vlog(clusterlog.info, "Created deletion marker for topic {}", tp_ns); } else { - mode = topic_lifecycle_transition_mode::oneshot_delete; + mode = topic_lifecycle_transition_mode::pending_gc; vlog(clusterlog.info, "Deleting topic {}", tp_ns); } From c35d93be4c71d78fb2b97d33f5c8bda5ff49404b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 14 Nov 2024 11:41:10 +0100 Subject: [PATCH 04/19] s/ntp_config: add topic revision field --- .../tests/cloud_storage_e2e_test.cc | 2 +- .../cluster/archival/archival_metadata_stm.cc | 4 +- .../archival/archiver_operations_impl.cc | 2 +- .../cluster/archival/ntp_archiver_service.cc | 2 +- .../tests/archival_metadata_stm_test.cc | 8 +-- src/v/cluster/controller_backend.cc | 56 +++++++++++-------- src/v/cluster/partition.cc | 10 +++- src/v/cluster/partition.h | 1 + src/v/cluster/partition_recovery_manager.cc | 2 +- src/v/cluster/topic_configuration.cc | 6 +- src/v/cluster/topic_configuration.h | 5 +- src/v/storage/ntp_config.h | 37 ++++++------ src/v/storage/types.cc | 10 ++-- 13 files changed, 84 insertions(+), 61 deletions(-) diff --git a/src/v/cloud_storage/tests/cloud_storage_e2e_test.cc b/src/v/cloud_storage/tests/cloud_storage_e2e_test.cc index 00a6ca3e8f22..c18f04b67236 100644 --- a/src/v/cloud_storage/tests/cloud_storage_e2e_test.cc +++ b/src/v/cloud_storage/tests/cloud_storage_e2e_test.cc @@ -309,7 +309,7 @@ TEST_P(EndToEndFixture, TestProduceConsumeFromCloudWithSpillover) { ASSERT_EQ(req.method, "PUT"); cloud_storage::partition_manifest spm( partition->get_ntp_config().ntp(), - partition->get_ntp_config().get_initial_revision()); + partition->get_ntp_config().get_remote_revision()); iobuf sbuf; sbuf.append(req.content.data(), req.content_length); vlog( diff --git a/src/v/cluster/archival/archival_metadata_stm.cc b/src/v/cluster/archival/archival_metadata_stm.cc index b5020d8b4e2a..4ea848029189 100644 --- a/src/v/cluster/archival/archival_metadata_stm.cc +++ b/src/v/cluster/archival/archival_metadata_stm.cc @@ -691,7 +691,7 @@ archival_metadata_stm::archival_metadata_stm( , _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)) + raft->ntp(), raft->log_config().get_remote_revision(), _mem_tracker)) , _cloud_storage_api(remote) , _feature_table(ft) , _remote_path_provider(remote_label, remote_topic_namespace_override) {} @@ -1272,7 +1272,7 @@ ss::future<> archival_metadata_stm::apply_local_snapshot( *_manifest = cloud_storage::partition_manifest( _raft->ntp(), - _raft->log_config().get_initial_revision(), + _raft->log_config().get_remote_revision(), _manifest->mem_tracker(), snap.start_offset, snap.last_offset, diff --git a/src/v/cluster/archival/archiver_operations_impl.cc b/src/v/cluster/archival/archiver_operations_impl.cc index 1f8d57e094b2..89e9782815bc 100644 --- a/src/v/cluster/archival/archiver_operations_impl.cc +++ b/src/v/cluster/archival/archiver_operations_impl.cc @@ -1088,7 +1088,7 @@ class cluster_partition : public detail::cluster_partition_api { } model::initial_revision_id get_initial_revision() const override { - return _part->log()->config().get_initial_revision(); + return _part->log()->config().get_remote_revision(); } // aborted transactions diff --git a/src/v/cluster/archival/ntp_archiver_service.cc b/src/v/cluster/archival/ntp_archiver_service.cc index b628098632d2..d3b076aa0e48 100644 --- a/src/v/cluster/archival/ntp_archiver_service.cc +++ b/src/v/cluster/archival/ntp_archiver_service.cc @@ -212,7 +212,7 @@ ntp_archiver::ntp_archiver( ss::shared_ptr ops, ss::shared_ptr> sched) : _ntp(ntp.ntp()) - , _rev(ntp.get_initial_revision()) + , _rev(ntp.get_remote_revision()) , _remote(remote) , _cache(c) , _parent(parent) diff --git a/src/v/cluster/archival/tests/archival_metadata_stm_test.cc b/src/v/cluster/archival/tests/archival_metadata_stm_test.cc index f23bf559869e..3e1b26795382 100644 --- a/src/v/cluster/archival/tests/archival_metadata_stm_test.cc +++ b/src/v/cluster/archival/tests/archival_metadata_stm_test.cc @@ -312,7 +312,7 @@ void check_snapshot_size( FIXTURE_TEST(test_snapshot_loading, archival_metadata_stm_base_fixture) { create_raft(); auto& ntp_cfg = _raft->log_config(); - partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_initial_revision()); + partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_remote_revision()); m.add( segment_name("0-1-v1.log"), segment_meta{ @@ -392,7 +392,7 @@ FIXTURE_TEST(test_snapshot_loading, archival_metadata_stm_base_fixture) { FIXTURE_TEST(test_sname_derivation, archival_metadata_stm_base_fixture) { create_raft(); auto& ntp_cfg = _raft->log_config(); - partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_initial_revision()); + partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_remote_revision()); // original segments m.add( @@ -509,7 +509,7 @@ FIXTURE_TEST( .committed_offset = model::offset(399), .archiver_term = model::term_id(1), .segment_term = model::term_id(1)}); - partition_manifest pm(ntp_cfg.ntp(), ntp_cfg.get_initial_revision()); + partition_manifest pm(ntp_cfg.ntp(), ntp_cfg.get_remote_revision()); for (const auto& s : m) { auto name = cloud_storage::generate_local_segment_name( s.base_offset, model::term_id{1}); @@ -651,7 +651,7 @@ FIXTURE_TEST( archival_metadata_stm_base_fixture) { create_raft(); auto& ntp_cfg = _raft->log_config(); - partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_initial_revision()); + partition_manifest m(ntp_cfg.ntp(), ntp_cfg.get_remote_revision()); m.add( segment_name("0-1-v1.log"), segment_meta{ diff --git a/src/v/cluster/controller_backend.cc b/src/v/cluster/controller_backend.cc index 640d6c987b90..bad8552bd6d5 100644 --- a/src/v/cluster/controller_backend.cc +++ b/src/v/cluster/controller_backend.cc @@ -1368,46 +1368,56 @@ ss::future controller_backend::create_partition( log_revision, initial_replicas); - auto cfg = _topics.local().get_topic_cfg(model::topic_namespace_view(ntp)); - if (!cfg) { - // partition was already removed, do nothing - co_return errc::success; - } - auto ec = co_await _shard_placement.prepare_create(ntp, log_revision); if (ec) { co_return ec; } + topic_configuration cfg; + model::revision_id topic_rev; + model::initial_revision_id remote_rev; + { + auto topic_md = _topics.local().get_topic_metadata_ref( + model::topic_namespace_view(ntp)); + if (!topic_md) { + // topic was already removed, do nothing + co_return errc::success; + } + cfg = topic_md->get().get_configuration(); + topic_rev = topic_md->get().get_revision(); + // Remote revision is used for cloud storage paths. If the topic was + // recovered, this is the value from the original manifest, and if topic + // is read replica, the value from remote topic manifest is used. + remote_rev = topic_md->get().get_remote_revision().value_or( + model::initial_revision_id{topic_rev}); + } + // handle partially created topic auto partition = _partition_manager.local().get(ntp); - // initial revision of the partition on the moment when it was created - // the value is used by shadow indexing - // if topic is read replica, the value from remote topic manifest is - // used - auto initial_rev = _topics.local().get_initial_revision(ntp); - if (!initial_rev) { - co_return errc::topic_not_exists; - } // no partition exists, create one if (likely(!partition)) { std::vector initial_brokers = create_brokers_set( initial_replicas, _members_table.local()); std::optional read_replica_bucket; - if (cfg->is_read_replica()) { + if (cfg.is_read_replica()) { read_replica_bucket = cloud_storage_clients::bucket_name( - cfg->properties.read_replica_bucket.value()); + cfg.properties.read_replica_bucket.value()); } std::optional xst_state; if (auto it = _xst_states.find(ntp); it != _xst_states.end()) { xst_state = it->second; } - auto ntp_config = cfg->make_ntp_config( - _data_directory, ntp.tp.partition, log_revision, initial_rev.value()); - auto rtp = cfg->properties.remote_topic_properties; + + auto ntp_config = cfg.make_ntp_config( + _data_directory, + ntp.tp.partition, + log_revision, + topic_rev, + remote_rev); + auto rtp = cfg.properties.remote_topic_properties; const bool is_cloud_topic = ntp_config.is_archival_enabled() || ntp_config.is_remote_fetch_enabled(); const bool is_internal = ntp.ns == model::kafka_internal_namespace; @@ -1423,7 +1433,7 @@ ss::future controller_backend::create_partition( // topic being cloud enabled implies existence of overrides ntp_config.get_overrides().recovery_enabled = storage::topic_recovery_enabled::yes; - rtp.emplace(*initial_rev, cfg->partition_count); + rtp.emplace(remote_rev, cfg.partition_count); } // we use offset as an rev as it is always increasing and it // increases while ntp is being created again @@ -1437,8 +1447,8 @@ ss::future controller_backend::create_partition( std::move(xst_state), rtp, read_replica_bucket, - cfg->properties.remote_label, - cfg->properties.remote_topic_namespace_override); + cfg.properties.remote_label, + cfg.properties.remote_topic_namespace_override); _xst_states.erase(ntp); @@ -1464,7 +1474,7 @@ ss::future controller_backend::create_partition( auto partition = _partition_manager.local().get(ntp); if (partition) { partition->set_topic_config( - std::make_unique(std::move(*cfg))); + std::make_unique(std::move(cfg))); } } diff --git a/src/v/cluster/partition.cc b/src/v/cluster/partition.cc index 3f8113756729..8febc0d33439 100644 --- a/src/v/cluster/partition.cc +++ b/src/v/cluster/partition.cc @@ -1205,7 +1205,7 @@ partition::unsafe_reset_remote_partition_manifest_from_json(iobuf json_buf) { // Deserialise provided manifest cloud_storage::partition_manifest req_m{ - _raft->ntp(), _raft->log_config().get_initial_revision()}; + _raft->ntp(), _raft->log_config().get_remote_revision()}; req_m.update_with_json(std::move(json_buf)); co_await replicate_unsafe_reset(std::move(req_m)); @@ -1292,7 +1292,7 @@ partition::fetch_latest_cloud_offset_from_manifest( co_return errc::invalid_partition_operation; } - const auto initial_rev = _raft->log_config().get_initial_revision(); + const auto initial_rev = _raft->log_config().get_remote_revision(); const auto bucket = [this]() { if (is_read_replica_mode_enabled()) { return get_read_replica_bucket(); @@ -1338,7 +1338,7 @@ partition::fetch_latest_cloud_offset_from_manifest( ss::future<> partition::do_unsafe_reset_remote_partition_manifest_from_cloud(bool force) { - const auto initial_rev = _raft->log_config().get_initial_revision(); + const auto initial_rev = _raft->log_config().get_remote_revision(); const auto bucket = [this]() { if (is_read_replica_mode_enabled()) { return get_read_replica_bucket(); @@ -1647,6 +1647,10 @@ model::revision_id partition::get_log_revision_id() const { return _raft->log_config().get_revision(); } +model::revision_id partition::get_topic_revision_id() const { + return _raft->log_config().get_topic_revision(); +} + std::optional partition::get_leader_id() const { return _raft->get_leader_id(); } diff --git a/src/v/cluster/partition.h b/src/v/cluster/partition.h index bbe4a8c38662..e571b4e9b737 100644 --- a/src/v/cluster/partition.h +++ b/src/v/cluster/partition.h @@ -191,6 +191,7 @@ class partition : public ss::enable_lw_shared_from_this { model::revision_id get_revision_id() const; model::revision_id get_log_revision_id() const; + model::revision_id get_topic_revision_id() const; std::optional get_leader_id() const; diff --git a/src/v/cluster/partition_recovery_manager.cc b/src/v/cluster/partition_recovery_manager.cc index 7c38aa3059ff..4f516fb670d9 100644 --- a/src/v/cluster/partition_recovery_manager.cc +++ b/src/v/cluster/partition_recovery_manager.cc @@ -711,7 +711,7 @@ 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)); + _ntpc.ntp(), _ntpc.get_remote_revision(), segm)); auto stream = [this, &stream_stats, diff --git a/src/v/cluster/topic_configuration.cc b/src/v/cluster/topic_configuration.cc index e8f320f7e0c5..4e60e29328b1 100644 --- a/src/v/cluster/topic_configuration.cc +++ b/src/v/cluster/topic_configuration.cc @@ -23,7 +23,8 @@ storage::ntp_config topic_configuration::make_ntp_config( const ss::sstring& work_dir, model::partition_id p_id, model::revision_id rev, - model::initial_revision_id init_rev) const { + model::revision_id topic_rev, + model::initial_revision_id remote_rev) const { auto has_overrides = properties.has_overrides() || is_internal(); std::unique_ptr overrides = nullptr; @@ -65,7 +66,8 @@ storage::ntp_config topic_configuration::make_ntp_config( work_dir, std::move(overrides), rev, - init_rev}; + topic_rev, + remote_rev}; } void topic_configuration::serde_write(iobuf& out) { diff --git a/src/v/cluster/topic_configuration.h b/src/v/cluster/topic_configuration.h index f71b036a9f15..ebe4db28c6a2 100644 --- a/src/v/cluster/topic_configuration.h +++ b/src/v/cluster/topic_configuration.h @@ -40,8 +40,9 @@ struct topic_configuration storage::ntp_config make_ntp_config( const ss::sstring&, model::partition_id, - model::revision_id, - model::initial_revision_id) const; + model::revision_id rev, + model::revision_id topic_rev, + model::initial_revision_id remote_rev) const; bool is_internal() const { return tp_ns.ns == model::kafka_internal_namespace diff --git a/src/v/storage/ntp_config.h b/src/v/storage/ntp_config.h index cf500a95cd67..eca03e6f9ef2 100644 --- a/src/v/storage/ntp_config.h +++ b/src/v/storage/ntp_config.h @@ -118,21 +118,25 @@ class ntp_config { model::ntp n, ss::sstring base_dir, std::unique_ptr overrides, - model::revision_id id, - model::initial_revision_id initial_id) noexcept + model::revision_id rev, + model::revision_id topic_rev, + model::initial_revision_id remote_rev) noexcept : _ntp(std::move(n)) , _base_dir(std::move(base_dir)) , _overrides(std::move(overrides)) - , _revision_id(id) - , _initial_rev(initial_id) {} + , _revision_id(rev) + , _topic_rev(topic_rev) + , _remote_rev(remote_rev) {} const model::ntp& ntp() const { return _ntp; } model::ntp& ntp() { return _ntp; } model::revision_id get_revision() const { return _revision_id; } - model::initial_revision_id get_initial_revision() const { - return _initial_rev; + model::revision_id get_topic_revision() const { return _topic_rev; } + + model::initial_revision_id get_remote_revision() const { + return _remote_rev; } const ss::sstring& base_directory() const { return _base_dir; } @@ -370,19 +374,18 @@ class ntp_config { std::unique_ptr _overrides; - /** - * A number indicating an id of the NTP in case it was created more - * than once (i.e. created, deleted and then created again) - */ + /// Revision of the command that resulted in this partition appearing on + /// this node (i.e. it will changed if the partition replica is moved back + /// and forth from/to the node, as well as if the topic is re-created). It + /// is used in constructing the local directory path. model::revision_id _revision_id{0}; - /** - * A number indicating an initial revision of the NTP. The revision - * of the NTP might change when the partition is moved between the - * nodes. The initial revision is the revision_id that was assigned - * to the topic when it was created. - */ - model::initial_revision_id _initial_rev{0}; + /// Revision of the topic creation command. + model::revision_id _topic_rev; + + /// This revision is used to construct cloud storage paths. It differs from + /// _topic_revision in case of recovered topics or read replicas. + model::initial_revision_id _remote_rev{0}; // in storage/types.cc friend std::ostream& operator<<(std::ostream&, const ntp_config&); diff --git a/src/v/storage/types.cc b/src/v/storage/types.cc index ef91edf16fe9..31bd671b7060 100644 --- a/src/v/storage/types.cc +++ b/src/v/storage/types.cc @@ -146,21 +146,23 @@ std::ostream& operator<<(std::ostream& o, const ntp_config& v) { fmt::print( o, "{{ntp: {}, base_dir: {}, overrides: {}, revision: {}, " - "initial_revision: {}}}", + "topic_revision: {}, remote_revision: {}}}", v.ntp(), v.base_directory(), v.get_overrides(), v.get_revision(), - v.get_initial_revision()); + v.get_topic_revision(), + v.get_remote_revision()); } else { fmt::print( o, "{{ntp: {}, base_dir: {}, overrides: nullptr, revision: {}, " - "initial_revision: {}}}", + "topic_revision: {}, remote_revision: {}}}", v.ntp(), v.base_directory(), v.get_revision(), - v.get_initial_revision()); + v.get_topic_revision(), + v.get_remote_revision()); } return o; } From 779e77bcc6de58cf65ad86d1256ed0e2181140d3 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 25 Nov 2024 14:53:48 +0100 Subject: [PATCH 05/19] datalake: use s3_impostor in fixture utests Previously we used http_impostor which was not providing adequate S3 API - e.g. if the test put some file into "S3", it then couldn't read that file back (the impostor returned 404). --- .../archival/tests/archival_service_fixture.h | 83 ++++++++++--------- src/v/cluster/tests/cluster_test_fixture.h | 39 +++++++-- src/v/datalake/tests/fixture.h | 63 ++++++-------- 3 files changed, 100 insertions(+), 85 deletions(-) diff --git a/src/v/cluster/archival/tests/archival_service_fixture.h b/src/v/cluster/archival/tests/archival_service_fixture.h index 1dc893629fd5..3acc7e8e0591 100644 --- a/src/v/cluster/archival/tests/archival_service_fixture.h +++ b/src/v/cluster/archival/tests/archival_service_fixture.h @@ -46,6 +46,46 @@ constexpr int16_t fixture_port_number = 7676; inline ss::logger arch_fixture_log("archival_service_fixture"); } // namespace +inline auto +get_cloud_storage_configurations(std::string_view hosthame, uint16_t port) { + net::unresolved_address server_addr(ss::sstring(hosthame), port); + cloud_storage_clients::s3_configuration s3_conf; + s3_conf.uri = cloud_storage_clients::access_point_uri(hosthame); + s3_conf.access_key = cloud_roles::public_key_str("access-key"); + s3_conf.secret_key = cloud_roles::private_key_str("secret-key"); + s3_conf.region = cloud_roles::aws_region_name("us-east-1"); + s3_conf.url_style = cloud_storage_clients::s3_url_style::virtual_host; + s3_conf._probe = ss::make_shared( + net::metrics_disabled::yes, + net::public_metrics_disabled::yes, + cloud_roles::aws_region_name{}, + cloud_storage_clients::endpoint_url{}); + s3_conf.server_addr = server_addr; + + archival::configuration a_conf{ + .cloud_storage_initial_backoff = config::mock_binding(100ms), + .segment_upload_timeout = config::mock_binding(1000ms), + .manifest_upload_timeout = config::mock_binding(1000ms), + .garbage_collect_timeout = config::mock_binding(1000ms), + .upload_loop_initial_backoff = config::mock_binding(100ms), + .upload_loop_max_backoff = config::mock_binding(5000ms)}; + a_conf.bucket_name = cloud_storage_clients::bucket_name("test-bucket"); + a_conf.ntp_metrics_disabled = archival::per_ntp_metrics_disabled::yes; + a_conf.svc_metrics_disabled = archival::service_metrics_disabled::yes; + a_conf.time_limit = std::nullopt; + + cloud_storage::configuration c_conf; + c_conf.client_config = s3_conf; + c_conf.bucket_name = cloud_storage_clients::bucket_name("test-bucket"); + c_conf.connection_limit = archival::connection_limit(2); + c_conf.cloud_credentials_source + = model::cloud_credentials_source::config_file; + return std::make_tuple( + std::move(s3_conf), + ss::make_lw_shared(std::move(a_conf)), + c_conf); +} + class archiver_cluster_fixture : public cluster_test_fixture , public http_imposter_fixture { @@ -54,51 +94,12 @@ class archiver_cluster_fixture static constexpr int proxy_port_base = 8082; static constexpr int schema_reg_port_base = 8081; - auto get_configurations() { - net::unresolved_address server_addr( - ss::sstring(httpd_host_name), httpd_port_number()); - cloud_storage_clients::s3_configuration s3_conf; - s3_conf.uri = cloud_storage_clients::access_point_uri(httpd_host_name); - s3_conf.access_key = cloud_roles::public_key_str("access-key"); - s3_conf.secret_key = cloud_roles::private_key_str("secret-key"); - s3_conf.region = cloud_roles::aws_region_name("us-east-1"); - s3_conf.url_style = cloud_storage_clients::s3_url_style::virtual_host; - s3_conf._probe = ss::make_shared( - net::metrics_disabled::yes, - net::public_metrics_disabled::yes, - cloud_roles::aws_region_name{}, - cloud_storage_clients::endpoint_url{}); - s3_conf.server_addr = server_addr; - - archival::configuration a_conf{ - .cloud_storage_initial_backoff = config::mock_binding(100ms), - .segment_upload_timeout = config::mock_binding(1000ms), - .manifest_upload_timeout = config::mock_binding(1000ms), - .garbage_collect_timeout = config::mock_binding(1000ms), - .upload_loop_initial_backoff = config::mock_binding(100ms), - .upload_loop_max_backoff = config::mock_binding(5000ms)}; - a_conf.bucket_name = cloud_storage_clients::bucket_name("test-bucket"); - a_conf.ntp_metrics_disabled = archival::per_ntp_metrics_disabled::yes; - a_conf.svc_metrics_disabled = archival::service_metrics_disabled::yes; - a_conf.time_limit = std::nullopt; - - cloud_storage::configuration c_conf; - c_conf.client_config = s3_conf; - c_conf.bucket_name = cloud_storage_clients::bucket_name("test-bucket"); - c_conf.connection_limit = archival::connection_limit(2); - c_conf.cloud_credentials_source - = model::cloud_credentials_source::config_file; - return std::make_tuple( - std::move(s3_conf), - ss::make_lw_shared(std::move(a_conf)), - c_conf); - } - public: model::node_id add_node() { auto id = model::node_id((int)apps.size()); - auto [s3_conf, a_conf, cs_conf] = get_configurations(); + auto [s3_conf, a_conf, cs_conf] = get_cloud_storage_configurations( + httpd_host_name, httpd_port_number()); auto node = create_node_application( id, kafka_port_base, diff --git a/src/v/cluster/tests/cluster_test_fixture.h b/src/v/cluster/tests/cluster_test_fixture.h index 02a21e4b219a..36054a46cc68 100644 --- a/src/v/cluster/tests/cluster_test_fixture.h +++ b/src/v/cluster/tests/cluster_test_fixture.h @@ -69,7 +69,7 @@ class cluster_test_fixture { std::filesystem::remove_all(std::filesystem::path(_base_dir)); } - virtual fixture_ptr make_redpanda_fixture( + fixture_ptr make_redpanda_fixture( model::node_id node_id, int16_t kafka_port, int16_t rpc_port, @@ -82,7 +82,8 @@ class cluster_test_fixture { = std::nullopt, std::optional archival_cfg = std::nullopt, std::optional cloud_cfg = std::nullopt, - bool enable_legacy_upload_mode = true) { + bool enable_legacy_upload_mode = true, + bool iceberg_enabled = false) { return std::make_unique( node_id, kafka_port, @@ -100,7 +101,8 @@ class cluster_test_fixture { empty_seed_starts_cluster_val, std::nullopt, false, - enable_legacy_upload_mode); + enable_legacy_upload_mode, + iceberg_enabled); } void add_node( @@ -117,7 +119,8 @@ class cluster_test_fixture { = std::nullopt, std::optional archival_cfg = std::nullopt, std::optional cloud_cfg = std::nullopt, - bool enable_legacy_upload_mode = true) { + bool enable_legacy_upload_mode = true, + bool iceberg_enabled = false) { _instances.emplace( node_id, make_redpanda_fixture( @@ -132,7 +135,8 @@ class cluster_test_fixture { s3_config, archival_cfg, cloud_cfg, - enable_legacy_upload_mode)); + enable_legacy_upload_mode, + iceberg_enabled)); } application* get_node_application(model::node_id id) { @@ -165,7 +169,8 @@ class cluster_test_fixture { = std::nullopt, std::optional archival_cfg = std::nullopt, std::optional cloud_cfg = std::nullopt, - bool legacy_upload_mode_enabled = true) { + bool legacy_upload_mode_enabled = true, + bool iceberg_enabled = false) { std::vector seeds = {}; if (!empty_seed_starts_cluster_val || node_id != 0) { seeds.push_back( @@ -183,7 +188,8 @@ class cluster_test_fixture { s3_config, archival_cfg, cloud_cfg, - legacy_upload_mode_enabled); + legacy_upload_mode_enabled, + iceberg_enabled); return get_node_application(node_id); } @@ -320,6 +326,25 @@ class cluster_test_fixture { } protected: + std::vector instance_ids() const { + std::vector ret; + for (const auto& [id, _] : _instances) { + ret.push_back(id); + } + return ret; + } + + model::node_id next_node_id() const { + model::node_id max; + for (const auto& [id, _] : _instances) { + max = std::max(max, id); + } + if (max < 0) { + return model::node_id{0}; + } + return max + model::node_id{1}; + } + redpanda_thread_fixture* instance(model::node_id id) { return _instances[id].get(); } diff --git a/src/v/datalake/tests/fixture.h b/src/v/datalake/tests/fixture.h index ff3cf7be6771..bb21fe055a78 100644 --- a/src/v/datalake/tests/fixture.h +++ b/src/v/datalake/tests/fixture.h @@ -17,51 +17,40 @@ static ss::logger logger{"datalake-test-logger"}; namespace datalake::tests { -class datalake_cluster_test_fixture : public archiver_cluster_fixture { +class datalake_cluster_test_fixture + : public cluster_test_fixture + , public s3_imposter_fixture { public: - datalake_cluster_test_fixture() - : archiver_cluster_fixture() { - listen(); - } + datalake_cluster_test_fixture() { set_expectations_and_listen({}); } ~datalake_cluster_test_fixture() { - for (auto& [id, _] : apps) { + for (auto id : instance_ids()) { remove_node_application(id); } } - fixture_ptr make_redpanda_fixture( - model::node_id node_id, - int16_t kafka_port, - int16_t rpc_port, - int16_t proxy_port, - int16_t schema_reg_port, - std::vector seeds, - configure_node_id use_node_id, - empty_seed_starts_cluster empty_seed_starts_cluster_val, - std::optional s3_config, - std::optional archival_cfg, - std::optional cloud_cfg, - bool enable_legacy_upload_mode) override { - return std::make_unique( - node_id, - kafka_port, - rpc_port, - proxy_port, - schema_reg_port, - seeds, - ssx::sformat("{}.{}", _base_dir, node_id()), - _sgroups, - true, - s3_config, - archival_cfg, - cloud_cfg, - use_node_id, - empty_seed_starts_cluster_val, - std::nullopt, + void add_node() { + static constexpr int kafka_port_base = 9092; + static constexpr int rpc_port_base = 11000; + static constexpr int proxy_port_base = 8082; + static constexpr int schema_reg_port_base = 8081; + + auto [s3_conf, a_conf, cs_conf] = get_cloud_storage_configurations( + httpd_host_name, httpd_port_number()); + + create_node_application( + next_node_id(), + kafka_port_base, + rpc_port_base, + proxy_port_base, + schema_reg_port_base, + configure_node_id::yes, + empty_seed_starts_cluster::yes, + s3_conf, + std::move(*a_conf), + cs_conf, false, - enable_legacy_upload_mode, - /* iceberg */ true); + /*iceberg_enabled=*/true); } ss::future<> create_iceberg_topic( From 3a946dcea6904a4b5f3ed3f9f137812e593e8ff6 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 14 Nov 2024 16:24:11 +0100 Subject: [PATCH 06/19] datalake: add topic revision to coordinator state and requests --- src/v/datalake/coordinator/coordinator.cc | 39 +++++++++-- src/v/datalake/coordinator/coordinator.h | 11 ++- src/v/datalake/coordinator/frontend.cc | 8 ++- .../coordinator/iceberg_file_committer.cc | 2 +- src/v/datalake/coordinator/state.cc | 1 + src/v/datalake/coordinator/state.h | 4 +- src/v/datalake/coordinator/state_update.cc | 69 ++++++++++++++++--- src/v/datalake/coordinator/state_update.h | 15 ++-- .../coordinator/tests/coordinator_test.cc | 56 +++++++++------ .../coordinator/tests/state_machine_test.cc | 4 +- .../coordinator/tests/state_test_utils.h | 5 +- .../coordinator/tests/state_update_test.cc | 40 ++++++----- src/v/datalake/coordinator/types.cc | 11 ++- src/v/datalake/coordinator/types.h | 32 +++++++-- src/v/datalake/tests/fixture.h | 2 + .../translation/partition_translator.cc | 12 ++-- 16 files changed, 230 insertions(+), 81 deletions(-) diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index 729122c15db0..16116e0381d3 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -49,6 +49,8 @@ std::ostream& operator<<(std::ostream& o, coordinator::errc e) { return o << "coordinator::errc::shutting_down"; case coordinator::errc::stm_apply_error: return o << "coordinator::errc::stm_apply_error"; + case coordinator::errc::revision_mismatch: + return o << "coordinator::errc::revision_mismatch"; case coordinator::errc::timedout: return o << "coordinator::errc::timedout"; } @@ -209,7 +211,9 @@ checked coordinator::maybe_gate() { ss::future> coordinator::sync_add_files( - model::topic_partition tp, chunked_vector entries) { + model::topic_partition tp, + model::revision_id topic_revision, + chunked_vector entries) { if (entries.empty()) { vlog(datalake_log.debug, "Empty entry requested {}", tp); co_return std::nullopt; @@ -220,8 +224,9 @@ coordinator::sync_add_files( } vlog( datalake_log.debug, - "Sync add files requested {}: [{}, {}], {} files", + "Sync add files requested {} (topic rev: {}): [{}, {}], {} files", tp, + topic_revision, entries.begin()->start_offset, entries.back().last_offset, entries.size()); @@ -231,7 +236,7 @@ coordinator::sync_add_files( } auto added_last_offset = entries.back().last_offset; auto update_res = add_files_update::build( - stm_->state(), tp, std::move(entries)); + stm_->state(), tp, topic_revision, std::move(entries)); if (update_res.has_error()) { // NOTE: rejection here is just an optimization -- the operation would // fail to be applied to the STM anyway. @@ -273,7 +278,8 @@ coordinator::sync_add_files( } ss::future, coordinator::errc>> -coordinator::sync_get_last_added_offset(model::topic_partition tp) { +coordinator::sync_get_last_added_offset( + model::topic_partition tp, model::revision_id requested_topic_rev) { auto gate = maybe_gate(); if (gate.has_error()) { co_return gate.error(); @@ -282,11 +288,30 @@ coordinator::sync_get_last_added_offset(model::topic_partition tp) { if (sync_res.has_error()) { co_return convert_stm_errc(sync_res.error()); } - auto prt_state_opt = stm_->state().partition_state(tp); - if (!prt_state_opt.has_value()) { + auto topic_it = stm_->state().topic_to_state.find(tp.topic); + if (topic_it == stm_->state().topic_to_state.end()) { co_return std::nullopt; } - const auto& prt_state = prt_state_opt->get(); + const auto& topic = topic_it->second; + if (requested_topic_rev < topic.revision) { + vlog( + datalake_log.debug, + "asked offsets for tp {} but rev {} is obsolete, current rev: {}", + tp, + requested_topic_rev, + topic.revision); + co_return errc::revision_mismatch; + } else if (requested_topic_rev > topic.revision) { + // Coordinator is ready to accept files for the new topic revision, + // but there is no stm record yet. Reply with "no offset". + co_return std::nullopt; + } + + auto partition_it = topic.pid_to_pending_files.find(tp.partition); + if (partition_it == topic.pid_to_pending_files.end()) { + co_return std::nullopt; + } + const auto& prt_state = partition_it->second; if (prt_state.pending_entries.empty()) { co_return prt_state.last_committed; } diff --git a/src/v/datalake/coordinator/coordinator.h b/src/v/datalake/coordinator/coordinator.h index 03e345631c02..24b145872329 100644 --- a/src/v/datalake/coordinator/coordinator.h +++ b/src/v/datalake/coordinator/coordinator.h @@ -27,6 +27,7 @@ class coordinator { enum class errc { not_leader, stm_apply_error, + revision_mismatch, timedout, shutting_down, }; @@ -40,10 +41,16 @@ class coordinator { void start(); ss::future<> stop_and_wait(); + ss::future> sync_add_files( - model::topic_partition tp, chunked_vector); + model::topic_partition tp, + model::revision_id topic_revision, + chunked_vector); + ss::future, errc>> - sync_get_last_added_offset(model::topic_partition tp); + sync_get_last_added_offset( + model::topic_partition tp, model::revision_id topic_rev); + void notify_leadership(std::optional); bool leader_loop_running() const { return term_as_.has_value(); } diff --git a/src/v/datalake/coordinator/frontend.cc b/src/v/datalake/coordinator/frontend.cc index ed72e83d05dd..426017ab0c66 100644 --- a/src/v/datalake/coordinator/frontend.cc +++ b/src/v/datalake/coordinator/frontend.cc @@ -32,6 +32,8 @@ errc to_rpc_errc(coordinator::errc e) { return errc::not_leader; case coordinator::errc::stm_apply_error: return errc::stale; + case coordinator::errc::revision_mismatch: + return errc::revision_mismatch; case coordinator::errc::timedout: return errc::timeout; } @@ -44,7 +46,8 @@ ss::future add_files( if (!crd) { co_return add_translated_data_files_reply{errc::not_leader}; } - auto ret = co_await crd->sync_add_files(req.tp, std::move(req.ranges)); + auto ret = co_await crd->sync_add_files( + req.tp, req.topic_revision, std::move(req.ranges)); if (ret.has_error()) { co_return to_rpc_errc(ret.error()); } @@ -58,7 +61,8 @@ ss::future fetch_latest_offset( if (!crd) { co_return fetch_latest_translated_offset_reply{errc::not_leader}; } - auto ret = co_await crd->sync_get_last_added_offset(req.tp); + auto ret = co_await crd->sync_get_last_added_offset( + req.tp, req.topic_revision); if (ret.has_error()) { co_return to_rpc_errc(ret.error()); } diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index 58d4f9a5e603..7539d4cb6cd0 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -162,7 +162,7 @@ iceberg_file_committer::commit_topic_files_to_catalog( for (const auto& [pid, committed_offset] : pending_commits) { auto tp = model::topic_partition(topic, pid); auto update_res = mark_files_committed_update::build( - state, tp, committed_offset); + state, tp, tp_state.revision, committed_offset); if (update_res.has_error()) { vlog( datalake_log.warn, diff --git a/src/v/datalake/coordinator/state.cc b/src/v/datalake/coordinator/state.cc index 9e2ed767e266..b24e9fdc6de5 100644 --- a/src/v/datalake/coordinator/state.cc +++ b/src/v/datalake/coordinator/state.cc @@ -26,6 +26,7 @@ partition_state partition_state::copy() const { topic_state topic_state::copy() const { topic_state result; + result.revision = revision; result.pid_to_pending_files.reserve(pid_to_pending_files.size()); for (const auto& [id, state] : pid_to_pending_files) { result.pid_to_pending_files[id] = state.copy(); diff --git a/src/v/datalake/coordinator/state.h b/src/v/datalake/coordinator/state.h index e5340c7f9f36..5f76ec1b8412 100644 --- a/src/v/datalake/coordinator/state.h +++ b/src/v/datalake/coordinator/state.h @@ -81,8 +81,10 @@ struct partition_state struct topic_state : public serde:: envelope, serde::compat_version<0>> { - auto serde_fields() { return std::tie(pid_to_pending_files); } + auto serde_fields() { return std::tie(revision, pid_to_pending_files); } + // Topic revision + model::revision_id revision; // Map from Redpanda partition id to the files pending per partition. chunked_hash_map pid_to_pending_files; diff --git a/src/v/datalake/coordinator/state_update.cc b/src/v/datalake/coordinator/state_update.cc index da988dc61668..e89f5757f5e3 100644 --- a/src/v/datalake/coordinator/state_update.cc +++ b/src/v/datalake/coordinator/state_update.cc @@ -28,9 +28,11 @@ std::ostream& operator<<(std::ostream& o, const update_key& u) { checked add_files_update::build( const topics_state& state, const model::topic_partition& tp, + model::revision_id topic_revision, chunked_vector entries) { add_files_update update{ .tp = tp, + .topic_revision = topic_revision, .entries = std::move(entries), }; auto allowed = update.can_apply(state); @@ -45,12 +47,30 @@ add_files_update::can_apply(const topics_state& state) { if (entries.empty()) { return stm_update_error{"No entries requested"}; } - auto prt_state_opt = state.partition_state(tp); - if (!prt_state_opt.has_value()) { - // No entries at all, this partition hasn't ever added any files. + auto topic_it = state.topic_to_state.find(tp.topic); + if (topic_it == state.topic_to_state.end()) { + return std::nullopt; + } + auto& cur_topic = topic_it->second; + if (topic_revision < cur_topic.revision) { + return stm_update_error{fmt::format( + "topic {} rev {} not yet registered (current rev {})", + tp.topic, + topic_revision, + cur_topic.revision)}; + } else if (topic_revision > cur_topic.revision) { + // We are ready to accept files for an instance with the higher revision + // id. return std::nullopt; } - const auto& prt_state = prt_state_opt.value().get(); + + auto partition_it = topic_it->second.pid_to_pending_files.find( + tp.partition); + if (partition_it == topic_it->second.pid_to_pending_files.end()) { + return std::nullopt; + } + const auto& prt_state = partition_it->second; + if ( prt_state.pending_entries.empty() && !prt_state.last_committed.has_value()) { @@ -83,7 +103,16 @@ add_files_update::apply(topics_state& state, model::offset applied_offset) { } const auto& topic = tp.topic; const auto& pid = tp.partition; + auto& tp_state = state.topic_to_state[topic]; + if (topic_revision > tp_state.revision) { + // We've got files for a topic instance with higher revision id, reset + // topic state + topic_state new_state; + new_state.revision = topic_revision; + tp_state = std::move(new_state); + } + // after this point tp_state.revision == topic_revision auto& partition_state = tp_state.pid_to_pending_files[pid]; for (auto& e : entries) { partition_state.pending_entries.emplace_back(pending_entry{ @@ -98,9 +127,11 @@ checked mark_files_committed_update::build( const topics_state& state, const model::topic_partition& tp, + model::revision_id topic_revision, kafka::offset o) { mark_files_committed_update update{ .tp = tp, + .topic_revision = topic_revision, .new_committed = o, }; auto allowed = update.can_apply(state); @@ -112,24 +143,40 @@ mark_files_committed_update::build( checked mark_files_committed_update::can_apply(const topics_state& state) { - auto prt_state = state.partition_state(tp); - if (!prt_state.has_value() || prt_state->get().pending_entries.empty()) { - // Can't mark files committed if there are no files. + auto topic_it = state.topic_to_state.find(tp.topic); + if (topic_it == state.topic_to_state.end()) { + return stm_update_error{fmt::format( + "topic {} rev {} not yet registered", tp.topic, topic_revision)}; + } + const auto& cur_topic = topic_it->second; + if (topic_revision != cur_topic.revision) { + return stm_update_error{fmt::format( + "topic {} revision mismatch: got {}, current rev {}", + tp.topic, + topic_revision, + cur_topic.revision)}; + } + + auto partition_it = cur_topic.pid_to_pending_files.find(tp.partition); + if ( + partition_it == cur_topic.pid_to_pending_files.end() + || partition_it->second.pending_entries.empty()) { return stm_update_error{ "Can't mark files committed if there are no files"}; } + const auto& prt_state = partition_it->second; if ( - prt_state->get().last_committed.has_value() - && prt_state->get().last_committed.value() >= new_committed) { + prt_state.last_committed.has_value() + && prt_state.last_committed.value() >= new_committed) { // The state already has committed up to the given offset. return stm_update_error{fmt::format( "The state has committed up to {} >= requested offset {}", - prt_state->get().last_committed.value(), + prt_state.last_committed.value(), new_committed)}; } // At this point, the desired offset looks okay. Examine the entries to // make sure the new committed offset corresponds to one of them. - for (const auto& entry_state : prt_state->get().pending_entries) { + for (const auto& entry_state : prt_state.pending_entries) { if (entry_state.data.last_offset == new_committed) { return std::nullopt; } diff --git a/src/v/datalake/coordinator/state_update.h b/src/v/datalake/coordinator/state_update.h index 77794d0fc77e..b2c97c008340 100644 --- a/src/v/datalake/coordinator/state_update.h +++ b/src/v/datalake/coordinator/state_update.h @@ -37,15 +37,16 @@ struct add_files_update static checked build( const topics_state&, const model::topic_partition&, + model::revision_id topic_revision, chunked_vector); - auto serde_fields() { return std::tie(tp, entries); } + auto serde_fields() { return std::tie(tp, topic_revision, entries); } checked can_apply(const topics_state&); checked apply(topics_state&, model::offset); model::topic_partition tp; - + model::revision_id topic_revision; // Expected to be ordered from lowest offset to highest offset. chunked_vector entries; }; @@ -57,14 +58,18 @@ struct mark_files_committed_update serde::version<0>, serde::compat_version<0>> { static constexpr auto key{update_key::mark_files_committed}; - static checked - build(const topics_state&, const model::topic_partition&, kafka::offset); - auto serde_fields() { return std::tie(tp, new_committed); } + static checked build( + const topics_state&, + const model::topic_partition&, + model::revision_id topic_revision, + kafka::offset); + auto serde_fields() { return std::tie(tp, topic_revision, new_committed); } checked can_apply(const topics_state&); checked apply(topics_state&); model::topic_partition tp; + model::revision_id topic_revision; // All pending entries whose offset range falls entirely below this offset // (inclusive) should be removed. diff --git a/src/v/datalake/coordinator/tests/coordinator_test.cc b/src/v/datalake/coordinator/tests/coordinator_test.cc index a179618f3a56..665e3cca14de 100644 --- a/src/v/datalake/coordinator/tests/coordinator_test.cc +++ b/src/v/datalake/coordinator/tests/coordinator_test.cc @@ -72,6 +72,7 @@ using pairs_t = std::vector>; ss::future<> file_adder_loop( const pairs_t& files, const model::topic_partition& tp, + model::revision_id topic_rev, coordinator_node& n, int fiber_id, bool& done) { @@ -82,7 +83,8 @@ ss::future<> file_adder_loop( while (!done) { co_await random_sleep_ms(30); vlog(datalake::datalake_log.debug, "[{}] getting last added", id); - auto last_res = co_await n.crd.sync_get_last_added_offset(tp); + auto last_res = co_await n.crd.sync_get_last_added_offset( + tp, topic_rev); if (last_res.has_error()) { continue; } @@ -123,7 +125,7 @@ ss::future<> file_adder_loop( files_to_send.size(), files_to_send.begin()->first); auto add_res = co_await n.crd.sync_add_files( - tp, make_pending_files(files_to_send)); + tp, topic_rev, make_pending_files(files_to_send)); if (add_res.has_error()) { // Leave this inner loop on error so we can refetch. break; @@ -284,7 +286,9 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); const auto tp01 = tp(0, 1); + const model::revision_id rev0{1}; const auto tp10 = tp(1, 0); + const model::revision_id rev1{2}; pairs_t total_expected_00; for (const auto& v : { pairs_t{{0, 100}}, @@ -293,7 +297,7 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { pairs_t{{401, 500}, {501, 600}}, }) { auto add_res - = leader.crd.sync_add_files(tp00, make_pending_files(v)).get(); + = leader.crd.sync_add_files(tp00, rev0, make_pending_files(v)).get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); @@ -308,7 +312,7 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { pairs_t total_expected_01; for (const auto& v : {pairs_t{{0, 100}}, pairs_t{{101, 200}}}) { auto add_res - = leader.crd.sync_add_files(tp01, make_pending_files(v)).get(); + = leader.crd.sync_add_files(tp01, rev0, make_pending_files(v)).get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); @@ -324,7 +328,7 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { pairs_t total_expected_10; for (const auto& v : {pairs_t{{100, 200}}, pairs_t{{201, 300}}}) { auto add_res - = leader.crd.sync_add_files(tp10, make_pending_files(v)).get(); + = leader.crd.sync_add_files(tp10, rev1, make_pending_files(v)).get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); @@ -346,20 +350,21 @@ TEST_F(CoordinatorTest, TestLastAddedHappyPath) { auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); const auto tp01 = tp(0, 1); + const model::revision_id rev{1}; pairs_t total_expected_00; for (const auto& v : {pairs_t{{101, 200}}, pairs_t{{201, 300}, {301, 400}}}) { auto add_res - = leader.crd.sync_add_files(tp00, make_pending_files(v)).get(); + = leader.crd.sync_add_files(tp00, rev, make_pending_files(v)).get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); } - auto last_res = leader.crd.sync_get_last_added_offset(tp00).get(); + auto last_res = leader.crd.sync_get_last_added_offset(tp00, rev).get(); ASSERT_FALSE(last_res.has_error()) << last_res.error(); ASSERT_TRUE(last_res.value().has_value()); ASSERT_EQ(400, last_res.value().value()()); - last_res = leader.crd.sync_get_last_added_offset(tp01).get(); + last_res = leader.crd.sync_get_last_added_offset(tp01, rev).get(); ASSERT_FALSE(last_res.has_error()) << last_res.error(); ASSERT_FALSE(last_res.value().has_value()); } @@ -377,15 +382,16 @@ TEST_F(CoordinatorTest, TestNotLeader) { ASSERT_TRUE(non_leader_opt.has_value()); auto& non_leader = non_leader_opt->get(); const auto tp00 = tp(0, 0); + const model::revision_id rev{1}; pairs_t total_expected_00; auto add_res = non_leader.crd - .sync_add_files(tp00, make_pending_files({{0, 100}})) + .sync_add_files(tp00, rev, make_pending_files({{0, 100}})) .get(); ASSERT_TRUE(add_res.has_error()); EXPECT_EQ(coordinator::errc::not_leader, add_res.error()); - auto last_res = non_leader.crd.sync_get_last_added_offset(tp00).get(); + auto last_res = non_leader.crd.sync_get_last_added_offset(tp00, rev).get(); ASSERT_TRUE(last_res.has_error()) << last_res.error(); EXPECT_EQ(coordinator::errc::not_leader, last_res.error()); } @@ -403,6 +409,7 @@ TEST_P(CoordinatorTestWithParams, TestConcurrentAddFiles) { std::pair{cur_start, next_start - 1}); } const auto tp00 = tp(0, 0); + const model::revision_id rev0{1}; bool done = false; std::vector> adders; int fiber_id = 0; @@ -413,7 +420,7 @@ TEST_P(CoordinatorTestWithParams, TestConcurrentAddFiles) { for (auto& n : crds) { for (size_t i = 0; i < num_adders_per_node; i++) { adders.push_back( - file_adder_loop(files, tp00, *n, fiber_id++, done)); + file_adder_loop(files, tp00, rev0, *n, fiber_id++, done)); } } std::optional> chaos; @@ -510,8 +517,10 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesHappyPath) { ASSERT_NO_FATAL_FAILURE(wait_for_leader(leader_opt).get()); auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); - auto add_res - = leader.crd.sync_add_files(tp00, make_pending_files({{0, 100}})).get(); + const model::revision_id rev0{1}; + auto add_res = leader.crd + .sync_add_files(tp00, rev0, make_pending_files({{0, 100}})) + .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); RPTEST_REQUIRE_EVENTUALLY(1s, [&] { @@ -537,8 +546,10 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesNotLeader) { auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); - auto add_res - = leader.crd.sync_add_files(tp00, make_pending_files({{0, 100}})).get(); + const model::revision_id rev0{1}; + auto add_res = leader.crd + .sync_add_files(tp00, rev0, make_pending_files({{0, 100}})) + .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); ss::sleep(500ms).get(); @@ -565,8 +576,9 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesNotLeader) { } // Newly added files are committed in the background. - add_res - = leader.crd.sync_add_files(tp00, make_pending_files({{101, 200}})).get(); + add_res = leader.crd + .sync_add_files(tp00, rev0, make_pending_files({{101, 200}})) + .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); @@ -582,7 +594,7 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesNotLeader) { ASSERT_NO_FATAL_FAILURE(wait_for_leader(leader_opt).get()); auto& new_leader = leader_opt->get(); add_res = new_leader.crd - .sync_add_files(tp00, make_pending_files({{201, 300}})) + .sync_add_files(tp00, rev0, make_pending_files({{201, 300}})) .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); wait_for_apply().get(); @@ -614,8 +626,12 @@ TEST_F(CoordinatorSleepingLoopTest, TestQuickShutdownOnLeadershipChange) { auto& leader = leader_opt->get(); for (int i = 0; i < 100; i++) { auto t = tp(i, 0); - auto add_res - = leader.crd.sync_add_files(t, make_pending_files({{0, 100}})).get(); + auto add_res = leader.crd + .sync_add_files( + t, + model::revision_id{i}, + make_pending_files({{0, 100}})) + .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); } ASSERT_TRUE(leader.crd.leader_loop_running()); diff --git a/src/v/datalake/coordinator/tests/state_machine_test.cc b/src/v/datalake/coordinator/tests/state_machine_test.cc index fa6893a8e806..d1404170905b 100644 --- a/src/v/datalake/coordinator/tests/state_machine_test.cc +++ b/src/v/datalake/coordinator/tests/state_machine_test.cc @@ -117,6 +117,7 @@ struct coordinator_stm_fixture : stm_raft_fixture { static constexpr int32_t max_partitions = 5; model::topic_partition tp{model::topic{"test"}, model::partition_id{0}}; + model::revision_id rev{123}; datalake::coordinator::simple_file_committer file_committer; absl::flat_hash_map coordinators; }; @@ -135,7 +136,7 @@ TEST_F_CORO(coordinator_stm_fixture, test_snapshots) { auto add_files_result = co_await retry_with_leader_coordinator( [&, this](coordinator& coordinator) mutable { auto tp = random_tp(); - return coordinator->sync_get_last_added_offset(tp).then( + return coordinator->sync_get_last_added_offset(tp, rev).then( [&, tp](auto result) { if (!result) { return ss::make_ready_future(false); @@ -152,6 +153,7 @@ TEST_F_CORO(coordinator_stm_fixture, test_snapshots) { return coordinator ->sync_add_files( tp, + rev, datalake::coordinator::make_pending_files(offset_pairs)) .then([](auto result) { return ss::make_ready_future( diff --git a/src/v/datalake/coordinator/tests/state_test_utils.h b/src/v/datalake/coordinator/tests/state_test_utils.h index 9474849b5edf..b5937324ddc4 100644 --- a/src/v/datalake/coordinator/tests/state_test_utils.h +++ b/src/v/datalake/coordinator/tests/state_test_utils.h @@ -44,7 +44,10 @@ class simple_file_committer : public file_committer { } model::topic_partition tp(t, pid); auto build_res = mark_files_committed_update::build( - state, tp, files.pending_entries.back().data.last_offset); + state, + tp, + t_state.revision, + files.pending_entries.back().data.last_offset); EXPECT_FALSE(build_res.has_error()); ret.emplace_back(std::move(build_res.value())); } diff --git a/src/v/datalake/coordinator/tests/state_update_test.cc b/src/v/datalake/coordinator/tests/state_update_test.cc index f9ed46510444..15e11bb1174e 100644 --- a/src/v/datalake/coordinator/tests/state_update_test.cc +++ b/src/v/datalake/coordinator/tests/state_update_test.cc @@ -22,6 +22,7 @@ using namespace datalake::coordinator; namespace { const model::topic topic{"test_topic"}; +const model::revision_id rev{123}; const model::partition_id pid{0}; const model::topic_partition tp{topic, pid}; @@ -32,12 +33,12 @@ void check_add_doesnt_apply( const std::vector>& offset_bounds) { // We should fail to build the update in the first place. auto update = add_files_update::build( - state, tp, make_pending_files(offset_bounds)); + state, tp, rev, make_pending_files(offset_bounds)); EXPECT_TRUE(update.has_error()); // Also explicitly build the bad update and make sure it doesn't apply. auto res - = add_files_update{.tp = tp, .entries = make_pending_files(offset_bounds)} + = add_files_update{.tp = tp, .topic_revision = rev, .entries = make_pending_files(offset_bounds)} .apply(state, model::offset{}); EXPECT_TRUE(res.has_error()); } @@ -49,12 +50,12 @@ void check_commit_doesnt_apply( int64_t commit_offset) { // We should fail to build the update in the first place. auto update = mark_files_committed_update::build( - state, tp, kafka::offset{commit_offset}); + state, tp, rev, kafka::offset{commit_offset}); EXPECT_TRUE(update.has_error()); // Also explicitly build the bad update and make sure it doesn't apply. auto res - = mark_files_committed_update{.tp = tp, .new_committed = kafka::offset{commit_offset}} + = mark_files_committed_update{.tp = tp, .topic_revision = rev, .new_committed = kafka::offset{commit_offset}} .apply(state); EXPECT_TRUE(res.has_error()); } @@ -64,7 +65,7 @@ void check_commit_doesnt_apply( TEST(StateUpdateTest, TestAddFile) { topics_state state; auto update = add_files_update::build( - state, tp, make_pending_files({{0, 100}})); + state, tp, rev, make_pending_files({{0, 100}})); // We can always add files to a topic or partition that isn't yet tracked. ASSERT_FALSE(update.has_error()); EXPECT_FALSE(state.partition_state(tp).has_value()); @@ -86,7 +87,7 @@ TEST(StateUpdateTest, TestAddFile) { // Now build one that does align properly. update = add_files_update::build( - state, tp, make_pending_files({{101, 200}})); + state, tp, rev, make_pending_files({{101, 200}})); ASSERT_FALSE(update.has_error()); res = update.value().apply(state, model::offset{}); ASSERT_FALSE(res.has_error()); @@ -98,8 +99,9 @@ TEST(StateUpdateTest, TestAddFileWithCommittedOffset) { // First, set up an existing committed offset, e.g. if we've committed all // our files up to offset 100. topics_state state; - state.topic_to_state[topic].pid_to_pending_files[pid].last_committed - = kafka::offset{100}; + auto& t_state = state.topic_to_state[topic]; + t_state.revision = rev; + t_state.pid_to_pending_files[pid].last_committed = kafka::offset{100}; // Try a few adds that don't apply because they don't align with the // committed offset. @@ -110,7 +112,7 @@ TEST(StateUpdateTest, TestAddFileWithCommittedOffset) { // Now successfully add some. auto update = add_files_update::build( - state, tp, make_pending_files({{101, 101}, {102, 200}})); + state, tp, rev, make_pending_files({{101, 101}, {102, 200}})); ASSERT_FALSE(update.has_error()); auto res = update.value().apply(state, model::offset{}); EXPECT_FALSE(res.has_error()); @@ -127,7 +129,7 @@ TEST(StateUpdateTest, TestAddFileWithCommittedOffset) { // Now successfully add some, this time with a non-empty pending files. update = add_files_update::build( - state, tp, make_pending_files({{201, 201}})); + state, tp, rev, make_pending_files({{201, 201}})); ASSERT_FALSE(update.has_error()); res = update.value().apply(state, model::offset{}); EXPECT_FALSE(res.has_error()); @@ -145,8 +147,9 @@ TEST(StateUpdateTest, TestMarkCommitted) { // Even if we explicitly have a committed offset already, we still have no // pending files and therefore can't commit. - state.topic_to_state[topic].pid_to_pending_files[pid].last_committed - = kafka::offset{100}; + auto& t_state = state.topic_to_state[topic]; + t_state.revision = rev; + t_state.pid_to_pending_files[pid].last_committed = kafka::offset{100}; ASSERT_NO_FATAL_FAILURE(check_commit_doesnt_apply(state, tp, 0)); ASSERT_NO_FATAL_FAILURE(check_commit_doesnt_apply(state, tp, 100)); ASSERT_NO_FATAL_FAILURE(check_commit_doesnt_apply(state, tp, 101)); @@ -154,7 +157,7 @@ TEST(StateUpdateTest, TestMarkCommitted) { // Now add some files. auto res = add_files_update::build( - state, tp, make_pending_files({{101, 200}})) + state, tp, rev, make_pending_files({{101, 200}})) .value() .apply(state, model::offset{}); EXPECT_FALSE(res.has_error()); @@ -167,7 +170,7 @@ TEST(StateUpdateTest, TestMarkCommitted) { ASSERT_NO_FATAL_FAILURE(check_commit_doesnt_apply(state, tp, 201)); ASSERT_NO_FATAL_FAILURE(check_partition(state, tp, 100, {{101, 200}})); - res = mark_files_committed_update::build(state, tp, kafka::offset{200}) + res = mark_files_committed_update::build(state, tp, rev, kafka::offset{200}) .value() .apply(state); EXPECT_FALSE(res.has_error()); @@ -176,7 +179,10 @@ TEST(StateUpdateTest, TestMarkCommitted) { // Now let's try commit when there are multiple pending files. // First, add multiple files. res = add_files_update::build( - state, tp, make_pending_files({{201, 205}, {206, 210}, {211, 220}})) + state, + tp, + rev, + make_pending_files({{201, 205}, {206, 210}, {211, 220}})) .value() .apply(state, model::offset{}); EXPECT_FALSE(res.has_error()); @@ -192,7 +198,7 @@ TEST(StateUpdateTest, TestMarkCommitted) { check_partition(state, tp, 200, {{201, 205}, {206, 210}, {211, 220}})); // But it should work with one of the inner files. - res = mark_files_committed_update::build(state, tp, kafka::offset{205}) + res = mark_files_committed_update::build(state, tp, rev, kafka::offset{205}) .value() .apply(state); EXPECT_FALSE(res.has_error()); @@ -200,7 +206,7 @@ TEST(StateUpdateTest, TestMarkCommitted) { check_partition(state, tp, 205, {{206, 210}, {211, 220}})); // And it should work with the last file. - res = mark_files_committed_update::build(state, tp, kafka::offset{220}) + res = mark_files_committed_update::build(state, tp, rev, kafka::offset{220}) .value() .apply(state); EXPECT_FALSE(res.has_error()); diff --git a/src/v/datalake/coordinator/types.cc b/src/v/datalake/coordinator/types.cc index fed78234dc83..5acb83826e42 100644 --- a/src/v/datalake/coordinator/types.cc +++ b/src/v/datalake/coordinator/types.cc @@ -37,6 +37,8 @@ std::ostream& operator<<(std::ostream& o, const errc& errc) { case errc::concurrent_requests: o << "errc::concurrent_requests"; break; + case errc::revision_mismatch: + o << "errc::revision_mismatch"; } return o; } @@ -51,8 +53,9 @@ std::ostream& operator<<(std::ostream& o, const add_translated_data_files_request& request) { fmt::print( o, - "{{partition: {}, files: {}, translation term: {}}}", + "{{partition: {}, topic_revision: {}, files: {}, translation term: {}}}", request.tp, + request.topic_revision, request.ranges, request.translator_term); return o; @@ -67,7 +70,11 @@ operator<<(std::ostream& o, const fetch_latest_translated_offset_reply& reply) { std::ostream& operator<<( std::ostream& o, const fetch_latest_translated_offset_request& request) { - fmt::print(o, "{{partition: {}}}", request.tp); + fmt::print( + o, + "{{partition: {}, topic_revision: {}}}", + request.tp, + request.topic_revision); return o; } } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/types.h b/src/v/datalake/coordinator/types.h index 27ec5e7cf33c..441de65b89a5 100644 --- a/src/v/datalake/coordinator/types.h +++ b/src/v/datalake/coordinator/types.h @@ -27,6 +27,7 @@ enum class errc : int16_t { fenced, stale, concurrent_requests, + revision_mismatch, }; constexpr bool is_retriable(errc errc) { @@ -66,19 +67,33 @@ struct add_translated_data_files_request add_translated_data_files_request() = default; model::topic_partition tp; + model::revision_id topic_revision; // Translated data files, expected to be contiguous, with no gaps or // overlaps, ordered in increasing offset order. chunked_vector ranges; model::term_id translator_term; + add_translated_data_files_request( + model::topic_partition tp, + model::revision_id topic_revision, + chunked_vector ranges, + model::term_id translator_term) + : tp(std::move(tp)) + , topic_revision(topic_revision) + , ranges(std::move(ranges)) + , translator_term(translator_term) {} + add_translated_data_files_request copy() const { - add_translated_data_files_request result; - result.tp = tp; + chunked_vector copied_ranges; for (auto& range : ranges) { - result.ranges.push_back(range.copy()); + copied_ranges.push_back(range.copy()); } - result.translator_term = translator_term; - return result; + return { + tp, + topic_revision, + std::move(copied_ranges), + translator_term, + }; } friend std::ostream& @@ -86,7 +101,9 @@ struct add_translated_data_files_request const model::topic_partition& topic_partition() const { return tp; } - auto serde_fields() { return std::tie(tp, ranges, translator_term); } + auto serde_fields() { + return std::tie(tp, topic_revision, ranges, translator_term); + } }; struct fetch_latest_translated_offset_reply @@ -129,13 +146,14 @@ struct fetch_latest_translated_offset_request fetch_latest_translated_offset_request() = default; model::topic_partition tp; + model::revision_id topic_revision; const model::topic_partition& topic_partition() const { return tp; } friend std::ostream& operator<<(std::ostream&, const fetch_latest_translated_offset_request&); - auto serde_fields() { return std::tie(tp); } + auto serde_fields() { return std::tie(tp, topic_revision); } }; struct stm_snapshot diff --git a/src/v/datalake/tests/fixture.h b/src/v/datalake/tests/fixture.h index bb21fe055a78..76f6dfbfbacd 100644 --- a/src/v/datalake/tests/fixture.h +++ b/src/v/datalake/tests/fixture.h @@ -114,12 +114,14 @@ class datalake_cluster_test_fixture if (!partition) { throw std::runtime_error("leader not found during validation"); } + auto topic_revision = partition->get_topic_revision_id(); const auto& ot = partition->get_offset_translator_state(); auto max_offset = kafka::prev_offset(model::offset_cast( ot->from_log_offset(partition->last_stable_offset()))); auto& fe = coordinator_frontend(fixture->app.controller->self()); coordinator::fetch_latest_translated_offset_request request; request.tp = ntp.tp; + request.topic_revision = topic_revision; vlog(logger.info, "Waiting for last added offet: {}", max_offset); co_await ::tests::cooperative_spin_wait_with_timeout(20s, [&] { return fe.local().fetch_latest_translated_offset(request).then( diff --git a/src/v/datalake/translation/partition_translator.cc b/src/v/datalake/translation/partition_translator.cc index 192055989097..56f71126176c 100644 --- a/src/v/datalake/translation/partition_translator.cc +++ b/src/v/datalake/translation/partition_translator.cc @@ -311,10 +311,13 @@ partition_translator::checkpoint_translated_data( translated_range.start_offset = reader_begin_offset; } auto last_offset = translated_range.last_offset; - coordinator::add_translated_data_files_request request; - request.tp = _partition->ntp().tp; - request.translator_term = _term; - request.ranges.emplace_back(std::move(translated_range)); + chunked_vector ranges; + ranges.push_back(std::move(translated_range)); + coordinator::add_translated_data_files_request request{ + _partition->ntp().tp, + _partition->get_topic_revision_id(), + std::move(ranges), + _term}; vlog(_logger.trace, "Adding translated data file, request: {}", request); auto result = co_await retry_with_backoff( rcn, @@ -334,6 +337,7 @@ ss::future> partition_translator::reconcile_with_coordinator() { auto request = coordinator::fetch_latest_translated_offset_request{}; request.tp = _partition->ntp().tp; + request.topic_revision = _partition->get_topic_revision_id(); vlog(_logger.trace, "fetch_latest_translated_offset, request: {}", request); auto resp = co_await _frontend->local().fetch_latest_translated_offset( request); From 91a2b5cb3add78a3599b0ec6db72c151b25e8922 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 15 Nov 2024 13:43:42 +0100 Subject: [PATCH 07/19] datalake: track topic lifecycle states in coordinator --- src/v/datalake/coordinator/BUILD | 1 + src/v/datalake/coordinator/coordinator.cc | 144 +++++++++++++++--- src/v/datalake/coordinator/coordinator.h | 7 + .../coordinator/coordinator_manager.cc | 3 + .../coordinator/coordinator_manager.h | 2 + .../coordinator/iceberg_file_committer.cc | 5 + src/v/datalake/coordinator/state.cc | 12 ++ src/v/datalake/coordinator/state.h | 19 ++- src/v/datalake/coordinator/state_machine.cc | 7 + src/v/datalake/coordinator/state_update.cc | 63 +++++++- src/v/datalake/coordinator/state_update.h | 20 +++ src/v/datalake/coordinator/tests/BUILD | 2 + .../coordinator/tests/coordinator_test.cc | 7 +- .../coordinator/tests/state_machine_test.cc | 18 ++- src/v/redpanda/application.cc | 1 + 15 files changed, 283 insertions(+), 28 deletions(-) diff --git a/src/v/datalake/coordinator/BUILD b/src/v/datalake/coordinator/BUILD index c6807aa83da5..89ca2e0864f3 100644 --- a/src/v/datalake/coordinator/BUILD +++ b/src/v/datalake/coordinator/BUILD @@ -48,6 +48,7 @@ redpanda_cc_library( ":file_committer", ":state_update", ":stm", + "//src/v/cluster", "//src/v/config", "//src/v/container:fragmented_vector", "//src/v/model", diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index 16116e0381d3..e312e56e7c64 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -10,6 +10,7 @@ #include "datalake/coordinator/coordinator.h" #include "base/vlog.h" +#include "cluster/topic_table.h" #include "container/fragmented_vector.h" #include "datalake/coordinator/state_update.h" #include "datalake/logger.h" @@ -162,23 +163,34 @@ coordinator::run_until_term_change(model::term_id term) { // TODO: apply table retention periodically too. auto updates = std::move(commit_res.value()); - if (updates.empty()) { - // Nothing to replicate. - continue; - } - storage::record_batch_builder builder( - model::record_batch_type::datalake_coordinator, model::offset{0}); - for (auto& update : updates) { - builder.add_raw_kv( - serde::to_iobuf(mark_files_committed_update::key), - serde::to_iobuf(std::move(update))); + if (!updates.empty()) { + storage::record_batch_builder builder( + model::record_batch_type::datalake_coordinator, + model::offset{0}); + for (auto& update : updates) { + builder.add_raw_kv( + serde::to_iobuf(mark_files_committed_update::key), + serde::to_iobuf(std::move(update))); + } + auto repl_res = co_await stm_->replicate_and_wait( + term, std::move(builder).build(), as_); + if (repl_res.has_error()) { + auto e = convert_stm_errc(repl_res.error()); + vlog(datalake_log.warn, "Replication failed {}", e); + co_return e; + } } - auto repl_res = co_await stm_->replicate_and_wait( - term, std::move(builder).build(), as_); - if (repl_res.has_error()) { - auto e = convert_stm_errc(repl_res.error()); - vlog(datalake_log.warn, "Replication failed {}", e); - co_return e; + + // check if the topic has been deleted and we need to clean up + // topic state. + while (true) { + auto update_res = co_await update_lifecycle_state(t, term); + if (update_res.has_error()) { + co_return update_res.error(); + } + if (update_res.value() == ss::stop_iteration::yes) { + break; + } } } auto sleep_res = co_await ss::coroutine::as_future( @@ -302,9 +314,28 @@ coordinator::sync_get_last_added_offset( topic.revision); co_return errc::revision_mismatch; } else if (requested_topic_rev > topic.revision) { - // Coordinator is ready to accept files for the new topic revision, - // but there is no stm record yet. Reply with "no offset". - co_return std::nullopt; + if (topic.lifecycle_state == topic_state::lifecycle_state_t::purged) { + // Coordinator is ready to accept files for the new topic revision, + // but there is no stm record yet. Reply with "no offset". + co_return std::nullopt; + } + + vlog( + datalake_log.debug, + "asked offsets for tp {} rev: {}, but rev: {} still not purged", + tp, + requested_topic_rev, + topic.revision); + co_return errc::revision_mismatch; + } + + if (topic.lifecycle_state != topic_state::lifecycle_state_t::live) { + vlog( + datalake_log.debug, + "asked offsets for tp {} rev: {}, but it is already closed", + tp, + requested_topic_rev); + co_return errc::revision_mismatch; } auto partition_it = topic.pid_to_pending_files.find(tp.partition); @@ -334,4 +365,79 @@ void coordinator::notify_leadership(std::optional leader_id) { term_as_->get().request_abort(); } } + +ss::future> +coordinator::update_lifecycle_state( + const model::topic& t, model::term_id term) { + auto topic_it = stm_->state().topic_to_state.find(t); + if (topic_it == stm_->state().topic_to_state.end()) { + co_return ss::stop_iteration::yes; + } + const auto& topic = topic_it->second; + auto revision = topic.revision; + + if (revision >= topic_table_.last_applied_revision()) { + // topic table not yet up-to-date + co_return ss::stop_iteration::yes; + } + + topic_state::lifecycle_state_t new_state; + switch (topic.lifecycle_state) { + case topic_state::lifecycle_state_t::live: { + auto topic_md = topic_table_.get_topic_metadata_ref( + model::topic_namespace_view{model::kafka_namespace, t}); + if (topic_md && revision >= topic_md->get().get_revision()) { + // topic still exists + co_return ss::stop_iteration::yes; + } + + new_state = topic_state::lifecycle_state_t::closed; + break; + } + case topic_state::lifecycle_state_t::closed: { + for (const auto& [_, partition_state] : topic.pid_to_pending_files) { + if (!partition_state.pending_entries.empty()) { + // still have entries to commit + co_return ss::stop_iteration::yes; + } + } + + new_state = topic_state::lifecycle_state_t::purged; + break; + } + case topic_state::lifecycle_state_t::purged: + co_return ss::stop_iteration::yes; + } + + topic_lifecycle_update update{ + .topic = t, + .revision = revision, + .new_state = new_state, + }; + auto check_res = update.can_apply(stm_->state()); + if (check_res.has_error()) { + vlog( + datalake_log.debug, + "Rejecting lifecycle transition request {}: {}", + update, + check_res.error()); + co_return errc::stm_apply_error; + } + storage::record_batch_builder builder( + model::record_batch_type::datalake_coordinator, model::offset{0}); + builder.add_raw_kv( + serde::to_iobuf(topic_lifecycle_update::key), + serde::to_iobuf(std::move(update))); + + auto repl_res = co_await stm_->replicate_and_wait( + term, std::move(builder).build(), as_); + if (repl_res.has_error()) { + auto e = convert_stm_errc(repl_res.error()); + vlog(datalake_log.warn, "Replication failed {}", e); + co_return e; + } + + co_return ss::stop_iteration::no; +} + } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/coordinator.h b/src/v/datalake/coordinator/coordinator.h index 24b145872329..8691720eec72 100644 --- a/src/v/datalake/coordinator/coordinator.h +++ b/src/v/datalake/coordinator/coordinator.h @@ -9,6 +9,7 @@ */ #pragma once +#include "cluster/fwd.h" #include "config/property.h" #include "container/fragmented_vector.h" #include "datalake/coordinator/file_committer.h" @@ -33,9 +34,11 @@ class coordinator { }; coordinator( ss::shared_ptr stm, + cluster::topic_table& topics, file_committer& file_committer, config::binding commit_interval) : stm_(std::move(stm)) + , topic_table_(topics) , file_committer_(file_committer) , commit_interval_(std::move(commit_interval)) {} @@ -69,7 +72,11 @@ class coordinator { ss::future> run_until_term_change(model::term_id); + ss::future> + update_lifecycle_state(const model::topic&, model::term_id); + ss::shared_ptr stm_; + cluster::topic_table& topic_table_; file_committer& file_committer_; config::binding commit_interval_; diff --git a/src/v/datalake/coordinator/coordinator_manager.cc b/src/v/datalake/coordinator/coordinator_manager.cc index c0a237f5f8bb..34c9b84be203 100644 --- a/src/v/datalake/coordinator/coordinator_manager.cc +++ b/src/v/datalake/coordinator/coordinator_manager.cc @@ -28,12 +28,14 @@ coordinator_manager::coordinator_manager( model::node_id self, ss::sharded& gm, ss::sharded& pm, + ss::sharded& topics, std::unique_ptr catalog_factory, ss::sharded& io, cloud_storage_clients::bucket_name bucket) : self_(self) , gm_(gm.local()) , pm_(pm.local()) + , topics_(topics.local()) , manifest_io_(io.local(), bucket) , catalog_factory_(std::move(catalog_factory)) {} @@ -102,6 +104,7 @@ void coordinator_manager::start_managing(cluster::partition& p) { } auto crd = ss::make_lw_shared( std::move(stm), + topics_, *file_committer_, config::shard_local_cfg().iceberg_catalog_commit_interval_ms.bind()); if (p.is_leader()) { diff --git a/src/v/datalake/coordinator/coordinator_manager.h b/src/v/datalake/coordinator/coordinator_manager.h index 3f21a43e554a..ad367750cebe 100644 --- a/src/v/datalake/coordinator/coordinator_manager.h +++ b/src/v/datalake/coordinator/coordinator_manager.h @@ -35,6 +35,7 @@ class coordinator_manager { model::node_id self, ss::sharded&, ss::sharded&, + ss::sharded&, std::unique_ptr, ss::sharded&, cloud_storage_clients::bucket_name); @@ -56,6 +57,7 @@ class coordinator_manager { model::node_id self_; raft::group_manager& gm_; cluster::partition_manager& pm_; + cluster::topic_table& topics_; // Underlying IO is expected to outlive this class. iceberg::manifest_io manifest_io_; diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index 7539d4cb6cd0..7d24c0833464 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -103,6 +103,11 @@ iceberg_file_committer::commit_topic_files_to_catalog( if (tp_it == state.topic_to_state.end()) { co_return chunked_vector{}; } + if ( + tp_it->second.lifecycle_state == topic_state::lifecycle_state_t::purged) { + co_return chunked_vector{}; + } + auto table_id = table_id_for_topic(topic); auto table_res = co_await load_or_create_table(table_id); if (table_res.has_error()) { diff --git a/src/v/datalake/coordinator/state.cc b/src/v/datalake/coordinator/state.cc index b24e9fdc6de5..a34a8f56475a 100644 --- a/src/v/datalake/coordinator/state.cc +++ b/src/v/datalake/coordinator/state.cc @@ -24,6 +24,17 @@ partition_state partition_state::copy() const { return result; } +std::ostream& operator<<(std::ostream& o, topic_state::lifecycle_state_t s) { + switch (s) { + case topic_state::lifecycle_state_t::live: + return o << "live"; + case topic_state::lifecycle_state_t::closed: + return o << "closed"; + case topic_state::lifecycle_state_t::purged: + return o << "purged"; + } +} + topic_state topic_state::copy() const { topic_state result; result.revision = revision; @@ -31,6 +42,7 @@ topic_state topic_state::copy() const { for (const auto& [id, state] : pid_to_pending_files) { result.pid_to_pending_files[id] = state.copy(); } + result.lifecycle_state = lifecycle_state; return result; } diff --git a/src/v/datalake/coordinator/state.h b/src/v/datalake/coordinator/state.h index 5f76ec1b8412..c97e673cd4ad 100644 --- a/src/v/datalake/coordinator/state.h +++ b/src/v/datalake/coordinator/state.h @@ -81,12 +81,29 @@ struct partition_state struct topic_state : public serde:: envelope, serde::compat_version<0>> { - auto serde_fields() { return std::tie(revision, pid_to_pending_files); } + auto serde_fields() { + return std::tie(revision, pid_to_pending_files, lifecycle_state); + } + + enum class lifecycle_state_t { + // ready to accept new files + live, + // topic deleted, new files can't be accepted (but already accepted + // files will be committed) + closed, + // all state related to this revision of the topic has been purged, + // files for new revisions of this topic can be accepted. + // TODO: GC purged topic states + purged, + }; + friend std::ostream& + operator<<(std::ostream&, topic_state::lifecycle_state_t); // Topic revision model::revision_id revision; // Map from Redpanda partition id to the files pending per partition. chunked_hash_map pid_to_pending_files; + lifecycle_state_t lifecycle_state = lifecycle_state_t::live; topic_state copy() const; diff --git a/src/v/datalake/coordinator/state_machine.cc b/src/v/datalake/coordinator/state_machine.cc index 1eab93536011..0a8355b18454 100644 --- a/src/v/datalake/coordinator/state_machine.cc +++ b/src/v/datalake/coordinator/state_machine.cc @@ -124,6 +124,13 @@ ss::future<> coordinator_stm::do_apply(const model::record_batch& b) { maybe_log_update_error(_log, key, o, res); continue; } + case update_key::topic_lifecycle_update: { + auto update = serde::read(val_p); + vlog(_log.debug, "Applying {} from offset {}: {}", key, o, update); + auto res = update.apply(state_); + maybe_log_update_error(_log, key, o, res); + continue; + } } vlog( _log.error, diff --git a/src/v/datalake/coordinator/state_update.cc b/src/v/datalake/coordinator/state_update.cc index e89f5757f5e3..e998e36ba651 100644 --- a/src/v/datalake/coordinator/state_update.cc +++ b/src/v/datalake/coordinator/state_update.cc @@ -22,6 +22,8 @@ std::ostream& operator<<(std::ostream& o, const update_key& u) { return o << "update_key::add_files"; case update_key::mark_files_committed: return o << "update_key::mark_files_committed"; + case update_key::topic_lifecycle_update: + return o << "update_key::topic_lifecycle_update"; } } @@ -59,14 +61,27 @@ add_files_update::can_apply(const topics_state& state) { topic_revision, cur_topic.revision)}; } else if (topic_revision > cur_topic.revision) { - // We are ready to accept files for an instance with the higher revision - // id. + if ( + cur_topic.lifecycle_state != topic_state::lifecycle_state_t::purged) { + return stm_update_error{fmt::format( + "topic {} rev {} not yet purged (new topic rev {})", + tp.topic, + cur_topic.revision, + topic_revision)}; + } + + // Previous topic instance has been fully purged, so we are ready to + // accept files for an instance with the higher revision id. return std::nullopt; } - auto partition_it = topic_it->second.pid_to_pending_files.find( - tp.partition); - if (partition_it == topic_it->second.pid_to_pending_files.end()) { + if (cur_topic.lifecycle_state != topic_state::lifecycle_state_t::live) { + return stm_update_error{fmt::format( + "topic {} rev {} already closed", tp.topic, cur_topic.revision)}; + } + + auto partition_it = cur_topic.pid_to_pending_files.find(tp.partition); + if (partition_it == cur_topic.pid_to_pending_files.end()) { return std::nullopt; } const auto& prt_state = partition_it->second; @@ -112,7 +127,7 @@ add_files_update::apply(topics_state& state, model::offset applied_offset) { new_state.revision = topic_revision; tp_state = std::move(new_state); } - // after this point tp_state.revision == topic_revision + // after this point tp_state.revision == topic_revision and state == live auto& partition_state = tp_state.pid_to_pending_files[pid]; for (auto& e : entries) { partition_state.pending_entries.emplace_back(pending_entry{ @@ -205,4 +220,40 @@ mark_files_committed_update::apply(topics_state& state) { return std::nullopt; } +checked +topic_lifecycle_update::can_apply(const topics_state& state) { + auto topic_it = state.topic_to_state.find(topic); + if (topic_it == state.topic_to_state.end()) { + return stm_update_error{fmt::format("topic {} not found", topic)}; + } + if (topic_it->second.revision != revision) { + return stm_update_error{fmt::format( + "topic {} revision mismatch, expected {}, got {}", + topic, + revision, + topic_it->second.revision)}; + } + return std::nullopt; +} + +checked +topic_lifecycle_update::apply(topics_state& state) { + auto allowed = can_apply(state); + if (allowed.has_error()) { + return allowed.error(); + } + state.topic_to_state[topic].lifecycle_state = new_state; + return std::nullopt; +} + +std::ostream& operator<<(std::ostream& o, topic_lifecycle_update u) { + fmt::print( + o, + "{{topic: {}, revision: {}, new_state: {}}}", + u.topic, + u.revision, + u.new_state); + return o; +} + } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/state_update.h b/src/v/datalake/coordinator/state_update.h index b2c97c008340..f59d69f74600 100644 --- a/src/v/datalake/coordinator/state_update.h +++ b/src/v/datalake/coordinator/state_update.h @@ -24,6 +24,7 @@ namespace datalake::coordinator { enum class update_key : uint8_t { add_files = 0, mark_files_committed = 1, + topic_lifecycle_update = 2, }; std::ostream& operator<<(std::ostream&, const update_key&); @@ -76,4 +77,23 @@ struct mark_files_committed_update kafka::offset new_committed; }; +// An update to change topic lifecycle state after it has been deleted. +struct topic_lifecycle_update + : public serde::envelope< + topic_lifecycle_update, + serde::version<0>, + serde::compat_version<0>> { + static constexpr auto key{update_key::topic_lifecycle_update}; + auto serde_fields() { return std::tie(topic, revision, new_state); } + + checked can_apply(const topics_state&); + checked apply(topics_state&); + + friend std::ostream& operator<<(std::ostream&, topic_lifecycle_update); + + model::topic topic; + model::revision_id revision; + topic_state::lifecycle_state_t new_state; +}; + } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/tests/BUILD b/src/v/datalake/coordinator/tests/BUILD index 1c4446f3b6b7..b60486f53c20 100644 --- a/src/v/datalake/coordinator/tests/BUILD +++ b/src/v/datalake/coordinator/tests/BUILD @@ -28,6 +28,7 @@ redpanda_cc_gtest( cpu = 1, deps = [ ":state_test_utils", + "//src/v/cluster", "//src/v/config", "//src/v/datalake:logger", "//src/v/datalake/coordinator", @@ -100,6 +101,7 @@ redpanda_cc_gtest( cpu = 1, deps = [ ":state_test_utils", + "//src/v/cluster", "//src/v/datalake/coordinator", "//src/v/datalake/coordinator:stm", "//src/v/raft/tests:stm_raft_fixture", diff --git a/src/v/datalake/coordinator/tests/coordinator_test.cc b/src/v/datalake/coordinator/tests/coordinator_test.cc index 665e3cca14de..26017594c553 100644 --- a/src/v/datalake/coordinator/tests/coordinator_test.cc +++ b/src/v/datalake/coordinator/tests/coordinator_test.cc @@ -7,6 +7,8 @@ * * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ +#include "cluster/data_migrated_resources.h" +#include "cluster/topic_table.h" #include "config/mock_property.h" #include "datalake/coordinator/coordinator.h" #include "datalake/coordinator/file_committer.h" @@ -52,11 +54,14 @@ struct coordinator_node { std::chrono::milliseconds commit_interval) : stm(*stm) , commit_interval_ms(commit_interval) + , topic_table(mr) , file_committer(std::move(committer)) - , crd(stm, *file_committer, commit_interval_ms.bind()) {} + , crd(stm, topic_table, *file_committer, commit_interval_ms.bind()) {} coordinator_stm& stm; config::mock_property commit_interval_ms; + cluster::data_migrations::migrated_resources mr; + cluster::topic_table topic_table; std::unique_ptr file_committer; coordinator crd; }; diff --git a/src/v/datalake/coordinator/tests/state_machine_test.cc b/src/v/datalake/coordinator/tests/state_machine_test.cc index d1404170905b..ac9471ed0672 100644 --- a/src/v/datalake/coordinator/tests/state_machine_test.cc +++ b/src/v/datalake/coordinator/tests/state_machine_test.cc @@ -8,6 +8,8 @@ * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ +#include "cluster/data_migrated_resources.h" +#include "cluster/topic_table.h" #include "datalake/coordinator/coordinator.h" #include "datalake/coordinator/state_machine.h" #include "datalake/coordinator/tests/state_test_utils.h" @@ -46,7 +48,13 @@ struct coordinator_stm_fixture : stm_raft_fixture { auto stm = get_stm<0>(node); coordinators[node.get_vnode()] = std::make_unique( - get_stm<0>(node), file_committer, commit_interval()); + get_stm<0>(node), + topic_table, + [this](const model::topic& t, model::revision_id r) { + return remove_tombstone(t, r); + }, + file_committer, + commit_interval()); coordinators[node.get_vnode()]->start(); return ss::now(); }); @@ -115,9 +123,17 @@ struct coordinator_stm_fixture : stm_raft_fixture { random_generators::get_int(0, max_partitions - 1))}; } + ss::future< + checked> + remove_tombstone(const model::topic&, model::revision_id) { + co_return std::nullopt; + } + static constexpr int32_t max_partitions = 5; model::topic_partition tp{model::topic{"test"}, model::partition_id{0}}; model::revision_id rev{123}; + cluster::data_migrations::migrated_resources mr; + cluster::topic_table topic_table{mr}; datalake::coordinator::simple_file_committer file_committer; absl::flat_hash_map coordinators; }; diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index a090cfe568bc..e78a108fe162 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -1435,6 +1435,7 @@ void application::wire_up_runtime_services( node_id, std::ref(raft_group_manager), std::ref(partition_manager), + std::ref(controller->get_topics_state()), ss::sharded_parameter( [bucket](cloud_io::remote& remote) -> std::unique_ptr { From e63d1d2ad851f0414da7c3eeb729f63da95d7bcb Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 15 Nov 2024 15:27:46 +0100 Subject: [PATCH 08/19] datalake: drop iceberg tables on topic deletion --- src/v/datalake/coordinator/coordinator.cc | 34 +++++++++++++++++++ src/v/datalake/coordinator/coordinator.h | 7 ++++ .../coordinator/coordinator_manager.cc | 34 +++++++++++++++++++ .../coordinator/coordinator_manager.h | 4 +++ src/v/datalake/coordinator/file_committer.h | 4 +++ src/v/datalake/coordinator/frontend.cc | 2 ++ .../coordinator/iceberg_file_committer.cc | 14 ++++++++ .../coordinator/iceberg_file_committer.h | 3 ++ .../coordinator/tests/coordinator_test.cc | 20 ++++++++++- .../coordinator/tests/state_test_utils.h | 6 ++++ src/v/datalake/coordinator/types.cc | 4 +++ src/v/datalake/coordinator/types.h | 1 + src/v/redpanda/application.cc | 1 + 13 files changed, 133 insertions(+), 1 deletion(-) diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index e312e56e7c64..4e26c888fefd 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -54,6 +54,8 @@ std::ostream& operator<<(std::ostream& o, coordinator::errc e) { return o << "coordinator::errc::revision_mismatch"; case coordinator::errc::timedout: return o << "coordinator::errc::timedout"; + case coordinator::errc::failed: + return o << "coordinator::errc::failed"; } } @@ -402,6 +404,38 @@ coordinator::update_lifecycle_state( } } + // Now that we don't have pending files, we can check if the + // corresponding iceberg tombstone is present, and if it is, drop the + // table. + + auto tombstone_it = topic_table_.get_iceberg_tombstones().find( + model::topic_namespace_view{model::kafka_namespace, t}); + if (tombstone_it != topic_table_.get_iceberg_tombstones().end()) { + auto tombstone_rev = tombstone_it->second.last_deleted_revision; + if (tombstone_rev >= topic.revision) { + auto drop_res = co_await file_committer_.drop_table(t); + if (drop_res.has_error()) { + switch (drop_res.error()) { + case file_committer::errc::shutting_down: + co_return errc::shutting_down; + case file_committer::errc::failed: + vlog( + datalake_log.warn, + "failed to drop table for topic {}", + t); + co_return ss::stop_iteration::yes; + } + } + } + auto ts_res = co_await remove_tombstone_(t, tombstone_rev); + if (ts_res.has_error()) { + if (ts_res.error() == errc::shutting_down) { + co_return ts_res.error(); + } + co_return ss::stop_iteration::yes; + } + } + new_state = topic_state::lifecycle_state_t::purged; break; } diff --git a/src/v/datalake/coordinator/coordinator.h b/src/v/datalake/coordinator/coordinator.h index 8691720eec72..a44657fabd45 100644 --- a/src/v/datalake/coordinator/coordinator.h +++ b/src/v/datalake/coordinator/coordinator.h @@ -31,14 +31,20 @@ class coordinator { revision_mismatch, timedout, shutting_down, + failed, }; + using remove_tombstone_f + = ss::noncopyable_function>( + const model::topic&, model::revision_id)>; coordinator( ss::shared_ptr stm, cluster::topic_table& topics, + remove_tombstone_f remove_tombstone, file_committer& file_committer, config::binding commit_interval) : stm_(std::move(stm)) , topic_table_(topics) + , remove_tombstone_(std::move(remove_tombstone)) , file_committer_(file_committer) , commit_interval_(std::move(commit_interval)) {} @@ -77,6 +83,7 @@ class coordinator { ss::shared_ptr stm_; cluster::topic_table& topic_table_; + remove_tombstone_f remove_tombstone_; file_committer& file_committer_; config::binding commit_interval_; diff --git a/src/v/datalake/coordinator/coordinator_manager.cc b/src/v/datalake/coordinator/coordinator_manager.cc index 34c9b84be203..8c10589f815d 100644 --- a/src/v/datalake/coordinator/coordinator_manager.cc +++ b/src/v/datalake/coordinator/coordinator_manager.cc @@ -10,6 +10,7 @@ #include "datalake/coordinator/coordinator_manager.h" #include "cluster/partition_manager.h" +#include "cluster/topics_frontend.h" #include "config/configuration.h" #include "datalake/coordinator/catalog_factory.h" #include "datalake/coordinator/coordinator.h" @@ -29,6 +30,7 @@ coordinator_manager::coordinator_manager( ss::sharded& gm, ss::sharded& pm, ss::sharded& topics, + ss::sharded& topics_fe, std::unique_ptr catalog_factory, ss::sharded& io, cloud_storage_clients::bucket_name bucket) @@ -36,6 +38,7 @@ coordinator_manager::coordinator_manager( , gm_(gm.local()) , pm_(pm.local()) , topics_(topics.local()) + , topics_fe_(topics_fe) , manifest_io_(io.local(), bucket) , catalog_factory_(std::move(catalog_factory)) {} @@ -105,6 +108,9 @@ void coordinator_manager::start_managing(cluster::partition& p) { auto crd = ss::make_lw_shared( std::move(stm), topics_, + [this](const model::topic& t, model::revision_id rev) { + return remove_tombstone(t, rev); + }, *file_committer_, config::shard_local_cfg().iceberg_catalog_commit_interval_ms.bind()); if (p.is_leader()) { @@ -160,4 +166,32 @@ void coordinator_manager::notify_leadership_change( crd->notify_leadership(leader_id); } +ss::future> +coordinator_manager::remove_tombstone( + const model::topic& topic, model::revision_id rev) { + auto topic_res = co_await topics_fe_.local().purged_topic( + cluster::nt_revision{ + .nt = model::topic_namespace{model::kafka_namespace, topic}, + .initial_revision_id = model::initial_revision_id{rev}}, + cluster::topic_purge_domain::iceberg, + 5s); + switch (topic_res.ec) { + case cluster::errc::success: + case cluster::errc::topic_not_exists: + co_return std::nullopt; + case cluster::errc::shutting_down: + co_return coordinator::errc::shutting_down; + case cluster::errc::timeout: + co_return coordinator::errc::timedout; + default: + vlog( + datalake_log.warn, + "failed to remove iceberg tombstone, topic {}, rev {}: {}", + topic, + rev, + topic_res.ec); + co_return coordinator::errc::failed; + } +} + } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/coordinator_manager.h b/src/v/datalake/coordinator/coordinator_manager.h index ad367750cebe..0b18cf06f591 100644 --- a/src/v/datalake/coordinator/coordinator_manager.h +++ b/src/v/datalake/coordinator/coordinator_manager.h @@ -36,6 +36,7 @@ class coordinator_manager { ss::sharded&, ss::sharded&, ss::sharded&, + ss::sharded&, std::unique_ptr, ss::sharded&, cloud_storage_clients::bucket_name); @@ -52,12 +53,15 @@ class coordinator_manager { raft::group_id group, model::term_id term, std::optional leader_id); + ss::future> + remove_tombstone(const model::topic&, model::revision_id); ss::gate gate_; model::node_id self_; raft::group_manager& gm_; cluster::partition_manager& pm_; cluster::topic_table& topics_; + ss::sharded& topics_fe_; // Underlying IO is expected to outlive this class. iceberg::manifest_io manifest_io_; diff --git a/src/v/datalake/coordinator/file_committer.h b/src/v/datalake/coordinator/file_committer.h index 9ce9e25fd763..50cfa3bc7a95 100644 --- a/src/v/datalake/coordinator/file_committer.h +++ b/src/v/datalake/coordinator/file_committer.h @@ -26,6 +26,10 @@ class file_committer { virtual ss::future< checked, errc>> commit_topic_files_to_catalog(model::topic, const topics_state&) const = 0; + + virtual ss::future> + drop_table(const model::topic&) const = 0; + virtual ~file_committer() = default; }; diff --git a/src/v/datalake/coordinator/frontend.cc b/src/v/datalake/coordinator/frontend.cc index 426017ab0c66..df0db19f951c 100644 --- a/src/v/datalake/coordinator/frontend.cc +++ b/src/v/datalake/coordinator/frontend.cc @@ -36,6 +36,8 @@ errc to_rpc_errc(coordinator::errc e) { return errc::revision_mismatch; case coordinator::errc::timedout: return errc::timeout; + case coordinator::errc::failed: + return errc::failed; } } ss::future add_files( diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index 7d24c0833464..d5b232c74bcf 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -208,6 +208,20 @@ iceberg_file_committer::commit_topic_files_to_catalog( } co_return updates; } + +ss::future> +iceberg_file_committer::drop_table(const model::topic& topic) const { + auto table_id = table_id_for_topic(topic); + auto drop_res = co_await catalog_.drop_table(table_id, true); + if ( + drop_res.has_error() + && drop_res.error() != iceberg::catalog::errc::not_found) { + co_return log_and_convert_catalog_errc( + drop_res.error(), fmt::format("Failed to drop {}", table_id)); + } + co_return std::nullopt; +} + iceberg::table_identifier iceberg_file_committer::table_id_for_topic(const model::topic& t) const { return iceberg::table_identifier{ diff --git a/src/v/datalake/coordinator/iceberg_file_committer.h b/src/v/datalake/coordinator/iceberg_file_committer.h index b1479db65589..fe49823a604f 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.h +++ b/src/v/datalake/coordinator/iceberg_file_committer.h @@ -47,6 +47,9 @@ class iceberg_file_committer : public file_committer { commit_topic_files_to_catalog( model::topic, const topics_state&) const final; + ss::future> + drop_table(const model::topic&) const final; + private: // TODO: pull this out into some helper? Seems useful for other actions. iceberg::table_identifier table_id_for_topic(const model::topic& t) const; diff --git a/src/v/datalake/coordinator/tests/coordinator_test.cc b/src/v/datalake/coordinator/tests/coordinator_test.cc index 26017594c553..67c36322bdb3 100644 --- a/src/v/datalake/coordinator/tests/coordinator_test.cc +++ b/src/v/datalake/coordinator/tests/coordinator_test.cc @@ -38,6 +38,12 @@ class noop_file_committer : public file_committer { model::topic, const topics_state&) const override { co_return chunked_vector{}; } + + ss::future> + drop_table(const model::topic&) const final { + co_return std::nullopt; + } + ~noop_file_committer() override = default; }; @@ -56,7 +62,19 @@ struct coordinator_node { , commit_interval_ms(commit_interval) , topic_table(mr) , file_committer(std::move(committer)) - , crd(stm, topic_table, *file_committer, commit_interval_ms.bind()) {} + , crd( + stm, + topic_table, + [this](const model::topic& t, model::revision_id r) { + return remove_tombstone(t, r); + }, + *file_committer, + commit_interval_ms.bind()) {} + + ss::future> + remove_tombstone(const model::topic&, model::revision_id) { + co_return std::nullopt; + } coordinator_stm& stm; config::mock_property commit_interval_ms; diff --git a/src/v/datalake/coordinator/tests/state_test_utils.h b/src/v/datalake/coordinator/tests/state_test_utils.h index b5937324ddc4..9ab35a7cf8dc 100644 --- a/src/v/datalake/coordinator/tests/state_test_utils.h +++ b/src/v/datalake/coordinator/tests/state_test_utils.h @@ -53,6 +53,12 @@ class simple_file_committer : public file_committer { } co_return ret; } + + ss::future> + drop_table(const model::topic&) const final { + co_return std::nullopt; + } + ~simple_file_committer() override = default; }; diff --git a/src/v/datalake/coordinator/types.cc b/src/v/datalake/coordinator/types.cc index 5acb83826e42..369117a9dda8 100644 --- a/src/v/datalake/coordinator/types.cc +++ b/src/v/datalake/coordinator/types.cc @@ -39,6 +39,10 @@ std::ostream& operator<<(std::ostream& o, const errc& errc) { break; case errc::revision_mismatch: o << "errc::revision_mismatch"; + break; + case errc::failed: + o << "errc::failed"; + break; } return o; } diff --git a/src/v/datalake/coordinator/types.h b/src/v/datalake/coordinator/types.h index 441de65b89a5..bb5789683f88 100644 --- a/src/v/datalake/coordinator/types.h +++ b/src/v/datalake/coordinator/types.h @@ -28,6 +28,7 @@ enum class errc : int16_t { stale, concurrent_requests, revision_mismatch, + failed, }; constexpr bool is_retriable(errc errc) { diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index e78a108fe162..501a435c5ce1 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -1436,6 +1436,7 @@ void application::wire_up_runtime_services( std::ref(raft_group_manager), std::ref(partition_manager), std::ref(controller->get_topics_state()), + std::ref(controller->get_topics_frontend()), ss::sharded_parameter( [bucket](cloud_io::remote& remote) -> std::unique_ptr { From c2a5252e2fefba4cb16bb2b323c6f9b0a1251d7c Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 19 Nov 2024 19:27:09 +0100 Subject: [PATCH 09/19] datalake: add coordinator::frontend::ensure_table_exists method Only register a topic state in the stm, don't create the table just yet. --- src/v/datalake/coordinator/BUILD | 3 + src/v/datalake/coordinator/coordinator.cc | 61 +++++++++++++++-- src/v/datalake/coordinator/coordinator.h | 5 ++ src/v/datalake/coordinator/frontend.cc | 42 +++++++++++- src/v/datalake/coordinator/frontend.h | 10 +++ src/v/datalake/coordinator/rpc.json | 5 ++ src/v/datalake/coordinator/service.cc | 6 ++ src/v/datalake/coordinator/service.h | 6 ++ src/v/datalake/coordinator/state.cc | 9 +++ src/v/datalake/coordinator/state.h | 2 + src/v/datalake/coordinator/state_machine.cc | 3 +- src/v/datalake/coordinator/state_update.cc | 74 +++++++++++++++++---- src/v/datalake/coordinator/state_update.h | 5 +- src/v/datalake/coordinator/types.cc | 12 ++++ src/v/datalake/coordinator/types.h | 54 ++++++++++++++- 15 files changed, 273 insertions(+), 24 deletions(-) diff --git a/src/v/datalake/coordinator/BUILD b/src/v/datalake/coordinator/BUILD index 89ca2e0864f3..bb0b74819607 100644 --- a/src/v/datalake/coordinator/BUILD +++ b/src/v/datalake/coordinator/BUILD @@ -39,6 +39,7 @@ redpanda_cc_library( implementation_deps = [ "//src/v/base", "//src/v/datalake:logger", + "//src/v/datalake:schema_identifier", "//src/v/ssx:future_util", "//src/v/ssx:sleep_abortable", "//src/v/storage:record_batch_builder", @@ -216,6 +217,7 @@ redpanda_cc_library( visibility = [":__subpackages__"], deps = [ ":translated_offset_range", + "//src/v/datalake:schema_identifier", "//src/v/datalake:types", "//src/v/datalake/coordinator:state", "//src/v/model", @@ -323,6 +325,7 @@ redpanda_cc_library( "//src/v/cluster", "//src/v/container:fragmented_vector", "//src/v/datalake:logger", + "//src/v/datalake:schema_identifier", "//src/v/datalake:types", "//src/v/model", "//src/v/raft", diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index 4e26c888fefd..6b2372c64317 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -223,6 +223,59 @@ checked coordinator::maybe_gate() { return gate_.hold(); } +ss::future> +coordinator::sync_ensure_table_exists( + model::topic topic, + model::revision_id topic_revision, + record_schema_components) { + auto gate = maybe_gate(); + if (gate.has_error()) { + co_return gate.error(); + } + + vlog( + datalake_log.debug, + "Sync ensure table exists requested, topic: {} rev: {}", + topic, + topic_revision); + + auto sync_res = co_await stm_->sync(10s); + if (sync_res.has_error()) { + co_return convert_stm_errc(sync_res.error()); + } + + topic_lifecycle_update update{ + .topic = topic, + .revision = topic_revision, + .new_state = topic_state::lifecycle_state_t::live, + }; + auto check_res = update.can_apply(stm_->state()); + if (check_res.has_error()) { + vlog( + datalake_log.debug, + "Rejecting ensure_table_exist for {} rev {}: {}", + topic, + topic_revision, + check_res.error()); + co_return errc::revision_mismatch; + } + if (check_res.value()) { + // update is non-trivial + storage::record_batch_builder builder( + model::record_batch_type::datalake_coordinator, model::offset{0}); + builder.add_raw_kv( + serde::to_iobuf(topic_lifecycle_update::key), + serde::to_iobuf(std::move(update))); + auto repl_res = co_await stm_->replicate_and_wait( + sync_res.value(), std::move(builder).build(), as_); + if (repl_res.has_error()) { + co_return convert_stm_errc(repl_res.error()); + } + } + // TODO: actually create table + co_return std::nullopt; +} + ss::future> coordinator::sync_add_files( model::topic_partition tp, @@ -397,11 +450,9 @@ coordinator::update_lifecycle_state( break; } case topic_state::lifecycle_state_t::closed: { - for (const auto& [_, partition_state] : topic.pid_to_pending_files) { - if (!partition_state.pending_entries.empty()) { - // still have entries to commit - co_return ss::stop_iteration::yes; - } + if (topic.has_pending_entries()) { + // can't purge yet, have to deal with pending entries first + co_return ss::stop_iteration::yes; } // Now that we don't have pending files, we can check if the diff --git a/src/v/datalake/coordinator/coordinator.h b/src/v/datalake/coordinator/coordinator.h index a44657fabd45..cd117ed597ef 100644 --- a/src/v/datalake/coordinator/coordinator.h +++ b/src/v/datalake/coordinator/coordinator.h @@ -51,6 +51,11 @@ class coordinator { void start(); ss::future<> stop_and_wait(); + ss::future> sync_ensure_table_exists( + model::topic topic, + model::revision_id topic_revision, + record_schema_components); + ss::future> sync_add_files( model::topic_partition tp, model::revision_id topic_revision, diff --git a/src/v/datalake/coordinator/frontend.cc b/src/v/datalake/coordinator/frontend.cc index df0db19f951c..d1d8ebb79dec 100644 --- a/src/v/datalake/coordinator/frontend.cc +++ b/src/v/datalake/coordinator/frontend.cc @@ -40,6 +40,21 @@ errc to_rpc_errc(coordinator::errc e) { return errc::failed; } } +ss::future do_ensure_table_exists( + coordinator_manager& mgr, + model::ntp coordinator_ntp, + ensure_table_exists_request req) { + auto crd = mgr.get(coordinator_ntp); + if (!crd) { + co_return ensure_table_exists_reply{errc::not_leader}; + } + auto ret = co_await crd->sync_ensure_table_exists( + req.topic, req.topic_revision, std::move(req.schema_components)); + if (ret.has_error()) { + co_return to_rpc_errc(ret.error()); + } + co_return ensure_table_exists_reply{errc::ok}; +} ss::future add_files( coordinator_manager& mgr, model::ntp coordinator_ntp, @@ -116,7 +131,7 @@ auto frontend::process(req_t req, bool local_only) { return ss::make_ready_future( resp_t{errc::coordinator_topic_not_exists}); } - auto cp = coordinator_partition(req.topic_partition().topic); + auto cp = coordinator_partition(req.get_topic()); if (!cp) { return ss::make_ready_future( resp_t{errc::coordinator_topic_not_exists}); @@ -258,6 +273,31 @@ ss::future frontend::ensure_topic_exists() { } } +ss::future frontend::ensure_table_exists_locally( + ensure_table_exists_request request, + const model::ntp& coordinator_partition, + ss::shard_id shard) { + co_return co_await _coordinator_mgr->invoke_on( + shard, + [coordinator_partition, + req = std::move(request)](coordinator_manager& mgr) mutable { + auto partition = mgr.get(coordinator_partition); + if (!partition) { + return ssx::now(ensure_table_exists_reply{errc::not_leader}); + } + return do_ensure_table_exists( + mgr, coordinator_partition, std::move(req)); + }); +} + +ss::future frontend::ensure_table_exists( + ensure_table_exists_request request, local_only local_only_exec) { + auto holder = _gate.hold(); + co_return co_await process< + &frontend::ensure_table_exists_locally, + &client::ensure_table_exists>(std::move(request), bool(local_only_exec)); +} + ss::future frontend::add_translated_data_files_locally( add_translated_data_files_request request, diff --git a/src/v/datalake/coordinator/frontend.h b/src/v/datalake/coordinator/frontend.h index 4cae8bb4a26a..de81c5853977 100644 --- a/src/v/datalake/coordinator/frontend.h +++ b/src/v/datalake/coordinator/frontend.h @@ -8,6 +8,8 @@ * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ +#pragma once + #include "base/outcome.h" #include "base/seastarx.h" #include "cluster/fwd.h" @@ -46,6 +48,9 @@ class frontend : public ss::peering_sharded_service { ss::future<> stop(); + ss::future ensure_table_exists( + ensure_table_exists_request, local_only = local_only::no); + ss::future add_translated_data_files( add_translated_data_files_request, local_only = local_only::no); @@ -85,6 +90,11 @@ class frontend : public ss::peering_sharded_service { std::optional coordinator_partition(const model::topic&) const; + ss::future ensure_table_exists_locally( + ensure_table_exists_request, + const model::ntp& coordinator_partition, + ss::shard_id); + ss::future add_translated_data_files_locally( add_translated_data_files_request, diff --git a/src/v/datalake/coordinator/rpc.json b/src/v/datalake/coordinator/rpc.json index 960b39b0d7d1..0275180d350a 100644 --- a/src/v/datalake/coordinator/rpc.json +++ b/src/v/datalake/coordinator/rpc.json @@ -5,6 +5,11 @@ "datalake/coordinator/types.h" ], "methods": [ + { + "name": "ensure_table_exists", + "input_type": "ensure_table_exists_request", + "output_type": "ensure_table_exists_reply" + }, { "name": "add_translated_data_files", "input_type": "add_translated_data_files_request", diff --git a/src/v/datalake/coordinator/service.cc b/src/v/datalake/coordinator/service.cc index c638d92d4870..cfac04284d03 100644 --- a/src/v/datalake/coordinator/service.cc +++ b/src/v/datalake/coordinator/service.cc @@ -21,6 +21,12 @@ service::service( : impl::datalake_coordinator_rpc_service(sg, smp_sg) , _frontend(frontend) {} +ss::future service::ensure_table_exists( + ensure_table_exists_request request, ::rpc::streaming_context&) { + return _frontend->local().ensure_table_exists( + std::move(request), frontend::local_only::yes); +} + ss::future service::add_translated_data_files( add_translated_data_files_request request, ::rpc::streaming_context&) { return _frontend->local().add_translated_data_files( diff --git a/src/v/datalake/coordinator/service.h b/src/v/datalake/coordinator/service.h index 0da343e8e48d..358398c125f0 100644 --- a/src/v/datalake/coordinator/service.h +++ b/src/v/datalake/coordinator/service.h @@ -7,6 +7,9 @@ * * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ + +#pragma once + #include "datalake/coordinator/rpc_service.h" #include "datalake/fwd.h" @@ -16,6 +19,9 @@ class service final : public impl::datalake_coordinator_rpc_service { service( ss::scheduling_group, ss::smp_service_group, ss::sharded*); + ss::future ensure_table_exists( + ensure_table_exists_request, ::rpc::streaming_context&) override; + ss::future add_translated_data_files( add_translated_data_files_request, ::rpc::streaming_context&) override; diff --git a/src/v/datalake/coordinator/state.cc b/src/v/datalake/coordinator/state.cc index a34a8f56475a..f763ed905f0c 100644 --- a/src/v/datalake/coordinator/state.cc +++ b/src/v/datalake/coordinator/state.cc @@ -69,4 +69,13 @@ topics_state topics_state::copy() const { return result; } +bool topic_state::has_pending_entries() const { + for (const auto& [_, partition_state] : pid_to_pending_files) { + if (!partition_state.pending_entries.empty()) { + return true; + } + } + return false; +} + } // namespace datalake::coordinator diff --git a/src/v/datalake/coordinator/state.h b/src/v/datalake/coordinator/state.h index c97e673cd4ad..bed7af1280b0 100644 --- a/src/v/datalake/coordinator/state.h +++ b/src/v/datalake/coordinator/state.h @@ -99,6 +99,8 @@ struct topic_state friend std::ostream& operator<<(std::ostream&, topic_state::lifecycle_state_t); + bool has_pending_entries() const; + // Topic revision model::revision_id revision; // Map from Redpanda partition id to the files pending per partition. diff --git a/src/v/datalake/coordinator/state_machine.cc b/src/v/datalake/coordinator/state_machine.cc index 0a8355b18454..3b91dd10e871 100644 --- a/src/v/datalake/coordinator/state_machine.cc +++ b/src/v/datalake/coordinator/state_machine.cc @@ -23,11 +23,12 @@ namespace datalake::coordinator { namespace { +template void maybe_log_update_error( prefix_logger& log, update_key key, model::offset o, - const checked& r) { + const checked& r) { if (r.has_value()) { return; } diff --git a/src/v/datalake/coordinator/state_update.cc b/src/v/datalake/coordinator/state_update.cc index e998e36ba651..16c6d93e740b 100644 --- a/src/v/datalake/coordinator/state_update.cc +++ b/src/v/datalake/coordinator/state_update.cc @@ -220,30 +220,78 @@ mark_files_committed_update::apply(topics_state& state) { return std::nullopt; } -checked +checked topic_lifecycle_update::can_apply(const topics_state& state) { auto topic_it = state.topic_to_state.find(topic); if (topic_it == state.topic_to_state.end()) { - return stm_update_error{fmt::format("topic {} not found", topic)}; + return true; } - if (topic_it->second.revision != revision) { + + auto& cur_state = topic_it->second; + if (revision < cur_state.revision) { return stm_update_error{fmt::format( - "topic {} revision mismatch, expected {}, got {}", + "topic {} rev {}: lifecycle update with obsolete revision {}", topic, - revision, - topic_it->second.revision)}; + cur_state.revision, + revision)}; } - return std::nullopt; + + if (revision > cur_state.revision) { + if ( + cur_state.lifecycle_state != topic_state::lifecycle_state_t::purged) { + return stm_update_error{fmt::format( + "topic {} rev {} not fully purged, but already trying to " + "register revision {}", + topic, + cur_state.revision, + revision)}; + } + + return true; + } + + // after this point revision matches + + if (new_state < cur_state.lifecycle_state) { + return stm_update_error{fmt::format( + "topic {} rev {} invalid lifecycle state transition", + topic, + cur_state.revision)}; + } + + if (new_state > cur_state.lifecycle_state) { + if ( + new_state == topic_state::lifecycle_state_t::purged + && cur_state.has_pending_entries()) { + return stm_update_error{fmt::format( + "can't purge topic {} rev {}: still has pending entries", + topic, + cur_state.revision)}; + } + return true; + } + + // no-op + return false; } -checked +checked topic_lifecycle_update::apply(topics_state& state) { - auto allowed = can_apply(state); - if (allowed.has_error()) { - return allowed.error(); + auto check_res = can_apply(state); + if (check_res.has_error()) { + return check_res.error(); } - state.topic_to_state[topic].lifecycle_state = new_state; - return std::nullopt; + if (!check_res.value()) { + return false; + } + auto& t_state = state.topic_to_state[topic]; + t_state.revision = revision; + t_state.lifecycle_state = new_state; + if (new_state == topic_state::lifecycle_state_t::purged) { + // release memory + t_state.pid_to_pending_files = decltype(t_state.pid_to_pending_files){}; + } + return true; } std::ostream& operator<<(std::ostream& o, topic_lifecycle_update u) { diff --git a/src/v/datalake/coordinator/state_update.h b/src/v/datalake/coordinator/state_update.h index f59d69f74600..fbae9a5f52a5 100644 --- a/src/v/datalake/coordinator/state_update.h +++ b/src/v/datalake/coordinator/state_update.h @@ -86,8 +86,9 @@ struct topic_lifecycle_update static constexpr auto key{update_key::topic_lifecycle_update}; auto serde_fields() { return std::tie(topic, revision, new_state); } - checked can_apply(const topics_state&); - checked apply(topics_state&); + // returns true if the update actually changes anything + checked can_apply(const topics_state&); + checked apply(topics_state&); friend std::ostream& operator<<(std::ostream&, topic_lifecycle_update); diff --git a/src/v/datalake/coordinator/types.cc b/src/v/datalake/coordinator/types.cc index 369117a9dda8..5aeccb363146 100644 --- a/src/v/datalake/coordinator/types.cc +++ b/src/v/datalake/coordinator/types.cc @@ -47,6 +47,18 @@ std::ostream& operator<<(std::ostream& o, const errc& errc) { return o; } +std::ostream& operator<<(std::ostream& o, const ensure_table_exists_reply& r) { + fmt::print(o, "{{errc: {}}}", r.errc); + return o; +} + +std::ostream& +operator<<(std::ostream& o, const ensure_table_exists_request& r) { + fmt::print( + o, "{{topic: {}, topic_revision: {}}}", r.topic, r.topic_revision); + return o; +} + std::ostream& operator<<(std::ostream& o, const add_translated_data_files_reply& reply) { fmt::print(o, "{{errc: {}}}", reply.errc); diff --git a/src/v/datalake/coordinator/types.h b/src/v/datalake/coordinator/types.h index bb5789683f88..5ed75b524297 100644 --- a/src/v/datalake/coordinator/types.h +++ b/src/v/datalake/coordinator/types.h @@ -13,6 +13,7 @@ #include "datalake/coordinator/state.h" #include "datalake/coordinator/translated_offset_range.h" #include "datalake/errors.h" +#include "datalake/schema_identifier.h" #include "model/fundamental.h" #include "serde/rw/enum.h" #include "serde/rw/envelope.h" @@ -39,6 +40,55 @@ constexpr bool is_retriable(errc errc) { std::ostream& operator<<(std::ostream&, const errc&); +struct ensure_table_exists_reply + : serde::envelope< + ensure_table_exists_reply, + serde::version<0>, + serde::compat_version<0>> { + using rpc_adl_exempt = std::true_type; + + ensure_table_exists_reply() = default; + explicit ensure_table_exists_reply(errc err) + : errc(err) {} + + friend std::ostream& + operator<<(std::ostream&, const ensure_table_exists_reply&); + + errc errc; + + auto serde_fields() { return std::tie(errc); } +}; +struct ensure_table_exists_request + : serde::envelope< + ensure_table_exists_request, + serde::version<0>, + serde::compat_version<0>> { + using rpc_adl_exempt = std::true_type; + using resp_t = ensure_table_exists_reply; + + ensure_table_exists_request() = default; + ensure_table_exists_request( + model::topic topic, + model::revision_id topic_revision, + record_schema_components schema_components) + : topic(std::move(topic)) + , topic_revision(topic_revision) + , schema_components(std::move(schema_components)) {} + + model::topic topic; + model::revision_id topic_revision; + record_schema_components schema_components; + + friend std::ostream& + operator<<(std::ostream&, const ensure_table_exists_request&); + + const model::topic& get_topic() const { return topic; } + + auto serde_fields() { + return std::tie(topic, topic_revision, schema_components); + } +}; + struct add_translated_data_files_reply : serde::envelope< add_translated_data_files_reply, @@ -100,7 +150,7 @@ struct add_translated_data_files_request friend std::ostream& operator<<(std::ostream&, const add_translated_data_files_request&); - const model::topic_partition& topic_partition() const { return tp; } + const model::topic& get_topic() const { return tp.topic; } auto serde_fields() { return std::tie(tp, topic_revision, ranges, translator_term); @@ -149,7 +199,7 @@ struct fetch_latest_translated_offset_request model::topic_partition tp; model::revision_id topic_revision; - const model::topic_partition& topic_partition() const { return tp; } + const model::topic& get_topic() const { return tp.topic; } friend std::ostream& operator<<(std::ostream&, const fetch_latest_translated_offset_request&); From 4362a2e613cb190f398d2ff4e5012e38b78bb9a0 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 21 Nov 2024 02:01:52 +0100 Subject: [PATCH 10/19] datalake: split out schema_manager::ensure_table_schema As we want to move table creation and schema changes to the coordinator, this functionality is split out to a separate method, and schema_manager::get_registered_ids expects the table to already exist with a correct schema. --- src/v/datalake/catalog_schema_manager.cc | 37 ++++++++++++++----- src/v/datalake/catalog_schema_manager.h | 19 +++++++--- src/v/datalake/record_multiplexer.cc | 37 +++++++++++++++++-- .../tests/catalog_schema_manager_test.cc | 34 +++++++++++++---- 4 files changed, 102 insertions(+), 25 deletions(-) diff --git a/src/v/datalake/catalog_schema_manager.cc b/src/v/datalake/catalog_schema_manager.cc index 0fd6a54462a8..362ee2131e92 100644 --- a/src/v/datalake/catalog_schema_manager.cc +++ b/src/v/datalake/catalog_schema_manager.cc @@ -89,6 +89,13 @@ std::ostream& operator<<(std::ostream& o, const schema_manager::errc& e) { return o << "schema_manager::errc::shutting_down"; } } + +ss::future> +simple_schema_manager::ensure_table_schema( + const model::topic&, const iceberg::struct_type&) { + co_return std::nullopt; +} + ss::future> simple_schema_manager::get_registered_ids( const model::topic&, iceberg::struct_type& desired_type) { @@ -103,29 +110,33 @@ simple_schema_manager::get_registered_ids( } ss::future> -catalog_schema_manager::get_registered_ids( - const model::topic& topic, iceberg::struct_type& dest_type) { +catalog_schema_manager::ensure_table_schema( + const model::topic& topic, const iceberg::struct_type& desired_type) { auto table_id = table_id_for_topic(topic); auto load_res = co_await catalog_.load_or_create_table( - table_id, dest_type, hour_partition_spec()); + table_id, desired_type, hour_partition_spec()); if (load_res.has_error()) { co_return log_and_convert_catalog_err( load_res.error(), fmt::format("Error loading table {}", table_id)); } + + // Check schema compatibility + auto type_copy = desired_type.copy(); auto get_res = get_ids_from_table_meta( - table_id, load_res.value(), dest_type); + table_id, load_res.value(), type_copy); if (get_res.has_error()) { co_return get_res.error(); } if (get_res.value()) { - // Success! We got all the field IDs. + // Success! Schema already matches what we need. co_return std::nullopt; } + // The current table schema is a prefix of the desired schema. Add the // schema to the table. iceberg::transaction txn(std::move(load_res.value())); auto update_res = co_await txn.set_schema(iceberg::schema{ - .schema_struct = dest_type.copy(), + .schema_struct = desired_type.copy(), .schema_id = iceberg::schema::unassigned_id, .identifier_field_ids = {}, }); @@ -151,21 +162,29 @@ catalog_schema_manager::get_registered_ids( fmt::format( "Error while committing schema update to table {}", table_id)); } - load_res = co_await catalog_.load_table(table_id); + co_return std::nullopt; +} + +ss::future> +catalog_schema_manager::get_registered_ids( + const model::topic& topic, iceberg::struct_type& dest_type) { + auto table_id = table_id_for_topic(topic); + auto load_res = co_await catalog_.load_table(table_id); if (load_res.has_error()) { co_return log_and_convert_catalog_err( load_res.error(), fmt::format( "Error while reloading table {} after schema update", table_id)); } - get_res = get_ids_from_table_meta(table_id, load_res.value(), dest_type); + auto get_res = get_ids_from_table_meta( + table_id, load_res.value(), dest_type); if (get_res.has_error()) { co_return get_res.error(); } if (!get_res.value()) { vlog( datalake_log.warn, - "Failed to fill field IDs after adding schema to table {}", + "expected to successfully fill field IDs for table {}", table_id); co_return errc::failed; } diff --git a/src/v/datalake/catalog_schema_manager.h b/src/v/datalake/catalog_schema_manager.h index b1735c188b0b..1d429ca59591 100644 --- a/src/v/datalake/catalog_schema_manager.h +++ b/src/v/datalake/catalog_schema_manager.h @@ -28,6 +28,9 @@ class schema_manager { }; friend std::ostream& operator<<(std::ostream&, const errc&); + virtual ss::future> ensure_table_schema( + const model::topic&, const iceberg::struct_type& desired_type) + = 0; virtual ss::future> get_registered_ids(const model::topic&, iceberg::struct_type& desired_type) = 0; @@ -37,6 +40,9 @@ class schema_manager { class simple_schema_manager : public schema_manager { public: ss::future> + ensure_table_schema( + const model::topic&, const iceberg::struct_type& desired_type) override; + ss::future> get_registered_ids( const model::topic&, iceberg::struct_type& desired_type) override; ~simple_schema_manager() override = default; @@ -50,13 +56,16 @@ class catalog_schema_manager : public schema_manager { explicit catalog_schema_manager(iceberg::catalog& catalog) : catalog_(catalog) {} + // Create the table with a desired schema, or, if the table exists and its + // current schema doesn't include all of the fields (e.g. we are going from + // the schemaless schema to a schema containing user fields), the table's + // schema is updated to the desired type. + ss::future> + ensure_table_schema( + const model::topic&, const iceberg::struct_type& desired_type) override; + // Loads the table metadata for the given topic and fills the field IDs of // the given type with those in the current schema. - // - // If the table's current schema doesn't include all of the fields (e.g. we - // are going from the schemaless schema to a schema containing user - // fields), the table's schema is updated to the desired type, and then the - // fill is attempted again. ss::future> get_registered_ids( const model::topic&, iceberg::struct_type& desired_type) override; diff --git a/src/v/datalake/record_multiplexer.cc b/src/v/datalake/record_multiplexer.cc index c43b82a766cc..79fdfbe4821d 100644 --- a/src/v/datalake/record_multiplexer.cc +++ b/src/v/datalake/record_multiplexer.cc @@ -115,10 +115,10 @@ record_multiplexer::operator()(model::record_batch batch) { std::move(val_type_res.value().type)); auto writer_iter = _writers.find(record_type.comps); if (writer_iter == _writers.end()) { - auto get_ids_res = co_await _schema_mgr.get_registered_ids( + auto ensure_res = co_await _schema_mgr.ensure_table_schema( _ntp.tp.topic, record_type.type); - if (get_ids_res.has_error()) { - auto e = get_ids_res.error(); + if (ensure_res.has_error()) { + auto e = ensure_res.error(); switch (e) { case schema_manager::errc::not_supported: { auto invalid_res = co_await handle_invalid_record( @@ -134,16 +134,36 @@ record_multiplexer::operator()(model::record_batch batch) { continue; } case schema_manager::errc::failed: + vlog( + _log.warn, + "Error ensuring table schema for record {}", + offset); + [[fallthrough]]; case schema_manager::errc::shutting_down: + _error = writer_error::parquet_conversion_error; + } + co_return ss::stop_iteration::yes; + } + + auto get_ids_res = co_await _schema_mgr.get_registered_ids( + _ntp.tp.topic, record_type.type); + if (get_ids_res.has_error()) { + auto e = get_ids_res.error(); + switch (e) { + case schema_manager::errc::not_supported: + case schema_manager::errc::failed: vlog( _log.warn, "Error getting field IDs for record {}: {}", offset, get_ids_res.error()); + [[fallthrough]]; + case schema_manager::errc::shutting_down: _error = writer_error::parquet_conversion_error; } co_return ss::stop_iteration::yes; } + auto [iter, _] = _writers.emplace( record_type.comps, std::make_unique( @@ -237,6 +257,17 @@ record_multiplexer::handle_invalid_record( // TODO: maybe this should be a writer specific for a dead-letter queue. auto writer_iter = _writers.find(record_type.comps); if (writer_iter == _writers.end()) { + auto ensure_res = co_await _schema_mgr.ensure_table_schema( + _ntp.tp.topic, record_type.type); + if (ensure_res.has_error()) { + vlog( + _log.warn, + "Error ensuring table schema for binary record {}: {}", + offset, + ensure_res.error()); + co_return writer_error::parquet_conversion_error; + } + auto get_ids_res = co_await _schema_mgr.get_registered_ids( _ntp.tp.topic, record_type.type); if (get_ids_res.has_error()) { diff --git a/src/v/datalake/tests/catalog_schema_manager_test.cc b/src/v/datalake/tests/catalog_schema_manager_test.cc index ef3101f1dddb..bf5e378ad91e 100644 --- a/src/v/datalake/tests/catalog_schema_manager_test.cc +++ b/src/v/datalake/tests/catalog_schema_manager_test.cc @@ -91,10 +91,15 @@ TEST_F(CatalogSchemaManagerTest, TestCreateTable) { auto type = std::get(test_nested_schema_type()); reset_field_ids(type); - // Trying to fill the field IDs requires querying the catalog, which will - // result in the table being created. - auto res = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); - ASSERT_FALSE(res.has_error()); + // Create the table + auto create_res + = schema_mgr.ensure_table_schema(model::topic{"foo"}, type).get(); + ASSERT_FALSE(create_res.has_error()); + + // Fill the field IDs in `type`. + auto fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); + ASSERT_FALSE(fill_res.has_error()); auto table_ident = table_identifier{.ns = {"redpanda"}, .table = "foo"}; auto schema = load_table_schema(table_ident).get(); @@ -172,9 +177,15 @@ TEST_F(CatalogSchemaManagerTest, TestFillSuperset) { field_required::no, std::move(nested))); } + // Alter the table schema + auto ensure_res + = schema_mgr.ensure_table_schema(model::topic{"foo"}, type).get(); + ASSERT_FALSE(ensure_res.has_error()); - auto res = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); - ASSERT_FALSE(res.has_error()); + // Fill the ids in `type` + auto fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); + ASSERT_FALSE(fill_res.has_error()); // Check the resulting schema. schema s{ @@ -204,8 +215,15 @@ TEST_F(CatalogSchemaManagerTest, TestFillSupersetSubtype) { field_required::no, int_type{})); } - auto res = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); - ASSERT_FALSE(res.has_error()); + // Alter the table schema + auto ensure_res + = schema_mgr.ensure_table_schema(model::topic{"foo"}, type).get(); + ASSERT_FALSE(ensure_res.has_error()); + + // Fill the ids + auto fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); + ASSERT_FALSE(fill_res.has_error()); // Check the resulting schema. schema s{ From 71356e92982600f1f2b7d9065e72ca9cd079bc8b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 21 Nov 2024 03:33:16 +0100 Subject: [PATCH 11/19] datalake: introduce type_resolver::resolve_identifier Resolve a type from a schema_identifier, not by parsing a message. --- src/v/datalake/record_schema_resolver.cc | 237 +++++++++++++++-------- src/v/datalake/record_schema_resolver.h | 8 + 2 files changed, 167 insertions(+), 78 deletions(-) diff --git a/src/v/datalake/record_schema_resolver.cc b/src/v/datalake/record_schema_resolver.cc index 9d5ab9b1e033..989969c69426 100644 --- a/src/v/datalake/record_schema_resolver.cc +++ b/src/v/datalake/record_schema_resolver.cc @@ -30,6 +30,63 @@ namespace datalake { namespace { namespace ppsr = pandaproxy::schema_registry; + +checked translate_avro_schema( + const ppsr::avro_schema_definition& avro_def, ppsr::schema_id id) { + const auto& avro_schema = avro_def(); + try { + auto result = datalake::type_to_iceberg(avro_schema.root()); + if (result.has_error()) { + vlog( + datalake_log.error, + "Avro schema translation failed: {}", + result.error()); + return type_resolver::errc::translation_error; + } + + return resolved_type{ + .schema = avro_schema, + .id = { .schema_id = id, .protobuf_offsets = std::nullopt, }, + .type = std::move(result.value()), + .type_name = avro_schema.root()->name().fullname(), + }; + } catch (...) { + vlog( + datalake_log.error, + "Avro schema translation failed: {}", + std::current_exception()); + return type_resolver::errc::translation_error; + } +} + +checked translate_protobuf_schema( + ppsr::protobuf_schema_definition&& pb_def, + ppsr::schema_id id, + std::vector protobuf_offsets) { + // TODO: maybe there's another caching opportunity here. + auto d_res = descriptor(pb_def, protobuf_offsets); + if (d_res.has_error()) { + return type_resolver::errc::bad_input; + } + const auto* d = &d_res.value().get(); + try { + auto type = type_to_iceberg(*d).value(); + return resolved_type{ + .schema = wrapped_protobuf_descriptor{*d, std::move(pb_def)}, + .id + = {.schema_id = id, .protobuf_offsets = std::move(protobuf_offsets)}, + .type = std::move(type), + .type_name = d->name(), + }; + } catch (...) { + vlog( + datalake_log.error, + "Protobuf schema translation failed: {}", + std::current_exception()); + return type_resolver::errc::translation_error; + } +} + struct schema_translating_visitor { schema_translating_visitor(iobuf b, ppsr::schema_id id) : buf_no_id(std::move(b)) @@ -38,75 +95,100 @@ struct schema_translating_visitor { iobuf buf_no_id; ppsr::schema_id id; - ss::future> + checked operator()(ppsr::avro_schema_definition&& avro_def) { - const auto& avro_schema = avro_def(); - try { - auto result = datalake::type_to_iceberg(avro_schema.root()); - if (result.has_error()) { - vlog( - datalake_log.error, - "Avro schema translation failed: {}", - result.error()); - co_return type_resolver::errc::translation_error; - } - - co_return type_and_buf{ - .type = resolved_type{ - .schema = avro_schema, - .id = { .schema_id = id, .protobuf_offsets = std::nullopt, }, - .type = std::move(result.value()), - .type_name = avro_schema.root()->name().fullname(), - }, - .parsable_buf = std::move(buf_no_id), - }; - - } catch (...) { - vlog( - datalake_log.error, - "Avro schema translation failed: {}", - std::current_exception()); - co_return type_resolver::errc::translation_error; + auto tr_res = translate_avro_schema(avro_def, id); + if (tr_res.has_error()) { + return tr_res.error(); } + return type_and_buf{ + .type = std::move(tr_res.value()), + .parsable_buf = std::move(buf_no_id)}; } - ss::future> + + checked operator()(ppsr::protobuf_schema_definition&& pb_def) { auto offsets_res = get_proto_offsets(buf_no_id); if (offsets_res.has_error()) { - co_return type_resolver::errc::bad_input; + return type_resolver::errc::bad_input; } auto offsets = std::move(offsets_res.value()); - // TODO: maybe there's another caching opportunity here. - auto d_res = descriptor(pb_def, offsets.protobuf_offsets); - if (d_res.has_error()) { - co_return type_resolver::errc::bad_input; + + auto tr_res = translate_protobuf_schema( + std::move(pb_def), id, std::move(offsets.protobuf_offsets)); + if (tr_res.has_error()) { + return tr_res.error(); } - const auto* d = &d_res.value().get(); - try { - auto type = type_to_iceberg(*d).value(); - co_return type_and_buf{ - .type = resolved_type{ - .schema = wrapped_protobuf_descriptor { *d, std::move(pb_def) }, - .id = {.schema_id = id, .protobuf_offsets = std::move(offsets.protobuf_offsets)}, - .type = std::move(type), - .type_name = d->name(), - }, - .parsable_buf = std::move(offsets.shared_message_data), - }; - } catch (...) { - vlog( - datalake_log.error, - "Protobuf schema translation failed: {}", - std::current_exception()); - co_return type_resolver::errc::translation_error; + + return type_and_buf{ + .type = std::move(tr_res.value()), + .parsable_buf = std::move(offsets.shared_message_data)}; + } + + checked + operator()(ppsr::json_schema_definition&&) { + return type_resolver::errc::bad_input; + } +}; + +struct from_identifier_visitor { + from_identifier_visitor(schema_identifier ident) + : ident(std::move(ident)) {} + + schema_identifier ident; + + checked + operator()(ppsr::avro_schema_definition&& avro_def) { + if (ident.protobuf_offsets) { + return type_resolver::errc::bad_input; } + return translate_avro_schema(avro_def, ident.schema_id); } - ss::future> + checked + operator()(ppsr::protobuf_schema_definition&& pb_def) { + if (!ident.protobuf_offsets) { + return type_resolver::errc::bad_input; + } + return translate_protobuf_schema( + std::move(pb_def), + ident.schema_id, + std::move(ident.protobuf_offsets.value())); + } + checked operator()(ppsr::json_schema_definition&&) { - co_return type_resolver::errc::bad_input; + return type_resolver::errc::bad_input; } }; +ss::future> +get_schema(schema::registry& sr, ppsr::schema_id id) { + if (!sr.is_enabled()) { + vlog(datalake_log.warn, "Schema registry is not enabled"); + // TODO: should we treat this as transient? + co_return type_resolver::errc::translation_error; + } + // TODO: It'd be nice to cache these -- translation interval instills a + // natural limit to concurrency so a cache wouldn't grow huge. + auto schema_fut = co_await ss::coroutine::as_future( + sr.get_valid_schema(id)); + if (schema_fut.failed()) { + vlog( + datalake_log.warn, + "Error getting schema from registry: {}", + schema_fut.get_exception()); + co_return type_resolver::errc::registry_error; + } + auto resolved_schema = std::move(schema_fut.get()); + if (!resolved_schema.has_value()) { + vlog( + datalake_log.trace, + "Schema ID {} not in registry; using binary type", + id); + co_return type_resolver::errc::bad_input; + } + co_return std::move(resolved_schema.value()); +} + } // namespace std::ostream& operator<<(std::ostream& o, const type_resolver::errc& e) { @@ -132,6 +214,13 @@ binary_type_resolver::resolve_buf_type(iobuf b) const { co_return type_and_buf::make_raw_binary(std::move(b)); } +ss::future> +binary_type_resolver::resolve_identifier(schema_identifier) const { + // method is not expected to be called, as this resolver always returns + // nullopt type. + co_return type_resolver::errc::translation_error; +} + ss::future> record_schema_resolver::resolve_buf_type(iobuf b) const { // NOTE: Kafka's serialization protocol relies on a magic byte to @@ -150,32 +239,24 @@ record_schema_resolver::resolve_buf_type(iobuf b) const { auto schema_id = schema_id_res.schema_id; auto buf_no_id = std::move(schema_id_res.shared_message_data); - if (!sr_.is_enabled()) { - vlog(datalake_log.warn, "Schema registry is not enabled"); - // TODO: should we treat this as transient? - co_return errc::translation_error; - } - // TODO: It'd be nice to cache these -- translation interval instills a - // natural limit to concurrency so a cache wouldn't grow huge. - auto schema_fut = co_await ss::coroutine::as_future( - sr_.get_valid_schema(schema_id)); - if (schema_fut.failed()) { - vlog( - datalake_log.warn, - "Error getting schema from registry: {}", - schema_fut.get_exception()); - co_return errc::registry_error; - } - auto resolved_schema = std::move(schema_fut.get()); - if (!resolved_schema.has_value()) { - vlog( - datalake_log.trace, - "Schema ID {} not in registry; using binary type", - schema_id); - co_return errc::bad_input; + auto schema_res = co_await get_schema(sr_, schema_id); + if (schema_res.has_error()) { + co_return schema_res.error(); } - co_return co_await std::move(*resolved_schema) + + co_return std::move(schema_res.value()) .visit(schema_translating_visitor{std::move(buf_no_id), schema_id}); } +ss::future> +record_schema_resolver::resolve_identifier(schema_identifier ident) const { + auto schema_res = co_await get_schema(sr_, ident.schema_id); + if (schema_res.has_error()) { + co_return schema_res.error(); + } + + co_return std::move(schema_res.value()) + .visit(from_identifier_visitor{std::move(ident)}); +} + } // namespace datalake diff --git a/src/v/datalake/record_schema_resolver.h b/src/v/datalake/record_schema_resolver.h index ce201d6ee266..c40089ffb9ab 100644 --- a/src/v/datalake/record_schema_resolver.h +++ b/src/v/datalake/record_schema_resolver.h @@ -74,6 +74,8 @@ class type_resolver { friend std::ostream& operator<<(std::ostream&, const errc&); virtual ss::future> resolve_buf_type(iobuf b) const = 0; + virtual ss::future> + resolve_identifier(schema_identifier) const = 0; virtual ~type_resolver() = default; }; @@ -81,6 +83,8 @@ class binary_type_resolver : public type_resolver { public: ss::future> resolve_buf_type(iobuf b) const override; + ss::future> + resolve_identifier(schema_identifier) const override; ~binary_type_resolver() override = default; }; @@ -91,6 +95,10 @@ class record_schema_resolver : public type_resolver { ss::future> resolve_buf_type(iobuf b) const override; + + ss::future> + resolve_identifier(schema_identifier) const override; + ~record_schema_resolver() override = default; private: From 35b0c61c14f16bd68b0745d1c4ed1c8046a603dd Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 21 Nov 2024 15:05:14 +0100 Subject: [PATCH 12/19] datalake: introduce table_creator Its job is to resolve the table schema from a schema identifier and ensure that the corresponding table exists and has a compatible schema. --- src/v/datalake/BUILD | 19 +++++++ src/v/datalake/CMakeLists.txt | 1 + src/v/datalake/catalog_schema_manager.cc | 2 +- src/v/datalake/catalog_schema_manager.h | 4 +- src/v/datalake/fwd.h | 1 + src/v/datalake/table_creator.cc | 66 ++++++++++++++++++++++++ src/v/datalake/table_creator.h | 52 +++++++++++++++++++ 7 files changed, 142 insertions(+), 3 deletions(-) create mode 100644 src/v/datalake/table_creator.cc create mode 100644 src/v/datalake/table_creator.h diff --git a/src/v/datalake/BUILD b/src/v/datalake/BUILD index a75e4a2fd12a..40cf070e1fa0 100644 --- a/src/v/datalake/BUILD +++ b/src/v/datalake/BUILD @@ -532,3 +532,22 @@ redpanda_cc_library( "@seastar", ], ) + +redpanda_cc_library( + name = "table_creator", + srcs = [ + "table_creator.cc", + ], + hdrs = [ + "table_creator.h", + ], + implementation_deps = [ + ":record_translator", + ], + include_prefix = "datalake", + visibility = [":__subpackages__"], + deps = [ + ":catalog_schema_manager", + ":record_schema_resolver", + ], +) diff --git a/src/v/datalake/CMakeLists.txt b/src/v/datalake/CMakeLists.txt index 5c81677e2804..9463a0dc2395 100644 --- a/src/v/datalake/CMakeLists.txt +++ b/src/v/datalake/CMakeLists.txt @@ -47,6 +47,7 @@ v_cc_library( values_protobuf.cc base_types.cc cloud_data_io.cc + table_creator.cc translation_task.cc schema_parquet.cc values_parquet.cc diff --git a/src/v/datalake/catalog_schema_manager.cc b/src/v/datalake/catalog_schema_manager.cc index 362ee2131e92..d07809d0410d 100644 --- a/src/v/datalake/catalog_schema_manager.cc +++ b/src/v/datalake/catalog_schema_manager.cc @@ -223,7 +223,7 @@ catalog_schema_manager::get_ids_from_table_meta( } iceberg::table_identifier -catalog_schema_manager::table_id_for_topic(const model::topic& t) const { +schema_manager::table_id_for_topic(const model::topic& t) const { return iceberg::table_identifier{ // TODO: namespace as a topic property? Keep it in the table metadata? .ns = {"redpanda"}, diff --git a/src/v/datalake/catalog_schema_manager.h b/src/v/datalake/catalog_schema_manager.h index 1d429ca59591..407b8b8c4c89 100644 --- a/src/v/datalake/catalog_schema_manager.h +++ b/src/v/datalake/catalog_schema_manager.h @@ -35,6 +35,8 @@ class schema_manager { get_registered_ids(const model::topic&, iceberg::struct_type& desired_type) = 0; virtual ~schema_manager() = default; + + iceberg::table_identifier table_id_for_topic(const model::topic& t) const; }; class simple_schema_manager : public schema_manager { @@ -71,8 +73,6 @@ class catalog_schema_manager : public schema_manager { const model::topic&, iceberg::struct_type& desired_type) override; private: - iceberg::table_identifier table_id_for_topic(const model::topic& t) const; - // Attempts to fill the field ids in the given type with those from the // current schema of the given table metadata. // diff --git a/src/v/datalake/fwd.h b/src/v/datalake/fwd.h index fdfd7695d262..776a6e846d29 100644 --- a/src/v/datalake/fwd.h +++ b/src/v/datalake/fwd.h @@ -15,6 +15,7 @@ struct data_writer_result; class record_translator; class schema_manager; class type_resolver; +class table_creator; namespace coordinator { class coordinator_manager; class frontend; diff --git a/src/v/datalake/table_creator.cc b/src/v/datalake/table_creator.cc new file mode 100644 index 000000000000..7ab07fcf35f3 --- /dev/null +++ b/src/v/datalake/table_creator.cc @@ -0,0 +1,66 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Licensed as a Redpanda Enterprise file under the Redpanda Community + * License (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md + */ +#include "datalake/table_creator.h" + +#include "datalake/record_translator.h" + +namespace datalake { + +std::ostream& operator<<(std::ostream& o, const table_creator::errc& e) { + switch (e) { + case table_creator::errc::incompatible_schema: + return o << "table_creator::errc::incompatible_schema"; + case table_creator::errc::failed: + return o << "table_creator::errc::failed"; + case table_creator::errc::shutting_down: + return o << "table_creator::errc::shutting_down"; + } +} + +direct_table_creator::direct_table_creator( + type_resolver& tr, schema_manager& sm) + : type_resolver_(tr) + , schema_mgr_(sm) {} + +ss::future> +direct_table_creator::ensure_table( + const model::topic& topic, + model::revision_id, + record_schema_components comps) const { + auto table_id = schema_mgr_.table_id_for_topic(topic); + + std::optional val_type; + if (comps.val_identifier) { + auto type_res = co_await type_resolver_.resolve_identifier( + comps.val_identifier.value()); + if (type_res.has_error()) { + co_return errc::failed; + } + val_type = std::move(type_res.value()); + } + + auto record_type = default_translator{}.build_type(std::move(val_type)); + auto ensure_res = co_await schema_mgr_.ensure_table_schema( + topic, record_type.type); + if (ensure_res.has_error()) { + switch (ensure_res.error()) { + case schema_manager::errc::not_supported: + co_return errc::incompatible_schema; + case schema_manager::errc::failed: + co_return errc::failed; + case schema_manager::errc::shutting_down: + co_return errc::shutting_down; + } + } + + co_return std::nullopt; +} + +} // namespace datalake diff --git a/src/v/datalake/table_creator.h b/src/v/datalake/table_creator.h new file mode 100644 index 000000000000..8262f2b642f2 --- /dev/null +++ b/src/v/datalake/table_creator.h @@ -0,0 +1,52 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Licensed as a Redpanda Enterprise file under the Redpanda Community + * License (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md + */ +#pragma once + +#include "datalake/catalog_schema_manager.h" +#include "datalake/record_schema_resolver.h" + +namespace datalake { + +class table_creator { +public: + enum class errc { + incompatible_schema, + // The operation failed because of a subsystem failure. + failed, + // The system is shutting down. + shutting_down, + }; + friend std::ostream& operator<<(std::ostream&, const errc&); + + virtual ss::future> ensure_table( + const model::topic&, + model::revision_id topic_revision, + record_schema_components) const + = 0; + + virtual ~table_creator() = default; +}; + +// Creates or alters the table by interfacing directly with a catalog. +class direct_table_creator : public table_creator { +public: + direct_table_creator(type_resolver&, schema_manager&); + + ss::future> ensure_table( + const model::topic&, + model::revision_id topic_revision, + record_schema_components) const final; + +private: + datalake::type_resolver& type_resolver_; + datalake::schema_manager& schema_mgr_; +}; + +} // namespace datalake From d70c12cd415f6b52d0ede1abcda3fb7dbcb44118 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 21 Nov 2024 16:11:43 +0100 Subject: [PATCH 13/19] datalake: actually ensure table in coordinator::ensure_table_exists --- src/v/datalake/coordinator/BUILD | 8 +++++++ src/v/datalake/coordinator/CMakeLists.txt | 1 + src/v/datalake/coordinator/coordinator.cc | 24 +++++++++++++++++-- src/v/datalake/coordinator/coordinator.h | 5 ++++ .../coordinator/coordinator_manager.cc | 17 +++++++++++-- .../coordinator/coordinator_manager.h | 13 +++++++++- src/v/datalake/coordinator/frontend.cc | 2 ++ src/v/datalake/coordinator/tests/BUILD | 1 + .../coordinator/tests/coordinator_test.cc | 2 ++ .../coordinator/tests/state_machine_test.cc | 2 ++ .../coordinator/tests/state_test_utils.h | 10 ++++++++ src/v/datalake/coordinator/types.cc | 3 +++ src/v/datalake/coordinator/types.h | 1 + src/v/datalake/translation/BUILD | 1 + src/v/redpanda/application.cc | 1 + 15 files changed, 86 insertions(+), 5 deletions(-) diff --git a/src/v/datalake/coordinator/BUILD b/src/v/datalake/coordinator/BUILD index bb0b74819607..a7dbfe9faa0e 100644 --- a/src/v/datalake/coordinator/BUILD +++ b/src/v/datalake/coordinator/BUILD @@ -40,6 +40,8 @@ redpanda_cc_library( "//src/v/base", "//src/v/datalake:logger", "//src/v/datalake:schema_identifier", + "//src/v/datalake:table_creator", + "//src/v/datalake:types", "//src/v/ssx:future_util", "//src/v/ssx:sleep_abortable", "//src/v/storage:record_batch_builder", @@ -69,8 +71,14 @@ redpanda_cc_library( ":iceberg_file_committer", "//src/v/base", "//src/v/config", + "//src/v/datalake:catalog_schema_manager", "//src/v/datalake:logger", + "//src/v/datalake:record_schema_resolver", + "//src/v/datalake:table_creator", + "//src/v/datalake:types", "//src/v/iceberg:filesystem_catalog", + "//src/v/pandaproxy", + "//src/v/schema:registry", "//src/v/storage:record_batch_builder", ], include_prefix = "datalake/coordinator", diff --git a/src/v/datalake/coordinator/CMakeLists.txt b/src/v/datalake/coordinator/CMakeLists.txt index 57a54b7cc731..2d3764958456 100644 --- a/src/v/datalake/coordinator/CMakeLists.txt +++ b/src/v/datalake/coordinator/CMakeLists.txt @@ -26,6 +26,7 @@ v_cc_library( generated_datalake_coordinator_rpc v::cluster v::datalake_common + v::datalake_writer v::model v::rpc Seastar::seastar diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index 6b2372c64317..bc86182aba49 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -14,6 +14,7 @@ #include "container/fragmented_vector.h" #include "datalake/coordinator/state_update.h" #include "datalake/logger.h" +#include "datalake/table_creator.h" #include "model/fundamental.h" #include "model/record_batch_reader.h" #include "ssx/future-util.h" @@ -52,6 +53,8 @@ std::ostream& operator<<(std::ostream& o, coordinator::errc e) { return o << "coordinator::errc::stm_apply_error"; case coordinator::errc::revision_mismatch: return o << "coordinator::errc::revision_mismatch"; + case coordinator::errc::incompatible_schema: + return o << "coordinator::errc::incompatible_schema"; case coordinator::errc::timedout: return o << "coordinator::errc::timedout"; case coordinator::errc::failed: @@ -227,7 +230,7 @@ ss::future> coordinator::sync_ensure_table_exists( model::topic topic, model::revision_id topic_revision, - record_schema_components) { + record_schema_components comps) { auto gate = maybe_gate(); if (gate.has_error()) { co_return gate.error(); @@ -244,6 +247,8 @@ coordinator::sync_ensure_table_exists( co_return convert_stm_errc(sync_res.error()); } + // TODO: add mutex to protect against the thundering herd problem + topic_lifecycle_update update{ .topic = topic, .revision = topic_revision, @@ -272,7 +277,22 @@ coordinator::sync_ensure_table_exists( co_return convert_stm_errc(repl_res.error()); } } - // TODO: actually create table + + // TODO: verify stm state after replication + + auto ensure_res = co_await table_creator_.ensure_table( + topic, topic_revision, std::move(comps)); + if (ensure_res.has_error()) { + switch (ensure_res.error()) { + case table_creator::errc::incompatible_schema: + co_return errc::incompatible_schema; + case table_creator::errc::failed: + co_return errc::failed; + case table_creator::errc::shutting_down: + co_return errc::shutting_down; + } + } + co_return std::nullopt; } diff --git a/src/v/datalake/coordinator/coordinator.h b/src/v/datalake/coordinator/coordinator.h index cd117ed597ef..b43b6fd869b3 100644 --- a/src/v/datalake/coordinator/coordinator.h +++ b/src/v/datalake/coordinator/coordinator.h @@ -15,6 +15,7 @@ #include "datalake/coordinator/file_committer.h" #include "datalake/coordinator/state_machine.h" #include "datalake/coordinator/state_update.h" +#include "datalake/fwd.h" #include "model/fundamental.h" namespace datalake::coordinator { @@ -29,6 +30,7 @@ class coordinator { not_leader, stm_apply_error, revision_mismatch, + incompatible_schema, timedout, shutting_down, failed, @@ -39,11 +41,13 @@ class coordinator { coordinator( ss::shared_ptr stm, cluster::topic_table& topics, + table_creator& table_creator, remove_tombstone_f remove_tombstone, file_committer& file_committer, config::binding commit_interval) : stm_(std::move(stm)) , topic_table_(topics) + , table_creator_(table_creator) , remove_tombstone_(std::move(remove_tombstone)) , file_committer_(file_committer) , commit_interval_(std::move(commit_interval)) {} @@ -88,6 +92,7 @@ class coordinator { ss::shared_ptr stm_; cluster::topic_table& topic_table_; + table_creator& table_creator_; remove_tombstone_f remove_tombstone_; file_committer& file_committer_; config::binding commit_interval_; diff --git a/src/v/datalake/coordinator/coordinator_manager.cc b/src/v/datalake/coordinator/coordinator_manager.cc index 8c10589f815d..7660733c4d91 100644 --- a/src/v/datalake/coordinator/coordinator_manager.cc +++ b/src/v/datalake/coordinator/coordinator_manager.cc @@ -12,14 +12,17 @@ #include "cluster/partition_manager.h" #include "cluster/topics_frontend.h" #include "config/configuration.h" +#include "datalake/catalog_schema_manager.h" #include "datalake/coordinator/catalog_factory.h" #include "datalake/coordinator/coordinator.h" #include "datalake/coordinator/iceberg_file_committer.h" #include "datalake/coordinator/state_machine.h" #include "datalake/logger.h" -#include "iceberg/filesystem_catalog.h" +#include "datalake/record_schema_resolver.h" +#include "datalake/table_creator.h" #include "iceberg/manifest_io.h" #include "model/fundamental.h" +#include "schema/registry.h" #include @@ -31,6 +34,7 @@ coordinator_manager::coordinator_manager( ss::sharded& pm, ss::sharded& topics, ss::sharded& topics_fe, + pandaproxy::schema_registry::api* sr_api, std::unique_ptr catalog_factory, ss::sharded& io, cloud_storage_clients::bucket_name bucket) @@ -39,11 +43,19 @@ coordinator_manager::coordinator_manager( , pm_(pm.local()) , topics_(topics.local()) , topics_fe_(topics_fe) + , schema_registry_(schema::registry::make_default(sr_api)) , manifest_io_(io.local(), bucket) - , catalog_factory_(std::move(catalog_factory)) {} + , catalog_factory_(std::move(catalog_factory)) + , type_resolver_( + std::make_unique(*schema_registry_)) {} + +coordinator_manager::~coordinator_manager() = default; ss::future<> coordinator_manager::start() { catalog_ = co_await catalog_factory_->create_catalog(); + schema_mgr_ = std::make_unique(*catalog_); + table_creator_ = std::make_unique( + *type_resolver_, *schema_mgr_); file_committer_ = std::make_unique( *catalog_, manifest_io_); @@ -108,6 +120,7 @@ void coordinator_manager::start_managing(cluster::partition& p) { auto crd = ss::make_lw_shared( std::move(stm), topics_, + *table_creator_, [this](const model::topic& t, model::revision_id rev) { return remove_tombstone(t, rev); }, diff --git a/src/v/datalake/coordinator/coordinator_manager.h b/src/v/datalake/coordinator/coordinator_manager.h index 0b18cf06f591..7752afe5c3f1 100644 --- a/src/v/datalake/coordinator/coordinator_manager.h +++ b/src/v/datalake/coordinator/coordinator_manager.h @@ -13,9 +13,10 @@ #include "cluster/fwd.h" #include "cluster/notification.h" #include "datalake/coordinator/coordinator.h" -#include "datalake/coordinator/file_committer.h" +#include "datalake/fwd.h" #include "iceberg/catalog.h" #include "model/fundamental.h" +#include "pandaproxy/schema_registry/fwd.h" #include "raft/fwd.h" #include "raft/notification.h" @@ -24,6 +25,10 @@ #include #include +namespace schema { +class registry; +} + namespace datalake::coordinator { class catalog_factory; @@ -37,9 +42,11 @@ class coordinator_manager { ss::sharded&, ss::sharded&, ss::sharded&, + pandaproxy::schema_registry::api*, std::unique_ptr, ss::sharded&, cloud_storage_clients::bucket_name); + ~coordinator_manager(); ss::future<> start(); ss::future<> stop(); @@ -62,11 +69,15 @@ class coordinator_manager { cluster::partition_manager& pm_; cluster::topic_table& topics_; ss::sharded& topics_fe_; + std::unique_ptr schema_registry_; // Underlying IO is expected to outlive this class. iceberg::manifest_io manifest_io_; std::unique_ptr catalog_factory_; std::unique_ptr catalog_; + std::unique_ptr schema_mgr_; + std::unique_ptr type_resolver_; + std::unique_ptr table_creator_; std::unique_ptr file_committer_; std::optional manage_notifications_; diff --git a/src/v/datalake/coordinator/frontend.cc b/src/v/datalake/coordinator/frontend.cc index d1d8ebb79dec..45fe4241f336 100644 --- a/src/v/datalake/coordinator/frontend.cc +++ b/src/v/datalake/coordinator/frontend.cc @@ -34,6 +34,8 @@ errc to_rpc_errc(coordinator::errc e) { return errc::stale; case coordinator::errc::revision_mismatch: return errc::revision_mismatch; + case coordinator::errc::incompatible_schema: + return errc::incompatible_schema; case coordinator::errc::timedout: return errc::timeout; case coordinator::errc::failed: diff --git a/src/v/datalake/coordinator/tests/BUILD b/src/v/datalake/coordinator/tests/BUILD index b60486f53c20..827fea3c54e9 100644 --- a/src/v/datalake/coordinator/tests/BUILD +++ b/src/v/datalake/coordinator/tests/BUILD @@ -11,6 +11,7 @@ redpanda_test_cc_library( include_prefix = "datalake/coordinator/tests", deps = [ "//src/v/container:fragmented_vector", + "//src/v/datalake:table_creator", "//src/v/datalake/coordinator:file_committer", "//src/v/datalake/coordinator:state", "//src/v/datalake/coordinator:translated_offset_range", diff --git a/src/v/datalake/coordinator/tests/coordinator_test.cc b/src/v/datalake/coordinator/tests/coordinator_test.cc index 67c36322bdb3..53ec636c10cf 100644 --- a/src/v/datalake/coordinator/tests/coordinator_test.cc +++ b/src/v/datalake/coordinator/tests/coordinator_test.cc @@ -65,6 +65,7 @@ struct coordinator_node { , crd( stm, topic_table, + table_creator, [this](const model::topic& t, model::revision_id r) { return remove_tombstone(t, r); }, @@ -80,6 +81,7 @@ struct coordinator_node { config::mock_property commit_interval_ms; cluster::data_migrations::migrated_resources mr; cluster::topic_table topic_table; + noop_table_creator table_creator; std::unique_ptr file_committer; coordinator crd; }; diff --git a/src/v/datalake/coordinator/tests/state_machine_test.cc b/src/v/datalake/coordinator/tests/state_machine_test.cc index ac9471ed0672..1967f80ad95d 100644 --- a/src/v/datalake/coordinator/tests/state_machine_test.cc +++ b/src/v/datalake/coordinator/tests/state_machine_test.cc @@ -50,6 +50,7 @@ struct coordinator_stm_fixture : stm_raft_fixture { = std::make_unique( get_stm<0>(node), topic_table, + table_creator, [this](const model::topic& t, model::revision_id r) { return remove_tombstone(t, r); }, @@ -134,6 +135,7 @@ struct coordinator_stm_fixture : stm_raft_fixture { model::revision_id rev{123}; cluster::data_migrations::migrated_resources mr; cluster::topic_table topic_table{mr}; + datalake::coordinator::noop_table_creator table_creator; datalake::coordinator::simple_file_committer file_committer; absl::flat_hash_map coordinators; }; diff --git a/src/v/datalake/coordinator/tests/state_test_utils.h b/src/v/datalake/coordinator/tests/state_test_utils.h index 9ab35a7cf8dc..c4c98f7df90e 100644 --- a/src/v/datalake/coordinator/tests/state_test_utils.h +++ b/src/v/datalake/coordinator/tests/state_test_utils.h @@ -13,6 +13,7 @@ #include "datalake/coordinator/file_committer.h" #include "datalake/coordinator/state.h" #include "datalake/coordinator/translated_offset_range.h" +#include "datalake/table_creator.h" #include "model/fundamental.h" #include @@ -21,6 +22,15 @@ namespace datalake::coordinator { +class noop_table_creator : public table_creator { + ss::future> ensure_table( + const model::topic&, + model::revision_id, + record_schema_components) const final { + co_return std::nullopt; + } +}; + // Simple committer that returns the set of updates that would mark all the // pending files as committed. Doesn't affect any external state. class simple_file_committer : public file_committer { diff --git a/src/v/datalake/coordinator/types.cc b/src/v/datalake/coordinator/types.cc index 5aeccb363146..f9bccad12482 100644 --- a/src/v/datalake/coordinator/types.cc +++ b/src/v/datalake/coordinator/types.cc @@ -40,6 +40,9 @@ std::ostream& operator<<(std::ostream& o, const errc& errc) { case errc::revision_mismatch: o << "errc::revision_mismatch"; break; + case errc::incompatible_schema: + o << "errc::incompatible_schema"; + break; case errc::failed: o << "errc::failed"; break; diff --git a/src/v/datalake/coordinator/types.h b/src/v/datalake/coordinator/types.h index 5ed75b524297..b802ac9a49aa 100644 --- a/src/v/datalake/coordinator/types.h +++ b/src/v/datalake/coordinator/types.h @@ -29,6 +29,7 @@ enum class errc : int16_t { stale, concurrent_requests, revision_mismatch, + incompatible_schema, failed, }; diff --git a/src/v/datalake/translation/BUILD b/src/v/datalake/translation/BUILD index 2cfccaac6a1f..3479eac9455e 100644 --- a/src/v/datalake/translation/BUILD +++ b/src/v/datalake/translation/BUILD @@ -54,6 +54,7 @@ redpanda_cc_library( "//src/v/datalake:record_schema_resolver", "//src/v/datalake:record_translator", "//src/v/datalake:serde_parquet_writer", + "//src/v/datalake:table_creator", "//src/v/datalake:translation_task", "//src/v/datalake:types", "//src/v/datalake:writer", diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 501a435c5ce1..f5e07407c6b4 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -1437,6 +1437,7 @@ void application::wire_up_runtime_services( std::ref(partition_manager), std::ref(controller->get_topics_state()), std::ref(controller->get_topics_frontend()), + _schema_registry.get(), ss::sharded_parameter( [bucket](cloud_io::remote& remote) -> std::unique_ptr { From cfce9c0d366396a174b671dd4e6ec7f3c73f0b5d Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 21 Nov 2024 16:46:54 +0100 Subject: [PATCH 14/19] datalake: call ensure_table_exists from record_multiplexer Also make coordinator stm transitions more strict: we now require a topic state to add files and don't create one automatically. Also remove table creation from the file commit path (it is expected that the table is created separately first). --- src/v/datalake/BUILD | 2 + .../coordinator/iceberg_file_committer.cc | 9 ++--- .../coordinator/iceberg_file_committer.h | 4 +- src/v/datalake/coordinator/state_update.cc | 40 +++++++------------ .../coordinator/tests/coordinator_test.cc | 28 +++++++++++-- .../tests/iceberg_file_committer_test.cc | 32 +++++++++++++-- .../coordinator/tests/state_machine_test.cc | 25 ++++++++---- .../coordinator/tests/state_update_test.cc | 25 +++++++++++- src/v/datalake/record_multiplexer.cc | 23 ++++++----- src/v/datalake/record_multiplexer.h | 7 +++- src/v/datalake/tests/BUILD | 1 + .../tests/gtest_record_multiplexer_test.cc | 27 ++++++++++--- .../datalake/tests/record_multiplexer_test.cc | 10 ++++- src/v/datalake/tests/translation_task_test.cc | 13 ++++-- .../translation/partition_translator.cc | 40 ++++++++++++++++++- .../translation/partition_translator.h | 1 + src/v/datalake/translation_task.cc | 11 +++-- src/v/datalake/translation_task.h | 5 ++- 18 files changed, 229 insertions(+), 74 deletions(-) diff --git a/src/v/datalake/BUILD b/src/v/datalake/BUILD index 40cf070e1fa0..362f3089cbd5 100644 --- a/src/v/datalake/BUILD +++ b/src/v/datalake/BUILD @@ -342,6 +342,7 @@ redpanda_cc_library( ":logger", ":record_schema_resolver", ":record_translator", + ":table_creator", "//src/v/base", "//src/v/storage:parser_utils", "//src/v/utils:prefix_logger", @@ -351,6 +352,7 @@ redpanda_cc_library( deps = [ ":partitioning_writer", ":schema_identifier", + ":types", ":writer", "//src/v/container:chunked_hash_map", "//src/v/model", diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index d5b232c74bcf..ec1c23ea5b3c 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -109,7 +109,7 @@ iceberg_file_committer::commit_topic_files_to_catalog( } auto table_id = table_id_for_topic(topic); - auto table_res = co_await load_or_create_table(table_id); + auto table_res = co_await load_table(table_id); if (table_res.has_error()) { co_return table_res.error(); } @@ -232,13 +232,12 @@ iceberg_file_committer::table_id_for_topic(const model::topic& t) const { } ss::future> -iceberg_file_committer::load_or_create_table( +iceberg_file_committer::load_table( const iceberg::table_identifier& table_id) const { - auto res = co_await catalog_.load_or_create_table( - table_id, schemaless_struct_type(), hour_partition_spec()); + auto res = co_await catalog_.load_table(table_id); if (res.has_error()) { co_return log_and_convert_catalog_errc( - res.error(), fmt::format("Failed to load or create {}", table_id)); + res.error(), fmt::format("Failed to load table {}", table_id)); } co_return std::move(res.value()); } diff --git a/src/v/datalake/coordinator/iceberg_file_committer.h b/src/v/datalake/coordinator/iceberg_file_committer.h index fe49823a604f..281eec1848d7 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.h +++ b/src/v/datalake/coordinator/iceberg_file_committer.h @@ -54,10 +54,8 @@ class iceberg_file_committer : public file_committer { // TODO: pull this out into some helper? Seems useful for other actions. iceberg::table_identifier table_id_for_topic(const model::topic& t) const; - // Loads the table from the catalog, or creates a table with a default - // schema and default partition spec. ss::future> - load_or_create_table(const iceberg::table_identifier&) const; + load_table(const iceberg::table_identifier&) const; // Must outlive this committer. iceberg::catalog& catalog_; diff --git a/src/v/datalake/coordinator/state_update.cc b/src/v/datalake/coordinator/state_update.cc index 16c6d93e740b..8c0a4e99efcd 100644 --- a/src/v/datalake/coordinator/state_update.cc +++ b/src/v/datalake/coordinator/state_update.cc @@ -51,30 +51,17 @@ add_files_update::can_apply(const topics_state& state) { } auto topic_it = state.topic_to_state.find(tp.topic); if (topic_it == state.topic_to_state.end()) { - return std::nullopt; + return stm_update_error{fmt::format( + "topic {} rev {} not yet registered", tp.topic, topic_revision)}; } auto& cur_topic = topic_it->second; - if (topic_revision < cur_topic.revision) { + if (topic_revision != cur_topic.revision) { return stm_update_error{fmt::format( - "topic {} rev {} not yet registered (current rev {})", + "topic {} revision mismatch (update rev {}, current rev {})", tp.topic, topic_revision, cur_topic.revision)}; - } else if (topic_revision > cur_topic.revision) { - if ( - cur_topic.lifecycle_state != topic_state::lifecycle_state_t::purged) { - return stm_update_error{fmt::format( - "topic {} rev {} not yet purged (new topic rev {})", - tp.topic, - cur_topic.revision, - topic_revision)}; - } - - // Previous topic instance has been fully purged, so we are ready to - // accept files for an instance with the higher revision id. - return std::nullopt; } - if (cur_topic.lifecycle_state != topic_state::lifecycle_state_t::live) { return stm_update_error{fmt::format( "topic {} rev {} already closed", tp.topic, cur_topic.revision)}; @@ -120,14 +107,15 @@ add_files_update::apply(topics_state& state, model::offset applied_offset) { const auto& pid = tp.partition; auto& tp_state = state.topic_to_state[topic]; - if (topic_revision > tp_state.revision) { - // We've got files for a topic instance with higher revision id, reset - // topic state - topic_state new_state; - new_state.revision = topic_revision; - tp_state = std::move(new_state); - } - // after this point tp_state.revision == topic_revision and state == live + vassert( + tp_state.revision == topic_revision + && tp_state.lifecycle_state == topic_state::lifecycle_state_t::live, + "topic {} unexpected state (rev {} lc_state {}) (expected rev {})", + topic, + tp_state.revision, + tp_state.lifecycle_state, + topic_revision); + auto& partition_state = tp_state.pid_to_pending_files[pid]; for (auto& e : entries) { partition_state.pending_entries.emplace_back(pending_entry{ @@ -247,6 +235,8 @@ topic_lifecycle_update::can_apply(const topics_state& state) { revision)}; } + // Old revision is fully purged, we can now transition to the new + // revision. return true; } diff --git a/src/v/datalake/coordinator/tests/coordinator_test.cc b/src/v/datalake/coordinator/tests/coordinator_test.cc index 53ec636c10cf..06c0905bd065 100644 --- a/src/v/datalake/coordinator/tests/coordinator_test.cc +++ b/src/v/datalake/coordinator/tests/coordinator_test.cc @@ -77,6 +77,14 @@ struct coordinator_node { co_return std::nullopt; } + void ensure_table(const model::topic& topic, model::revision_id rev) { + auto res = crd + .sync_ensure_table_exists( + topic, rev, datalake::record_schema_components{}) + .get(); + ASSERT_FALSE(res.has_error()) << res.error(); + } + coordinator_stm& stm; config::mock_property commit_interval_ms; cluster::data_migrations::migrated_resources mr; @@ -113,6 +121,11 @@ ss::future<> file_adder_loop( if (last_res.has_error()) { continue; } + auto ensure_res = co_await n.crd.sync_ensure_table_exists( + tp.topic, topic_rev, datalake::record_schema_components{}); + if (ensure_res.has_error()) { + continue; + } auto cur_last_opt = last_res.value(); while (true) { co_await random_sleep_ms(30); @@ -314,6 +327,8 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { const model::revision_id rev0{1}; const auto tp10 = tp(1, 0); const model::revision_id rev1{2}; + + leader.ensure_table(tp00.topic, rev0); pairs_t total_expected_00; for (const auto& v : { pairs_t{{0, 100}}, @@ -333,6 +348,7 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { c->stm.state(), tp00, std::nullopt, total_expected_00)); } } + // Now try adding to a different partition of the same topic. pairs_t total_expected_01; for (const auto& v : {pairs_t{{0, 100}}, pairs_t{{101, 200}}}) { @@ -350,6 +366,7 @@ TEST_F(CoordinatorTest, TestAddFilesHappyPath) { } } // And finally a different topic entirely. + leader.ensure_table(tp10.topic, rev1); pairs_t total_expected_10; for (const auto& v : {pairs_t{{100, 200}}, pairs_t{{201, 300}}}) { auto add_res @@ -376,6 +393,7 @@ TEST_F(CoordinatorTest, TestLastAddedHappyPath) { const auto tp00 = tp(0, 0); const auto tp01 = tp(0, 1); const model::revision_id rev{1}; + leader.ensure_table(tp00.topic, rev); pairs_t total_expected_00; for (const auto& v : {pairs_t{{101, 200}}, pairs_t{{201, 300}, {301, 400}}}) { @@ -408,6 +426,7 @@ TEST_F(CoordinatorTest, TestNotLeader) { auto& non_leader = non_leader_opt->get(); const auto tp00 = tp(0, 0); const model::revision_id rev{1}; + leader_opt.value().get().ensure_table(tp00.topic, rev); pairs_t total_expected_00; auto add_res = non_leader.crd @@ -543,6 +562,7 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesHappyPath) { auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); const model::revision_id rev0{1}; + leader.ensure_table(tp00.topic, rev0); auto add_res = leader.crd .sync_add_files(tp00, rev0, make_pending_files({{0, 100}})) .get(); @@ -572,6 +592,7 @@ TEST_F(CoordinatorLoopTest, TestCommitFilesNotLeader) { auto& leader = leader_opt->get(); const auto tp00 = tp(0, 0); const model::revision_id rev0{1}; + leader.ensure_table(tp00.topic, rev0); auto add_res = leader.crd .sync_add_files(tp00, rev0, make_pending_files({{0, 100}})) .get(); @@ -651,11 +672,10 @@ TEST_F(CoordinatorSleepingLoopTest, TestQuickShutdownOnLeadershipChange) { auto& leader = leader_opt->get(); for (int i = 0; i < 100; i++) { auto t = tp(i, 0); + auto rev = model::revision_id{i}; + leader.ensure_table(t.topic, rev); auto add_res = leader.crd - .sync_add_files( - t, - model::revision_id{i}, - make_pending_files({{0, 100}})) + .sync_add_files(t, rev, make_pending_files({{0, 100}})) .get(); ASSERT_FALSE(add_res.has_error()) << add_res.error(); } diff --git a/src/v/datalake/coordinator/tests/iceberg_file_committer_test.cc b/src/v/datalake/coordinator/tests/iceberg_file_committer_test.cc index f6f6d91d11dc..802a871db916 100644 --- a/src/v/datalake/coordinator/tests/iceberg_file_committer_test.cc +++ b/src/v/datalake/coordinator/tests/iceberg_file_committer_test.cc @@ -90,19 +90,30 @@ class FileCommitterTest FileCommitterTest() : sr(cloud_io::scoped_remote::create(10, conf)) , catalog(remote(), bucket_name, ss::sstring(base_location)) + , schema_mgr(catalog) , manifest_io(remote(), bucket_name) , committer(catalog, manifest_io) { set_expectations_and_listen({}); } cloud_io::remote& remote() { return sr->remote.local(); } + void create_table() { + auto res = schema_mgr + .ensure_table_schema( + topic, datalake::schemaless_struct_type()) + .get(); + ASSERT_FALSE(res.has_error()); + } + std::unique_ptr sr; iceberg::filesystem_catalog catalog; + datalake::catalog_schema_manager schema_mgr; iceberg::manifest_io manifest_io; iceberg_file_committer committer; }; TEST_F(FileCommitterTest, TestCommit) { + create_table(); topics_state state; state.topic_to_state[topic] = make_topic_state({ {{0, 99}, {100, 199}}, @@ -125,7 +136,7 @@ TEST_F(FileCommitterTest, TestCommit) { ASSERT_EQ(updates[2].new_committed(), 299); } -TEST_F(FileCommitterTest, TestLoadOrCreateTable) { +TEST_F(FileCommitterTest, TestMissingTable) { auto load_res = catalog .load_table( iceberg::table_identifier{{"redpanda"}, "test-topic"}) @@ -137,17 +148,22 @@ TEST_F(FileCommitterTest, TestLoadOrCreateTable) { topics_state state; state.topic_to_state[topic] = make_topic_state({}); + // Requires a table (which is not created yet) auto res = committer.commit_topic_files_to_catalog(topic, state).get(); + ASSERT_TRUE(res.has_error()); + + create_table(); + + res = committer.commit_topic_files_to_catalog(topic, state).get(); ASSERT_FALSE(res.has_error()); ASSERT_TRUE(res.value().empty()); load_res = catalog .load_table( iceberg::table_identifier{{"redpanda"}, "test-topic"}) .get(); - ASSERT_FALSE(load_res.value().snapshots.has_value()); - // The table should be created. ASSERT_FALSE(load_res.has_error()); + ASSERT_FALSE(load_res.value().snapshots.has_value()); // Now try again with some data. state.topic_to_state[topic] = make_topic_state({{{0, 100}}}); @@ -185,6 +201,8 @@ TEST_F(FileCommitterTest, TestMissingTopic) { } TEST_F(FileCommitterTest, TestFilesGetPartitionKey) { + create_table(); + using namespace iceberg; // Constructs topic state with offset ranges added to partition 0. model::offset added_at_counter{1000}; @@ -272,6 +290,8 @@ TEST_F(FileCommitterTest, TestFilesGetPartitionKey) { // Test that deduplication happens when all of the pending files are already // committed to Iceberg. TEST_F(FileCommitterTest, TestDeduplicateAllFiles) { + create_table(); + topics_state state; state.topic_to_state[topic] = make_topic_state( { @@ -322,6 +342,8 @@ TEST_F(FileCommitterTest, TestDeduplicateAllFiles) { // Test that deduplication happens when some of the pending files are already // committed to Iceberg. TEST_F(FileCommitterTest, TestDeduplicateSomeFiles) { + create_table(); + topics_state state; state.topic_to_state[topic] = make_topic_state( {{{0, 99}, {100, 199}}}, model::offset{1000}, true); @@ -371,6 +393,8 @@ TEST_F(FileCommitterTest, TestDeduplicateSomeFiles) { // Test that deduplication happens when the Iceberg commit metadata is not in // the latest snapshot. TEST_F(FileCommitterTest, TestDeduplicateFromAncestor) { + create_table(); + topics_state state; state.topic_to_state[topic] = make_topic_state( { @@ -437,6 +461,8 @@ TEST_F(FileCommitterTest, TestDeduplicateFromAncestor) { } TEST_F(FileCommitterTest, TestDeduplicateConcurrently) { + create_table(); + std::vector> committers; constexpr auto num_committers = 10; constexpr auto num_chunks = 50; diff --git a/src/v/datalake/coordinator/tests/state_machine_test.cc b/src/v/datalake/coordinator/tests/state_machine_test.cc index 1967f80ad95d..0abe1b477b2d 100644 --- a/src/v/datalake/coordinator/tests/state_machine_test.cc +++ b/src/v/datalake/coordinator/tests/state_machine_test.cc @@ -169,13 +169,24 @@ TEST_F_CORO(coordinator_stm_fixture, test_snapshots) { next_offset = next_offset + 6; } return coordinator - ->sync_add_files( - tp, - rev, - datalake::coordinator::make_pending_files(offset_pairs)) - .then([](auto result) { - return ss::make_ready_future( - result.has_value()); + ->sync_ensure_table_exists( + tp.topic, rev, datalake::record_schema_components{}) + .then([this, tp, offset_pairs, &coordinator]( + auto ensure_res) { + if (!ensure_res) { + return ss::make_ready_future(false); + } + + return coordinator + ->sync_add_files( + tp, + rev, + datalake::coordinator::make_pending_files( + offset_pairs)) + .then([](auto result) { + return ss::make_ready_future( + result.has_value()); + }); }); }); }); diff --git a/src/v/datalake/coordinator/tests/state_update_test.cc b/src/v/datalake/coordinator/tests/state_update_test.cc index 15e11bb1174e..5956ed58ea80 100644 --- a/src/v/datalake/coordinator/tests/state_update_test.cc +++ b/src/v/datalake/coordinator/tests/state_update_test.cc @@ -26,6 +26,18 @@ const model::revision_id rev{123}; const model::partition_id pid{0}; const model::topic_partition tp{topic, pid}; +checked apply_lc_transition( + topics_state& state, + model::revision_id rev, + topic_state::lifecycle_state_t new_state) { + topic_lifecycle_update upd{ + .topic = topic, + .revision = rev, + .new_state = new_state, + }; + return upd.apply(state); +} + // Asserts that the given ranges can't be applied to the given partition state. void check_add_doesnt_apply( topics_state& state, @@ -64,11 +76,20 @@ void check_commit_doesnt_apply( TEST(StateUpdateTest, TestAddFile) { topics_state state; + + // We can't add files to a topic or partition that isn't yet tracked. + ASSERT_TRUE( + add_files_update::build(state, tp, rev, make_pending_files({{0, 100}})) + .has_error()); + + // create table state + ASSERT_FALSE( + apply_lc_transition(state, rev, topic_state::lifecycle_state_t::live) + .has_error()); + auto update = add_files_update::build( state, tp, rev, make_pending_files({{0, 100}})); - // We can always add files to a topic or partition that isn't yet tracked. ASSERT_FALSE(update.has_error()); - EXPECT_FALSE(state.partition_state(tp).has_value()); // Now apply the update and check that we have the expected tracked file. auto res = update.value().apply(state, model::offset{}); diff --git a/src/v/datalake/record_multiplexer.cc b/src/v/datalake/record_multiplexer.cc index 79fdfbe4821d..f377d33faab4 100644 --- a/src/v/datalake/record_multiplexer.cc +++ b/src/v/datalake/record_multiplexer.cc @@ -15,6 +15,7 @@ #include "datalake/logger.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/table_creator.h" #include "model/record.h" #include "storage/parser_utils.h" @@ -24,16 +25,20 @@ namespace datalake { record_multiplexer::record_multiplexer( const model::ntp& ntp, + model::revision_id topic_revision, std::unique_ptr writer_factory, schema_manager& schema_mgr, type_resolver& type_resolver, - record_translator& record_translator) + record_translator& record_translator, + table_creator& table_creator) : _log(datalake_log, fmt::format("{}", ntp)) , _ntp(ntp) + , _topic_revision(topic_revision) , _writer_factory{std::move(writer_factory)} , _schema_mgr(schema_mgr) , _type_resolver(type_resolver) - , _record_translator(record_translator) {} + , _record_translator(record_translator) + , _table_creator(table_creator) {} ss::future record_multiplexer::operator()(model::record_batch batch) { @@ -115,12 +120,12 @@ record_multiplexer::operator()(model::record_batch batch) { std::move(val_type_res.value().type)); auto writer_iter = _writers.find(record_type.comps); if (writer_iter == _writers.end()) { - auto ensure_res = co_await _schema_mgr.ensure_table_schema( - _ntp.tp.topic, record_type.type); + auto ensure_res = co_await _table_creator.ensure_table( + _ntp.tp.topic, _topic_revision, record_type.comps); if (ensure_res.has_error()) { auto e = ensure_res.error(); switch (e) { - case schema_manager::errc::not_supported: { + case table_creator::errc::incompatible_schema: { auto invalid_res = co_await handle_invalid_record( offset, record.share_key(), @@ -133,13 +138,13 @@ record_multiplexer::operator()(model::record_batch batch) { } continue; } - case schema_manager::errc::failed: + case table_creator::errc::failed: vlog( _log.warn, "Error ensuring table schema for record {}", offset); [[fallthrough]]; - case schema_manager::errc::shutting_down: + case table_creator::errc::shutting_down: _error = writer_error::parquet_conversion_error; } co_return ss::stop_iteration::yes; @@ -257,8 +262,8 @@ record_multiplexer::handle_invalid_record( // TODO: maybe this should be a writer specific for a dead-letter queue. auto writer_iter = _writers.find(record_type.comps); if (writer_iter == _writers.end()) { - auto ensure_res = co_await _schema_mgr.ensure_table_schema( - _ntp.tp.topic, record_type.type); + auto ensure_res = co_await _table_creator.ensure_table( + _ntp.tp.topic, _topic_revision, record_type.comps); if (ensure_res.has_error()) { vlog( _log.warn, diff --git a/src/v/datalake/record_multiplexer.h b/src/v/datalake/record_multiplexer.h index 5eb95582bcdf..c9950c0233a7 100644 --- a/src/v/datalake/record_multiplexer.h +++ b/src/v/datalake/record_multiplexer.h @@ -11,6 +11,7 @@ #include "container/chunked_hash_map.h" #include "datalake/data_writer_interface.h" +#include "datalake/fwd.h" #include "datalake/partitioning_writer.h" #include "datalake/schema_identifier.h" #include "model/record.h" @@ -41,10 +42,12 @@ class record_multiplexer { }; explicit record_multiplexer( const model::ntp& ntp, + model::revision_id topic_revision, std::unique_ptr writer, schema_manager& schema_mgr, type_resolver& type_resolver, - record_translator& record_translator); + record_translator& record_translator, + table_creator&); ss::future operator()(model::record_batch batch); ss::future> end_of_stream(); @@ -63,10 +66,12 @@ class record_multiplexer { prefix_logger _log; const model::ntp& _ntp; + model::revision_id _topic_revision; std::unique_ptr _writer_factory; schema_manager& _schema_mgr; type_resolver& _type_resolver; record_translator& _record_translator; + table_creator& _table_creator; chunked_hash_map< record_schema_components, std::unique_ptr> diff --git a/src/v/datalake/tests/BUILD b/src/v/datalake/tests/BUILD index 4a1ec08d39af..aa2ed91c75f7 100644 --- a/src/v/datalake/tests/BUILD +++ b/src/v/datalake/tests/BUILD @@ -213,6 +213,7 @@ redpanda_cc_gtest( "//src/v/datalake:record_multiplexer", "//src/v/datalake:record_schema_resolver", "//src/v/datalake:record_translator", + "//src/v/datalake:table_creator", "//src/v/datalake:table_definition", "//src/v/iceberg:filesystem_catalog", "//src/v/model", diff --git a/src/v/datalake/tests/gtest_record_multiplexer_test.cc b/src/v/datalake/tests/gtest_record_multiplexer_test.cc index 37332f297182..27d48a7a783f 100644 --- a/src/v/datalake/tests/gtest_record_multiplexer_test.cc +++ b/src/v/datalake/tests/gtest_record_multiplexer_test.cc @@ -14,6 +14,7 @@ #include "datalake/record_multiplexer.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/table_creator.h" #include "datalake/tests/catalog_and_registry_fixture.h" #include "datalake/tests/record_generator.h" #include "datalake/tests/test_data_writer.h" @@ -35,8 +36,10 @@ using namespace datalake; namespace { simple_schema_manager simple_schema_mgr; binary_type_resolver bin_resolver; +direct_table_creator t_creator{bin_resolver, simple_schema_mgr}; const model::ntp ntp(model::ns{"rp"}, model::topic{"t"}, model::partition_id{0}); +const model::revision_id rev{123}; default_translator translator; } // namespace @@ -48,10 +51,12 @@ TEST(DatalakeMultiplexerTest, TestMultiplexer) { false); datalake::record_multiplexer multiplexer( ntp, + rev, std::move(writer_factory), simple_schema_mgr, bin_resolver, - translator); + translator, + t_creator); model::test::record_batch_spec batch_spec; batch_spec.records = record_count; @@ -86,10 +91,12 @@ TEST(DatalakeMultiplexerTest, TestMultiplexerWriteError) { true); datalake::record_multiplexer multiplexer( ntp, + rev, std::move(writer_factory), simple_schema_mgr, bin_resolver, - translator); + translator, + t_creator); model::test::record_batch_spec batch_spec; batch_spec.records = record_count; @@ -125,10 +132,12 @@ TEST(DatalakeMultiplexerTest, WritesDataFiles) { datalake::record_multiplexer multiplexer( ntp, + rev, std::move(writer_factory), simple_schema_mgr, bin_resolver, - translator); + translator, + t_creator); model::test::record_batch_spec batch_spec; batch_spec.records = record_count; @@ -218,9 +227,11 @@ class RecordMultiplexerParquetTest public: RecordMultiplexerParquetTest() : schema_mgr(catalog) - , type_resolver(registry) {} + , type_resolver(registry) + , t_creator(type_resolver, schema_mgr) {} catalog_schema_manager schema_mgr; record_schema_resolver type_resolver; + direct_table_creator t_creator; }; TEST_F(RecordMultiplexerParquetTest, TestSimple) { @@ -265,7 +276,13 @@ TEST_F(RecordMultiplexerParquetTest, TestSimple) { "data", ss::make_shared(100, 10000)); record_multiplexer mux( - ntp, std::move(writer_factory), schema_mgr, type_resolver, translator); + ntp, + rev, + std::move(writer_factory), + schema_mgr, + type_resolver, + translator, + t_creator); auto res = reader.consume(std::move(mux), model::no_timeout).get(); ASSERT_FALSE(res.has_error()) << res.error(); EXPECT_EQ(res.value().start_offset(), start_offset()); diff --git a/src/v/datalake/tests/record_multiplexer_test.cc b/src/v/datalake/tests/record_multiplexer_test.cc index 6ae3cb2d8f93..ca090115fa87 100644 --- a/src/v/datalake/tests/record_multiplexer_test.cc +++ b/src/v/datalake/tests/record_multiplexer_test.cc @@ -11,6 +11,7 @@ #include "datalake/record_multiplexer.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/table_creator.h" #include "datalake/table_definition.h" #include "datalake/tests/catalog_and_registry_fixture.h" #include "datalake/tests/record_generator.h" @@ -32,6 +33,7 @@ namespace { structured_data_translator translator; const model::ntp ntp(model::ns{"rp"}, model::topic{"t"}, model::partition_id{0}); +const model::revision_id topic_rev{123}; // v1: struct field with one field. constexpr std::string_view avro_schema_v1_str = R"({ "type": "record", @@ -96,7 +98,8 @@ class RecordMultiplexerTestBase public: RecordMultiplexerTestBase() : schema_mgr(catalog) - , type_resolver(registry) {} + , type_resolver(registry) + , t_creator(type_resolver, schema_mgr) {} // Runs the multiplexer on records generated with cb() based on the test // parameters. @@ -130,10 +133,12 @@ class RecordMultiplexerTestBase std::move(batches)); record_multiplexer mux( ntp, + topic_rev, std::make_unique(false), schema_mgr, type_resolver, - translator); + translator, + t_creator); auto res = reader.consume(std::move(mux), model::no_timeout).get(); if (expect_error) { EXPECT_TRUE(res.has_error()); @@ -170,6 +175,7 @@ class RecordMultiplexerTestBase catalog_schema_manager schema_mgr; record_schema_resolver type_resolver; + direct_table_creator t_creator; static constexpr records_param default_param = { .records_per_batch = 1, diff --git a/src/v/datalake/tests/translation_task_test.cc b/src/v/datalake/tests/translation_task_test.cc index ecfa0ad8d919..69aef68f782a 100644 --- a/src/v/datalake/tests/translation_task_test.cc +++ b/src/v/datalake/tests/translation_task_test.cc @@ -16,6 +16,7 @@ #include "datalake/local_parquet_file_writer.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/table_creator.h" #include "datalake/translation_task.h" #include "model/record_batch_reader.h" #include "storage/record_batch_builder.h" @@ -33,7 +34,10 @@ namespace { auto schema_mgr = std::make_unique(); auto schema_resolver = std::make_unique(); auto translator = std::make_unique(); +auto t_creator = std::make_unique( + *schema_resolver, *schema_mgr); const auto ntp = model::ntp{}; +const auto rev = model::revision_id{123}; } // namespace class TranslateTaskTest @@ -160,11 +164,12 @@ struct deleter { TEST_F(TranslateTaskTest, TestHappyPathTranslation) { datalake::translation_task task( - cloud_io, *schema_mgr, *schema_resolver, *translator); + cloud_io, *schema_mgr, *schema_resolver, *translator, *t_creator); auto result = task .translate( ntp, + rev, get_writer_factory(), make_batches(10, 16), datalake::remote_path("test/location/1"), @@ -188,7 +193,7 @@ TEST_F(TranslateTaskTest, TestHappyPathTranslation) { TEST_F(TranslateTaskTest, TestDataFileMissing) { datalake::translation_task task( - cloud_io, *schema_mgr, *schema_resolver, *translator); + cloud_io, *schema_mgr, *schema_resolver, *translator, *t_creator); // create deleting task to cause local io error deleter del(tmp_dir.get_path().string()); del.start(); @@ -196,6 +201,7 @@ TEST_F(TranslateTaskTest, TestDataFileMissing) { auto result = task .translate( ntp, + rev, get_writer_factory(), make_batches(10, 16), datalake::remote_path("test/location/1"), @@ -209,7 +215,7 @@ TEST_F(TranslateTaskTest, TestDataFileMissing) { TEST_F(TranslateTaskTest, TestUploadError) { datalake::translation_task task( - cloud_io, *schema_mgr, *schema_resolver, *translator); + cloud_io, *schema_mgr, *schema_resolver, *translator, *t_creator); // fail all PUT requests fail_request_if( [](const http_test_utils::request_info& req) -> bool { @@ -222,6 +228,7 @@ TEST_F(TranslateTaskTest, TestUploadError) { auto result = task .translate( ntp, + model::revision_id{123}, get_writer_factory(), make_batches(10, 16), datalake::remote_path("test/location/1"), diff --git a/src/v/datalake/translation/partition_translator.cc b/src/v/datalake/translation/partition_translator.cc index 56f71126176c..0725dbce84a1 100644 --- a/src/v/datalake/translation/partition_translator.cc +++ b/src/v/datalake/translation/partition_translator.cc @@ -20,6 +20,7 @@ #include "datalake/record_multiplexer.h" #include "datalake/record_translator.h" #include "datalake/serde_parquet_writer.h" +#include "datalake/table_creator.h" #include "datalake/translation/state_machine.h" #include "datalake/translation_task.h" #include "kafka/utils/txn_reader.h" @@ -76,6 +77,36 @@ ss::futurize_t retry_with_backoff( } } +// Creates or alters the table by delegating to the coordinator. +class coordinator_table_creator : public table_creator { +public: + explicit coordinator_table_creator(coordinator::frontend& fe) + : coordinator_fe_(fe) {} + + ss::future> ensure_table( + const model::topic& topic, + model::revision_id topic_revision, + record_schema_components comps) const final { + auto ensure_res = co_await coordinator_fe_.ensure_table_exists( + coordinator::ensure_table_exists_request{ + topic, + topic_revision, + comps, + }); + switch (ensure_res.errc) { + case coordinator::errc::ok: + co_return std::nullopt; + case coordinator::errc::incompatible_schema: + co_return errc::incompatible_schema; + default: + co_return errc::failed; + } + } + +private: + coordinator::frontend& coordinator_fe_; +}; + } // namespace static constexpr std::chrono::milliseconds translation_jitter{500}; @@ -105,6 +136,8 @@ partition_translator::partition_translator( , _schema_mgr(schema_mgr) , _type_resolver(std::move(type_resolver)) , _record_translator(std::move(record_translator)) + , _table_creator( + std::make_unique(_frontend->local())) , _partition_proxy(std::make_unique( kafka::make_partition_proxy(_partition))) , _jitter{translation_interval, translation_jitter} @@ -187,7 +220,11 @@ partition_translator::do_translation_for_range( ss::make_shared()); auto task = translation_task{ - **_cloud_io, *_schema_mgr, *_type_resolver, *_record_translator}; + **_cloud_io, + *_schema_mgr, + *_type_resolver, + *_record_translator, + *_table_creator}; const auto& ntp = _partition->ntp(); auto remote_path_prefix = remote_path{ fmt::format("{}/{}/{}", iceberg_file_path_prefix, ntp.path(), _term)}; @@ -197,6 +234,7 @@ partition_translator::do_translation_for_range( }}; auto result = co_await task.translate( ntp, + _partition->get_topic_revision_id(), std::move(writer_factory), std::move(rdr), remote_path_prefix, diff --git a/src/v/datalake/translation/partition_translator.h b/src/v/datalake/translation/partition_translator.h index cf04a6d52fbc..57aa2a595d9a 100644 --- a/src/v/datalake/translation/partition_translator.h +++ b/src/v/datalake/translation/partition_translator.h @@ -122,6 +122,7 @@ class partition_translator { schema_manager* _schema_mgr; std::unique_ptr _type_resolver; std::unique_ptr _record_translator; + std::unique_ptr _table_creator; std::unique_ptr _partition_proxy; using jitter_t = simple_time_jitter; diff --git a/src/v/datalake/translation_task.cc b/src/v/datalake/translation_task.cc index eb7adcf95541..4056d627c774 100644 --- a/src/v/datalake/translation_task.cc +++ b/src/v/datalake/translation_task.cc @@ -38,16 +38,19 @@ translation_task::translation_task( cloud_data_io& cloud_io, schema_manager& schema_mgr, type_resolver& type_resolver, - record_translator& record_translator) + record_translator& record_translator, + table_creator& table_creator) : _cloud_io(&cloud_io) , _schema_mgr(&schema_mgr) , _type_resolver(&type_resolver) - , _record_translator(&record_translator) {} + , _record_translator(&record_translator) + , _table_creator(&table_creator) {} ss::future< checked> translation_task::translate( const model::ntp& ntp, + model::revision_id topic_revision, std::unique_ptr writer_factory, model::record_batch_reader reader, const remote_path& remote_path_prefix, @@ -55,10 +58,12 @@ translation_task::translate( lazy_abort_source& lazy_as) { record_multiplexer mux( ntp, + topic_revision, std::move(writer_factory), *_schema_mgr, *_type_resolver, - *_record_translator); + *_record_translator, + *_table_creator); // Write local files auto mux_result = co_await std::move(reader).consume( std::move(mux), _read_timeout + model::timeout_clock::now()); diff --git a/src/v/datalake/translation_task.h b/src/v/datalake/translation_task.h index 1656b9a46499..3217525b3b29 100644 --- a/src/v/datalake/translation_task.h +++ b/src/v/datalake/translation_task.h @@ -29,7 +29,8 @@ class translation_task { cloud_data_io& uploader, schema_manager& schema_mgr, type_resolver& type_resolver, - record_translator& record_translator); + record_translator& record_translator, + table_creator&); enum class errc { file_io_error, cloud_io_error, @@ -41,6 +42,7 @@ class translation_task { */ ss::future> translate( const model::ntp& ntp, + model::revision_id topic_revision, std::unique_ptr writer_factory, model::record_batch_reader reader, const remote_path& remote_path_prefix, @@ -67,5 +69,6 @@ class translation_task { schema_manager* _schema_mgr; type_resolver* _type_resolver; record_translator* _record_translator; + table_creator* _table_creator; }; } // namespace datalake From 8539cb58aa086d7078a7a0f39e98c86d6fea7a5b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 20 Nov 2024 00:08:10 +0100 Subject: [PATCH 15/19] datalake: add coordinator stm topic lifecycle transitions utest --- .../coordinator/tests/state_update_test.cc | 88 +++++++++++++++++-- 1 file changed, 83 insertions(+), 5 deletions(-) diff --git a/src/v/datalake/coordinator/tests/state_update_test.cc b/src/v/datalake/coordinator/tests/state_update_test.cc index 5956ed58ea80..660eeb280b4e 100644 --- a/src/v/datalake/coordinator/tests/state_update_test.cc +++ b/src/v/datalake/coordinator/tests/state_update_test.cc @@ -77,11 +77,6 @@ void check_commit_doesnt_apply( TEST(StateUpdateTest, TestAddFile) { topics_state state; - // We can't add files to a topic or partition that isn't yet tracked. - ASSERT_TRUE( - add_files_update::build(state, tp, rev, make_pending_files({{0, 100}})) - .has_error()); - // create table state ASSERT_FALSE( apply_lc_transition(state, rev, topic_state::lifecycle_state_t::live) @@ -233,3 +228,86 @@ TEST(StateUpdateTest, TestMarkCommitted) { EXPECT_FALSE(res.has_error()); ASSERT_NO_FATAL_FAILURE(check_partition(state, tp, 220, {})); } + +TEST(StateUpdateTest, TestLifecycle) { + topics_state state; + + // We can't add files to a topic or partition that isn't yet tracked. + ASSERT_FALSE( + add_files_update::build(state, tp, rev, make_pending_files({{0, 100}})) + .has_value()); + + auto rev2 = model::revision_id{345}; + auto rev3 = model::revision_id{678}; + + ASSERT_TRUE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::live) + .has_value()); + + // files for obsolete revision can't be added + ASSERT_FALSE( + add_files_update::build( + state, tp, rev, make_pending_files({{0, 100}}, /*with_file=*/true)) + .has_value()); + + // add files + { + auto upd = add_files_update::build( + state, tp, rev2, make_pending_files({{0, 100}}, /*with_file=*/true)); + ASSERT_TRUE(upd.has_value()); + ASSERT_TRUE(upd.value().apply(state, model::offset{}).has_value()); + } + + // can't go back to obsolete revision + ASSERT_FALSE( + apply_lc_transition(state, rev, topic_state::lifecycle_state_t::live) + .has_value()); + + // can't go to the next revision as well without purging first + ASSERT_FALSE( + apply_lc_transition(state, rev3, topic_state::lifecycle_state_t::live) + .has_value()); + + // state transitions are idempotent + ASSERT_TRUE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::live) + .has_value()); + + // can transition to closed + ASSERT_TRUE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::closed) + .has_value()); + // but can't go back to live + ASSERT_FALSE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::live) + .has_value()); + + // can't add new files after topic has been closed + ASSERT_FALSE( + add_files_update::build( + state, tp, rev2, make_pending_files({{100, 200}}, /*with_file=*/true)) + .has_value()); + + // can't transition to purged while files are still pending + ASSERT_FALSE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::purged) + .has_value()); + + { + auto upd = mark_files_committed_update::build( + state, tp, rev2, kafka::offset{100}); + ASSERT_TRUE(upd.has_value()); + ASSERT_TRUE(upd.value().apply(state).has_value()); + } + + // now we can transition to purged + ASSERT_TRUE( + apply_lc_transition(state, rev2, topic_state::lifecycle_state_t::purged) + .has_value()); + ASSERT_EQ(state.topic_to_state.at(topic).pid_to_pending_files.size(), 0); + + // ...and to the next revision, even straight to the closed state + ASSERT_TRUE( + apply_lc_transition(state, rev3, topic_state::lifecycle_state_t::closed) + .has_value()); +} From 17a83ad48731c788ceeb420903924a31444fb752 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 22 Nov 2024 20:30:03 +0100 Subject: [PATCH 16/19] datalake: check that topic revision is constant after loading table --- src/v/datalake/coordinator/iceberg_file_committer.cc | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index ec1c23ea5b3c..c025c8f71d05 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -107,6 +107,7 @@ iceberg_file_committer::commit_topic_files_to_catalog( tp_it->second.lifecycle_state == topic_state::lifecycle_state_t::purged) { co_return chunked_vector{}; } + auto topic_revision = tp_it->second.revision; auto table_id = table_id_for_topic(topic); auto table_res = co_await load_table(table_id); @@ -126,6 +127,14 @@ iceberg_file_committer::commit_topic_files_to_catalog( } auto iceberg_commit_meta_opt = meta_res.value(); + // update the iterator after a scheduling point + tp_it = state.topic_to_state.find(topic); + if ( + tp_it == state.topic_to_state.end() + || tp_it->second.revision != topic_revision) { + co_return chunked_vector{}; + } + chunked_hash_map pending_commits; chunked_vector icb_files; std::optional new_committed_offset; @@ -167,7 +176,7 @@ iceberg_file_committer::commit_topic_files_to_catalog( for (const auto& [pid, committed_offset] : pending_commits) { auto tp = model::topic_partition(topic, pid); auto update_res = mark_files_committed_update::build( - state, tp, tp_state.revision, committed_offset); + state, tp, topic_revision, committed_offset); if (update_res.has_error()) { vlog( datalake_log.warn, From 167846f721b29987595cda65e898f3564bc86de6 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 22 Nov 2024 20:42:22 +0100 Subject: [PATCH 17/19] datalake: return correct errc for add_files in case of revision mismatch --- src/v/datalake/coordinator/coordinator.cc | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/src/v/datalake/coordinator/coordinator.cc b/src/v/datalake/coordinator/coordinator.cc index bc86182aba49..e4ba57903124 100644 --- a/src/v/datalake/coordinator/coordinator.cc +++ b/src/v/datalake/coordinator/coordinator.cc @@ -321,6 +321,18 @@ coordinator::sync_add_files( if (sync_res.has_error()) { co_return convert_stm_errc(sync_res.error()); } + + auto topic_it = stm_->state().topic_to_state.find(tp.topic); + if ( + topic_it == stm_->state().topic_to_state.end() + || topic_it->second.revision != topic_revision) { + vlog( + datalake_log.debug, + "Rejecting request to add files for {}: unexpected topic revision", + tp); + co_return errc::revision_mismatch; + } + auto added_last_offset = entries.back().last_offset; auto update_res = add_files_update::build( stm_->state(), tp, topic_revision, std::move(entries)); @@ -349,6 +361,17 @@ coordinator::sync_add_files( // NOTE: a mismatch here just means there was a race to update the STM, and // this should be handled by callers. // TODO: would be nice to encapsulate this in some update validator. + + if ( + topic_it == stm_->state().topic_to_state.end() + || topic_it->second.revision != topic_revision) { + vlog( + datalake_log.debug, + "Unexpected topic revision for {} after STM update", + tp); + co_return errc::stm_apply_error; + } + auto prt_opt = stm_->state().partition_state(tp); if ( !prt_opt.has_value() || prt_opt->get().pending_entries.empty() From 1e88454919f66448c5c0625e7d6a707a4c039241 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 20 Nov 2024 21:01:48 +0100 Subject: [PATCH 18/19] tests: start redpanda in DatalakeServices We have to do it in DatalakeServices to correctly configure redpanda with the preferred catalog. --- .../tests/datalake/compaction_gaps_test.py | 4 ++++ .../datalake/coordinator_retention_test.py | 4 ++++ .../tests/datalake/datalake_e2e_test.py | 4 ++++ .../tests/datalake/datalake_services.py | 21 +++++++++++++++++++ 4 files changed, 33 insertions(+) diff --git a/tests/rptest/tests/datalake/compaction_gaps_test.py b/tests/rptest/tests/datalake/compaction_gaps_test.py index 07767e244e16..e7ebeeee60ed 100644 --- a/tests/rptest/tests/datalake/compaction_gaps_test.py +++ b/tests/rptest/tests/datalake/compaction_gaps_test.py @@ -41,6 +41,10 @@ def __init__(self, test_ctx, *args, **kwargs): self.segment_size = 5 * 1024 * 1024 self.kafka_cat = KafkaCat(self.redpanda) + def setUp(self): + # redpanda will be started by DatalakeServices + pass + def partition_segments(self) -> int: assert len(self.redpanda.nodes) == 1, self.redpanda.nodes node = self.redpanda.nodes[0] diff --git a/tests/rptest/tests/datalake/coordinator_retention_test.py b/tests/rptest/tests/datalake/coordinator_retention_test.py index cf8a66819cd2..fadea1e2ee90 100644 --- a/tests/rptest/tests/datalake/coordinator_retention_test.py +++ b/tests/rptest/tests/datalake/coordinator_retention_test.py @@ -35,6 +35,10 @@ def __init__(self, test_ctx, *args, **kwargs): self.test_ctx = test_ctx self.topic_name = "test" + def setUp(self): + # redpanda will be started by DatalakeServices + pass + def wait_until_coordinator_snapshots(self): try: replica_last_snapshot_offsets = [] diff --git a/tests/rptest/tests/datalake/datalake_e2e_test.py b/tests/rptest/tests/datalake/datalake_e2e_test.py index 71c886f511f5..f41f78092e92 100644 --- a/tests/rptest/tests/datalake/datalake_e2e_test.py +++ b/tests/rptest/tests/datalake/datalake_e2e_test.py @@ -38,6 +38,10 @@ def __init__(self, test_ctx, *args, **kwargs): self.test_ctx = test_ctx self.topic_name = "test" + def setUp(self): + # redpanda will be started by DatalakeServices + pass + def _get_serde_client( self, schema_type: SchemaType, diff --git a/tests/rptest/tests/datalake/datalake_services.py b/tests/rptest/tests/datalake/datalake_services.py index f2af2ccc3481..e2376d452cdf 100644 --- a/tests/rptest/tests/datalake/datalake_services.py +++ b/tests/rptest/tests/datalake/datalake_services.py @@ -44,7 +44,28 @@ def __init__(self, self.query_engines: list[Service] = [] def setUp(self): + assert len(self.redpanda.started_nodes()) == 0, \ + "DatalakeServices expects to start redpanda itself" + + # create bucket first, or the catalog won't start + self.redpanda.start_si() + self.catalog_service.start() + + if not self.catalog_service.filesystem_wrapper_mode: + # REST catalog mode + self.redpanda.add_extra_rp_conf({ + "iceberg_catalog_type": + "rest", + "iceberg_rest_catalog_endpoint": + self.catalog_service.catalog_url, + "iceberg_rest_catalog_client_id": + "panda-user", + "iceberg_rest_catalog_client_secret": + "panda-secret", + }) + self.redpanda.start(start_si=False) + for engine in self.included_query_engines: svc_cls = get_query_engine_by_type(engine) svc = svc_cls(self.test_ctx, self.catalog_service.catalog_url) From 123ccdd37be70427a7b582ec9dc609976627f5a0 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 20 Nov 2024 23:53:09 +0100 Subject: [PATCH 19/19] tests: add datalake topic lifecycle test --- .../tests/datalake/datalake_e2e_test.py | 53 +++++++++++++++++++ .../tests/datalake/datalake_services.py | 19 +++++-- 2 files changed, 67 insertions(+), 5 deletions(-) diff --git a/tests/rptest/tests/datalake/datalake_e2e_test.py b/tests/rptest/tests/datalake/datalake_e2e_test.py index f41f78092e92..6ef294829a99 100644 --- a/tests/rptest/tests/datalake/datalake_e2e_test.py +++ b/tests/rptest/tests/datalake/datalake_e2e_test.py @@ -9,6 +9,7 @@ from typing import Optional from rptest.clients.serde_client_utils import SchemaType, SerdeClientType from rptest.clients.types import TopicSpec +from rptest.clients.rpk import RpkTool from rptest.services.cluster import cluster from rptest.services.redpanda import PandaproxyConfig, SchemaRegistryConfig, SISettings @@ -19,6 +20,7 @@ from rptest.tests.datalake.query_engine_base import QueryEngineType from rptest.tests.datalake.utils import supported_storage_types from ducktape.mark import matrix +from ducktape.utils.util import wait_until class DatalakeE2ETests(RedpandaTest): @@ -129,3 +131,54 @@ def test_avro_schema(self, cloud_storage_type, query_engine): f"describe {table_name}") assert spark_describe_out == spark_expected_out, str( spark_describe_out) + + @cluster(num_nodes=4) + @matrix(cloud_storage_type=supported_storage_types(), + filesystem_catalog_mode=[False]) + def test_topic_lifecycle(self, cloud_storage_type, + filesystem_catalog_mode): + count = 100 + with DatalakeServices(self.test_ctx, + redpanda=self.redpanda, + filesystem_catalog_mode=filesystem_catalog_mode, + include_query_engines=[QueryEngineType.SPARK + ]) as dl: + rpk = RpkTool(self.redpanda) + + # produce some data then delete the topic + dl.create_iceberg_enabled_topic(self.topic_name, partitions=10) + dl.produce_to_topic(self.topic_name, 1024, count) + dl.wait_for_translation(self.topic_name, msg_count=count) + + rpk.alter_topic_config(self.topic_name, "redpanda.iceberg.delete", + "false") + rpk.delete_topic(self.topic_name) + + # table is not deleted, it will contain messages from both topic instances + dl.create_iceberg_enabled_topic(self.topic_name, partitions=15) + dl.produce_to_topic(self.topic_name, 1024, count) + dl.wait_for_translation(self.topic_name, msg_count=2 * count) + + # now table should be deleted + rpk.delete_topic(self.topic_name) + + catalog_client = dl.catalog_client() + + def table_deleted(): + return not dl.table_exists(self.topic_name, + client=catalog_client) + + wait_until(table_deleted, + timeout_sec=30, + backoff_sec=5, + err_msg="table was not deleted") + + # recreate an empty topic a few times + for _ in range(3): + dl.create_iceberg_enabled_topic(self.topic_name, partitions=10) + rpk.delete_topic(self.topic_name) + + # check that the table is recreated after we start producing again + dl.create_iceberg_enabled_topic(self.topic_name, partitions=5) + dl.produce_to_topic(self.topic_name, 1024, count) + dl.wait_for_translation(self.topic_name, msg_count=count) diff --git a/tests/rptest/tests/datalake/datalake_services.py b/tests/rptest/tests/datalake/datalake_services.py index e2376d452cdf..09a09b234fc7 100644 --- a/tests/rptest/tests/datalake/datalake_services.py +++ b/tests/rptest/tests/datalake/datalake_services.py @@ -117,14 +117,23 @@ def set_iceberg_mode_on_topic(self, topic: str, mode: str): rpk = RpkTool(self.redpanda) rpk.alter_topic_config(topic, "redpanda.iceberg.mode", mode) + def catalog_client(self): + return self.catalog_service.client("redpanda-iceberg-catalog") + + def table_exists(self, table, namespace="redpanda", client=None): + if client is None: + client = self.catalog_client() + + namespaces = client.list_namespaces() + self.redpanda.logger.debug(f"namespaces: {namespaces}") + return (namespace, ) in namespaces and ( + namespace, table) in client.list_tables(namespace) + def wait_for_iceberg_table(self, namespace, table, timeout, backoff_sec): - client = self.catalog_service.client("redpanda-iceberg-catalog") + client = self.catalog_client() def table_created(): - namespaces = client.list_namespaces() - self.redpanda.logger.debug(f"namespaces: {namespaces}") - return (namespace, ) in namespaces and ( - namespace, table) in client.list_tables(namespace) + return self.table_exists(table, namespace=namespace, client=client) wait_until( table_created,