diff --git a/src/v/cloud_storage/tests/topic_manifest_test.cc b/src/v/cloud_storage/tests/topic_manifest_test.cc index 00bae5c1445d5..4a6e0c766e1f5 100644 --- a/src/v/cloud_storage/tests/topic_manifest_test.cc +++ b/src/v/cloud_storage/tests/topic_manifest_test.cc @@ -509,6 +509,7 @@ SEASTAR_THREAD_TEST_CASE(test_topic_manifest_serde_feature_table) { std::nullopt, std::nullopt, model::redpanda_storage_mode::tiered, + std::nullopt, }; auto random_initial_revision_id diff --git a/src/v/cluster/BUILD b/src/v/cluster/BUILD index 52e8d5e37f169..6d408427d48cd 100644 --- a/src/v/cluster/BUILD +++ b/src/v/cluster/BUILD @@ -236,6 +236,7 @@ redpanda_cc_library( deps = [ "//src/v/base", "//src/v/cloud_storage:remote_label", + "//src/v/config", "//src/v/model", "//src/v/pandaproxy/schema_registry:subject_name_strategy", "//src/v/reflection:adl", diff --git a/src/v/cluster/controller.cc b/src/v/cluster/controller.cc index 92f5fa05abf9d..001e234af600c 100644 --- a/src/v/cluster/controller.cc +++ b/src/v/cluster/controller.cc @@ -1016,6 +1016,7 @@ ss::future<> controller::stop() { co_await _data_migrated_resources.stop(); co_await _roles.stop(); co_await _credentials.stop(); + co_await _partition_balancer_state.stop(); co_await _tp_state.stop(); co_await _members_manager.stop(); co_await _epoch_service.stop(); diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 468fd8bc598c8..33d86ce32bc69 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -22,6 +22,7 @@ #include "config/configuration.h" #include "config/endpoint_tls_config.h" #include "config/node_config.h" +#include "config/replicas_preference.h" #include "config/sasl_mechanisms.h" #include "config/tls_config.h" #include "config/types.h" @@ -273,6 +274,22 @@ feature_manager::report_enterprise_features() const { report.set( features::license_required_feature::leadership_pinning, leadership_pinning_enabled()); + auto replica_pinning_enabled = [this] { + const config::replicas_preference no_replicas_preference{}; + for (const auto& topic : _topic_table.local().topics_map()) { + const auto replicas_preference + = topic.second.get_configuration().properties.replicas_preference; + if ( + replicas_preference.has_value() + && replicas_preference.value() != no_replicas_preference) { + return true; + } + } + return false; + }; + report.set( + features::license_required_feature::replica_pinning, + replica_pinning_enabled()); report.set( features::license_required_feature::shadow_linking, cfg.enable_shadow_linking()); diff --git a/src/v/cluster/partition_balancer_backend.cc b/src/v/cluster/partition_balancer_backend.cc index 3c3b0ab0bbeea..26d3f70df16d9 100644 --- a/src/v/cluster/partition_balancer_backend.cc +++ b/src/v/cluster/partition_balancer_backend.cc @@ -105,6 +105,13 @@ bool partition_balancer_backend::is_enabled() const { return is_leader() && !config::node().recovery_mode_enabled(); } +void partition_balancer_backend::maybe_advance_term() { + if (!_cur_term || _raft0->term() != _cur_term->id) { + _cur_term = per_term_state(_raft0->term()); + _state.reset_pinning_cache(); + } +} + void partition_balancer_backend::start() { _topic_table_updates = _state.topics().register_lw_ntp_notification( [this]() { on_topic_table_update(); }); @@ -137,9 +144,7 @@ ss::future partition_balancer_backend::request_rebalance() { co_return errc::not_leader; } - if (!_cur_term || _raft0->term() != _cur_term->id) { - _cur_term = per_term_state(_raft0->term()); - } + maybe_advance_term(); if ( _cur_term->_ondemand_rebalance_requested @@ -359,9 +364,7 @@ ss::future<> partition_balancer_backend::do_tick() { vlog(clusterlog.debug, "tick"); - if (!_cur_term || _raft0->term() != _cur_term->id) { - _cur_term = per_term_state(_raft0->term()); - } + maybe_advance_term(); co_await _controller_stm.wait( _raft0->committed_offset(), @@ -448,6 +451,8 @@ ss::future<> partition_balancer_backend::do_tick() { _cur_term->last_tick_time = clock_t::now(); _cur_term->last_violations = std::move(plan_data.violations); + _cur_term->last_pinning_violations_count + = plan_data.last_pinning_violations_count; _cur_term->last_tick_reallocation_failures = std::move( plan_data.reallocation_failures); if ( diff --git a/src/v/cluster/partition_balancer_backend.h b/src/v/cluster/partition_balancer_backend.h index 8eb58f9e691fb..5889307afd764 100644 --- a/src/v/cluster/partition_balancer_backend.h +++ b/src/v/cluster/partition_balancer_backend.h @@ -75,6 +75,14 @@ class partition_balancer_backend { partition_balancer_overview_reply overview() const; + // Pinning-violation count observed during the most recent planning tick + // of the current leader term. Zero if no tick has run yet for this term. + // TODO: expose through partition_balancer_overview_reply (serde version + // bump) once we're ready to make this visible via the admin API. + size_t last_pinning_violations_count() const { + return _cur_term ? _cur_term->last_pinning_violations_count : 0; + } + ss::future request_rebalance(); private: @@ -84,6 +92,11 @@ class partition_balancer_backend { /// If now, rearms to run immediately, else rearms to _tick_interval or /// current timeout whichever is minimum. void maybe_rearm_timer(bool now = false); + + /// If the raft0 term has advanced since we last observed it, reset + /// per-term state. Must be called on shard 0 before any code that + /// relies on _cur_term or on per-term caches in partition_balancer_state. + void maybe_advance_term(); void on_members_update(model::node_id, model::membership_state); void on_topic_table_update(); void on_health_monitor_update( @@ -144,6 +157,7 @@ class partition_balancer_backend { partition_balancer_status last_status = partition_balancer_status::starting; size_t last_tick_in_progress_updates = 0; + size_t last_pinning_violations_count = 0; chunked_hash_map last_tick_reallocation_failures; diff --git a/src/v/cluster/partition_balancer_planner.cc b/src/v/cluster/partition_balancer_planner.cc index d800a3ff030f8..5c379a6f3e451 100644 --- a/src/v/cluster/partition_balancer_planner.cc +++ b/src/v/cluster/partition_balancer_planner.cc @@ -25,6 +25,7 @@ #include "cluster/scheduling/types.h" #include "cluster/topic_table.h" #include "cluster/types.h" +#include "config/replicas_preference.h" #include "container/chunked_hash_map.h" #include "model/fundamental.h" #include "model/metadata.h" @@ -104,6 +105,32 @@ reallocation_failure_details map_result_to_failure_details( } } // namespace +void partition_balancer_planner::warn_if_pinning_capacity_insufficient( + const model::topic_namespace& tp_ns, + const config::replicas_preference& pref, + const std::vector& capacity_by_rack_group, + int16_t replication_factor) { + uint32_t total_preference_capacity = 0; + for (auto rack_capacity : capacity_by_rack_group) { + total_preference_capacity += rack_capacity; + } + + // Static mismatch between topology/preference/RF. Paired with the + // revert-path info log below, this tells live-site operators why + // a pinning constraint isn't converging: structural vs. transient. + if (total_preference_capacity < static_cast(replication_factor)) { + vlog( + clusterlog.warn, + "topic {} has replicas_preference {} but preferred-group " + "capacity {} is less than replication factor {}; pinning " + "cannot be fully satisfied", + tp_ns, + pref, + total_preference_capacity, + replication_factor); + } +} + partition_balancer_planner::partition_balancer_planner( planner_config config, partition_balancer_state& state, @@ -171,6 +198,7 @@ class partition_balancer_planner::request_context { // full retention case change_reason::node_unavailable: case change_reason::rack_constraint_repair: + case change_reason::replica_pinning_repair: case change_reason::disk_full: return reconfiguration_policy::full_local_retention; } @@ -1271,8 +1299,23 @@ partition_balancer_planner::reassignable_partition::get_allocation_constraints( // soft constraints + // replica_pinning_preferred biases the allocator toward nodes in + // higher-priority rack groups (group 0 outranks group 1, etc.), with + // rackless and unpreferred nodes scoring zero. This shapes the + // destination chosen by get_replica_pinning_repair_actions. Placed at + // L0 in the reassignment path (rack diversity lives elsewhere here); + // partition_allocator.cc::do_allocate() puts it at L1 after rack + // diversity. + auto cfg = _ctx.state().topics().get_topic_cfg( + model::topic_namespace_view(ntp())); + if (cfg && cfg->properties.replicas_preference) { + constraints.add(replica_pinning_preferred( + *cfg->properties.replicas_preference, _ctx.state().members())); + } + if (_ctx.config().topic_aware) { // Add constraint for balanced topic-wise replica counts + constraints.ensure_new_level(); constraints.add(min_count_in_map( "min topic-wise count", _ctx.get_topic_to_node_count(model::topic_namespace_view(ntp())))); @@ -1730,6 +1773,347 @@ ss::future<> partition_balancer_planner::get_rack_constraint_repair_actions( } } +absl::flat_hash_map +partition_balancer_planner::build_rack_node_counts( + const members_table& members, bool rack_awareness) { + absl::flat_hash_map counts; + for (auto node_id : members.node_ids()) { + auto maybe_rack = members.get_node_rack_id(node_id); + if (!maybe_rack) { + continue; + } + const auto& rack = *maybe_rack; + if (rack_awareness) { + // rack-awareness forces one replica per rack -> capacity is + // rack presence, not node count. + counts[rack] = 1; + } else { + ++counts[rack]; + } + } + return counts; +} + +std::vector partition_balancer_planner::compute_pinning_capacity( + const config::replicas_preference& pref, + const absl::flat_hash_map& rack_node_counts) { + uint32_t num_groups = pref.num_groups(); + std::vector capacity(num_groups, 0); + for (const auto& [rack, count] : rack_node_counts) { + auto maybe_group = pref.group_index_for(rack); + if (!maybe_group) { + continue; + } + capacity[*maybe_group] += count; + } + return capacity; +} + +std::vector +partition_balancer_planner::compute_ideal_pinning_assignment( + size_t replication_factor, const std::vector& capacity_per_group) { + std::vector ideal; + ideal.reserve(replication_factor); + for (uint32_t group_idx = 0; group_idx < capacity_per_group.size() + && ideal.size() < replication_factor; + ++group_idx) { + uint32_t slots = std::min( + capacity_per_group[group_idx], + static_cast(replication_factor - ideal.size())); + ideal.insert(ideal.end(), slots, group_idx); + } + // If preferred groups lack total capacity, remaining replicas are + // unpreferred -- represent with sentinel max. + ideal.resize(replication_factor, std::numeric_limits::max()); + return ideal; +} + +bool partition_balancer_planner::is_pinning_violated( + const std::vector& replicas, + const config::replicas_preference& pref, + const std::vector& ideal, + const members_table& members) { + if (pref.type == config::replicas_preference::type_t::none) { + return false; + } + + constexpr uint32_t unpreferred = std::numeric_limits::max(); + std::vector actual; + actual.reserve(replicas.size()); + for (const auto& bs : replicas) { + auto maybe_rack = members.get_node_rack_id(bs.node_id); + if (!maybe_rack) { + actual.push_back(unpreferred); + continue; + } + auto maybe_group = pref.group_index_for(*maybe_rack); + actual.push_back(maybe_group.value_or(unpreferred)); + } + std::ranges::sort(actual); + + return actual != ideal; +} + +std::pair +partition_balancer_planner::find_worst_replica_and_group( + const std::vector& replicas, + const config::replicas_preference& pref, + const members_table& members) { + constexpr uint32_t unpreferred = std::numeric_limits::max(); + + model::node_id worst_replica{-1}; + uint32_t worst_rack_group = 0; + for (const auto& bs : replicas) { + auto maybe_rack = members.get_node_rack_id(bs.node_id); + uint32_t rack_group = unpreferred; + if (maybe_rack) { + auto maybe_g = pref.group_index_for(*maybe_rack); + if (maybe_g) { + rack_group = *maybe_g; + } + } + if (rack_group >= worst_rack_group || rack_group == unpreferred) { + worst_rack_group = rack_group; + worst_replica = bs.node_id; + } + } + return {worst_replica, worst_rack_group}; +} + +// given a reassignable partition with suboptimal pinning, try to move the worst +// replica to a new node that improves the pinning score +void partition_balancer_planner::try_repair_replica_pinning( + request_context& ctx, + reassignable_partition& rpart, + model::node_id worst_replica, + uint32_t previous_rack_group, + const config::replicas_preference& pref, + const members_table& members) { + constexpr uint32_t unpreferred = std::numeric_limits::max(); + + // moved by someone else, not our issue at the moment + if (!rpart.is_original(worst_replica)) { + return; + } + auto r = rpart.move_replica( + worst_replica, + ctx.config().max_disk_usage_ratio, + change_reason::replica_pinning_repair); + if (r.has_error()) { + ctx.report_reallocation_failure( + rpart.ntp(), + map_result_to_failure_details( + change_reason::replica_pinning_repair, r.error(), worst_replica)); + return; + } + // same node, no move occurred + auto new_node = r.value().current().node_id; + if (new_node == worst_replica) { + return; + } + + // get the preference index of the new node + uint32_t new_rack_group = unpreferred; + auto maybe_new_rack = members.get_node_rack_id(new_node); + if (maybe_new_rack) { + auto maybe_group = pref.group_index_for(*maybe_new_rack); + if (maybe_group) { + new_rack_group = *maybe_group; + } + } + + // new node didn't improve the score, don't bother with the move + if (new_rack_group >= previous_rack_group) { + vlog( + clusterlog.info, + "ntp {}: pinning repair could not find a strictly better " + "preferred group this tick (previous_rack_group={}, " + "new_rack_group={}, " + "preference={}); reverting move", + rpart.ntp(), + previous_rack_group, + new_rack_group, + pref); + rpart.revert(r.value()); + } +} + +partition_balancer_planner::pinning_violations +partition_balancer_planner::detect_pinning_violations(request_context& ctx) { + using preference = config::replicas_preference::type_t; + + pinning_violations result; + + const auto& topics = ctx.state().topics(); + const auto& members = ctx.state().members(); + bool rack_awareness = ctx.state().is_rack_awareness_enabled(); + auto rack_node_counts = build_rack_node_counts(members, rack_awareness); + + ctx.state().ensure_pinning_cache_seeded(); + + // Synchronous — no co_await, so iteration is safe without a snapshot. + for (const auto& tp_ns : ctx.state().topics_with_replica_pinning()) { + auto md_ref = topics.get_topic_metadata_ref(tp_ns); + if (!md_ref) { + // Raced with a topic delete — delta handler will drop it + // next tick. + continue; + } + const auto& cfg = md_ref->get().get_configuration(); + const auto& maybe_pref = cfg.properties.replicas_preference; + if (!maybe_pref || maybe_pref->type == preference::none) { + continue; + } + + auto capacity = compute_pinning_capacity(*maybe_pref, rack_node_counts); + topic_pinning_plan plan{ + .preference = *maybe_pref, + .ideal = compute_ideal_pinning_assignment( + cfg.replication_factor, capacity), + .violating_partitions = {}, + }; + + warn_if_pinning_capacity_insufficient( + tp_ns, plan.preference, capacity, cfg.replication_factor); + + const auto& assignments = md_ref->get().get_assignments(); + for (const auto& [p_id, assignment] : assignments) { + if ( + is_pinning_violated( + assignment.replicas, plan.preference, plan.ideal, members)) { + plan.violating_partitions.insert(model::partition_id{p_id}); + ++result.count; + } + } + + if (!plan.violating_partitions.empty()) { + result.per_topic.emplace(tp_ns, std::move(plan)); + } + } + + return result; +} + +ss::future<> partition_balancer_planner::get_replica_pinning_repair_actions( + request_context& ctx, + plan_data& result, + const pinning_violations& violations) { + if (violations.per_topic.empty()) { + co_return; + } + + const auto& topics = ctx.state().topics(); + const auto& members = ctx.state().members(); + + // Snapshot topic_namespace keys from the detection map so we don't + // hold iterators into topic_table-adjacent state across co_await + // points. + chunked_vector pinned_topics_snapshot; + pinned_topics_snapshot.reserve(violations.per_topic.size()); + for (const auto& [tp_ns, _] : violations.per_topic) { + pinned_topics_snapshot.push_back(tp_ns); + } + + for (const auto& tp_ns : pinned_topics_snapshot) { + auto plan_it = violations.per_topic.find(tp_ns); + if (plan_it == violations.per_topic.end()) { + continue; + } + const auto& plan = plan_it->second; + + // Snapshot the violating partition ids for the same reason. + chunked_vector violating_snapshot; + violating_snapshot.reserve(plan.violating_partitions.size()); + for (auto p_id : plan.violating_partitions) { + violating_snapshot.push_back(p_id); + } + + for (auto p_id : violating_snapshot) { + if (!ctx.can_add_reassignment()) { + co_return; + } + + // Re-fetch and re-verify. Between detect_pinning_violations + // and now, other same-tick actions (drain, rack repair, + // full-disk) may have moved the replica off the bad node, or + // the topic may have been altered or deleted via + // AlterConfigs. + auto md_ref = topics.get_topic_metadata_ref(tp_ns); + if (!md_ref) { + continue; + } + const auto& cfg = md_ref->get().get_configuration(); + const auto& maybe_pref = cfg.properties.replicas_preference; + if ( + !maybe_pref + || maybe_pref->type + == config::replicas_preference::type_t::none) { + continue; + } + const auto& assignments = md_ref->get().get_assignments(); + auto a_it = assignments.find(p_id); + if (a_it == assignments.end()) { + continue; + } + // Copy replicas so the inner co_await below doesn't hold a + // reference across the suspension. + std::vector replicas = a_it->second.replicas; + + // Use the cached ideal. If the preference changed between + // detection and repair, the cached ideal is slightly stale + // for one tick; the next tick's detection uses the new + // preference. + if (!is_pinning_violated( + replicas, plan.preference, plan.ideal, members)) { + co_await ss::maybe_yield(); + continue; + } + + model::ntp ntp(tp_ns.ns, tp_ns.tp, p_id); + co_await ctx.with_partition(ntp, [&](partition& part) { + auto [worst_replica, previous_rack_group] + = find_worst_replica_and_group( + part.replicas(), plan.preference, members); + + part.match_variant( + [&](reassignable_partition& rpart) { + try_repair_replica_pinning( + ctx, + rpart, + worst_replica, + previous_rack_group, + plan.preference, + members); + }, + [&](immutable_partition& ipart) { + ipart.report_failure( + change_reason::replica_pinning_repair); + ipart.report_immutable_partition_as_reallocation_failure( + ctx, + {worst_replica}, + change_reason::replica_pinning_repair); + }, + [&](moving_partition& mpart) { + vlog( + clusterlog.debug, + "replica pinning will not attempt to move {} because " + "partition is {}", + mpart.ntp(), + "moving"); + }, + [&](force_reassignable_partition& fpart) { + vlog( + clusterlog.debug, + "replica pinning will not attempt to move {} because " + "partition is {}", + fpart.ntp(), + "forced"); + }); + }); + } + } +} + /** * This is the place where we decide about the order in which partitions will be * moved in the case when node disk is being full. @@ -2401,6 +2785,9 @@ partition_balancer_planner::plan_actions( get_auto_decommission_actions(ctx, health_report); } + auto pinning = detect_pinning_violations(ctx); + result.last_pinning_violations_count = pinning.count; + // early exit if theres nothing to be done if ( result.violations.is_empty() && ctx.decommissioning_nodes.empty() @@ -2408,7 +2795,7 @@ partition_balancer_planner::plan_actions( && _state.nodes_to_rebalance().empty() && _state.topics().partitions_to_force_recover().empty() && !_config.ondemand_rebalance_requested - && !ctx._maybe_node_to_auto_decommission.has_value()) { + && !ctx._maybe_node_to_auto_decommission.has_value() && !pinning.any()) { result.status = status::empty; co_return result; } @@ -2426,6 +2813,7 @@ partition_balancer_planner::plan_actions( change_reason::node_unavailable); co_await get_full_node_actions(ctx); co_await get_rack_constraint_repair_actions(ctx); + co_await get_replica_pinning_repair_actions(ctx, result, pinning); } co_await get_counts_rebalancing_actions(ctx); co_await get_force_repair_actions(ctx); diff --git a/src/v/cluster/partition_balancer_planner.h b/src/v/cluster/partition_balancer_planner.h index add1463fc7c00..6f0bbde79c8ab 100644 --- a/src/v/cluster/partition_balancer_planner.h +++ b/src/v/cluster/partition_balancer_planner.h @@ -10,12 +10,15 @@ #pragma once +#include "absl/container/btree_set.h" #include "absl/container/flat_hash_map.h" #include "cluster/fwd.h" #include "cluster/health_monitor_types.h" #include "cluster/partition_balancer_types.h" #include "cluster/scheduling/types.h" #include "cluster/types.h" +#include "config/replicas_preference.h" +#include "container/chunked_hash_map.h" #include "container/chunked_vector.h" #include "model/metadata.h" @@ -98,6 +101,7 @@ class partition_balancer_planner { std::optional maybe_node_to_autodecommission; bool counts_rebalancing_finished = false; size_t failed_actions_count = 0; + size_t last_pinning_violations_count = 0; status status = status::empty; void maybe_add_reallocation_failure(); @@ -129,7 +133,64 @@ class partition_balancer_planner { const absl::flat_hash_set&, change_reason reason); + /// Per-topic state carried from detection into repair. The preference + /// is copied (not referenced into topic_table) so it survives co_await + /// points during the repair phase even if the topic is deleted or its + /// properties change mid-iteration. + struct topic_pinning_plan { + config::replicas_preference preference; + std::vector ideal; + absl::btree_set violating_partitions; + }; + + /// Result of the detection pass. `count` is the total violating + /// partitions across all topics; `per_topic` maps each topic with at + /// least one violation to its precomputed plan. Callers use `any()` + /// to short-circuit the planner's early-exit predicate without + /// iterating the map. + struct pinning_violations { + bool any() const { return count > 0; } + size_t count{0}; + chunked_hash_map per_topic; + }; + + /// Detection pass: walk the pinned-topics cache, compute the ideal + /// pinning for each topic, and record every partition whose current + /// replica set violates it. Synchronous; reads only `state().topics()`, + /// `state().members()`, and `state().is_rack_awareness_enabled()`. + /// Safe to call before `init_ntp_sizes_from_health_report` — it does + /// not inspect partition sizes or the allocator. + pinning_violations detect_pinning_violations(request_context& ctx); + static ss::future<> get_rack_constraint_repair_actions(request_context&); + static ss::future<> get_replica_pinning_repair_actions( + request_context&, plan_data&, const pinning_violations&); + /// Logs a warning when the topic's preferred-group capacity is below the + /// replication factor. Signals a structural (topology vs preference vs RF) + /// mismatch where pinning cannot be fully satisfied. + static void warn_if_pinning_capacity_insufficient( + const model::topic_namespace& tp_ns, + const config::replicas_preference& pref, + const std::vector& capacity_by_rack_group, + int16_t replication_factor); + /// Pick the worst replica: highest-group wins, unpreferred beats any + /// preferred group, last-seen wins on ties. Returns the replica's node id + /// and its rack group (numeric_limits::max() if unpreferred). + static std::pair find_worst_replica_and_group( + const std::vector& replicas, + const config::replicas_preference& pref, + const members_table& members); + /// Attempt to move \p worst_replica off its current node to satisfy the + /// topic's replicas_preference. On failure, records a reallocation + /// failure. On success, applies an anti-spurious-move guard that reverts + /// any move which does not strictly improve the replica's pinning group. + static void try_repair_replica_pinning( + request_context& ctx, + reassignable_partition& rpart, + model::node_id worst_replica, + uint32_t previous_rack_group, + const config::replicas_preference& pref, + const members_table& members); static ss::future<> get_full_node_actions(request_context&); static ss::future<> get_counts_rebalancing_actions(request_context&); static ss::future<> get_force_repair_actions(request_context&); @@ -167,6 +228,34 @@ class partition_balancer_planner { candidate_nodes_to_decommission, const absl::flat_hash_set& decommissioning_nodes); + /// Count live nodes per rack from the members table. With rack awareness + /// enabled, each rack contributes 1 (rack presence); disabled, each rack + /// contributes the number of nodes it hosts. Callers use this as the + /// per-tick source of truth for replica-pinning capacity calculations. + static absl::flat_hash_map + build_rack_node_counts(const members_table& members, bool rack_awareness); + + /// Capacity (node count, per the precomputed map) available in each + /// priority group of the preference. + static std::vector compute_pinning_capacity( + const config::replicas_preference& pref, + const absl::flat_hash_map& rack_node_counts); + + /// Fill-then-overflow ideal: greedily fill group 0 up to its capacity, + /// then group 1, etc. Any remaining slots are marked unpreferred via + /// numeric_limits::max() sentinels. + static std::vector compute_ideal_pinning_assignment( + size_t replication_factor, + const std::vector& capacity_per_group); + + /// True if the sorted per-replica group assignment doesn't match the + /// fill-then-overflow ideal. + static bool is_pinning_violated( + const std::vector& replicas, + const config::replicas_preference& pref, + const std::vector& ideal, + const members_table& members); + planner_config _config; partition_balancer_state& _state; partition_allocator& _partition_allocator; diff --git a/src/v/cluster/partition_balancer_state.cc b/src/v/cluster/partition_balancer_state.cc index 4274207fded65..58ddd51739b74 100644 --- a/src/v/cluster/partition_balancer_state.cc +++ b/src/v/cluster/partition_balancer_state.cc @@ -16,7 +16,10 @@ #include "cluster/members_table.h" #include "cluster/node_status_table.h" #include "cluster/scheduling/partition_allocator.h" +#include "cluster/topic_table.h" +#include "cluster/types.h" #include "config/configuration.h" +#include "config/replicas_preference.h" #include "metrics/metrics.h" #include "metrics/prometheus_sanitize.h" @@ -33,7 +36,77 @@ partition_balancer_state::partition_balancer_state( , _members_table(members_table.local()) , _partition_allocator(pa.local()) , _node_status(nst.local()) - , _probe(*this) {} + , _probe(*this) { + if (ss::this_shard_id() == 0) { + _topic_deltas_handle = _topic_table.register_topic_delta_notification( + [this](const chunked_vector& deltas) { + handle_topic_deltas(deltas); + }); + } +} + +ss::future<> partition_balancer_state::stop() { + if (_topic_deltas_handle != cluster::notification_id_type_invalid) { + _topic_table.unregister_topic_delta_notification(_topic_deltas_handle); + _topic_deltas_handle = cluster::notification_id_type_invalid; + } + return ss::now(); +} + +bool partition_balancer_state::is_rack_awareness_enabled() const { + return _partition_allocator.is_rack_awareness_enabled(); +} + +void partition_balancer_state::ensure_pinning_cache_seeded() { + if (_pinning_cache_seeded) { + return; + } + _topics_with_replica_pinning.clear(); + for (const auto& [tp_ns, md] : _topic_table.topics_map()) { + const auto& pref + = md.get_configuration().properties.replicas_preference; + if (pref && pref->type != config::replicas_preference::type_t::none) { + _topics_with_replica_pinning.insert(tp_ns); + } + } + _pinning_cache_seeded = true; +} + +void partition_balancer_state::handle_topic_deltas( + const chunked_vector& deltas) { + // Before first seed there is nothing to maintain; the next + // ensure_pinning_cache_seeded() call reads live topic_table state. + if (!_pinning_cache_seeded) { + return; + } + + using delta_type = topic_table_topic_delta_type; + for (const auto& d : deltas) { + switch (d.type) { + case delta_type::added: + case delta_type::properties_updated: { + auto md_ref = _topic_table.get_topic_metadata_ref(d.ns_tp); + if (!md_ref) { + _topics_with_replica_pinning.erase(d.ns_tp); + break; + } + const auto& pref = md_ref->get() + .get_configuration() + .properties.replicas_preference; + if ( + pref && pref->type != config::replicas_preference::type_t::none) { + _topics_with_replica_pinning.insert(d.ns_tp); + } else { + _topics_with_replica_pinning.erase(d.ns_tp); + } + break; + } + case delta_type::removed: + _topics_with_replica_pinning.erase(d.ns_tp); + break; + } + } +} void partition_balancer_state::handle_ntp_move_begin_or_cancel( const model::ns& ns, @@ -41,54 +114,87 @@ void partition_balancer_state::handle_ntp_move_begin_or_cancel( model::partition_id p_id, const std::vector& prev, const std::vector& next) { - if (_partition_allocator.is_rack_awareness_enabled()) { - absl::flat_hash_set racks; - bool is_rack_constraint_violated = false; - for (const auto& bs : next) { - auto rack = _members_table.get_node_rack_id(bs.node_id); - if (rack) { - auto res = racks.insert(std::move(*rack)); - if (!res.second) { - is_rack_constraint_violated = true; - break; - } + if (!_partition_allocator.is_rack_awareness_enabled()) { + return; + } + + model::ntp ntp(ns, tp, p_id); + + absl::flat_hash_set racks; + bool is_rack_constraint_violated = false; + for (const auto& bs : next) { + auto rack = _members_table.get_node_rack_id(bs.node_id); + if (rack) { + auto res = racks.insert(std::move(*rack)); + if (!res.second) { + is_rack_constraint_violated = true; + break; } } + } - model::ntp ntp(ns, tp, p_id); - if (is_rack_constraint_violated) { - auto res = _ntps_with_broken_rack_constraint.insert(ntp); - _ntps_with_broken_rack_constraint_revision++; - if (res.second) { - vlog( - clusterlog.debug, - "rack constraint violated for ntp: {}, " - "replica set change: {} -> {}", - ntp, - prev, - next); - } - } else { - auto erased = _ntps_with_broken_rack_constraint.erase(ntp); - _ntps_with_broken_rack_constraint_revision++; - if (erased > 0) { - vlog( - clusterlog.debug, - "rack constraint restored for ntp: {}, " - "replica set change: {} -> {}", - ntp, - prev, - next); + if (is_rack_constraint_violated) { + auto res = _ntps_with_broken_rack_constraint.insert(ntp); + _ntps_with_broken_rack_constraint_revision++; + if (res.second) { + vlog( + clusterlog.debug, + "rack constraint violated for ntp: {}, " + "replica set change: {} -> {}", + ntp, + prev, + next); + } + } else { + auto erased = _ntps_with_broken_rack_constraint.erase(ntp); + _ntps_with_broken_rack_constraint_revision++; + if (erased > 0) { + vlog( + clusterlog.debug, + "rack constraint restored for ntp: {}, " + "replica set change: {} -> {}", + ntp, + prev, + next); + } + } +} + +void partition_balancer_state::handle_ntp_move_finish( + const model::ntp& ntp, const std::vector& replicas) { + if (!_partition_allocator.is_rack_awareness_enabled()) { + return; + } + + absl::flat_hash_set racks; + bool is_rack_constraint_violated = false; + for (const auto& bs : replicas) { + auto rack = _members_table.get_node_rack_id(bs.node_id); + if (rack) { + auto res = racks.insert(std::move(*rack)); + if (!res.second) { + is_rack_constraint_violated = true; + break; } } } + + if (is_rack_constraint_violated) { + _ntps_with_broken_rack_constraint.insert(ntp); + } else { + _ntps_with_broken_rack_constraint.erase(ntp); + } + _ntps_with_broken_rack_constraint_revision++; +} + +void partition_balancer_state::handle_ntp_delete(const model::ntp& ntp) { + _ntps_with_broken_rack_constraint.erase(ntp); + _ntps_with_broken_rack_constraint_revision++; } ss::future<> partition_balancer_state::apply_snapshot(const controller_snapshot& snap) { - if (!_partition_allocator.is_rack_awareness_enabled()) { - co_return; - } + bool rack_awareness = _partition_allocator.is_rack_awareness_enabled(); absl::flat_hash_map node2rack; for (const auto& [id, node] : snap.members.nodes) { @@ -111,6 +217,12 @@ partition_balancer_state::apply_snapshot(const controller_snapshot& snap) { _ntps_with_broken_rack_constraint.clear(); _ntps_with_broken_rack_constraint_revision++; + + // topic_table is being rebuilt from the snapshot, so any deltas the + // pinning-cache notification callback may have observed are no longer + // authoritative. Force a re-seed on the next read. + reset_pinning_cache(); + for (const auto& [ns_tp, topic] : snap.topics.topics) { for (const auto& [p_id, partition] : topic.partitions) { const std::vector* replicas @@ -123,7 +235,7 @@ partition_balancer_state::apply_snapshot(const controller_snapshot& snap) { } } - if (!is_rack_placement_valid(*replicas)) { + if (rack_awareness && !is_rack_placement_valid(*replicas)) { _ntps_with_broken_rack_constraint.emplace( ns_tp.ns, ns_tp.tp, p_id); _ntps_with_broken_rack_constraint_revision++; diff --git a/src/v/cluster/partition_balancer_state.h b/src/v/cluster/partition_balancer_state.h index 89c0a25c64aa8..7eb9d6707d279 100644 --- a/src/v/cluster/partition_balancer_state.h +++ b/src/v/cluster/partition_balancer_state.h @@ -13,6 +13,9 @@ #include "absl/container/btree_set.h" #include "absl/container/flat_hash_set.h" #include "cluster/fwd.h" +#include "cluster/notification.h" +#include "cluster/types.h" +#include "container/chunked_vector.h" #include "metrics/metrics.h" #include "model/fundamental.h" #include "model/metadata.h" @@ -32,6 +35,14 @@ class partition_balancer_state { ss::sharded&, ss::sharded&, ss::sharded&); + partition_balancer_state(const partition_balancer_state&) = delete; + partition_balancer_state(partition_balancer_state&&) = delete; + partition_balancer_state& + operator=(const partition_balancer_state&) = delete; + partition_balancer_state& operator=(partition_balancer_state&&) = delete; + ~partition_balancer_state() = default; + + ss::future<> stop(); topic_table& topics() const { return _topic_table; } @@ -39,6 +50,8 @@ class partition_balancer_state { node_status_table& node_status() const { return _node_status; } + bool is_rack_awareness_enabled() const; + const absl::btree_set& ntps_with_broken_rack_constraint() const { return _ntps_with_broken_rack_constraint; @@ -60,6 +73,29 @@ class partition_balancer_state { _ntps_with_broken_rack_constraint.end()); } + /// Topics whose `replicas_preference` is currently set to a non-"none" + /// type. Seeded by ensure_pinning_cache_seeded() from a full topic_table + /// scan on first use, then maintained incrementally via topic_table + /// delta notifications. Only populated on shard 0 (the balancer shard). + const absl::btree_set& + topics_with_replica_pinning() const { + return _topics_with_replica_pinning; + } + + /// Bootstrap the pinning cache from topic_table if not yet seeded. + /// Idempotent; cheap after the first call within a term. + void ensure_pinning_cache_seeded(); + + /// Invalidate the pinning cache so the next + /// ensure_pinning_cache_seeded() call re-scans topic_table. Also clears + /// the current contents so a caller reading between reset and re-seed + /// does not observe stale data. Call on raft0 term change or when + /// topic_table is rebuilt from a controller snapshot. + void reset_pinning_cache() { + _pinning_cache_seeded = false; + _topics_with_replica_pinning.clear(); + } + /// Called when the replica set of an ntp changes. Note that this doesn't /// account for in-progress moves - the function is called only once when /// the move is started. @@ -108,6 +144,13 @@ class partition_balancer_state { model::revision_id _ntps_with_broken_rack_constraint_revision; absl::flat_hash_set _nodes_to_rebalance; probe _probe; + + absl::btree_set _topics_with_replica_pinning; + bool _pinning_cache_seeded{false}; + cluster::notification_id_type _topic_deltas_handle{ + cluster::notification_id_type_invalid}; + + void handle_topic_deltas(const chunked_vector&); }; } // namespace cluster diff --git a/src/v/cluster/partition_balancer_types.cc b/src/v/cluster/partition_balancer_types.cc index adbb9ef55b0f7..daafd3e0f920d 100644 --- a/src/v/cluster/partition_balancer_types.cc +++ b/src/v/cluster/partition_balancer_types.cc @@ -160,6 +160,8 @@ fmt::iterator format_to(change_reason e, fmt::iterator out) { return fmt::format_to(out, "node_unavailable"); case change_reason::disk_full: return fmt::format_to(out, "disk_full"); + case change_reason::replica_pinning_repair: + return fmt::format_to(out, "replica_pinning_repair"); } } fmt::iterator format_to(reallocation_error e, fmt::iterator out) { diff --git a/src/v/cluster/partition_balancer_types.h b/src/v/cluster/partition_balancer_types.h index 1dff41a360e24..8d7406baa3980 100644 --- a/src/v/cluster/partition_balancer_types.h +++ b/src/v/cluster/partition_balancer_types.h @@ -135,6 +135,7 @@ enum class change_reason { node_decommissioning, node_unavailable, disk_full, + replica_pinning_repair, }; fmt::iterator format_to(change_reason rep, fmt::iterator); diff --git a/src/v/cluster/scheduling/constraints.cc b/src/v/cluster/scheduling/constraints.cc index 0a90a9f24a21c..75499489981c2 100644 --- a/src/v/cluster/scheduling/constraints.cc +++ b/src/v/cluster/scheduling/constraints.cc @@ -320,6 +320,41 @@ soft_constraint least_disk_filled( max_disk_usage_ratio, partition_size, node_disk_reports)); } +soft_constraint replica_pinning_preferred( + const config::replicas_preference& pref, const members_table& members) { + class impl : public soft_constraint::impl { + public: + impl(config::replicas_preference pref, const members_table& members) + : _pref(std::move(pref)) + , _members(members) {} + + soft_constraint_evaluator make_evaluator( + const allocated_partition&, + std::optional) const final { + return [this](const allocation_node& node) -> uint64_t { + auto rack = _members.get_node_rack_id(node.id()); + if (!rack) { + return uint64_t{0}; + } + auto group = _pref.group_index_for(*rack); + if (!group) { + return uint64_t{0}; + } + return soft_constraint::max_score / (*group + 1); + }; + } + + ss::sstring name() const final { return "replica pinning preferred"; } + + private: + config::replicas_preference _pref; + const members_table& _members; + }; + + return soft_constraint( + std::make_unique(config::replicas_preference{pref}, members)); +} + soft_constraint distinct_rack_preferred(const members_table& members) { return distinct_labels_preferred( rack_label.data(), diff --git a/src/v/cluster/scheduling/constraints.h b/src/v/cluster/scheduling/constraints.h index 0912f59f082ac..f064ef4fb474e 100644 --- a/src/v/cluster/scheduling/constraints.h +++ b/src/v/cluster/scheduling/constraints.h @@ -15,6 +15,7 @@ #include "cluster/partition_balancer_types.h" #include "cluster/scheduling/allocation_node.h" #include "cluster/scheduling/types.h" +#include "config/replicas_preference.h" #include "model/metadata.h" namespace cluster { @@ -137,6 +138,11 @@ distinct_labels_preferred(const char* label_name, Mapper&& mapper) { soft_constraint distinct_rack_preferred(const members_table&); +/// Pure priority scoring by rack group index. Rackless and unpreferred nodes +/// score 0. Group 0 = max_score, group k = max_score/(k+1). +soft_constraint replica_pinning_preferred( + const config::replicas_preference&, const members_table&); + /// Scores nodes based on replica count queried from the supplied map (as /// opposed to total node count). E.g. if the map represents topic partition /// counts, this constraint will implement topic-aware replica placement. diff --git a/src/v/cluster/scheduling/partition_allocator.cc b/src/v/cluster/scheduling/partition_allocator.cc index 9184d849764a8..b0309ef5747ef 100644 --- a/src/v/cluster/scheduling/partition_allocator.cc +++ b/src/v/cluster/scheduling/partition_allocator.cc @@ -337,6 +337,7 @@ partition_allocator::allocate(simple_allocation_request simple_req) { model::partition_id(p), simple_req.replication_factor); } req.existing_replica_counts = std::move(simple_req.existing_replica_counts); + req.replicas_preference = std::move(simple_req.replicas_preference); co_return co_await do_allocate(std::move(req)); } @@ -393,6 +394,11 @@ partition_allocator::do_allocate(allocation_request request) { } auto effective_constraints = default_constraints(); + if (request.replicas_preference) { + effective_constraints.ensure_new_level(); + effective_constraints.add(replica_pinning_preferred( + *request.replicas_preference, _members.local())); + } if (node2count) { effective_constraints.ensure_new_level(); effective_constraints.add( diff --git a/src/v/cluster/scheduling/types.h b/src/v/cluster/scheduling/types.h index 0658c99aabaeb..fede2517a9284 100644 --- a/src/v/cluster/scheduling/types.h +++ b/src/v/cluster/scheduling/types.h @@ -17,6 +17,7 @@ #include "base/oncore.h" #include "base/vassert.h" #include "cluster/types.h" +#include "config/replicas_preference.h" #include "model/fundamental.h" #include "model/metadata.h" @@ -357,6 +358,9 @@ struct allocation_request { // objective. std::optional existing_replica_counts; + // if present, the allocator inserts a replica pinning soft constraint + std::optional replicas_preference; + fmt::iterator format_to(fmt::iterator it) const; }; @@ -392,6 +396,9 @@ struct simple_allocation_request { // objective. std::optional existing_replica_counts; + // if present, the allocator inserts a replica pinning soft constraint + std::optional replicas_preference; + fmt::iterator format_to(fmt::iterator it) const; }; diff --git a/src/v/cluster/tests/BUILD b/src/v/cluster/tests/BUILD index ac5a29ba0f6fc..8bb586df6efdc 100644 --- a/src/v/cluster/tests/BUILD +++ b/src/v/cluster/tests/BUILD @@ -96,6 +96,25 @@ redpanda_test_cc_library( ], ) +redpanda_test_cc_library( + name = "partition_balancer_sim_fixture", + hdrs = [ + "partition_balancer_sim_fixture.h", + ], + deps = [ + ":partition_balancer_planner_fixture", + "//src/v/base", + "//src/v/cluster", + "//src/v/config", + "//src/v/model", + "//src/v/random:generators", + "@abseil-cpp//absl/container:btree", + "@abseil-cpp//absl/container:flat_hash_map", + "@boost//:test", + "@seastar", + ], +) + redpanda_test_cc_library( name = "rebalancing_tests_fixture", hdrs = [ @@ -532,14 +551,57 @@ redpanda_cc_btest( "//src/v/cluster:node_status_table", "//src/v/cluster:scheduling_allocation", "//src/v/cluster:topic_table", + "//src/v/config", "//src/v/test_utils:seastar_boost", - "//src/v/utils:stable_iterator", + "@abseil-cpp//absl/container:flat_hash_set", "@boost//:test", "@seastar", "@seastar//:testing", ], ) +redpanda_cc_gtest( + name = "partition_balancer_state_test", + timeout = "short", + srcs = [ + "partition_balancer_state_test.cc", + ], + deps = [ + "//src/v/cluster", + "//src/v/cluster:commands", + "//src/v/cluster:data_migrated_resources", + "//src/v/cluster:members_table", + "//src/v/cluster:node_status_table", + "//src/v/cluster:scheduling_allocation", + "//src/v/cluster:topic_table", + "//src/v/cluster:types", + "//src/v/config", + "//src/v/features", + "//src/v/model", + "//src/v/test_utils:gtest", + "@abseil-cpp//absl/container:btree", + "@googletest//:gtest", + "@seastar", + ], +) + +redpanda_cc_btest( + name = "partition_balancer_pinning_scoring_test", + timeout = "short", + srcs = [ + "partition_balancer_pinning_scoring_test.cc", + ], + deps = [ + ":partition_balancer_planner_fixture", + "//src/v/cluster", + "//src/v/cluster:scheduling_allocation", + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_btest( name = "partition_balancer_simulator_test", timeout = "long", @@ -547,8 +609,20 @@ redpanda_cc_btest( "partition_balancer_simulator_test.cc", ], deps = [ - ":partition_balancer_planner_fixture", - "//src/v/base", + ":partition_balancer_sim_fixture", + "//src/v/test_utils:seastar_boost", + "@seastar//:testing", + ], +) + +redpanda_cc_btest( + name = "partition_balancer_pinning_sim_test", + timeout = "long", + srcs = [ + "partition_balancer_pinning_sim_test.cc", + ], + deps = [ + ":partition_balancer_sim_fixture", "//src/v/cluster", "//src/v/cluster:commands", "//src/v/cluster:data_migrated_resources", @@ -557,6 +631,7 @@ redpanda_cc_btest( "//src/v/cluster:node_status_table", "//src/v/cluster:scheduling_allocation", "//src/v/cluster:topic_table", + "//src/v/config", "//src/v/model", "//src/v/random:generators", "//src/v/test_utils:seastar_boost", diff --git a/src/v/cluster/tests/partition_balancer_pinning_scoring_test.cc b/src/v/cluster/tests/partition_balancer_pinning_scoring_test.cc new file mode 100644 index 0000000000000..686d3d5f67bfe --- /dev/null +++ b/src/v/cluster/tests/partition_balancer_pinning_scoring_test.cc @@ -0,0 +1,190 @@ +// Copyright 2026 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "cluster/scheduling/constraints.h" +#include "cluster/tests/partition_balancer_planner_fixture.h" +#include "config/replicas_preference.h" +#include "test_utils/boost_fixture.h" + +#include + +/* + * Constraint scoring tests for replica_pinning_preferred. + * + * These verify the soft constraint evaluator produces the correct scores: + * group 0 = max_score, group k = max_score/(k+1), + * rackless = 0, unpreferred = 0. + */ +FIXTURE_TEST( + test_pinning_constraint_group_zero_scores_max, + partition_balancer_planner_fixture) { + allocator_register_nodes(3, {"A", "B", "C"}); + + // Preference: A (group 0) + auto pref = config::replicas_preference::parse("racks: A"); + + auto constraint = cluster::replica_pinning_preferred( + pref, workers.members.local()); + + // Create a partition to get a valid allocated_partition reference. + create_topic("dummy", 1, 1); + auto ap = workers.allocator.local().make_allocated_partition( + model::ntp(test_ns, model::topic("dummy"), model::partition_id(0)), + {model::broker_shard{model::node_id(0), 0}}); + + auto evaluator = constraint.make_evaluator(ap, std::nullopt); + + const auto& nodes = workers.allocator.local().state().allocation_nodes(); + + // node 0 is on rack A = group 0 => max_score + auto score_node0 = evaluator(*nodes.at(model::node_id(0))); + BOOST_REQUIRE_EQUAL(score_node0, cluster::soft_constraint::max_score); +} + +FIXTURE_TEST( + test_pinning_constraint_group_k_scoring, partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"A", "B", "C", "D"}); + + // Preference: A (group 0), {B, C} (group 1), D (group 2) + auto pref = config::replicas_preference::parse("racks: A, {B, C}, D"); + + auto constraint = cluster::replica_pinning_preferred( + pref, workers.members.local()); + + create_topic("dummy", 1, 1); + auto ap = workers.allocator.local().make_allocated_partition( + model::ntp(test_ns, model::topic("dummy"), model::partition_id(0)), + {model::broker_shard{model::node_id(0), 0}}); + + auto evaluator = constraint.make_evaluator(ap, std::nullopt); + const auto& nodes = workers.allocator.local().state().allocation_nodes(); + + constexpr auto max_score = cluster::soft_constraint::max_score; + + // group 0 (A) => max_score / 1 = 10,000,000 + auto score_a = evaluator(*nodes.at(model::node_id(0))); + BOOST_REQUIRE_EQUAL(score_a, max_score); + + // group 1 (B) => max_score / 2 = 5,000,000 + auto score_b = evaluator(*nodes.at(model::node_id(1))); + BOOST_REQUIRE_EQUAL(score_b, max_score / 2); + + // group 1 (C) => max_score / 2 = 5,000,000 (same group as B) + auto score_c = evaluator(*nodes.at(model::node_id(2))); + BOOST_REQUIRE_EQUAL(score_c, max_score / 2); + + // group 2 (D) => max_score / 3 = 3,333,333 + auto score_d = evaluator(*nodes.at(model::node_id(3))); + BOOST_REQUIRE_EQUAL(score_d, max_score / 3); +} + +FIXTURE_TEST( + test_pinning_constraint_same_group_same_score, + partition_balancer_planner_fixture) { + allocator_register_nodes(3, {"B", "C", "A"}); + + // Preference: A (group 0), {B, C} (group 1) + auto pref = config::replicas_preference::parse("racks: A, {B, C}"); + + auto constraint = cluster::replica_pinning_preferred( + pref, workers.members.local()); + + create_topic("dummy", 1, 1); + auto ap = workers.allocator.local().make_allocated_partition( + model::ntp(test_ns, model::topic("dummy"), model::partition_id(0)), + {model::broker_shard{model::node_id(0), 0}}); + + auto evaluator = constraint.make_evaluator(ap, std::nullopt); + const auto& nodes = workers.allocator.local().state().allocation_nodes(); + + // node 0 = B (group 1), node 1 = C (group 1) + auto score_b = evaluator(*nodes.at(model::node_id(0))); + auto score_c = evaluator(*nodes.at(model::node_id(1))); + BOOST_REQUIRE_EQUAL(score_b, score_c); +} + +FIXTURE_TEST( + test_pinning_constraint_unpreferred_scores_zero, + partition_balancer_planner_fixture) { + allocator_register_nodes(3, {"A", "B", "X"}); + + auto pref = config::replicas_preference::parse("racks: A, B"); + + auto constraint = cluster::replica_pinning_preferred( + pref, workers.members.local()); + + create_topic("dummy", 1, 1); + auto ap = workers.allocator.local().make_allocated_partition( + model::ntp(test_ns, model::topic("dummy"), model::partition_id(0)), + {model::broker_shard{model::node_id(0), 0}}); + + auto evaluator = constraint.make_evaluator(ap, std::nullopt); + const auto& nodes = workers.allocator.local().state().allocation_nodes(); + + // node 2 = X, not in preference => 0 + auto score_x = evaluator(*nodes.at(model::node_id(2))); + BOOST_REQUIRE_EQUAL(score_x, uint64_t{0}); + + // node 0 = A (group 0) should still be max_score + auto score_a = evaluator(*nodes.at(model::node_id(0))); + BOOST_REQUIRE_EQUAL(score_a, cluster::soft_constraint::max_score); +} + +FIXTURE_TEST( + test_pinning_constraint_rackless_scores_zero, + partition_balancer_planner_fixture) { + // Register 2 nodes with racks and 1 without. + // allocator_register_nodes doesn't support per-node optional racks, + // so register manually. + auto& members_table = workers.members.local(); + + // node 0: rack_A + workers.allocator.local().register_node( + create_allocation_node(model::node_id(0), 4)); + // node 1: no rack + workers.allocator.local().register_node( + create_allocation_node(model::node_id(1), 4)); + + std::vector brokers; + brokers.emplace_back( + model::node_id(0), + net::unresolved_address{}, + net::unresolved_address{}, + model::rack_id{"A"}, + model::broker_properties{.cores = 4}); + brokers.emplace_back( + model::node_id(1), + net::unresolved_address{}, + net::unresolved_address{}, + std::nullopt, + model::broker_properties{.cores = 4}); + members_table.set_initial_brokers(std::move(brokers)); + last_node_idx = 2; + + auto pref = config::replicas_preference::parse("racks: A"); + + auto constraint = cluster::replica_pinning_preferred( + pref, workers.members.local()); + + create_topic("dummy", 1, 1); + auto ap = workers.allocator.local().make_allocated_partition( + model::ntp(test_ns, model::topic("dummy"), model::partition_id(0)), + {model::broker_shard{model::node_id(0), 0}}); + + auto evaluator = constraint.make_evaluator(ap, std::nullopt); + const auto& nodes = workers.allocator.local().state().allocation_nodes(); + + // node 0 = A (group 0) => max_score + auto score_a = evaluator(*nodes.at(model::node_id(0))); + BOOST_REQUIRE_EQUAL(score_a, cluster::soft_constraint::max_score); + + // node 1 = rackless => 0 + auto score_none = evaluator(*nodes.at(model::node_id(1))); + BOOST_REQUIRE_EQUAL(score_none, uint64_t{0}); +} diff --git a/src/v/cluster/tests/partition_balancer_pinning_sim_test.cc b/src/v/cluster/tests/partition_balancer_pinning_sim_test.cc new file mode 100644 index 0000000000000..c9c43241be8ca --- /dev/null +++ b/src/v/cluster/tests/partition_balancer_pinning_sim_test.cc @@ -0,0 +1,176 @@ +// Copyright 2026 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "cluster/commands.h" +#include "cluster/tests/partition_balancer_sim_fixture.h" +#include "config/replicas_preference.h" +#include "test_utils/boost_fixture.h" + +#include + +struct partition_balancer_pinning_sim_no_rack_fixture + : partition_balancer_sim_fixture { + partition_balancer_pinning_sim_no_rack_fixture() + : partition_balancer_sim_fixture(false) {} +}; + +// Rack awareness ON: 6 nodes across 4 racks. Preference "racks: A". +// With rack awareness, each partition picks 3 of 4 racks. Some partitions +// will miss rack A entirely, creating pinning violations. +// Expected: balancer repairs all violations by moving replicas to rack A. +FIXTURE_TEST( + test_replica_pinning_repair_rack_on, partition_balancer_sim_fixture) { + add_node(model::node_id{0}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{1}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{2}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{3}, 100_GiB, 4, model::rack_id{"C"}); + add_node(model::node_id{4}, 100_GiB, 4, model::rack_id{"C"}); + add_node(model::node_id{5}, 100_GiB, 4, model::rack_id{"D"}); + + add_topic("pinned_topic", 30, 3, 10_MiB); + + auto tp_ns = model::topic_namespace(test_ns, model::topic("pinned_topic")); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference::parse( + "racks: A"); + dispatch_topic_command( + cluster::update_topic_properties_cmd{tp_ns, std::move(updates)}); + + do_run_balancer(); + auto initial_violations = last_pinning_violations_count(); + logger.info("initial pinning violations: {}", initial_violations); + + BOOST_REQUIRE_GT(initial_violations, 0); + + BOOST_REQUIRE(run_to_completion(500)); + + auto final_violations = last_pinning_violations_count(); + logger.info("final pinning violations: {}", final_violations); + + BOOST_REQUIRE_EQUAL(final_violations, 0); +} + +// Rack awareness OFF: 4 nodes in rack A, 1 in B, 1 in C. +// Preference "racks: A". With rack awareness OFF, capacity of A = 4 nodes, +// which can hold all RF=3 replicas. Ideal = {A, A, A}. +// Some partitions will have replicas on B/C, creating violations. +// Expected: balancer repairs all violations by moving replicas to A nodes. +FIXTURE_TEST( + test_replica_pinning_repair_rack_off, + partition_balancer_pinning_sim_no_rack_fixture) { + add_node(model::node_id{0}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{1}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{2}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{3}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{4}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{5}, 100_GiB, 4, model::rack_id{"C"}); + + add_topic("pinned_topic", 30, 3, 10_MiB); + + auto tp_ns = model::topic_namespace(test_ns, model::topic("pinned_topic")); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference::parse( + "racks: A"); + dispatch_topic_command( + cluster::update_topic_properties_cmd{tp_ns, std::move(updates)}); + + do_run_balancer(); + auto initial_violations = last_pinning_violations_count(); + logger.info("initial pinning violations: {}", initial_violations); + + BOOST_REQUIRE_GT(initial_violations, 0); + + BOOST_REQUIRE(run_to_completion(500)); + + auto final_violations = last_pinning_violations_count(); + logger.info("final pinning violations: {}", final_violations); + + BOOST_REQUIRE_EQUAL(final_violations, 0); +} + +// Structurally under-capacity: pref `racks: A` with rack-aware ON and only +// one A rack means capacity[0] = 1 while RF = 3. Ideal = [0, unpref, unpref]. +// Every initial placement already has an A replica (rack-aware forces 3 +// distinct racks and A is the only one available), so actual matches ideal +// on day 1 and no repair moves should be generated. Also exercises the +// "pinning cannot be fully satisfied" warning log path. +FIXTURE_TEST( + test_replica_pinning_no_spurious_moves_when_under_capacity, + partition_balancer_sim_fixture) { + add_node(model::node_id{0}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{1}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{2}, 100_GiB, 4, model::rack_id{"C"}); + + add_topic("pinned_topic", 30, 3, 10_MiB); + + auto tp_ns = model::topic_namespace(test_ns, model::topic("pinned_topic")); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference::parse( + "racks: A"); + dispatch_topic_command( + cluster::update_topic_properties_cmd{tp_ns, std::move(updates)}); + + auto plan_data = do_run_balancer(); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 0); + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 0); + + // Run a few more ticks to ensure steady-state stays quiet. + BOOST_REQUIRE(run_to_completion(50)); + BOOST_REQUIRE_EQUAL(last_pinning_violations_count(), 0); +} + +// Anti-spurious-move guard: preferred-group capacity is 2 (rack A has 2 +// nodes, rack-aware OFF) with RF=3, so ideal = [0, 0, unpref]. Node 1 (the +// second rack-A node) is marked disk-full so the allocator rejects it. For +// partitions that reach a state with exactly one A replica and are asked +// to move a B replica, no strictly-better destination exists -- the other +// A node is either already a replica (distinct-nodes excluded) or +// disk-full (hard-excluded) -- so the allocator would pick a random +// same-rack B peer. The guard should revert that would-be move, leaving +// the partition at its current (still-violating) placement without +// churning replicas. +FIXTURE_TEST( + test_replica_pinning_guard_blocks_spurious_moves, + partition_balancer_pinning_sim_no_rack_fixture) { + add_node(model::node_id{0}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{1}, 100_GiB, 4, model::rack_id{"A"}); + add_node(model::node_id{2}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{3}, 100_GiB, 4, model::rack_id{"B"}); + add_node(model::node_id{4}, 100_GiB, 4, model::rack_id{"B"}); + + add_topic("pinned_topic", 30, 3, 10_MiB); + + auto tp_ns = model::topic_namespace(test_ns, model::topic("pinned_topic")); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference::parse( + "racks: A"); + dispatch_topic_command( + cluster::update_topic_properties_cmd{tp_ns, std::move(updates)}); + + // Wedge the second A node: any pinning move toward group 0 must now + // either land back on the already-present A node (distinct-nodes + // excluded) or on the disk-full A node (hard-excluded). + set_node_disk_fill(model::node_id{1}, 0.95); + + // Run to completion. Partitions with zero A replicas can still improve + // (move one to node 0). Partitions with exactly one A replica cannot + // improve; the guard must catch the would-be same-rack B-to-B move. + run_to_completion(500); + + // Steady state: at least one follow-up tick must produce no new + // reassignments. If the guard were buggy, pinning would keep shuffling + // B-rack replicas indefinitely and never reach a fixed point. + auto steady = do_run_balancer(); + BOOST_REQUIRE_EQUAL(steady.reassignments.size(), 0); + BOOST_REQUIRE_EQUAL(steady.cancellations.size(), 0); +} diff --git a/src/v/cluster/tests/partition_balancer_planner_fixture.h b/src/v/cluster/tests/partition_balancer_planner_fixture.h index 6bbacac7d234f..daf05eb50eb79 100644 --- a/src/v/cluster/tests/partition_balancer_planner_fixture.h +++ b/src/v/cluster/tests/partition_balancer_planner_fixture.h @@ -69,7 +69,7 @@ create_allocation_node(model::node_id nid, uint32_t cores) { struct controller_workers { public: - controller_workers() + explicit controller_workers(bool rack_awareness_enabled = true) : dispatcher(allocator, table, state) { migrated_resources.start().get(); table @@ -96,7 +96,7 @@ struct controller_workers { {model::kafka_audit_logging_topic, "__consumer_offsets", "_schemas"}}), - config::mock_binding(true)) + config::mock_binding(rack_awareness_enabled)) .get(); config::shard_local_cfg().topic_memory_per_partition.set_value( std::nullopt); @@ -185,6 +185,10 @@ struct controller_workers { }; struct partition_balancer_planner_fixture { + explicit partition_balancer_planner_fixture( + bool rack_awareness_enabled = true) + : workers(rack_awareness_enabled) {} + cluster::partition_balancer_planner make_planner( model::partition_autobalancing_mode mode = model::partition_autobalancing_mode::continuous, @@ -428,3 +432,10 @@ struct partition_balancer_planner_fixture { int last_node_idx{}; ss::abort_source as; }; + +/// Fixture variant with rack awareness disabled in the allocator. +struct partition_balancer_planner_no_rack_fixture + : partition_balancer_planner_fixture { + partition_balancer_planner_no_rack_fixture() + : partition_balancer_planner_fixture(false) {} +}; diff --git a/src/v/cluster/tests/partition_balancer_planner_test.cc b/src/v/cluster/tests/partition_balancer_planner_test.cc index e2105b8f54668..bf3ed5854b37b 100644 --- a/src/v/cluster/tests/partition_balancer_planner_test.cc +++ b/src/v/cluster/tests/partition_balancer_planner_test.cc @@ -8,10 +8,12 @@ // by the Apache License, Version 2.0 #include "base/vlog.h" +#include "cluster/commands.h" #include "cluster/controller_snapshot.h" #include "cluster/data_migrated_resources.h" #include "cluster/health_monitor_types.h" #include "cluster/tests/partition_balancer_planner_fixture.h" +#include "config/replicas_preference.h" #include "test_utils/boost_fixture.h" #include @@ -1073,3 +1075,458 @@ FIXTURE_TEST( BOOST_REQUIRE(failures > 0); BOOST_REQUIRE(reassignments > 0); } + +/* + * 4 nodes in 4 racks; 1 topic with 1 partition, RF=3. + * Replicas placed on racks B, C, D. Preference is "racks: A, B, C". + * Planner should move the replica from rack D to rack A. + */ +FIXTURE_TEST(test_replica_pinning_repair, partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + create_topic("topic-1", {{n(1), n(2), n(3)}}); + + // Set replicas_preference on the topic. + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + check_violations(plan_data, {}, {}); + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 1); + + // The reassignment should move the replica from node 3 (rack_D) to + // node 0 (rack_A). + const auto& new_replicas = plan_data.reassignments[0].allocated.replicas(); + BOOST_REQUIRE_EQUAL(new_replicas.size(), 3); + absl::flat_hash_set new_nodes; + for (const auto& bs : new_replicas) { + new_nodes.insert(bs.node_id); + } + BOOST_REQUIRE(new_nodes.contains(n(0))); + BOOST_REQUIRE(!new_nodes.contains(n(3))); + BOOST_REQUIRE_EQUAL(plan_data.cancellations.size(), 0); + BOOST_REQUIRE_EQUAL(plan_data.failed_actions_count, 0); +} + +/* + * Test is_pinning_violated detection through topic property changes and + * partition lifecycle events, and that the planner detects and repairs + * violations. + */ +FIXTURE_TEST( + test_planner_reports_pinning_violations_count, + partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + + model::topic topic{"topic-1"}; + + // Create topic with replicas: p0 on {B, C, D}, p1 on {A, B, C}. + create_topic(topic(), {{n(1), n(2), n(3)}, {n(0), n(1), n(2)}}); + + // Set preference: racks: rack_A, rack_B, rack_C. + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns(topic()), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + // Run planner — p0 on {B, C, D} is violated (rack_D not in preference), + // p1 on {A, B, C} is satisfied. Should produce 1 reassignment. + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 1); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 1); +} + +/* + * Test that removing replicas_preference clears all pinning violations + * for a topic. + */ +FIXTURE_TEST( + test_pinning_violations_cleared_when_preference_removed, + partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + + model::topic topic{"topic-1"}; + model::ntp ntp0{test_ns, topic, 0}; + + // Create topic with replicas on B, C, D (nodes 1, 2, 3). + create_topic(topic(), {{n(1), n(2), n(3)}}); + + // Set preference: rack_A, rack_B, rack_C -> p0 violated (on rack_D). + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns(topic()), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + // Run planner to detect violations. + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 1); + + // Remove preference by setting to none. + cluster::incremental_topic_updates remove_updates; + remove_updates.replicas_preference.op + = cluster::incremental_update_operation::remove; + cluster::update_topic_properties_cmd remove_cmd{ + make_tp_ns(topic()), std::move(remove_updates)}; + workers.dispatch_topic_command(std::move(remove_cmd)); + + // Re-run planner — no violations after preference removed. + auto planner2 = make_planner(); + auto plan_data2 = planner2.plan_actions(hr, as).get(); + BOOST_REQUIRE_EQUAL(plan_data2.last_pinning_violations_count, 0); +} + +/* + * Test that when all replicas are already on preferred racks, no pinning + * violation is detected and the planner produces no moves. + */ +FIXTURE_TEST( + test_replica_pinning_no_op_when_optimal, partition_balancer_planner_fixture) { + allocator_register_nodes(3, {"rack_A", "rack_B", "rack_C"}); + create_topic("topic-1", {{n(0), n(1), n(2)}}); + + // Set preference: rack_A, rack_B, rack_C — matches current placement. + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 0); + BOOST_REQUIRE_EQUAL(plan_data.failed_actions_count, 0); +} + +/* + * Test that multiple partitions with pinning violations are all tracked + * and the planner generates repair actions for each. + */ +FIXTURE_TEST( + test_replica_pinning_multiple_violations, + partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + + model::topic topic{"topic-1"}; + model::ntp ntp0{test_ns, topic, 0}; + model::ntp ntp1{test_ns, topic, 1}; + + // Both partitions have a replica on rack_D (node 3). + create_topic(topic(), {{n(1), n(2), n(3)}, {n(1), n(2), n(3)}}); + + // Set preference: rack_A, rack_B, rack_C. + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns(topic()), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + // Both partitions should get repair actions. + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 2); + BOOST_REQUIRE_EQUAL(plan_data.failed_actions_count, 0); +} + +/* + * Test that is_pinning_violated returns false when preference type is none + * (exercised through the state tracking path). + */ +FIXTURE_TEST( + test_state_no_violation_with_preference_none, + partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + + // Create topic on rack_B, rack_C, rack_D. + create_topic("topic-1", {{n(1), n(2), n(3)}}); + + // Set preference to type none explicitly. + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference{}; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 0); + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 0); +} + +/* + * Test A: rack awareness OFF, preference "racks: rack_A". + * 3 nodes in rack_A (nodes 0,1,2), 1 in rack_B (node 3), 1 in rack_C (node 4). + * RF=3, replicas on nodes 0, 3, 4 (racks A, B, C). + * With rack awareness OFF, capacity of group 0 (rack_A) = 3 nodes, which can + * hold all 3 replicas. So ideal = {A, A, A}. Actual has replicas on B and C. + * Expected: violation detected, repair moves replicas off B and C to A nodes. + */ +FIXTURE_TEST( + test_pinning_rack_off_all_on_preferred, + partition_balancer_planner_no_rack_fixture) { + // 3 nodes in rack_A, 1 in rack_B, 1 in rack_C + allocator_register_nodes( + 5, {"rack_A", "rack_A", "rack_A", "rack_B", "rack_C"}); + // RF=3, replicas on nodes 0 (A), 3 (B), 4 (C) + create_topic("topic-1", {{n(0), n(3), n(4)}}); + + auto pref = config::replicas_preference::parse("racks: rack_A"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + // Planner should generate one reassignment moving the worst replica to A. + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 1); + + // The reassignment should have moved one non-A replica to a rack_A node. + // Original: {0(A), 3(B), 4(C)}. After one repair pass, one of B/C moves + // to an A node, producing e.g. {0(A), 3(B), 1(A)} or {0(A), 1(A), 4(C)}. + const auto& new_replicas = plan_data.reassignments[0].allocated.replicas(); + BOOST_REQUIRE_EQUAL(new_replicas.size(), 3); + int rack_a_count = 0; + for (const auto& bs : new_replicas) { + auto rack = workers.members.local().get_node_rack_id(bs.node_id); + BOOST_REQUIRE(rack); + if (*rack == model::rack_id{"rack_A"}) { + rack_a_count++; + } + } + // Should have strictly more A replicas than the original (which had 1). + BOOST_REQUIRE_GE(rack_a_count, 2); +} + +/* + * Test B: rack awareness OFF, preference "racks: rack_A, rack_B". + * 4 nodes in rack_A (nodes 0-3), 1 in rack_B (node 4). + * RF=3, replicas on nodes 0, 1, 4 (racks A, A, B). + * With rack awareness OFF, capacity of group 0 = 4 nodes. Ideal fills A first: + * {A, A, A}. Actual has one on B. + * Expected: violation detected. + */ +FIXTURE_TEST( + test_pinning_rack_off_overflow_not_needed, + partition_balancer_planner_no_rack_fixture) { + // 4 nodes in rack_A, 1 in rack_B + allocator_register_nodes( + 5, {"rack_A", "rack_A", "rack_A", "rack_A", "rack_B"}); + // RF=3, replicas on nodes 0 (A), 1 (A), 4 (B) + create_topic("topic-1", {{n(0), n(1), n(4)}}); + + auto pref = config::replicas_preference::parse("racks: rack_A, rack_B"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + // Violation: A has capacity 4 > RF 3, so all 3 should fit on A. + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 1); +} + +/* + * Test C: rack awareness OFF, preference "racks: rack_A, rack_B, rack_C". + * 2 nodes in A (0,1), 2 in B (2,3), 1 in C (4). + * RF=3, replicas on nodes 0, 2, 4 (racks A, B, C). + * With rack awareness OFF: A capacity=2, B capacity=2, C capacity=1. + * Ideal: fill A first -> 2 on A, overflow 1 to B. Ideal = {A, A, B}. + * Actual = {A, B, C}. Violation: C replica should move to A. + */ +FIXTURE_TEST( + test_pinning_rack_off_fill_overflow, + partition_balancer_planner_no_rack_fixture) { + allocator_register_nodes( + 5, {"rack_A", "rack_A", "rack_B", "rack_B", "rack_C"}); + create_topic("topic-1", {{n(0), n(2), n(4)}}); + + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + // Violation: ideal is A=2, B=1 but actual is A=1, B=1, C=1. + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 1); +} + +/* + * Test D: rack awareness OFF, preference "racks: rack_A, rack_B". No-op. + * 2 nodes in A (0,1), 2 in B (2,3), 1 in C (4). + * RF=3, replicas on nodes 0, 1, 2 (racks A, A, B). + * With rack awareness OFF: A capacity=2, B capacity=2. Fill A first -> 2, then + * overflow 1 to B. Ideal = {A, A, B} = actual. + * Expected: NO violation. + */ +FIXTURE_TEST( + test_pinning_rack_off_no_violation_matches_ideal, + partition_balancer_planner_no_rack_fixture) { + allocator_register_nodes( + 5, {"rack_A", "rack_A", "rack_B", "rack_B", "rack_C"}); + create_topic("topic-1", {{n(0), n(1), n(2)}}); + + auto pref = config::replicas_preference::parse("racks: rack_A, rack_B"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 0); +} + +/* + * Test E: rack awareness ON, preference "racks: rack_A, rack_B, rack_C". + * 3 nodes in rack_A (0,1,2), 1 in B (3), 1 in C (4). + * RF=3, replicas on nodes 0, 3, 4 (racks A, B, C). + * With rack awareness ON, each rack can hold at most 1 replica (rack + * diversity). So capacity per group: A=1, B=1, C=1. Ideal = {A, B, C} = actual. + * Expected: NO violation. + */ +FIXTURE_TEST( + test_pinning_rack_on_diversity_caps_capacity, + partition_balancer_planner_fixture) { + allocator_register_nodes( + 5, {"rack_A", "rack_A", "rack_A", "rack_B", "rack_C"}); + create_topic("topic-1", {{n(0), n(3), n(4)}}); + + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + // No violation: rack awareness ON caps each rack at 1 replica. + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 0); +} + +/* + * Test that plan_actions returns empty when pinning is enabled and all + * replicas already satisfy the preference. + */ +FIXTURE_TEST( + test_plan_actions_early_exits_when_pinning_satisfied, + partition_balancer_planner_fixture) { + allocator_register_nodes(3, {"rack_A", "rack_B", "rack_C"}); + create_topic("topic-1", {{n(0), n(1), n(2)}}); + + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + // Pinning enabled, no violations -> empty early-exit. + BOOST_REQUIRE( + plan_data.status == cluster::partition_balancer_planner::status::empty); + BOOST_REQUIRE_EQUAL(plan_data.reassignments.size(), 0); + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 0); +} + +/* + * Test that plan_data.last_pinning_violations_count is populated even + * when violations cause the planner to proceed past the early-exit. + */ +FIXTURE_TEST( + test_plan_actions_exposes_pinning_count_pre_action, + partition_balancer_planner_fixture) { + allocator_register_nodes(4, {"rack_A", "rack_B", "rack_C", "rack_D"}); + create_topic("topic-1", {{n(1), n(2), n(3)}}); + + auto pref = config::replicas_preference::parse( + "racks: rack_A, rack_B, rack_C"); + cluster::incremental_topic_updates updates; + updates.replicas_preference.op = cluster::incremental_update_operation::set; + updates.replicas_preference.value = pref; + cluster::update_topic_properties_cmd cmd{ + make_tp_ns("topic-1"), std::move(updates)}; + workers.dispatch_topic_command(std::move(cmd)); + + auto hr = create_health_report(); + populate_node_status_table().get(); + auto planner = make_planner(); + auto plan_data = planner.plan_actions(hr, as).get(); + + BOOST_REQUIRE_EQUAL(plan_data.last_pinning_violations_count, 1); + BOOST_REQUIRE( + plan_data.status != cluster::partition_balancer_planner::status::empty); +} diff --git a/src/v/cluster/tests/partition_balancer_sim_fixture.h b/src/v/cluster/tests/partition_balancer_sim_fixture.h new file mode 100644 index 0000000000000..9cf1c8055bcf0 --- /dev/null +++ b/src/v/cluster/tests/partition_balancer_sim_fixture.h @@ -0,0 +1,770 @@ +/* + * Copyright 2026 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#pragma once + +#include "absl/container/btree_map.h" +#include "absl/container/flat_hash_map.h" +#include "base/vlog.h" +#include "cluster/commands.h" +#include "cluster/health_monitor_types.h" +#include "cluster/tests/partition_balancer_planner_fixture.h" +#include "model/fundamental.h" +#include "model/metadata.h" +#include "random/generators.h" + +#include + +#include + +#include +#include + +static ss::logger logger("balancer_sim"); + +static constexpr size_t produce_batch_size = 1_MiB; + +// Recovery rate magic numbers. Recovery bandwidth is not directly controllable, +// but it is related to the tick length: node bandwidth allows exactly 1 batch +// per tick. For 100MiB/s bandwidth this gives us 5ms ticks. +static constexpr size_t recovery_batch_size = 512_KiB; +static constexpr size_t recovery_throttle_burst = 100_MiB; +static constexpr size_t recovery_throttle_ticks_between_refills = 10; + +static constexpr auto node_responsiveness_timeout = std::chrono::seconds(10); + +class partition_balancer_sim_fixture { +public: + explicit partition_balancer_sim_fixture(bool rack_awareness_enabled = true) + : _workers(rack_awareness_enabled) {} + + void add_node( + model::node_id id, + size_t total_size, + uint32_t n_cores = 4, + std::optional rack = std::nullopt) { + vassert(!_nodes.contains(id), "duplicate node id: {}", id); + + model::broker broker( + id, + net::unresolved_address{}, + net::unresolved_address{}, + rack, + model::broker_properties{ + .cores = n_cores, + .available_memory_gb = 2, + .available_disk_gb = uint32_t(total_size / 1_GiB), + .available_memory_bytes = 2 * 1_GiB}); + + BOOST_REQUIRE(!_workers.members.local().apply( + model::offset{}, cluster::add_node_cmd(id, broker))); + _workers.allocator.local().register_node( + std::make_unique( + id, + n_cores, + config::mock_binding(1000), + config::mock_binding(0), + config::mock_binding>({}))); + + // add some random initial used space + size_t initial_used = random_generators::get_int( + 3 * total_size / 100, 5 * total_size / 100); + + _nodes.emplace( + id, node_state{.id = id, .total = total_size, .used = initial_used}); + } + + void remove_node(model::node_id id) { _nodes.erase(id); } + + const auto& nodes() const { return _nodes; } + + const cluster::allocation_state::underlying_t& allocation_nodes() const { + return _workers.allocator.local().state().allocation_nodes(); + } + + const cluster::topic_table& topics() const { + return _workers.table.local(); + } + + template + void dispatch_topic_command(Cmd cmd) { + _workers.dispatch_topic_command(std::move(cmd)); + } + + const cluster::partition_balancer_state& balancer_state() const { + return _workers.state.local(); + } + + cluster::partition_balancer_state& balancer_state() { + return _workers.state.local(); + } + + /// Violation count observed by the planner during the most recent tick. + /// Populated by run_balancer() / do_run_balancer(). Zero before any + /// tick has run. + size_t last_pinning_violations_count() const { + return _last_pinning_violations_count; + } + + void add_topic( + const ss::sstring& name, + int partitions, + int16_t replication_factor, + size_t mean_partition_size, + std::optional stddev = std::nullopt) { + auto tp_ns = model::topic_namespace(test_ns, model::topic(name)); + auto topic_conf = _workers.make_tp_configuration( + name, partitions, replication_factor); + _workers.dispatch_topic_command( + cluster::create_topic_cmd(tp_ns, topic_conf)); + + if (!stddev) { + stddev = produce_batch_size + * sqrt(double(mean_partition_size) / produce_batch_size); + } + + std::normal_distribution<> dist(0, 1); + for (const auto& as : topic_conf.assignments) { + model::ntp ntp{tp_ns.ns, tp_ns.tp, as.id}; + auto jitter = std::max( + -int64_t(mean_partition_size), + int64_t(*stddev * dist(random_generators::global().engine()))); + auto size = mean_partition_size + jitter; + auto partition = ss::make_lw_shared(ntp, size); + _partitions.emplace(ntp, partition); + + for (const auto& bs : as.replicas) { + auto& node = _nodes.at(bs.node_id); + node.replicas[ntp] = replica{ + .partition = partition, .local_size = size}; + node.used += size; + } + + elect_leader(ntp); + + logger.info( + "added ntp {}, replicas {}, size {}", + ntp, + as.replicas, + human::bytes(size)); + } + _total_replicas += static_cast(replication_factor) * partitions; + } + + void set_decommissioning(model::node_id id) { + _workers.set_decommissioning(id); + } + + /// Set a node's disk usage ratio. Used by tests to simulate a disk-full + /// node that the allocator excludes from reallocation destinations. + void set_node_disk_fill(model::node_id id, double ratio) { + auto it = _nodes.find(id); + vassert(it != _nodes.end(), "unknown node id: {}", id); + it->second.used = static_cast(double(it->second.total) * ratio); + } + + void add_node_to_rebalance(model::node_id id) { + _workers.state.local().add_node_to_rebalance(id); + } + + size_t cur_tick() const { return _cur_tick; } + + bool run_to_completion( + size_t max_balancer_actions, std::function tick_cb = [] {}) { + print_state(); + + size_t num_actions = 0; + while (num_actions < max_balancer_actions) { + tick(); + tick_cb(); + + if (should_schedule_balancer_run()) { + num_actions += run_balancer(); + print_state(); + + if (last_run_in_progress_updates() == 0) { + logger.info("finished after {} ticks", cur_tick()); + return true; + } + } + } + + print_state(); + return false; + } + + void tick() { + for (auto& [id, node] : _nodes) { + node.ticks_since_refill += 1; + if ( + node.ticks_since_refill + >= recovery_throttle_ticks_between_refills) { + node.ticks_since_refill = 0; + + if (node.bandwidth_left <= recovery_throttle_burst) { + node.bandwidth_left + += recovery_throttle_ticks_between_refills + * recovery_batch_size; + } + } + } + + absl::flat_hash_map> + node2pending_rs; + for (const auto& ntp : + _workers.table.local().all_updates_in_progress()) { + auto part = _partitions.at(ntp); + if (!part->leader) { + continue; + } + + auto learners = get_learners(ntp); + for (const auto& id : learners) { + node2pending_rs[*part->leader].push_back( + recovery_stream{.ntp = ntp, .from = *part->leader, .to = id}); + } + } + + for (auto& [node_id, recovery_streams] : node2pending_rs) { + auto& node = _nodes.at(node_id); + + std::shuffle( + recovery_streams.begin(), + recovery_streams.end(), + random_generators::global().engine()); + + for (const auto& rs : recovery_streams) { + if (node.bandwidth_left >= recovery_batch_size) { + perform_recovery_step(rs); + node.bandwidth_left -= recovery_batch_size; + } else { + break; + } + } + } + + for (const auto& ntp : + _workers.table.local().all_updates_in_progress()) { + maybe_finish_update(ntp); + } + + _cur_tick += 1; + } + + size_t run_balancer() { + auto plan_data = do_run_balancer(); + + logger.info( + "planned action counts: reassignments: {}, cancellations: {}, " + "failed: {}, pinning violations: {}", + plan_data.reassignments.size(), + plan_data.cancellations.size(), + plan_data.failed_actions_count, + plan_data.last_pinning_violations_count); + + size_t actions_count = plan_data.reassignments.size() + + plan_data.cancellations.size(); + + _last_run_in_progress_updates + = _workers.table.local().updates_in_progress().size() + actions_count; + + for (const auto& reassignment : plan_data.reassignments) { + dispatch_move(reassignment.ntp, reassignment.allocated.replicas()); + } + for (const auto& ntp : plan_data.cancellations) { + dispatch_cancel(ntp); + } + + return actions_count; + } + + cluster::partition_balancer_planner::plan_data do_run_balancer() { + auto hr = create_health_report(); + populate_node_status_table(); + + auto planner = make_planner(); + + ss::abort_source as; + + auto plan_data = planner.plan_actions(hr, as).get(); + + _last_pinning_violations_count + = plan_data.last_pinning_violations_count; + + return plan_data; + } + + size_t last_run_in_progress_updates() const { + return _last_run_in_progress_updates; + } + + void print_state() const { + logger.info( + "TICK {}: {} nodes, {} partitions, {} updates in progress", + _cur_tick, + _nodes.size(), + _partitions.size(), + _workers.table.local().updates_in_progress().size()); + for (const auto& [id, node] : nodes()) { + logger.info( + "node id: {}, used: {:.4} GiB ({}%), num replicas: {} (final: " + "{})", + id, + double(node.used) / 1_GiB, + (node.used * 100) / node.total, + node.replicas.size(), + allocation_nodes().at(id)->final_partitions()); + } + } + + void print_replica_map() const { + for (const auto& t : topics().topics_map()) { + for (const auto& [_, a] : t.second.get_assignments()) { + auto ntp = model::ntp(t.first.ns, t.first.tp, a.id); + std::vector replicas; + for (const auto& bs : a.replicas) { + replicas.push_back(bs.node_id); + } + std::sort(replicas.begin(), replicas.end()); + logger.info("ntp {}: {}", ntp, replicas); + } + } + } + + void validate_even_replica_distribution() { + static constexpr double max_skew = 0.01; + + absl::flat_hash_map node2replicas; + size_t total_replicas = 0; + size_t total_capacity = 0; + for (auto& [id, n] : allocation_nodes()) { + node2replicas[id] = n->allocated_partitions(); + total_replicas += n->allocated_partitions(); + total_capacity += n->max_capacity(); + } + + for (auto& [id, replicas] : node2replicas) { + size_t capacity = allocation_nodes().at(id)->max_capacity(); + auto expected = floor( + double(total_replicas) * capacity / total_capacity); + logger.info( + "node {} has {} replicas, expected: {}", id, replicas, expected); + auto expected_min = expected - ceil(max_skew * expected); + auto expected_max = expected + ceil(max_skew * expected); + if (replicas < expected_min || replicas > expected_max) { + print_replica_map(); + BOOST_REQUIRE_MESSAGE( + false, + "node " << id << ": unexpected replicas count: " << replicas + << "(expected interval: [" << expected_min << ", " + << expected_max << "]"); + } + } + } + + void validate_even_topic_distribution() { + size_t total_capacity = 0; + for (auto& [id, n] : allocation_nodes()) { + total_capacity += n->max_capacity(); + } + + absl::node_hash_map< + model::topic_namespace, + absl::flat_hash_map> + topic_replica_distribution; + + absl::node_hash_map + total_topic_replicas; + + for (auto& [tp_ns, topic_md] : + _workers.table.local().all_topics_metadata()) { + for (auto& [_, p_as] : topic_md.get_assignments()) { + total_topic_replicas[tp_ns] += p_as.replicas.size(); + for (auto& r : p_as.replicas) { + topic_replica_distribution[tp_ns][r.node_id]++; + } + } + } + + for (auto& [tp, node_replicas] : topic_replica_distribution) { + if (total_topic_replicas[tp] < nodes().size()) { + continue; + } + + double total_replicas = static_cast( + total_topic_replicas[tp]); + for (auto& [id, alloc_node] : allocation_nodes()) { + auto it = node_replicas.find(id); + const auto replicas_on_node = it == node_replicas.end() + ? 0 + : it->second; + + auto expected = ceil( + total_replicas * alloc_node->max_capacity() / total_capacity); + + logger.info( + "topic {} has {} replicas on {}, expected: {}, " + "total replicas: {}", + tp, + replicas_on_node, + id, + expected, + total_replicas); + + static constexpr double max_skew = 0.03; + auto expected_min = expected - ceil(max_skew * expected); + auto expected_max = expected + ceil(max_skew * expected); + BOOST_CHECK_MESSAGE( + replicas_on_node >= expected_min + && replicas_on_node <= expected_max, + "topic " << tp.tp() << ": unexpected replicas count on node " + << id); + } + } + } + + void validate_topic_replica_pair_frequencies(const ss::sstring& topic) { + do_validate_replica_pair_frequencies(topic); + } + + void validate_replica_pair_frequencies() { + for (const auto& [tp_ns, topic_md] : + _workers.table.local().all_topics_metadata()) { + do_validate_replica_pair_frequencies(tp_ns.tp()); + } + do_validate_replica_pair_frequencies(std::nullopt); + } + + bool should_schedule_balancer_run() const { + auto current_in_progress + = _workers.table.local().updates_in_progress().size(); + return current_in_progress == 0 + || double(current_in_progress) + < 0.8 * double(_last_run_in_progress_updates); + } + + size_t total_replicas() const { return _total_replicas; } + +private: + void + do_validate_replica_pair_frequencies(std::optional topic) { + absl::flat_hash_map< + model::node_id, + absl::flat_hash_map> + pair_freqs; + size_t total_pairs = 0; + for (const auto& [tp_ns, topic_md] : + _workers.table.local().all_topics_metadata()) { + if (topic && tp_ns.tp() != topic) { + continue; + } + + for (const auto& [_, p_as] : topic_md.get_assignments()) { + for (const auto& r1 : p_as.replicas) { + for (const auto& r2 : p_as.replicas) { + if (r1.node_id != r2.node_id) { + pair_freqs[r1.node_id][r2.node_id] += 1; + total_pairs += 1; + } + } + } + } + } + + size_t node_count = allocation_nodes().size(); + double expected_freq = double(total_pairs) + / (node_count * (node_count - 1)); + + double expected_min = expected_freq - sqrt(expected_freq) * 3; + double expected_max = expected_freq + sqrt(expected_freq) * 4; + + logger.info( + "validating replica pair frequencies, topic filter: {}, " + "expected: {:.4} (interval: [{:.4}, {:.4}])", + topic, + expected_freq, + expected_min, + expected_max); + std::optional> offending_pair; + for (const auto& [id1, _] : allocation_nodes()) { + for (const auto& [id2, _] : allocation_nodes()) { + if (id1 == id2) { + continue; + } + size_t freq = pair_freqs[id1][id2]; + logger.info("node pair {} - {} frequency: {}", id1, id2, freq); + if (freq < expected_min || freq > expected_max) { + offending_pair = {id1, id2}; + } + } + } + + auto failure_msg = offending_pair.has_value() + ? fmt::format( + "validation failed, offending pair: {}, {}", + offending_pair->first, + offending_pair->second) + : ""; + BOOST_REQUIRE_MESSAGE(!offending_pair, failure_msg); + } + + cluster::cluster_health_report create_health_report() const { + cluster::cluster_health_report report; + for (const auto& [id, state] : _nodes) { + report.node_reports.push_back(state.get_health_report()); + } + return report; + } + + void populate_node_status_table() { + std::vector status_updates; + for (const auto& [id, state] : _nodes) { + auto last_seen = raft::clock_type::now(); + status_updates.push_back( + cluster::node_status{ + .node_id = id, + .last_seen = last_seen, + }); + } + + _workers.node_status_table + .invoke_on_all([status_updates](cluster::node_status_table& nts) { + nts.update_peers(status_updates); + }) + .get(); + } + + cluster::partition_balancer_planner make_planner( + model::partition_autobalancing_mode mode + = model::partition_autobalancing_mode::continuous) { + return cluster::partition_balancer_planner( + cluster::planner_config{ + .mode = mode, + .max_disk_usage_ratio = 0.8, + .max_concurrent_actions = 50, + .node_availability_timeout_sec = std::chrono::minutes(1), + .segment_fallocation_step = 16_MiB, + .node_responsiveness_timeout = node_responsiveness_timeout, + .topic_aware = true, + .node_autodecommission_timeout = {}}, + _workers.state.local(), + _workers.allocator.local()); + } + + std::vector get_voters(const model::ntp& ntp) const { + std::vector ret; + for (const auto& [id, node] : _nodes) { + auto it = node.replicas.find(ntp); + if ( + it != node.replicas.end() + && it->second.local_size == it->second.partition->size) { + ret.push_back(id); + } + } + return ret; + } + + std::vector get_learners(const model::ntp& ntp) const { + std::vector ret; + for (const auto& [id, node] : _nodes) { + auto it = node.replicas.find(ntp); + if ( + it != node.replicas.end() + && it->second.local_size < it->second.partition->size) { + ret.push_back(id); + } + } + return ret; + } + + void elect_leader(const model::ntp& ntp) { + auto voters = get_voters(ntp); + _partitions.at(ntp)->leader = random_generators::random_choice(voters); + } + + void dispatch_move( + model::ntp ntp, std::vector new_replicas) { + _workers.dispatch_topic_command( + cluster::move_partition_replicas_cmd(ntp, new_replicas)); + + auto partition = _partitions.at(ntp); + for (const auto& bs : new_replicas) { + auto& node = _nodes.at(bs.node_id); + node.replicas.emplace( + ntp, replica{.partition = partition, .local_size = 0}); + } + } + + void dispatch_cancel(model::ntp ntp) { + cluster::cancel_moving_partition_replicas_cmd cmd{ + std::move(ntp), + cluster::cancel_moving_partition_replicas_cmd_data{ + cluster::force_abort_update{false}}}; + _workers.dispatch_topic_command(std::move(cmd)); + } + + struct recovery_stream { + model::ntp ntp; + model::node_id from; + model::node_id to; + + friend bool + operator==(const recovery_stream&, const recovery_stream&) = default; + }; + + void perform_recovery_step(const recovery_stream& rs) { + auto& dest_node = _nodes.at(rs.to); + auto& dest_replica = dest_node.replicas.at(rs.ntp); + const auto& partition = *dest_replica.partition; + auto step = std::min( + recovery_batch_size, partition.size - dest_replica.local_size); + dest_replica.local_size += step; + dest_node.used += step; + } + + bool maybe_finish_update(const model::ntp& ntp) { + const auto& partition = *_partitions.at(ntp); + if (!partition.leader) { + return false; + } + + const auto& cur_update + = _workers.table.local().updates_in_progress().at(ntp); + + bool all_replicas_recovered = true; + for (const auto& bs : cur_update.get_target_replicas()) { + const auto& node = _nodes.at(bs.node_id); + if (node.replicas.at(ntp).local_size < partition.size) { + all_replicas_recovered = false; + break; + } + } + + switch (cur_update.get_state()) { + case cluster::reconfiguration_state::in_progress: + if (!all_replicas_recovered) { + return false; + } + + _workers.dispatch_topic_command( + cluster::finish_moving_partition_replicas_cmd{ + ntp, cur_update.get_target_replicas()}); + break; + case cluster::reconfiguration_state::cancelled: + if (all_replicas_recovered) { + _workers.dispatch_topic_command( + cluster::revert_cancel_partition_move_cmd{ + 0, + cluster::revert_cancel_partition_move_cmd_data{ + .ntp = ntp}}); + } else { + _workers.dispatch_topic_command( + cluster::finish_moving_partition_replicas_cmd{ + ntp, cur_update.get_previous_replicas()}); + } + break; + default: + BOOST_REQUIRE(false); + } + + auto cur_assignment = _workers.table.local().get_partition_assignment( + ntp); + BOOST_REQUIRE(cur_assignment); + + absl::flat_hash_set cur_replicas; + for (const auto& bs : cur_assignment->replicas) { + cur_replicas.insert(bs.node_id); + } + + for (auto& [id, node] : _nodes) { + if (!cur_replicas.contains(id)) { + auto it = node.replicas.find(ntp); + if (it != node.replicas.end()) { + node.used -= it->second.local_size; + node.replicas.erase(it); + } + } + } + + elect_leader(ntp); + + return true; + } + + struct partition_state { + partition_state(model::ntp ntp, size_t size) + : ntp(std::move(ntp)) + , size(size) {} + + model::ntp ntp; + size_t size = 0; + std::optional leader; + + using ptr_t = ss::lw_shared_ptr; + }; + + struct replica { + partition_state::ptr_t partition; + size_t local_size = 0; + }; + + struct node_state { + model::node_id id; + size_t total = 0; + size_t used = 0; + absl::flat_hash_map replicas; + size_t bandwidth_left = recovery_throttle_burst; + size_t ticks_since_refill = 0; + + cluster::node_health_report_ptr get_health_report() const { + cluster::node::local_state local_state; + storage::disk node_disk{.free = total - used, .total = total}; + local_state.set_disk(node_disk); + local_state.log_data_size = { + .data_target_size = total, + .data_current_size = used, + .data_reclaimable_size = 0}; + + absl::flat_hash_map< + model::topic_namespace, + cluster::partition_statuses_t> + topic2partitions; + for (const auto& [ntp, repl] : replicas) { + topic2partitions[model::topic_namespace(ntp.ns, ntp.tp.topic)] + .push_back( + cluster::partition_status{ + .id = ntp.tp.partition, .size_bytes = repl.local_size}); + } + + chunked_vector topics; + for (auto& [topic, partitions] : topic2partitions) { + topics.push_back( + cluster::topic_status(topic, std::move(partitions))); + } + + return ss::make_foreign( + ss::make_lw_shared( + id, + local_state, + std::move(topics), + std::nullopt, + cluster::node_liveness_report{})); + } + }; + + absl::btree_map _nodes; + absl::flat_hash_map _partitions; + size_t _cur_tick = 0; + size_t _last_run_in_progress_updates = 0; + size_t _last_pinning_violations_count = 0; + controller_workers _workers; + size_t _total_replicas = 0; +}; diff --git a/src/v/cluster/tests/partition_balancer_simulator_test.cc b/src/v/cluster/tests/partition_balancer_simulator_test.cc index 471b7cd12b7f9..ca7a6282af86b 100644 --- a/src/v/cluster/tests/partition_balancer_simulator_test.cc +++ b/src/v/cluster/tests/partition_balancer_simulator_test.cc @@ -7,751 +7,12 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -#include "cluster/health_monitor_types.h" -#include "cluster/tests/partition_balancer_planner_fixture.h" -#include "model/fundamental.h" -#include "model/metadata.h" -#include "random/generators.h" +#include "cluster/tests/partition_balancer_sim_fixture.h" #include "test_utils/boost_fixture.h" -#include +#include -static ss::logger logger("balancer_sim"); - -static constexpr size_t produce_batch_size = 1_MiB; - -// Recovery rate magic numbers. Recovery bandwidth is not directly controllable, -// but it is related to the tick length: node bandwidth allows exactly 1 batch -// per tick. For 100MiB/s bandwidth this gives us 5ms ticks. -static constexpr size_t recovery_batch_size = 512_KiB; -static constexpr size_t recovery_throttle_burst = 100_MiB; -static constexpr size_t recovery_throttle_ticks_between_refills = 10; - -static constexpr auto node_responsiveness_timeout = std::chrono::seconds(10); - -class partition_balancer_sim_fixture { -public: - void add_node( - model::node_id id, - size_t total_size, - uint32_t n_cores = 4, - std::optional rack = std::nullopt) { - vassert(!_nodes.contains(id), "duplicate node id: {}", id); - - model::broker broker( - id, - net::unresolved_address{}, - net::unresolved_address{}, - rack, - model::broker_properties{ - .cores = n_cores, - .available_memory_gb = 2, - .available_disk_gb = uint32_t(total_size / 1_GiB), - .available_memory_bytes = 2 * 1_GiB}); - - BOOST_REQUIRE(!_workers.members.local().apply( - model::offset{}, cluster::add_node_cmd(id, broker))); - _workers.allocator.local().register_node( - std::make_unique( - id, - n_cores, - config::mock_binding(1000), - config::mock_binding(0), - config::mock_binding>({}))); - - // add some random initial used space - size_t initial_used = random_generators::get_int( - 3 * total_size / 100, 5 * total_size / 100); - - _nodes.emplace( - id, node_state{.id = id, .total = total_size, .used = initial_used}); - } - - void remove_node(model::node_id id) { _nodes.erase(id); } - - const auto& nodes() const { return _nodes; } - - const cluster::allocation_state::underlying_t& allocation_nodes() const { - return _workers.allocator.local().state().allocation_nodes(); - } - - const cluster::topic_table& topics() const { - return _workers.table.local(); - } - - void add_topic( - const ss::sstring& name, - int partitions, - int16_t replication_factor, - size_t mean_partition_size, - std::optional stddev = std::nullopt) { - auto tp_ns = model::topic_namespace(test_ns, model::topic(name)); - auto topic_conf = _workers.make_tp_configuration( - name, partitions, replication_factor); - _workers.dispatch_topic_command( - cluster::create_topic_cmd(tp_ns, topic_conf)); - - if (!stddev) { - /// If a value is a sum of a large number of random variables that - /// with some probability add item_size to the value and the result - /// has a mean of `mean` (example: producing to a topic, randomly - /// choosing the partition for each produced batch), the - /// distribution will be approximately gaussian with stddev - /// sqrt(mean number of items). - stddev = produce_batch_size - * sqrt(double(mean_partition_size) / produce_batch_size); - } - - std::normal_distribution<> dist(0, 1); - for (const auto& as : topic_conf.assignments) { - model::ntp ntp{tp_ns.ns, tp_ns.tp, as.id}; - auto jitter = std::max( - -int64_t(mean_partition_size), - int64_t(*stddev * dist(random_generators::global().engine()))); - auto size = mean_partition_size + jitter; - auto partition = ss::make_lw_shared(ntp, size); - _partitions.emplace(ntp, partition); - - for (const auto& bs : as.replicas) { - auto& node = _nodes.at(bs.node_id); - node.replicas[ntp] = replica{ - .partition = partition, .local_size = size}; - node.used += size; - } - - elect_leader(ntp); - - logger.info( - "added ntp {}, replicas {}, size {}", - ntp, - as.replicas, - human::bytes(size)); - } - _total_replicas += static_cast(replication_factor) * partitions; - } - - void set_decommissioning(model::node_id id) { - _workers.set_decommissioning(id); - } - - void add_node_to_rebalance(model::node_id id) { - _workers.state.local().add_node_to_rebalance(id); - } - - size_t cur_tick() const { return _cur_tick; } - - // returns true if all partition movements stopped - bool run_to_completion( - size_t max_balancer_actions, std::function tick_cb = [] {}) { - print_state(); - - size_t num_actions = 0; - while (num_actions < max_balancer_actions) { - tick(); - tick_cb(); - - if (should_schedule_balancer_run()) { - num_actions += run_balancer(); - print_state(); - - if (last_run_in_progress_updates() == 0) { - logger.info("finished after {} ticks", cur_tick()); - return true; - } - } - } - - print_state(); - return false; - } - - void tick() { - // refill the bandwidth - for (auto& [id, node] : _nodes) { - node.ticks_since_refill += 1; - if ( - node.ticks_since_refill - >= recovery_throttle_ticks_between_refills) { - node.ticks_since_refill = 0; - - if (node.bandwidth_left <= recovery_throttle_burst) { - node.bandwidth_left - += recovery_throttle_ticks_between_refills - * recovery_batch_size; - } - } - } - - // gather all active recovery streams - absl::flat_hash_map> - node2pending_rs; - for (const auto& ntp : - _workers.table.local().all_updates_in_progress()) { - auto part = _partitions.at(ntp); - if (!part->leader) { - continue; - } - - auto learners = get_learners(ntp); - for (const auto& id : learners) { - node2pending_rs[*part->leader].push_back( - recovery_stream{.ntp = ntp, .from = *part->leader, .to = id}); - } - } - - // perform recovery for lucky streams that have won the bandwidth - // lottery. - for (auto& [node_id, recovery_streams] : node2pending_rs) { - auto& node = _nodes.at(node_id); - - std::shuffle( - recovery_streams.begin(), - recovery_streams.end(), - random_generators::global().engine()); - - for (const auto& rs : recovery_streams) { - if (node.bandwidth_left >= recovery_batch_size) { - perform_recovery_step(rs); - node.bandwidth_left -= recovery_batch_size; - } else { - break; - } - } - } - - // finish the updates that are ready - for (const auto& ntp : - _workers.table.local().all_updates_in_progress()) { - maybe_finish_update(ntp); - } - - _cur_tick += 1; - } - - // return the number of scheduled actions - size_t run_balancer() { - auto plan_data = do_run_balancer(); - - logger.info( - "planned action counts: reassignments: {}, cancellations: {}, " - "failed: {}", - plan_data.reassignments.size(), - plan_data.cancellations.size(), - plan_data.failed_actions_count); - - size_t actions_count = plan_data.reassignments.size() - + plan_data.cancellations.size(); - - _last_run_in_progress_updates - = _workers.table.local().updates_in_progress().size() + actions_count; - - for (const auto& reassignment : plan_data.reassignments) { - dispatch_move(reassignment.ntp, reassignment.allocated.replicas()); - } - for (const auto& ntp : plan_data.cancellations) { - dispatch_cancel(ntp); - } - - return actions_count; - } - - // run the balancer planner, return its plan - cluster::partition_balancer_planner::plan_data do_run_balancer() { - auto hr = create_health_report(); - populate_node_status_table(); - - auto planner = make_planner(); - - ss::abort_source as; - - auto plan_data = planner.plan_actions(hr, as).get(); - - return plan_data; - } - - size_t last_run_in_progress_updates() const { - return _last_run_in_progress_updates; - } - - void print_state() const { - logger.info( - "TICK {}: {} nodes, {} partitions, {} updates in progress", - _cur_tick, - _nodes.size(), - _partitions.size(), - _workers.table.local().updates_in_progress().size()); - for (const auto& [id, node] : nodes()) { - logger.info( - "node id: {}, used: {:.4} GiB ({}%), num replicas: {} (final: " - "{})", - id, - double(node.used) / 1_GiB, - (node.used * 100) / node.total, - node.replicas.size(), - allocation_nodes().at(id)->final_partitions()); - } - } - - void print_replica_map() const { - for (const auto& t : topics().topics_map()) { - for (const auto& [_, a] : t.second.get_assignments()) { - auto ntp = model::ntp(t.first.ns, t.first.tp, a.id); - std::vector replicas; - for (const auto& bs : a.replicas) { - replicas.push_back(bs.node_id); - } - std::sort(replicas.begin(), replicas.end()); - logger.info("ntp {}: {}", ntp, replicas); - } - } - } - - void validate_even_replica_distribution() { - static constexpr double max_skew = 0.01; - - absl::flat_hash_map node2replicas; - size_t total_replicas = 0; - size_t total_capacity = 0; - for (auto& [id, n] : allocation_nodes()) { - node2replicas[id] = n->allocated_partitions(); - total_replicas += n->allocated_partitions(); - total_capacity += n->max_capacity(); - } - - for (auto& [id, replicas] : node2replicas) { - size_t capacity = allocation_nodes().at(id)->max_capacity(); - auto expected = floor( - double(total_replicas) * capacity / total_capacity); - logger.info( - "node {} has {} replicas, expected: {}", id, replicas, expected); - auto expected_min = expected - ceil(max_skew * expected); - auto expected_max = expected + ceil(max_skew * expected); - if (replicas < expected_min || replicas > expected_max) { - print_replica_map(); - BOOST_REQUIRE_MESSAGE( - false, - "node " << id << ": unexpected replicas count: " << replicas - << "(expected interval: [" << expected_min << ", " - << expected_max << "]"); - } - } - } - - void validate_even_topic_distribution() { - size_t total_capacity = 0; - for (auto& [id, n] : allocation_nodes()) { - total_capacity += n->max_capacity(); - } - - absl::node_hash_map< - model::topic_namespace, - absl::flat_hash_map> - topic_replica_distribution; - - absl::node_hash_map - total_topic_replicas; - - for (auto& [tp_ns, topic_md] : - _workers.table.local().all_topics_metadata()) { - for (auto& [_, p_as] : topic_md.get_assignments()) { - total_topic_replicas[tp_ns] += p_as.replicas.size(); - for (auto& r : p_as.replicas) { - topic_replica_distribution[tp_ns][r.node_id]++; - } - } - } - - for (auto& [tp, node_replicas] : topic_replica_distribution) { - if (total_topic_replicas[tp] < nodes().size()) { - continue; - } - - double total_replicas = static_cast( - total_topic_replicas[tp]); - for (auto& [id, alloc_node] : allocation_nodes()) { - auto it = node_replicas.find(id); - const auto replicas_on_node = it == node_replicas.end() - ? 0 - : it->second; - - auto expected = ceil( - total_replicas * alloc_node->max_capacity() / total_capacity); - - logger.info( - "topic {} has {} replicas on {}, expected: {}, " - "total replicas: {}", - tp, - replicas_on_node, - id, - expected, - total_replicas); - - static constexpr double max_skew = 0.03; - auto expected_min = expected - ceil(max_skew * expected); - auto expected_max = expected + ceil(max_skew * expected); - BOOST_CHECK_MESSAGE( - replicas_on_node >= expected_min - && replicas_on_node <= expected_max, - "topic " << tp.tp() << ": unexpected replicas count on node " - << id); - } - } - } - - /// Validate that all possible replica pairings are represented - /// approximately equally in topic partition assignments. - void validate_topic_replica_pair_frequencies(const ss::sstring& topic) { - do_validate_replica_pair_frequencies(topic); - } - - /// Validate that all possible replica pairings are represented - /// approximately equally in overall partition assignments, as well as for - /// each topic assignments. - void validate_replica_pair_frequencies() { - for (const auto& [tp_ns, topic_md] : - _workers.table.local().all_topics_metadata()) { - do_validate_replica_pair_frequencies(tp_ns.tp()); - } - do_validate_replica_pair_frequencies(std::nullopt); - } - - bool should_schedule_balancer_run() const { - auto current_in_progress - = _workers.table.local().updates_in_progress().size(); - return current_in_progress == 0 - || double(current_in_progress) - < 0.8 * double(_last_run_in_progress_updates); - } - - size_t total_replicas() const { return _total_replicas; } - -private: - void - do_validate_replica_pair_frequencies(std::optional topic) { - absl::flat_hash_map< - model::node_id, - absl::flat_hash_map> - pair_freqs; - size_t total_pairs = 0; - for (const auto& [tp_ns, topic_md] : - _workers.table.local().all_topics_metadata()) { - if (topic && tp_ns.tp() != topic) { - continue; - } - - for (const auto& [_, p_as] : topic_md.get_assignments()) { - for (const auto& r1 : p_as.replicas) { - for (const auto& r2 : p_as.replicas) { - if (r1.node_id != r2.node_id) { - pair_freqs[r1.node_id][r2.node_id] += 1; - total_pairs += 1; - } - } - } - } - } - - size_t node_count = allocation_nodes().size(); - double expected_freq = double(total_pairs) - / (node_count * (node_count - 1)); - - // Generous boundaries to allow for fluctuations. But they will catch - // pathological cases. Set empirically to avoid flakes (i.e., if the - // upper bound is * 3, we get ~1% flakes in the pair test). - double expected_min = expected_freq - sqrt(expected_freq) * 3; - double expected_max = expected_freq + sqrt(expected_freq) * 4; - - logger.info( - "validating replica pair frequencies, topic filter: {}, " - "expected: {:.4} (interval: [{:.4}, {:.4}])", - topic, - expected_freq, - expected_min, - expected_max); - std::optional> offending_pair; - for (const auto& [id1, _] : allocation_nodes()) { - for (const auto& [id2, _] : allocation_nodes()) { - if (id1 == id2) { - continue; - } - size_t freq = pair_freqs[id1][id2]; - logger.info("node pair {} - {} frequency: {}", id1, id2, freq); - if (freq < expected_min || freq > expected_max) { - offending_pair = {id1, id2}; - } - } - } - - auto failure_msg = offending_pair.has_value() - ? fmt::format( - "validation failed, offending pair: {}, {}", - offending_pair->first, - offending_pair->second) - : ""; - BOOST_REQUIRE_MESSAGE(!offending_pair, failure_msg); - } - - cluster::cluster_health_report create_health_report() const { - cluster::cluster_health_report report; - for (const auto& [id, state] : _nodes) { - report.node_reports.push_back(state.get_health_report()); - } - return report; - } - - void populate_node_status_table() { - std::vector status_updates; - for (const auto& [id, state] : _nodes) { - auto last_seen = raft::clock_type::now(); - // TODO: add ability to add unavailable nodes - status_updates.push_back( - cluster::node_status{ - .node_id = id, - .last_seen = last_seen, - }); - } - - _workers.node_status_table - .invoke_on_all([status_updates](cluster::node_status_table& nts) { - nts.update_peers(status_updates); - }) - .get(); - } - - cluster::partition_balancer_planner make_planner( - model::partition_autobalancing_mode mode - = model::partition_autobalancing_mode::continuous) { - return cluster::partition_balancer_planner( - cluster::planner_config{ - .mode = mode, - .max_disk_usage_ratio = 0.8, - .max_concurrent_actions = 50, - .node_availability_timeout_sec = std::chrono::minutes(1), - .segment_fallocation_step = 16_MiB, - .node_responsiveness_timeout = node_responsiveness_timeout, - .topic_aware = true, - .node_autodecommission_timeout = {}}, - _workers.state.local(), - _workers.allocator.local()); - } - - std::vector get_voters(const model::ntp& ntp) const { - std::vector ret; - for (const auto& [id, node] : _nodes) { - auto it = node.replicas.find(ntp); - if ( - it != node.replicas.end() - && it->second.local_size == it->second.partition->size) { - ret.push_back(id); - } - } - return ret; - } - - std::vector get_learners(const model::ntp& ntp) const { - std::vector ret; - for (const auto& [id, node] : _nodes) { - auto it = node.replicas.find(ntp); - if ( - it != node.replicas.end() - && it->second.local_size < it->second.partition->size) { - ret.push_back(id); - } - } - return ret; - } - - void elect_leader(const model::ntp& ntp) { - auto voters = get_voters(ntp); - _partitions.at(ntp)->leader = random_generators::random_choice(voters); - } - - void dispatch_move( - model::ntp ntp, std::vector new_replicas) { - _workers.dispatch_topic_command( - cluster::move_partition_replicas_cmd(ntp, new_replicas)); - - auto partition = _partitions.at(ntp); - for (const auto& bs : new_replicas) { - auto& node = _nodes.at(bs.node_id); - node.replicas.emplace( - ntp, replica{.partition = partition, .local_size = 0}); - } - } - - void dispatch_cancel(model::ntp ntp) { - cluster::cancel_moving_partition_replicas_cmd cmd{ - std::move(ntp), - cluster::cancel_moving_partition_replicas_cmd_data{ - cluster::force_abort_update{false}}}; - _workers.dispatch_topic_command(std::move(cmd)); - } - - struct recovery_stream { - model::ntp ntp; - model::node_id from; - model::node_id to; - - friend bool - operator==(const recovery_stream&, const recovery_stream&) = default; - }; - - void perform_recovery_step(const recovery_stream& rs) { - auto& dest_node = _nodes.at(rs.to); - auto& dest_replica = dest_node.replicas.at(rs.ntp); - const auto& partition = *dest_replica.partition; - auto step = std::min( - recovery_batch_size, partition.size - dest_replica.local_size); - dest_replica.local_size += step; - dest_node.used += step; - } - - bool maybe_finish_update(const model::ntp& ntp) { - const auto& partition = *_partitions.at(ntp); - if (!partition.leader) { - // can't finish anything for a leaderless partition - return false; - } - - const auto& cur_update - = _workers.table.local().updates_in_progress().at(ntp); - - bool all_replicas_recovered = true; - for (const auto& bs : cur_update.get_target_replicas()) { - const auto& node = _nodes.at(bs.node_id); - if (node.replicas.at(ntp).local_size < partition.size) { - // some nodes are still learners - all_replicas_recovered = false; - break; - } - } - - // dispatch the finish command - - switch (cur_update.get_state()) { - case cluster::reconfiguration_state::in_progress: - if (!all_replicas_recovered) { - return false; - } - - _workers.dispatch_topic_command( - cluster::finish_moving_partition_replicas_cmd{ - ntp, cur_update.get_target_replicas()}); - break; - case cluster::reconfiguration_state::cancelled: - if (all_replicas_recovered) { - _workers.dispatch_topic_command( - cluster::revert_cancel_partition_move_cmd{ - 0, - cluster::revert_cancel_partition_move_cmd_data{ - .ntp = ntp}}); - } else { - _workers.dispatch_topic_command( - cluster::finish_moving_partition_replicas_cmd{ - ntp, cur_update.get_previous_replicas()}); - } - break; - default: - // other states can't appear because they can only be created - // manually, not by the balancer. - BOOST_REQUIRE(false); - } - - // remove excess replicas - - auto cur_assignment = _workers.table.local().get_partition_assignment( - ntp); - BOOST_REQUIRE(cur_assignment); - - absl::flat_hash_set cur_replicas; - for (const auto& bs : cur_assignment->replicas) { - cur_replicas.insert(bs.node_id); - } - - for (auto& [id, node] : _nodes) { - if (!cur_replicas.contains(id)) { - auto it = node.replicas.find(ntp); - if (it != node.replicas.end()) { - node.used -= it->second.local_size; - node.replicas.erase(it); - } - } - } - - // for the case when the old leader is not in the new replica set. - elect_leader(ntp); - - return true; - } - - struct partition_state { - partition_state(model::ntp ntp, size_t size) - : ntp(std::move(ntp)) - , size(size) {} - - model::ntp ntp; - size_t size = 0; - std::optional leader; - - using ptr_t = ss::lw_shared_ptr; - }; - - struct replica { - partition_state::ptr_t partition; - size_t local_size = 0; - }; - - struct node_state { - model::node_id id; - size_t total = 0; - size_t used = 0; - absl::flat_hash_map replicas; - size_t bandwidth_left = recovery_throttle_burst; - size_t ticks_since_refill = 0; - - cluster::node_health_report_ptr get_health_report() const { - cluster::node::local_state local_state; - storage::disk node_disk{.free = total - used, .total = total}; - local_state.set_disk(node_disk); - local_state.log_data_size = { - .data_target_size = total, - .data_current_size = used, - .data_reclaimable_size = 0}; - - absl::flat_hash_map< - model::topic_namespace, - cluster::partition_statuses_t> - topic2partitions; - for (const auto& [ntp, repl] : replicas) { - topic2partitions[model::topic_namespace(ntp.ns, ntp.tp.topic)] - .push_back( - cluster::partition_status{ - .id = ntp.tp.partition, .size_bytes = repl.local_size}); - } - - chunked_vector topics; - for (auto& [topic, partitions] : topic2partitions) { - topics.push_back( - cluster::topic_status(topic, std::move(partitions))); - } - - return ss::make_foreign( - ss::make_lw_shared( - id, - local_state, - std::move(topics), - /* drain_status */ std::nullopt, - cluster::node_liveness_report{})); - } - }; - - absl::btree_map _nodes; - absl::flat_hash_map _partitions; - size_t _cur_tick = 0; - size_t _last_run_in_progress_updates = 0; - controller_workers _workers; - size_t _total_replicas = 0; -}; +using namespace std::chrono_literals; FIXTURE_TEST(test_decommission, partition_balancer_sim_fixture) { for (model::node_id::type i = 0; i < 4; ++i) { diff --git a/src/v/cluster/tests/partition_balancer_state_test.cc b/src/v/cluster/tests/partition_balancer_state_test.cc new file mode 100644 index 0000000000000..d2e26f9db3dce --- /dev/null +++ b/src/v/cluster/tests/partition_balancer_state_test.cc @@ -0,0 +1,222 @@ +/* + * Copyright 2026 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 "cluster/commands.h" +#include "cluster/data_migrated_resources.h" +#include "cluster/members_table.h" +#include "cluster/node_status_table.h" +#include "cluster/partition_balancer_state.h" +#include "cluster/scheduling/allocation_node.h" +#include "cluster/scheduling/partition_allocator.h" +#include "cluster/topic_table.h" +#include "cluster/types.h" +#include "config/configuration.h" +#include "config/mock_property.h" +#include "config/replicas_preference.h" +#include "features/feature_table.h" +#include "model/fundamental.h" +#include "model/metadata.h" +#include "model/namespace.h" +#include "test_utils/test.h" + +#include +#include + +#include + +namespace { + +constexpr uint32_t partitions_per_shard = 7000; +constexpr uint32_t partitions_reserve_shard0 = 2; + +struct pb_state_fixture : public seastar_test { + ss::future<> SetUpAsync() override { + co_await migrated_resources.start(); + co_await topics.start( + ss::sharded_parameter( + [this] { return std::ref(migrated_resources.local()); }), + model::node_id{1}); + co_await members.start_single(); + co_await features.start(); + co_await allocator.start_single( + std::ref(members), + std::ref(features), + config::mock_binding>(std::nullopt), + config::mock_binding(uint32_t{partitions_per_shard}), + config::mock_binding(uint32_t{partitions_reserve_shard0}), + config::mock_binding>({}), + config::mock_binding(false)); + allocator.local().register_node( + std::make_unique( + model::node_id{1}, + 8, + config::mock_binding(uint32_t{partitions_per_shard}), + config::mock_binding(uint32_t{partitions_reserve_shard0}), + config::mock_binding>({}))); + co_await node_status.start_single(model::node_id{123}); + co_await pb_state.start_single( + std::ref(topics), + std::ref(members), + std::ref(allocator), + std::ref(node_status)); + } + + ss::future<> TearDownAsync() override { + co_await pb_state.stop(); + co_await node_status.stop(); + co_await allocator.stop(); + co_await features.stop(); + co_await members.stop(); + co_await topics.stop(); + co_await migrated_resources.stop(); + } + + model::topic_namespace make_tp_ns(std::string_view tp) { + return {model::kafka_namespace, model::topic(ss::sstring{tp})}; + } + + ss::future<> create_topic(std::string_view name, int partitions = 1) { + cluster::topic_configuration cfg( + model::kafka_namespace, + model::topic(ss::sstring{name}), + partitions, + /*replication_factor=*/1); + ss::chunked_fifo pas; + for (int p = 0; p < partitions; ++p) { + pas.push_back( + cluster::partition_assignment( + raft::group_id{_next_group++}, + model::partition_id(p), + std::vector{{model::node_id{1}, 0}})); + } + cluster::topic_configuration_assignment cfg_a( + std::move(cfg), std::move(pas)); + cluster::create_topic_cmd cmd(make_tp_ns(name), std::move(cfg_a)); + auto ec = co_await topics.local().apply( + std::move(cmd), model::offset(_next_offset++)); + ASSERT_EQ_CORO(ec, cluster::errc::success); + } + + ss::future<> + set_pinning(std::string_view name, std::string_view preference_spec) { + cluster::incremental_topic_updates updates; + updates.replicas_preference.op + = cluster::incremental_update_operation::set; + updates.replicas_preference.value = config::replicas_preference::parse( + preference_spec); + cluster::update_topic_properties_cmd cmd( + make_tp_ns(name), std::move(updates)); + auto ec = co_await topics.local().apply( + std::move(cmd), model::offset(_next_offset++)); + ASSERT_EQ_CORO(ec, cluster::errc::success); + } + + ss::future<> clear_pinning(std::string_view name) { + cluster::incremental_topic_updates updates; + updates.replicas_preference.op + = cluster::incremental_update_operation::remove; + cluster::update_topic_properties_cmd cmd( + make_tp_ns(name), std::move(updates)); + auto ec = co_await topics.local().apply( + std::move(cmd), model::offset(_next_offset++)); + ASSERT_EQ_CORO(ec, cluster::errc::success); + } + + ss::future<> delete_topic(std::string_view name) { + cluster::delete_topic_cmd cmd(make_tp_ns(name), make_tp_ns(name)); + auto ec = co_await topics.local().apply( + std::move(cmd), model::offset(_next_offset++)); + ASSERT_EQ_CORO(ec, cluster::errc::success); + } + + ss::sharded + migrated_resources; + ss::sharded topics; + ss::sharded members; + ss::sharded features; + ss::sharded allocator; + ss::sharded node_status; + ss::sharded pb_state; + + int64_t _next_offset{0}; + int64_t _next_group{1}; +}; + +TEST_F_CORO(pb_state_fixture, pinning_cache_seeds_from_existing_topics) { + co_await create_topic("pinned"); + co_await create_topic("plain"); + co_await set_pinning("pinned", "racks: rack_A, rack_B"); + + auto& state = pb_state.local(); + + // Before seeding: cache empty (notifications before seeding are + // intentionally ignored; the seed reads authoritative state). + EXPECT_TRUE(state.topics_with_replica_pinning().empty()); + + state.ensure_pinning_cache_seeded(); + + const auto& pinned = state.topics_with_replica_pinning(); + ASSERT_EQ_CORO(pinned.size(), 1u); + EXPECT_TRUE(pinned.contains(make_tp_ns("pinned"))); + EXPECT_FALSE(pinned.contains(make_tp_ns("plain"))); +} + +TEST_F_CORO(pb_state_fixture, pinning_cache_tracks_properties_updated) { + co_await create_topic("t1"); + + auto& state = pb_state.local(); + state.ensure_pinning_cache_seeded(); + EXPECT_TRUE(state.topics_with_replica_pinning().empty()); + + // Set pinning — notification must update the cache. + co_await set_pinning("t1", "racks: rack_A"); + + ASSERT_EQ_CORO(state.topics_with_replica_pinning().size(), 1u); + EXPECT_TRUE(state.topics_with_replica_pinning().contains(make_tp_ns("t1"))); + + // Clear pinning via remove op — cache must drop the topic. + co_await clear_pinning("t1"); + + EXPECT_TRUE(state.topics_with_replica_pinning().empty()); +} + +TEST_F_CORO(pb_state_fixture, pinning_cache_drops_topic_on_delete) { + co_await create_topic("doomed"); + co_await set_pinning("doomed", "racks: rack_A"); + + auto& state = pb_state.local(); + state.ensure_pinning_cache_seeded(); + ASSERT_EQ_CORO(state.topics_with_replica_pinning().size(), 1u); + + co_await delete_topic("doomed"); + + EXPECT_TRUE(state.topics_with_replica_pinning().empty()); +} + +TEST_F_CORO(pb_state_fixture, pinning_cache_reset_forces_reseed) { + co_await create_topic("t1"); + co_await set_pinning("t1", "racks: rack_A"); + + auto& state = pb_state.local(); + state.ensure_pinning_cache_seeded(); + ASSERT_EQ_CORO(state.topics_with_replica_pinning().size(), 1u); + + // Reset clears the cache so callers don't observe stale contents in + // the window before ensure_pinning_cache_seeded() re-scans. + state.reset_pinning_cache(); + EXPECT_TRUE(state.topics_with_replica_pinning().empty()); + + // Re-seed rebuilds from authoritative topic_table state. + state.ensure_pinning_cache_seeded(); + ASSERT_EQ_CORO(state.topics_with_replica_pinning().size(), 1u); + EXPECT_TRUE(state.topics_with_replica_pinning().contains(make_tp_ns("t1"))); +} + +} // namespace diff --git a/src/v/cluster/tests/topic_properties_generator.h b/src/v/cluster/tests/topic_properties_generator.h index 6cf2bb17a8011..0bce471a5ba83 100644 --- a/src/v/cluster/tests/topic_properties_generator.h +++ b/src/v/cluster/tests/topic_properties_generator.h @@ -10,6 +10,7 @@ #include "base/units.h" #include "cluster/types.h" +#include "config/replicas_preference.h" #include "model/tests/randoms.h" #include "random/generators.h" #include "test_utils/randoms.h" @@ -78,6 +79,8 @@ inline cluster::topic_properties random_topic_properties() { {model::redpanda_storage_mode::local, model::redpanda_storage_mode::tiered, model::redpanda_storage_mode::cloud}); + properties.replicas_preference = tests::random_optional( + [] { return config::replicas_preference::parse("racks: A, {B, C}, D"); }); return properties; } diff --git a/src/v/cluster/tests/topic_properties_test.cc b/src/v/cluster/tests/topic_properties_test.cc index dedb6f5c9d0df..3c58853a2bde5 100644 --- a/src/v/cluster/tests/topic_properties_test.cc +++ b/src/v/cluster/tests/topic_properties_test.cc @@ -27,4 +27,181 @@ TEST(TopicProperties, ostream) { ASSERT_FALSE(result.contains("{}")) << result; } +TEST(TopicProperties, serde_roundtrip_with_replicas_preference) { + topic_properties orig; + orig.replicas_preference = config::replicas_preference::parse( + "racks: A, {B, C}, D"); + + iobuf buf; + serde::write(buf, std::move(orig)); + + auto parser = iobuf_parser(std::move(buf)); + auto decoded = serde::read(parser); + + ASSERT_TRUE(decoded.replicas_preference.has_value()); + EXPECT_EQ(decoded.replicas_preference->num_groups(), 3); + EXPECT_EQ( + decoded.replicas_preference->group_index_for(model::rack_id{"A"}).value(), + 0); + EXPECT_EQ( + decoded.replicas_preference->group_index_for(model::rack_id{"B"}).value(), + 1); + EXPECT_EQ( + decoded.replicas_preference->group_index_for(model::rack_id{"C"}).value(), + 1); + EXPECT_EQ( + decoded.replicas_preference->group_index_for(model::rack_id{"D"}).value(), + 2); +} + +TEST(TopicProperties, serde_roundtrip_without_replicas_preference) { + topic_properties orig; + // replicas_preference defaults to nullopt + + iobuf buf; + serde::write(buf, std::move(orig)); + + auto parser = iobuf_parser(std::move(buf)); + auto decoded = serde::read(parser); + + EXPECT_FALSE(decoded.replicas_preference.has_value()); +} + +/// A v13 topic_properties struct (before replicas_preference was added). +/// Used to verify backward compatibility: deserializing a v13 payload as +/// v14 should leave replicas_preference as nullopt. +struct topic_properties_v13 + : serde::envelope< + topic_properties_v13, + serde::version<13>, + serde::compat_version<0>> { + std::optional compression; + std::optional cleanup_policy_bitflags; + std::optional compaction_strategy; + std::optional timestamp_type; + std::optional segment_size; + tristate retention_bytes{std::nullopt}; + tristate retention_duration{std::nullopt}; + std::optional recovery; + std::optional shadow_indexing; + std::optional read_replica; + std::optional read_replica_bucket; + std::optional remote_topic_properties; + std::optional batch_max_bytes; + tristate retention_local_target_bytes{std::nullopt}; + tristate retention_local_target_ms{std::nullopt}; + bool remote_delete{false}; + tristate segment_ms{std::nullopt}; + std::optional record_key_schema_id_validation; + std::optional record_key_schema_id_validation_compat; + std::optional + record_key_subject_name_strategy; + std::optional + record_key_subject_name_strategy_compat; + std::optional record_value_schema_id_validation; + std::optional record_value_schema_id_validation_compat; + std::optional + record_value_subject_name_strategy; + std::optional + record_value_subject_name_strategy_compat; + tristate initial_retention_local_target_bytes{std::nullopt}; + tristate initial_retention_local_target_ms{ + std::nullopt}; + std::optional mpx_virtual_cluster_id; + std::optional write_caching; + std::optional flush_ms; + std::optional flush_bytes; + std::optional remote_label; + std::optional remote_topic_namespace_override; + model::iceberg_mode iceberg_mode{model::iceberg_mode::disabled}; + std::optional leaders_preference; + bool deprecated_cloud_topic_enabled{false}; + tristate delete_retention_ms{disable_tristate}; + std::optional iceberg_delete; + std::optional iceberg_partition_spec; + std::optional + iceberg_invalid_record_action; + std::optional iceberg_target_lag_ms{}; + tristate min_cleanable_dirty_ratio{std::nullopt}; + std::optional remote_topic_allow_gaps; + std::optional min_compaction_lag_ms{}; + std::optional max_compaction_lag_ms{}; + std::optional message_timestamp_before_max_ms{}; + std::optional message_timestamp_after_max_ms{}; + model::redpanda_storage_mode storage_mode{ + model::redpanda_storage_mode::local}; + // No replicas_preference — this is v13 + + auto serde_fields() { + return std::tie( + compression, + cleanup_policy_bitflags, + compaction_strategy, + timestamp_type, + segment_size, + retention_bytes, + retention_duration, + recovery, + shadow_indexing, + read_replica, + read_replica_bucket, + remote_topic_properties, + batch_max_bytes, + retention_local_target_bytes, + retention_local_target_ms, + remote_delete, + segment_ms, + record_key_schema_id_validation, + record_key_schema_id_validation_compat, + record_key_subject_name_strategy, + record_key_subject_name_strategy_compat, + record_value_schema_id_validation, + record_value_schema_id_validation_compat, + record_value_subject_name_strategy, + record_value_subject_name_strategy_compat, + initial_retention_local_target_bytes, + initial_retention_local_target_ms, + mpx_virtual_cluster_id, + write_caching, + flush_ms, + flush_bytes, + remote_label, + remote_topic_namespace_override, + iceberg_mode, + leaders_preference, + deprecated_cloud_topic_enabled, + delete_retention_ms, + iceberg_delete, + iceberg_partition_spec, + iceberg_invalid_record_action, + iceberg_target_lag_ms, + min_cleanable_dirty_ratio, + remote_topic_allow_gaps, + min_compaction_lag_ms, + max_compaction_lag_ms, + message_timestamp_before_max_ms, + message_timestamp_after_max_ms, + storage_mode); + } +}; + +TEST(TopicProperties, serde_backward_compat_v13_to_v14) { + // Serialize a v13 payload (no replicas_preference field). + topic_properties_v13 v13; + v13.compression = model::compression::gzip; + v13.segment_size = 1024; + + iobuf buf; + serde::write(buf, std::move(v13)); + + // Deserialize as v14 topic_properties — replicas_preference should + // be default-initialized to nullopt. + auto parser = iobuf_parser(std::move(buf)); + auto decoded = serde::read(parser); + + EXPECT_EQ(decoded.compression, model::compression::gzip); + EXPECT_EQ(decoded.segment_size, 1024); + EXPECT_FALSE(decoded.replicas_preference.has_value()); +} + } // namespace cluster diff --git a/src/v/cluster/topic_properties.cc b/src/v/cluster/topic_properties.cc index 97c95dd0d3d5d..0e253c63f3b14 100644 --- a/src/v/cluster/topic_properties.cc +++ b/src/v/cluster/topic_properties.cc @@ -53,7 +53,8 @@ fmt::iterator topic_properties::format_to(fmt::iterator it) const { "max_compaction_lag_ms: {}, " "message_timestamp_before_max_ms: {}, " "message_timestamp_after_max_ms: {}, " - "redpanda_storage_mode: {}}}", + "redpanda_storage_mode: {}, " + "replicas_preference: {}}}", compression, cleanup_policy_bitflags, compaction_strategy, @@ -100,7 +101,8 @@ fmt::iterator topic_properties::format_to(fmt::iterator it) const { max_compaction_lag_ms, message_timestamp_before_max_ms, message_timestamp_after_max_ms, - storage_mode); + storage_mode, + replicas_preference); } bool topic_properties::is_compacted() const { if (!cleanup_policy_bitflags) { @@ -146,7 +148,8 @@ bool topic_properties::has_overrides() const { || remote_topic_allow_gaps.has_value() || message_timestamp_before_max_ms.has_value() || message_timestamp_after_max_ms.has_value() - || storage_mode != storage::ntp_config::default_storage_mode; + || storage_mode != storage::ntp_config::default_storage_mode + || replicas_preference.has_value(); return overrides; } @@ -346,6 +349,7 @@ adl::from(iobuf_parser& parser) { std::nullopt, std::nullopt, model::redpanda_storage_mode::local, + std::nullopt, }; } diff --git a/src/v/cluster/topic_properties.h b/src/v/cluster/topic_properties.h index cadad747f41d3..7c6ec32ac7c69 100644 --- a/src/v/cluster/topic_properties.h +++ b/src/v/cluster/topic_properties.h @@ -12,6 +12,7 @@ #include "base/format_to.h" #include "cloud_storage/remote_label.h" #include "cluster/remote_topic_properties.h" +#include "config/replicas_preference.h" #include "model/compression.h" #include "model/fundamental.h" #include "model/metadata.h" @@ -34,7 +35,7 @@ namespace cluster { */ struct topic_properties : serde:: - envelope, serde::compat_version<0>> { + envelope, serde::compat_version<0>> { topic_properties() noexcept = default; topic_properties( std::optional compression, @@ -87,7 +88,8 @@ struct topic_properties std::optional remote_topic_allow_gaps, std::optional message_timestamp_before_max_ms, std::optional message_timestamp_after_max_ms, - model::redpanda_storage_mode storage_mode) + model::redpanda_storage_mode storage_mode, + std::optional replicas_preference) : compression(compression) , cleanup_policy_bitflags(cleanup_policy_bitflags) , compaction_strategy(compaction_strategy) @@ -138,7 +140,8 @@ struct topic_properties , max_compaction_lag_ms(max_compaction_lag_ms) , message_timestamp_before_max_ms(message_timestamp_before_max_ms) , message_timestamp_after_max_ms(message_timestamp_after_max_ms) - , storage_mode(storage_mode) {} + , storage_mode(storage_mode) + , replicas_preference(std::move(replicas_preference)) {} std::optional compression; std::optional cleanup_policy_bitflags; @@ -236,6 +239,10 @@ struct topic_properties model::redpanda_storage_mode storage_mode{ storage::ntp_config::default_storage_mode}; + // Per-topic replica placement preference (priority-ordered rack groups). + // nullopt means no pinning preference. + std::optional replicas_preference; + bool is_cloud_topic() const { return storage_mode == model::redpanda_storage_mode::cloud || storage_mode == model::redpanda_storage_mode::tiered_cloud; @@ -315,7 +322,8 @@ struct topic_properties max_compaction_lag_ms, message_timestamp_before_max_ms, message_timestamp_after_max_ms, - storage_mode); + storage_mode, + replicas_preference); } friend bool diff --git a/src/v/cluster/topic_table.cc b/src/v/cluster/topic_table.cc index 4472efff1280e..15068ead3f263 100644 --- a/src/v/cluster/topic_table.cc +++ b/src/v/cluster/topic_table.cc @@ -1194,6 +1194,8 @@ topic_properties topic_table::update_topic_properties( storage::ntp_config::default_iceberg_mode); incremental_update( updated_properties.leaders_preference, overrides.leaders_preference); + incremental_update( + updated_properties.replicas_preference, overrides.replicas_preference); incremental_update( updated_properties.delete_retention_ms, overrides.delete_retention_ms); incremental_update( diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index e348e347acf56..5daa37da79d59 100644 --- a/src/v/cluster/topics_frontend.cc +++ b/src/v/cluster/topics_frontend.cc @@ -104,6 +104,15 @@ get_enterprise_features(const cluster::topic_configuration& cfg) { features.emplace_back("leadership pinning"); } + // We are always enforcing replica preference restrictions + if ( + const auto& replicas_pref = cfg.properties.replicas_preference; + replicas_pref.has_value() + && replicas_pref.value().type + == config::replicas_preference::type_t::racks) { + features.emplace_back("replica pinning"); + } + if (config::shard_local_cfg().iceberg_enabled.is_restricted()) { if (cfg.properties.iceberg_mode != model::iceberg_mode::disabled) { features.emplace_back("iceberg"); @@ -240,6 +249,7 @@ cluster::simple_allocation_request make_simple_allocation_request( if (topic_aware) { req.existing_replica_counts = cluster::node2count_t{}; } + req.replicas_preference = ca_cfg.cfg.properties.replicas_preference; return req; } diff --git a/src/v/cluster/types.cc b/src/v/cluster/types.cc index e6e0f288c5621..c48b06926270c 100644 --- a/src/v/cluster/types.cc +++ b/src/v/cluster/types.cc @@ -341,7 +341,7 @@ fmt::iterator incremental_topic_updates::format_to(fmt::iterator it) const { "iceberg_invalid_record_action: {}, " "iceberg_target_lag_ms: {}, " "remote_allow_gaps: {}, " - "topic_id: {}}}", + "topic_id: {}, replicas_preference: {}}}", compression, cleanup_policy_bitflags, compaction_strategy, @@ -377,7 +377,8 @@ fmt::iterator incremental_topic_updates::format_to(fmt::iterator it) const { iceberg_invalid_record_action, iceberg_target_lag_ms, remote_allow_gaps, - topic_id); + topic_id, + replicas_preference); } std::istream& operator>>(std::istream& i, replication_factor& cs) { diff --git a/src/v/cluster/types.h b/src/v/cluster/types.h index 1883ed99ce5e1..605e08e2ca604 100644 --- a/src/v/cluster/types.h +++ b/src/v/cluster/types.h @@ -591,7 +591,7 @@ struct property_update> struct incremental_topic_updates : serde::envelope< incremental_topic_updates, - serde::version<10>, + serde::version<11>, serde::compat_version<0>> { static constexpr int8_t version_with_data_policy = -1; static constexpr int8_t version_with_shadow_indexing = -3; @@ -662,6 +662,8 @@ struct incremental_topic_updates incremental_update_operation::none}; property_update> leaders_preference; + property_update> + replicas_preference; property_update> delete_retention_ms; property_update> iceberg_delete; property_update> iceberg_partition_spec; @@ -745,7 +747,8 @@ struct incremental_topic_updates message_timestamp_before_max_ms, message_timestamp_after_max_ms, remote_label, - storage_mode); + storage_mode, + replicas_preference); } fmt::iterator format_to(fmt::iterator it) const; diff --git a/src/v/cluster_link/model/types.h b/src/v/cluster_link/model/types.h index b18842a470a44..e6722da5353a3 100644 --- a/src/v/cluster_link/model/types.h +++ b/src/v/cluster_link/model/types.h @@ -77,6 +77,7 @@ inline auto disallowed_topic_properties = std::to_array({ kafka::topic_property_remote_allow_gaps, kafka::topic_property_mpx_virtual_cluster_id, kafka::topic_property_leaders_preference, + kafka::topic_property_replicas_preference, }); /** diff --git a/src/v/cluster_link/utils/topic_properties_utils.cc b/src/v/cluster_link/utils/topic_properties_utils.cc index d6f1c9baa75c2..63c20ad60e8a9 100644 --- a/src/v/cluster_link/utils/topic_properties_utils.cc +++ b/src/v/cluster_link/utils/topic_properties_utils.cc @@ -242,6 +242,14 @@ bool maybe_append_update( topic_config.properties.leaders_preference, kafka::noop_validator{}); } + if (config_name == kafka::topic_property_replicas_preference) { + return parse_and_set( + topic_config.tp_ns, + update.properties.replicas_preference, + config_value, + topic_config.properties.replicas_preference, + kafka::noop_validator{}); + } if (config_name == kafka::topic_property_delete_retention_ms) { return parse_and_set( topic_config.tp_ns, diff --git a/src/v/config/BUILD b/src/v/config/BUILD index 511d55f07d9e4..647318387e089 100644 --- a/src/v/config/BUILD +++ b/src/v/config/BUILD @@ -20,6 +20,7 @@ redpanda_cc_library( "logger.cc", "node_config.cc", "node_overrides.cc", + "replicas_preference.cc", "rest_authn_endpoint.cc", "rjson_serialization.cc", "sasl_mechanisms.cc", @@ -46,6 +47,7 @@ redpanda_cc_library( "node_config.h", "node_overrides.h", "property.h", + "replicas_preference.h", "rest_authn_endpoint.h", "rjson_serialization.h", "sasl_mechanisms.h", @@ -84,6 +86,7 @@ redpanda_cc_library( "//src/v/utils:unresolved_address", "@abseil-cpp//absl/container:flat_hash_set", "@abseil-cpp//absl/container:node_hash_set", + "@abseil-cpp//absl/strings", "@boost//:algorithm", "@boost//:filesystem", "@boost//:lexical_cast", diff --git a/src/v/config/replicas_preference.cc b/src/v/config/replicas_preference.cc new file mode 100644 index 0000000000000..154c1558a5245 --- /dev/null +++ b/src/v/config/replicas_preference.cc @@ -0,0 +1,246 @@ +// Copyright 2026 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "config/replicas_preference.h" + +#include "absl/strings/str_split.h" +#include "absl/strings/strip.h" +#include "base/vassert.h" + +#include + +#include +#include +#include +#include +#include +#include + +namespace config { + +namespace { + +constexpr std::string_view parse_error_prefix + = "couldn't parse replicas_preference: "; + +template +uint32_t checked_uint32_cast(T v, std::string_view what) { + vassert( + std::cmp_greater_equal(v, 0) + && std::cmp_less_equal(v, std::numeric_limits::max()), + "{} value {} does not fit in uint32_t", + what, + v); + return static_cast(v); +} + +/// Append a parsed rack token to `ret`, enforcing non-empty, no stray braces, +/// and uniqueness. Bumps `current_offset` on success. +void consume_rack( + std::string_view rack_str, + replicas_preference& ret, + std::unordered_set& seen, + uint32_t& current_offset) { + rack_str = absl::StripAsciiWhitespace(rack_str); + if (rack_str.empty()) { + throw std::runtime_error( + fmt::format("{}empty rack token", parse_error_prefix)); + } + if ( + rack_str.find('{') != std::string_view::npos + || rack_str.find('}') != std::string_view::npos) { + throw std::runtime_error( + fmt::format("{}nested braces", parse_error_prefix)); + } + auto rack = model::rack_id(ss::sstring(rack_str)); + if (seen.contains(rack)) { + throw std::runtime_error( + fmt::format("{}duplicate rack", parse_error_prefix)); + } + seen.insert(rack); + ret.rack_ids.push_back(std::move(rack)); + vassert( + current_offset < std::numeric_limits::max(), + "current_offset overflow"); + current_offset++; +} + +} // namespace + +std::optional +replicas_preference::group_index_for(const model::rack_id& rack) const { + // CSR lookup: rack_ids is a flat array; group_offsets[i] is the first + // index in rack_ids belonging to group i (with a trailing sentinel so + // group i spans [group_offsets[i], group_offsets[i+1])). To find the + // owning group for a rack, locate the rack in rack_ids and take the + // upper_bound of its index in group_offsets — the preceding entry is + // the owning group. + auto it = std::ranges::find(rack_ids, rack); + if (it == rack_ids.end()) { + return std::nullopt; + } + auto rack_index = checked_uint32_cast( + std::distance(rack_ids.begin(), it), "rack index"); + auto group_iter = std::ranges::upper_bound(group_offsets, rack_index); + vassert( + group_iter != group_offsets.begin(), + "group_offsets invariant broken: upper_bound at begin for rack index {}", + rack_index); + return checked_uint32_cast( + std::distance(group_offsets.begin(), group_iter) - 1, "group index"); +} + +uint32_t replicas_preference::num_groups() const { + if (group_offsets.empty()) { + return 0; + } + // group_offsets has num_groups + 1 entries (includes sentinel) + return checked_uint32_cast(group_offsets.size() - 1, "num_groups"); +} + +replicas_preference replicas_preference::parse(std::string_view s) { + auto trimmed = absl::StripAsciiWhitespace(s); + + if (trimmed == none_prefix || trimmed.empty()) { + return replicas_preference{}; + } + + if (!trimmed.starts_with(racks_prefix)) { + throw std::runtime_error( + fmt::format( + "{}should be \"{}\" or start with \"{}\"", + parse_error_prefix, + none_prefix, + racks_prefix)); + } + + auto body = trimmed.substr(racks_prefix.size()); + + replicas_preference ret; + ret.type = type_t::racks; + + std::unordered_set seen; + bool in_group = false; + uint32_t current_offset = 0; + + // Start the first group + ret.group_offsets.push_back(0); + + for (std::string_view token_sv : absl::StrSplit(body, ',')) { + auto tok = absl::StripAsciiWhitespace(token_sv); + if (tok.empty()) { + throw std::runtime_error( + fmt::format("{}empty rack token", parse_error_prefix)); + } + + bool starts_group = tok.starts_with("{"); + bool ends_group = tok.ends_with("}"); + + if (starts_group && ends_group) { + // Single-element group like {A}: strip '{' prefix and '}' suffix + // with substr(1, tok.size() - 2). + if (in_group) { + throw std::runtime_error( + fmt::format("{}nested braces", parse_error_prefix)); + } + consume_rack( + tok.substr(1, tok.size() - 2), ret, seen, current_offset); + ret.group_offsets.push_back(current_offset); + } else if (starts_group) { + // Group-opening token: strip leading '{' with substr(1). + if (in_group) { + throw std::runtime_error( + fmt::format("{}nested braces", parse_error_prefix)); + } + in_group = true; + consume_rack(tok.substr(1), ret, seen, current_offset); + } else if (ends_group) { + // Group-closing token: strip trailing '}' with + // substr(0, tok.size() - 1). + if (!in_group) { + throw std::runtime_error( + fmt::format( + "{}unexpected closing brace", parse_error_prefix)); + } + in_group = false; + consume_rack( + tok.substr(0, tok.size() - 1), ret, seen, current_offset); + ret.group_offsets.push_back(current_offset); + } else { + consume_rack(tok, ret, seen, current_offset); + if (!in_group) { + ret.group_offsets.push_back(current_offset); + } + } + } + + if (in_group) { + throw std::runtime_error( + fmt::format("{}unclosed brace", parse_error_prefix)); + } + + if (ret.rack_ids.empty()) { + throw std::runtime_error( + fmt::format("{}no racks specified", parse_error_prefix)); + } + + // Canonicalize: sort rack_ids within each group so that permutations + // like "racks: A, {C, B}" and "racks: A, {B, C}" compare equal and + // serialize identically. Groups themselves remain in the user-specified + // priority order. + for (uint32_t group_idx = 0; group_idx + 1 < ret.group_offsets.size(); + ++group_idx) { + auto group_start = ret.rack_ids.begin() + ret.group_offsets[group_idx]; + auto group_end = ret.rack_ids.begin() + + ret.group_offsets[group_idx + 1]; + std::ranges::sort(group_start, group_end); + } + + return ret; +} + +fmt::iterator replicas_preference::format_to(fmt::iterator it) const { + if (type == type_t::none) { + return fmt::format_to(it, "{}", none_str); + } + + it = fmt::format_to(it, "{} ", racks_prefix); + auto total_groups = num_groups(); + for (uint32_t group_idx = 0; group_idx < total_groups; ++group_idx) { + if (group_idx > 0) { + it = fmt::format_to(it, ", "); + } + auto group_start = group_offsets[group_idx]; + auto group_end = group_offsets[group_idx + 1]; + auto group_size = group_end - group_start; + + if (group_size > 1) { + it = fmt::format_to(it, "{{"); + for (auto i = group_start; i < group_end; ++i) { + if (i > group_start) { + it = fmt::format_to(it, ", "); + } + it = fmt::format_to(it, "{}", rack_ids[i]); + } + it = fmt::format_to(it, "}}"); + } else { + it = fmt::format_to(it, "{}", rack_ids[group_start]); + } + } + return it; +} + +std::istream& operator>>(std::istream& is, replicas_preference& res) { + std::stringstream ss; + ss << is.rdbuf(); + res = replicas_preference::parse(ss.str()); + return is; +} + +} // namespace config diff --git a/src/v/config/replicas_preference.h b/src/v/config/replicas_preference.h new file mode 100644 index 0000000000000..ee8c3186fcf99 --- /dev/null +++ b/src/v/config/replicas_preference.h @@ -0,0 +1,71 @@ +/* + * Copyright 2026 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#pragma once + +#include "base/format_to.h" +#include "model/metadata.h" +#include "serde/envelope.h" + +#include +#include +#include +#include + +namespace config { + +/// Per-topic replica placement preference using priority-ordered rack groups. +/// +/// Format: "racks: A, {B, C}, D" means prefer rack A (highest priority), +/// then racks B and C (equal priority), then rack D. +/// +/// Uses CSR-style storage: a flat rack_ids array with group_offsets indexing +/// into it. For the example above: +/// rack_ids = [A, B, C, D], group_offsets = [0, 1, 3, 4] +struct replicas_preference + : public serde::envelope< + replicas_preference, + serde::version<0>, + serde::compat_version<0>> { + enum class type_t { + none, + racks, + }; + + static constexpr std::string_view none_str = "none"; + static constexpr std::string_view racks_str = "racks"; + static constexpr std::string_view none_prefix = "none"; + static constexpr std::string_view racks_prefix = "racks:"; + + type_t type = type_t::none; + + // CSR-style storage for priority-ordered rack groups. + std::vector rack_ids; + std::vector group_offsets; + + /// Returns the priority group index for a rack, or nullopt if not found. + std::optional group_index_for(const model::rack_id&) const; + + /// Number of priority groups. + uint32_t num_groups() const; + + static replicas_preference parse(std::string_view); + + fmt::iterator format_to(fmt::iterator it) const; + friend std::istream& operator>>(std::istream&, replicas_preference&); + + friend bool operator==( + const replicas_preference&, const replicas_preference&) = default; + + auto serde_fields() { return std::tie(type, rack_ids, group_offsets); } +}; + +} // namespace config diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 92b5f4c0f8a75..74c7705efafc4 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -223,6 +223,28 @@ redpanda_cc_btest( ], ) +redpanda_cc_gtest( + name = "replicas_preference_test", + timeout = "short", + srcs = [ + "replicas_preference_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/bytes:iobuf", + "//src/v/bytes:iobuf_parser", + "//src/v/config", + "//src/v/serde", + "//src/v/serde:enum", + "//src/v/serde:named_type", + "//src/v/serde:sstring", + "//src/v/serde:vector", + "//src/v/test_utils:gtest", + "@fmt", + "@googletest//:gtest", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", diff --git a/src/v/config/tests/replicas_preference_test.cc b/src/v/config/tests/replicas_preference_test.cc new file mode 100644 index 0000000000000..363fbba41ddb9 --- /dev/null +++ b/src/v/config/tests/replicas_preference_test.cc @@ -0,0 +1,203 @@ +/* + * Copyright 2026 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#include "bytes/iobuf.h" +#include "bytes/iobuf_parser.h" +#include "config/replicas_preference.h" +#include "serde/rw/enum.h" +#include "serde/rw/envelope.h" +#include "serde/rw/named_type.h" +#include "serde/rw/sstring.h" +#include "serde/rw/vector.h" + +#include +#include + +#include +#include + +namespace { + +using rp = config::replicas_preference; +using rp_t = config::replicas_preference::type_t; + +TEST(ReplicasPreference, ParseNone) { + for (std::string_view input : {"none", "", " ", "\t \n"}) { + SCOPED_TRACE(fmt::format("input: '{}'", input)); + auto p = rp::parse(input); + EXPECT_EQ(p.type, rp_t::none); + EXPECT_TRUE(p.rack_ids.empty()); + EXPECT_EQ(p.num_groups(), 0); + EXPECT_FALSE(p.group_index_for(model::rack_id{"A"}).has_value()); + } +} + +TEST(ReplicasPreference, ParseSingleRack) { + auto p = rp::parse("racks: A"); + EXPECT_EQ(p.type, rp_t::racks); + EXPECT_EQ(p.num_groups(), 1); + EXPECT_EQ(p.group_index_for(model::rack_id{"A"}).value(), 0U); + EXPECT_FALSE(p.group_index_for(model::rack_id{"Z"}).has_value()); +} + +TEST(ReplicasPreference, ParseMultipleRacksEachItsOwnGroup) { + auto p = rp::parse("racks: A, B, C"); + EXPECT_EQ(p.num_groups(), 3); + EXPECT_EQ(p.group_index_for(model::rack_id{"A"}).value(), 0U); + EXPECT_EQ(p.group_index_for(model::rack_id{"B"}).value(), 1U); + EXPECT_EQ(p.group_index_for(model::rack_id{"C"}).value(), 2U); + EXPECT_FALSE(p.group_index_for(model::rack_id{"D"}).has_value()); +} + +TEST(ReplicasPreference, ParseGroups) { + // Mixed single rack + multi-rack group + single rack. + { + auto p = rp::parse("racks: A, {B, C}, D"); + EXPECT_EQ(p.num_groups(), 3); + EXPECT_EQ(p.group_index_for(model::rack_id{"A"}).value(), 0U); + EXPECT_EQ(p.group_index_for(model::rack_id{"B"}).value(), 1U); + EXPECT_EQ(p.group_index_for(model::rack_id{"C"}).value(), 1U); + EXPECT_EQ(p.group_index_for(model::rack_id{"D"}).value(), 2U); + } + // Multiple multi-rack groups. + { + auto p = rp::parse("racks: {A, B}, {C, D}, E"); + EXPECT_EQ(p.num_groups(), 3); + EXPECT_EQ(p.group_index_for(model::rack_id{"A"}).value(), 0U); + EXPECT_EQ(p.group_index_for(model::rack_id{"B"}).value(), 0U); + EXPECT_EQ(p.group_index_for(model::rack_id{"C"}).value(), 1U); + EXPECT_EQ(p.group_index_for(model::rack_id{"D"}).value(), 1U); + EXPECT_EQ(p.group_index_for(model::rack_id{"E"}).value(), 2U); + } + // Explicit single-element group (equivalent to a bare rack). + { + auto p = rp::parse("racks: {A}, B"); + EXPECT_EQ(p.num_groups(), 2); + EXPECT_EQ(p.group_index_for(model::rack_id{"A"}).value(), 0U); + EXPECT_EQ(p.group_index_for(model::rack_id{"B"}).value(), 1U); + } +} + +TEST(ReplicasPreference, ParseToleratesWhitespace) { + // Extra internal whitespace and surrounding whitespace. + auto p1 = rp::parse(" racks: A , { B , C } , D "); + // No whitespace at all. + auto p2 = rp::parse("racks:A,{B,C},D"); + // Canonical spacing. + auto p3 = rp::parse("racks: A, {B, C}, D"); + EXPECT_EQ(p1, p2); + EXPECT_EQ(p1, p3); +} + +TEST(ReplicasPreference, ParseErrors) { + for (std::string_view bad : { + "invalid", // no prefix + "racks: {}", // empty braces + "racks: {{A}}", // nested braces + "racks: {A, B", // unclosed brace + "racks: A}", // unexpected close + "racks: A, A", // duplicate + "racks: A, {B, A}", // duplicate across groups + "racks: A,,B", // empty token + "racks: ", // no racks + "racks: A,", // trailing comma + }) { + SCOPED_TRACE(fmt::format("input: '{}'", bad)); + EXPECT_THROW(rp::parse(bad), std::runtime_error); + } +} + +TEST(ReplicasPreference, FormatRoundtrip) { + // parse(input) == parse(format(parse(input))), and format is stable. + for (std::string_view input : { + "none", + "racks: A", + "racks: A, B, C", + "racks: A, {B, C}, D", + "racks: {A, B}, {C, D}, E", + "racks: {A}, B", + }) { + SCOPED_TRACE(fmt::format("input: '{}'", input)); + auto orig = rp::parse(input); + auto str = fmt::format("{}", orig); + auto reparsed = rp::parse(str); + EXPECT_EQ(orig, reparsed); + EXPECT_EQ(str, fmt::format("{}", reparsed)); + } +} + +TEST(ReplicasPreference, FormatExact) { + EXPECT_EQ(fmt::format("{}", rp{}), "none"); + EXPECT_EQ(fmt::format("{}", rp::parse("racks: A")), "racks: A"); + EXPECT_EQ( + fmt::format("{}", rp::parse("racks: A, {B, C}, D")), + "racks: A, {B, C}, D"); + EXPECT_EQ( + fmt::format("{}", rp::parse("racks: {A, B}, {C, D}, E")), + "racks: {A, B}, {C, D}, E"); +} + +TEST(ReplicasPreference, SerdeRoundtrip) { + for (std::string_view input : { + "none", + "racks: A", + "racks: A, B, C", + "racks: A, {B, C}, D", + "racks: {A, B}, {C, D}, E", + }) { + SCOPED_TRACE(fmt::format("input: '{}'", input)); + auto orig = rp::parse(input); + + iobuf buf; + serde::write(buf, orig); + + auto parser = iobuf_parser(std::move(buf)); + auto decoded = serde::read(parser); + + EXPECT_EQ(orig, decoded); + } +} + +TEST(ReplicasPreference, Inequality) { + auto a = rp::parse("racks: A, B, C"); + EXPECT_NE(a, rp::parse("racks: A, B")); // fewer racks + EXPECT_NE(a, rp::parse("racks: A, {B, C}")); // different grouping + EXPECT_NE(a, rp{}); // vs. none +} + +// Intra-group rack order must not affect equality or serialization: +// "racks: A, {C, B}" is equivalent to "racks: A, {B, C}", and the +// canonical form is the lexicographically sorted one. +TEST(ReplicasPreference, CanonicalOrderingEquality) { + EXPECT_EQ(rp::parse("racks: A, {B, C}"), rp::parse("racks: A, {C, B}")); + EXPECT_EQ( + rp::parse("racks: {A, B}, {C, D}"), rp::parse("racks: {B, A}, {D, C}")); +} + +TEST(ReplicasPreference, CanonicalSerialization) { + // Both orderings serialize to the sorted canonical form. + EXPECT_EQ( + fmt::format("{}", rp::parse("racks: A, {C, B}")), "racks: A, {B, C}"); + EXPECT_EQ( + fmt::format("{}", rp::parse("racks: {B, A}, {D, C}")), + "racks: {A, B}, {C, D}"); +} + +TEST(ReplicasPreference, IstreamOperator) { + auto orig = rp::parse("racks: A, {B, C}, D"); + std::istringstream is(fmt::format("{}", orig)); + + rp parsed; + is >> parsed; + EXPECT_EQ(orig, parsed); +} + +} // namespace diff --git a/src/v/features/enterprise_feature_messages.h b/src/v/features/enterprise_feature_messages.h index 50770b892454e..b46a0bb317b91 100644 --- a/src/v/features/enterprise_feature_messages.h +++ b/src/v/features/enterprise_feature_messages.h @@ -89,6 +89,13 @@ inline ss::sstring default_leaders_preference() { request_or_trial); } +inline ss::sstring default_replicas_preference() { + return ssx::sformat( + R"({} to use the enterprise feature "replica pinning". This feature is disabled. The values of the topic property "redpanda.replicas.preference" are being ignored. {})", + required, + request_or_trial); +} + inline ss::sstring topic_property(auto features) { return ssx::sformat( "{} to enable ({}). {}.", diff --git a/src/v/features/enterprise_features.h b/src/v/features/enterprise_features.h index 0cf218f966b5b..7ac4ec6c3bbd1 100644 --- a/src/v/features/enterprise_features.h +++ b/src/v/features/enterprise_features.h @@ -32,6 +32,7 @@ enum class license_required_feature { fips, datalake_iceberg, leadership_pinning, + replica_pinning, shadow_linking, cloud_topics, topic_deletion_disabled, @@ -61,6 +62,8 @@ inline fmt::iterator format_to(license_required_feature f, fmt::iterator out) { return fmt::format_to(out, "datalake_iceberg"); case license_required_feature::leadership_pinning: return fmt::format_to(out, "leadership_pinning"); + case license_required_feature::replica_pinning: + return fmt::format_to(out, "replica_pinning"); case license_required_feature::shadow_linking: return fmt::format_to(out, "shadow_linking"); case license_required_feature::cloud_topics: diff --git a/src/v/kafka/protocol/topic_properties.h b/src/v/kafka/protocol/topic_properties.h index 6952abdd7ec92..284bf0cc81996 100644 --- a/src/v/kafka/protocol/topic_properties.h +++ b/src/v/kafka/protocol/topic_properties.h @@ -76,6 +76,9 @@ inline constexpr std::string_view topic_property_mpx_virtual_cluster_id inline constexpr std::string_view topic_property_leaders_preference = "redpanda.leaders.preference"; +inline constexpr std::string_view topic_property_replicas_preference + = "redpanda.replicas.preference"; + inline constexpr std::string_view topic_property_redpanda_storage_mode = "redpanda.storage.mode"; diff --git a/src/v/kafka/server/handlers/alter_configs.cc b/src/v/kafka/server/handlers/alter_configs.cc index 97057576fb530..4dcdb400aa05f 100644 --- a/src/v/kafka/server/handlers/alter_configs.cc +++ b/src/v/kafka/server/handlers/alter_configs.cc @@ -13,6 +13,8 @@ #include "cluster/types.h" #include "config/configuration.h" #include "config/leaders_preference.h" +#include "config/replicas_preference.h" +#include "features/enterprise_feature_messages.h" #include "features/feature_table.h" #include "kafka/protocol/errors.h" #include "kafka/protocol/schemata/alter_configs_request.h" @@ -98,7 +100,7 @@ create_topic_properties_update( std::apply(apply_op(op_t::none), update.custom_properties.serde_fields()); static_assert( - std::tuple_size_v == 44, + std::tuple_size_v == 45, "If you add a property, decide on its default alter config " "policy, and handle the update in the loop below"); static_assert( @@ -403,6 +405,27 @@ create_topic_properties_update( config::leaders_preference::parse); continue; } + if (cfg.name == topic_property_replicas_preference) { + auto enterprise_validator = + [&feature_table = ctx.feature_table().local()]( + const ss::sstring&, const config::replicas_preference& rp) + -> std::optional { + if ( + rp.type == config::replicas_preference::type_t::racks + && feature_table.should_sanction()) { + return features::enterprise_error_message:: + default_replicas_preference(); + } + return std::nullopt; + }; + parse_and_set_optional( + update.properties.replicas_preference, + cfg.value, + kafka::config_resource_operation::set, + enterprise_validator, + config::replicas_preference::parse); + continue; + } if (cfg.name == topic_property_delete_retention_ms) { parse_and_set_tristate( update.properties.delete_retention_ms, diff --git a/src/v/kafka/server/handlers/configs/config_response_utils.cc b/src/v/kafka/server/handlers/configs/config_response_utils.cc index 0f77cb45051e0..07c01a9012c2e 100644 --- a/src/v/kafka/server/handlers/configs/config_response_utils.cc +++ b/src/v/kafka/server/handlers/configs/config_response_utils.cc @@ -16,6 +16,7 @@ #include "cluster/types.h" #include "config/configuration.h" #include "config/node_config.h" +#include "config/replicas_preference.h" #include "kafka/server/handlers/topics/types.h" #include "model/metadata.h" @@ -183,7 +184,8 @@ consteval describe_configs_type property_config_type() { std::is_same_v || std::is_same_v || std::is_same_v || - std::is_same_v || std::is_same_v || + std::is_same_v || + std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v; @@ -1051,6 +1053,23 @@ config_response_container_t make_topic_configs( "Preferred location (e.g. rack) for partition leaders of this topic."), &describe_as_string); + add_topic_config_if_requested( + config_keys, + result, + topic_property_replicas_preference, + topic_properties.replicas_preference, + include_synonyms, + maybe_make_documentation( + include_documentation, + "Preferred rack placement for replicas of this topic. " + "Priority-ordered with optional grouping, e.g. " + "\"racks: A, {B, C}, D\"."), + [](const std::optional& property) + -> std::optional { + return describe_as_string( + property.value_or(config::replicas_preference{})); + }); + if (topic_properties.iceberg_mode != model::iceberg_mode::disabled) { add_topic_config_if_requested( config_keys, diff --git a/src/v/kafka/server/handlers/configs/storage_mode_properties.h b/src/v/kafka/server/handlers/configs/storage_mode_properties.h index b0033d7806977..8a18c0df0236d 100644 --- a/src/v/kafka/server/handlers/configs/storage_mode_properties.h +++ b/src/v/kafka/server/handlers/configs/storage_mode_properties.h @@ -73,6 +73,7 @@ inline constexpr auto storage_mode_properties {topic_property_message_timestamp_before_max_ms, storage_mode_mask::all}, {topic_property_message_timestamp_after_max_ms, storage_mode_mask::all}, {topic_property_leaders_preference, storage_mode_mask::all}, + {topic_property_replicas_preference, storage_mode_mask::all}, {topic_property_mpx_virtual_cluster_id, storage_mode_mask::all}, {topic_property_recovery, storage_mode_mask::all}, {topic_property_compaction_strategy, storage_mode_mask::all}, diff --git a/src/v/kafka/server/handlers/create_topics.cc b/src/v/kafka/server/handlers/create_topics.cc index d2e80b8084659..9302ce5a3f255 100644 --- a/src/v/kafka/server/handlers/create_topics.cc +++ b/src/v/kafka/server/handlers/create_topics.cc @@ -73,6 +73,7 @@ bool is_supported(std::string_view name) { topic_property_flush_bytes, topic_property_iceberg_mode, topic_property_leaders_preference, + topic_property_replicas_preference, topic_property_delete_retention_ms, topic_property_iceberg_delete, topic_property_iceberg_partition_spec, diff --git a/src/v/kafka/server/handlers/incremental_alter_configs.cc b/src/v/kafka/server/handlers/incremental_alter_configs.cc index 62f4e1a66e3ff..8e39cb7664991 100644 --- a/src/v/kafka/server/handlers/incremental_alter_configs.cc +++ b/src/v/kafka/server/handlers/incremental_alter_configs.cc @@ -11,6 +11,9 @@ #include "cluster/config_frontend.h" #include "cluster/types.h" #include "config/configuration.h" +#include "config/node_config.h" +#include "config/replicas_preference.h" +#include "features/enterprise_feature_messages.h" #include "features/feature_table.h" #include "kafka/protocol/errors.h" #include "kafka/protocol/incremental_alter_configs.h" @@ -388,6 +391,27 @@ create_topic_properties_update( config::leaders_preference::parse); continue; } + if (cfg.name == topic_property_replicas_preference) { + auto enterprise_validator = + [&feature_table = ctx.feature_table().local()]( + const ss::sstring&, const config::replicas_preference& rp) + -> std::optional { + if ( + rp.type == config::replicas_preference::type_t::racks + && feature_table.should_sanction()) { + return features::enterprise_error_message:: + default_replicas_preference(); + } + return std::nullopt; + }; + parse_and_set_optional( + update.properties.replicas_preference, + cfg.value, + op, + enterprise_validator, + config::replicas_preference::parse); + continue; + } if (cfg.name == topic_property_delete_retention_ms) { parse_and_set_tristate( update.properties.delete_retention_ms, cfg.value, op); diff --git a/src/v/kafka/server/handlers/topics/types.cc b/src/v/kafka/server/handlers/topics/types.cc index b5edaa0a8e766..0e65ce48831c0 100644 --- a/src/v/kafka/server/handlers/topics/types.cc +++ b/src/v/kafka/server/handlers/topics/types.cc @@ -11,6 +11,8 @@ #include "cluster/types.h" #include "config/configuration.h" +#include "config/replicas_preference.h" +#include "container/chunked_vector.h" #include "kafka/server/handlers/configs/config_response_utils.h" #include "kafka/server/handlers/configs/config_utils.h" #include "model/compression.h" @@ -167,6 +169,16 @@ get_leaders_preference(const config_map_t& config) { return std::nullopt; } +static std::optional +get_replicas_preference(const config_map_t& config) { + if ( + auto it = config.find(topic_property_replicas_preference); + it != config.end()) { + return config::replicas_preference::parse(it->second); + } + return std::nullopt; +} + static tristate get_delete_retention_ms(const config_map_t& config) { auto delete_retention_ms = get_tristate_value( @@ -300,6 +312,9 @@ cluster::topic_configuration to_topic_config( cfg.properties.leaders_preference = get_leaders_preference(config_entries); + cfg.properties.replicas_preference = get_replicas_preference( + config_entries); + cfg.properties.delete_retention_ms = get_delete_retention_ms( config_entries); diff --git a/src/v/kafka/server/tests/alter_config_test.cc b/src/v/kafka/server/tests/alter_config_test.cc index 434f2242701b7..10352fd7cfced 100644 --- a/src/v/kafka/server/tests/alter_config_test.cc +++ b/src/v/kafka/server/tests/alter_config_test.cc @@ -750,6 +750,7 @@ FIXTURE_TEST( "flush.bytes", "redpanda.iceberg.mode", "redpanda.leaders.preference", + "redpanda.replicas.preference", "delete.retention.ms", "min.cleanable.dirty.ratio", "redpanda.remote.allowgaps", diff --git a/tests/rptest/tests/describe_topics_test.py b/tests/rptest/tests/describe_topics_test.py index e2c70970ec457..b1f2b737126cd 100644 --- a/tests/rptest/tests/describe_topics_test.py +++ b/tests/rptest/tests/describe_topics_test.py @@ -249,6 +249,13 @@ def test_describe_topics_with_documentation_and_types(self): value="none", doc_string="Preferred location (e.g. rack) for partition leaders of this topic.", ), + "redpanda.replicas.preference": ConfigProperty( + config_type="STRING", + value="none", + doc_string="Preferred rack placement for replicas of this topic. " + "Priority-ordered with optional grouping, e.g. " + '"racks: A, {B, C}, D".', + ), "delete.retention.ms": ConfigProperty( config_type="LONG", value="-1", diff --git a/tests/rptest/tests/enterprise_features_license_test.py b/tests/rptest/tests/enterprise_features_license_test.py index b093d11cec711..8ad9276f647e2 100644 --- a/tests/rptest/tests/enterprise_features_license_test.py +++ b/tests/rptest/tests/enterprise_features_license_test.py @@ -5,7 +5,7 @@ from ducktape.mark import matrix from ducktape.utils.util import wait_until -from rptest.clients.rpk import RpkTool +from rptest.clients.rpk import RpkException, RpkTool from rptest.services.admin import ( Admin, EnterpriseLicenseStatus, @@ -49,6 +49,7 @@ class Feature(IntEnum): shadow_linking = 13 cloud_topics = 14 topic_deletion_disabled = 15 + replica_pinning = 16 def to_enterprise_feature(feature): @@ -82,6 +83,7 @@ def to_enterprise_feature(feature): Feature.shadow_linking: "enable_shadow_linking", Feature.cloud_topics: CLOUD_TOPICS_CONFIG_STR, Feature.topic_deletion_disabled: "delete_topic_enable", + Feature.replica_pinning: "redpanda.replicas.preference", } SKIP_FEATURES = [ @@ -255,6 +257,13 @@ def has_role(r: str): replicas=1, config={"redpanda.leaders.preference": "racks:rack1"}, ) + elif feature == Feature.replica_pinning: + RpkTool(self.redpanda).create_topic( + "foo", + partitions=1, + replicas=1, + config={"redpanda.replicas.preference": "racks:rack1"}, + ) elif feature == Feature.shadow_linking: self.redpanda.set_cluster_config({"enable_shadow_linking": "true"}) elif feature == Feature.cloud_topics: @@ -295,10 +304,21 @@ def test_enable_features(self, feature, install_license, disable_trial): if has_license: self.try_enable_feature(feature) else: + + def matches(e): + if isinstance(e, RpkException): + # Kafka-side enterprise rejection references the license, + # not the topic property name -- e.g. + # "A Redpanda Enterprise Edition license is required to + # enable (replica pinning). ..." + return "Enterprise" in str(e) and "license" in str(e) + return ( + e.response.status_code == 403 + or FEATURE_DEPENDENT_CONFIG[feature] in e.response.json().keys() + ) + with expect_exception( - requests.exceptions.HTTPError, - lambda e: e.response.status_code == 403 - or FEATURE_DEPENDENT_CONFIG[feature] in e.response.json().keys(), + (requests.exceptions.HTTPError, RpkException), matches ): self.try_enable_feature(feature) diff --git a/tests/rptest/tests/replica_pinning_test.py b/tests/rptest/tests/replica_pinning_test.py new file mode 100644 index 0000000000000..15beab935130d --- /dev/null +++ b/tests/rptest/tests/replica_pinning_test.py @@ -0,0 +1,749 @@ +# Copyright 2026 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 + +import time +from collections import defaultdict + +from ducktape.tests.test import TestContext +from ducktape.utils.util import wait_until + +from rptest.clients.rpk import RpkTool +from rptest.services.admin import Admin +from rptest.services.cluster import cluster +from rptest.tests.redpanda_test import RedpandaTest +from rptest.util import wait_until_result + +REPLICAS_PREFERENCE_KEY = "redpanda.replicas.preference" + +# Allowed deviation from expected replica counts in statistical tests. +# Internal topic replicas cause small shifts in the distribution. +JITTER_PERCENT = 0.05 + + +class ReplicaPinningTest(RedpandaTest): + """ + Integration tests for the per-topic replicas_preference property that + influences replica placement using a priority-ordered list of racks. + + All tests allocate 8 nodes (the max needed across all tests) and start + only the subset required via _start_cluster(rack_layout). + """ + + def __init__(self, test_context: TestContext): + super().__init__( + test_context=test_context, + num_brokers=8, + skip_if_no_redpanda_log=True, + ) + + def setUp(self): + # Defer start to each test method via _start_cluster. + pass + + def _start_cluster( + self, + rack_layout: list[str | None], + rack_awareness: bool = True, + ): + """ + Configure and start a cluster with the given rack layout. + + rack_layout is a list of rack names, one per broker. Use None for + a rackless node. The list length determines how many of the + pre-allocated nodes to start. + """ + nodes_to_start = self.redpanda.nodes[: len(rack_layout)] + + self.redpanda.add_extra_rp_conf( + { + "enable_rack_awareness": rack_awareness, + "partition_autobalancing_mode": "continuous", + "partition_autobalancing_node_availability_timeout_sec": 10, + "partition_autobalancing_tick_interval_ms": 5000, + "raft_learner_recovery_rate": 100_000_000, + } + ) + + for ix, node in enumerate(nodes_to_start): + if rack_layout[ix] is not None: + self.redpanda.set_extra_node_conf(node, {"rack": rack_layout[ix]}) + + self.redpanda.start(nodes=nodes_to_start) + self._rack_layout = rack_layout + + def _node_id_to_rack(self) -> dict[int, str | None]: + """Return a map of node_id -> rack from the admin API.""" + admin = Admin(self.redpanda) + brokers = admin.get_brokers() + result: dict[int, str | None] = {} + for b in brokers: + result[b["node_id"]] = b.get("rack") + return result + + def _get_partition_rack_counts( + self, topic: str, n2r: dict[int, str | None] + ) -> dict[str | None, int]: + """ + For a single-partition topic, return {rack: replica_count}. + """ + rpk = RpkTool(self.redpanda) + partitions = list(rpk.describe_topic(topic)) + assert len(partitions) == 1, f"Expected 1 partition, got {len(partitions)}" + counts: dict[str | None, int] = defaultdict(int) + for replica_id in partitions[0].replicas: + counts[n2r.get(replica_id)] += 1 + return dict(counts) + + def _get_multi_partition_rack_counts( + self, + topic: str, + n2r: dict[int, str | None], + expected_partitions: int, + ) -> dict[str | None, int]: + """ + For a multi-partition topic, return total replica count per rack + across all partitions. Waits until all partitions are ready. + """ + rpk = RpkTool(self.redpanda) + + def ready(): + try: + parts = list(rpk.describe_topic(topic)) + except Exception: + return False + if len(parts) != expected_partitions: + return False + for p in parts: + if len(p.replicas) == 0: + return False + return (True, parts) + + parts = wait_until_result( + ready, timeout_sec=60, backoff_sec=1, err_msg="partitions not ready" + ) + + counts: dict[str | None, int] = defaultdict(int) + for p in parts: + for replica_id in p.replicas: + counts[n2r.get(replica_id)] += 1 + return dict(counts) + + def _get_multi_partition_node_counts( + self, + topic: str, + expected_partitions: int, + ) -> dict[int, int]: + """ + For a multi-partition topic, return total replica count per node. + Waits until all partitions are ready. + """ + rpk = RpkTool(self.redpanda) + + def ready(): + try: + parts = list(rpk.describe_topic(topic)) + except Exception: + return False + if len(parts) != expected_partitions: + return False + for p in parts: + if len(p.replicas) == 0: + return False + return (True, parts) + + parts = wait_until_result( + ready, timeout_sec=60, backoff_sec=1, err_msg="partitions not ready" + ) + + counts: dict[int, int] = defaultdict(int) + for p in parts: + for replica_id in p.replicas: + counts[replica_id] += 1 + return dict(counts) + + def _wait_no_reconfigurations(self, timeout_sec: int = 120): + """Wait until there are no pending partition reconfigurations.""" + admin = Admin(self.redpanda) + + def no_reconfigs(): + try: + return len(admin.list_reconfigurations()) == 0 + except Exception: + return False + + wait_until( + no_reconfigs, + timeout_sec=timeout_sec, + backoff_sec=2, + err_msg="Reconfigurations did not complete in time", + ) + + # ------------------------------------------------------------------ + # Test 1: Pinning + rack awareness ON + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_with_rack_awareness(self): + """ + Rack awareness (L0) spreads across racks, pinning (L1) selects WHICH + racks. Unpinned rack D is skipped. + + 5 nodes, 4 racks. rack_awareness=ON. RF=3, preference="racks: A, B, C" + Nodes: 1,2 in rack A; 3 in rack B; 4 in rack C; 5 in rack D + + Expected: 1 replica in each of racks A, B, C. 0 in rack D. + """ + self._start_cluster(["A", "A", "B", "C", "D"]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-rack-aware", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B, C"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-rack-aware", n2r) + + self.logger.info(f"Rack counts: {rc}") + assert rc.get("A", 0) == 1, ( + f"Expected 1 replica in rack A, got {rc.get('A', 0)}" + ) + assert rc.get("B", 0) == 1, ( + f"Expected 1 replica in rack B, got {rc.get('B', 0)}" + ) + assert rc.get("C", 0) == 1, ( + f"Expected 1 replica in rack C, got {rc.get('C', 0)}" + ) + assert rc.get("D", 0) == 0, ( + f"Expected 0 replicas in rack D, got {rc.get('D', 0)}" + ) + + # ------------------------------------------------------------------ + # Test 2: Pinning without rack awareness + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_without_rack_awareness(self): + """ + With rack awareness OFF, pinning concentrates replicas in the + highest-priority rack, then overflows to the next. + + 5 nodes, 3 racks. rack_awareness=OFF. RF=3, preference="racks: A, B" + Nodes: 1,2,3 in rack A; 4 in rack B; 5 in rack C + Rack A has capacity 3 >= RF, so all 3 replicas should land on A. + + Expected: 3 replicas in rack A, 0 in B, 0 in C. + """ + self._start_cluster(["A", "A", "A", "B", "C"], rack_awareness=False) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-no-rack", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-no-rack", n2r) + + self.logger.info(f"Rack counts: {rc}") + assert rc.get("A", 0) == 3, ( + f"Expected 3 replicas in rack A, got {rc.get('A', 0)}" + ) + assert rc.get("B", 0) == 0, ( + f"Expected 0 replicas in rack B, got {rc.get('B', 0)}" + ) + assert rc.get("C", 0) == 0, ( + f"Expected 0 replicas in rack C, got {rc.get('C', 0)}" + ) + + # ------------------------------------------------------------------ + # Test 3: Priority ordering + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_priority_ordering(self): + """ + The allocator respects priority ordering: A > B > C > D. With RF=3, + the top-3 priority racks are chosen. + + 4 nodes, 4 racks. rack_awareness=ON. RF=3, + preference="racks: A, B, C, D" + """ + self._start_cluster(["A", "B", "C", "D"]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-priority", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B, C, D"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-priority", n2r) + + self.logger.info(f"Rack counts: {rc}") + assert rc.get("A", 0) == 1, ( + f"Expected 1 replica in rack A, got {rc.get('A', 0)}" + ) + assert rc.get("B", 0) == 1, ( + f"Expected 1 replica in rack B, got {rc.get('B', 0)}" + ) + assert rc.get("C", 0) == 1, ( + f"Expected 1 replica in rack C, got {rc.get('C', 0)}" + ) + assert rc.get("D", 0) == 0, ( + f"Expected 0 replicas in rack D, got {rc.get('D', 0)}" + ) + + # ------------------------------------------------------------------ + # Test 4: Group notation -- statistical + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_with_group_notation(self): + """ + Group notation {B, C, D} treats racks within braces as equal + priority. Over many partitions, B, C, D should get roughly equal + replica counts. + + 8 nodes, 4 racks. rack_awareness=ON. 60 partitions, RF=3, + preference="racks: A, {B, C, D}" + Nodes: 2 in A; 2 in B; 2 in C; 2 in D + """ + self._start_cluster(["A", "A", "B", "B", "C", "C", "D", "D"]) + + num_partitions = 60 + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-groups", + partitions=num_partitions, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, {B, C, D}"}, + ) + + self._wait_no_reconfigurations() + + n2r = self._node_id_to_rack() + rc = self._get_multi_partition_rack_counts( + "pinning-groups", n2r, num_partitions + ) + + self.logger.info(f"Rack counts: {rc}") + + count_a = rc.get("A", 0) + count_b = rc.get("B", 0) + count_c = rc.get("C", 0) + count_d = rc.get("D", 0) + + # Rack A is highest priority; with rack awareness, each partition + # gets one replica per rack, so A should get ~1 per partition. + assert count_a >= num_partitions * 0.95, ( + f"Rack A should have >= {num_partitions * 0.95} replicas, got {count_a}" + ) + + # B, C, D are in the same group -- roughly equal share of non-A. + non_a = count_b + count_c + count_d + expected_per_rack = non_a / 3.0 + tolerance = 0.4 # 40% tolerance for statistical variation + + for rack, count in [("B", count_b), ("C", count_c), ("D", count_d)]: + low = expected_per_rack * (1 - tolerance) + high = expected_per_rack * (1 + tolerance) + assert low <= count <= high, ( + f"Rack {rack} count {count} outside expected range " + f"[{low:.0f}, {high:.0f}] (expected ~{expected_per_rack:.0f})" + ) + + # ------------------------------------------------------------------ + # Test 5: Fallback to unpreferred racks + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_fallback_to_unpreferred(self): + """ + When the preference list has fewer racks than RF, the allocator + falls back to unpreferred racks while still respecting rack awareness. + + 4 nodes, 4 racks. rack_awareness=ON. RF=3, preference="racks: A" + """ + self._start_cluster(["A", "B", "C", "D"]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-fallback", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-fallback", n2r) + + self.logger.info(f"Rack counts: {rc}") + assert rc.get("A", 0) == 1, ( + f"Expected 1 replica in rack A, got {rc.get('A', 0)}" + ) + fallback_count = sum(rc.get(r, 0) for r in ["B", "C", "D"]) + assert fallback_count == 2, ( + f"Expected 2 replicas in fallback racks, got {fallback_count}" + ) + for r in ["B", "C", "D"]: + assert rc.get(r, 0) <= 1, ( + f"Rack {r} has {rc.get(r, 0)} replicas, expected at most 1" + ) + + # ------------------------------------------------------------------ + # Test 6: Balancer repairs pinning violation + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_balancer_repairs_pinning_violation(self): + """ + After all nodes in the highest-priority rack go down and come back, + the balancer moves replicas back to the preferred rack. + + 5 nodes, 4 racks. rack_awareness=ON. RF=3, preference="racks: A, B, C" + Nodes: 1,2 in rack A; 3 in rack B; 4 in rack C; 5 in rack D + """ + self._start_cluster(["A", "A", "B", "C", "D"]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-repair", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B, C"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-repair", n2r) + self.logger.info(f"Initial rack counts: {rc}") + + assert rc.get("A", 0) == 1 + assert rc.get("B", 0) == 1 + assert rc.get("C", 0) == 1 + assert rc.get("D", 0) == 0 + + # Kill BOTH nodes in rack A + node_a0 = self.redpanda.nodes[0] + node_a1 = self.redpanda.nodes[1] + self.redpanda.stop_node(node_a0) + self.redpanda.stop_node(node_a1) + + # Wait for balancer to move A's replica to D + def replicas_moved_to_d(): + try: + rc = self._get_partition_rack_counts("pinning-repair", n2r) + self.logger.info(f"After kill rack counts: {rc}") + return rc.get("A", 0) == 0 and rc.get("D", 0) == 1 + except Exception: + return False + + wait_until( + replicas_moved_to_d, + timeout_sec=120, + backoff_sec=5, + err_msg="Balancer did not move replica from A to D", + ) + + # Restore both A nodes + self.redpanda.start_node(node_a0) + self.redpanda.start_node(node_a1) + + # Wait for balancer to repair: move replica from D back to A + def replicas_back_in_abc(): + try: + rc = self._get_partition_rack_counts("pinning-repair", n2r) + self.logger.info(f"After restore rack counts: {rc}") + return ( + rc.get("A", 0) == 1 + and rc.get("B", 0) == 1 + and rc.get("C", 0) == 1 + and rc.get("D", 0) == 0 + ) + except Exception: + return False + + wait_until( + replicas_back_in_abc, + timeout_sec=180, + backoff_sec=5, + err_msg="Balancer did not repair pinning violation (D->A)", + ) + + # ------------------------------------------------------------------ + # Test 7: Alter replicas_preference + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_alter_replicas_preference(self): + """ + Altering a topic's replicas_preference triggers the balancer to + rebalance existing replicas. Removing preference stops enforcement. + + 5 nodes, 4 racks. rack_awareness=ON. RF=3 + Nodes: 1 in A; 2 in B; 3 in C; 4 in D; 5 in D + """ + self._start_cluster(["A", "B", "C", "D", "D"]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic("pinning-alter", partitions=1, replicas=3) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-alter", n2r) + self.logger.info(f"Initial rack counts (no pref): {rc}") + + # Set replicas_preference to A, B, C + rpk.alter_topic_config( + "pinning-alter", REPLICAS_PREFERENCE_KEY, "racks: A, B, C" + ) + + # Wait for balancer to move replicas to A, B, C (and away from D) + def replicas_in_abc(): + try: + rc = self._get_partition_rack_counts("pinning-alter", n2r) + self.logger.info(f"After alter rack counts: {rc}") + return ( + rc.get("A", 0) == 1 + and rc.get("B", 0) == 1 + and rc.get("C", 0) == 1 + and rc.get("D", 0) == 0 + ) + except Exception: + return False + + wait_until( + replicas_in_abc, + timeout_sec=120, + backoff_sec=5, + err_msg="Balancer did not move replicas to A,B,C after alter", + ) + + rc_before = self._get_partition_rack_counts("pinning-alter", n2r) + + # Remove preference + rpk.delete_topic_config("pinning-alter", REPLICAS_PREFERENCE_KEY) + + # Wait for several balancer tick intervals (5s each) to allow any + # rebalancing to trigger if it were going to. + time.sleep(15) + + # Verify no reconfigurations were started + admin = Admin(self.redpanda) + reconfigs = admin.list_reconfigurations() + assert len(reconfigs) == 0, ( + f"Expected no reconfigurations after removing preference, " + f"got {len(reconfigs)}" + ) + + rc_after = self._get_partition_rack_counts("pinning-alter", n2r) + self.logger.info(f"After remove pref: before={rc_before}, after={rc_after}") + assert rc_before == rc_after, ( + "Removing replicas_preference should not trigger rebalancing" + ) + + # ------------------------------------------------------------------ + # Test 8: Pinning with rackless nodes + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_pinning_with_rackless_nodes(self): + """ + Nodes without a rack assignment are deprioritized (score 0). + When preferred racks have capacity, rackless nodes get no replicas. + + 4 nodes, 2 racks + 1 rackless. rack_awareness=ON. RF=3, + preference="racks: A, B" + Nodes: 1 in rack A; 2 in rack B; 3 in rack B; 4 has NO rack + """ + self._start_cluster(["A", "B", "B", None]) + + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "pinning-rackless", + partitions=1, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B"}, + ) + + n2r = self._node_id_to_rack() + rc = self._get_partition_rack_counts("pinning-rackless", n2r) + + self.logger.info(f"Rack counts: {rc}") + assert rc.get("A", 0) == 1, ( + f"Expected 1 replica in rack A, got {rc.get('A', 0)}" + ) + assert rc.get("B", 0) == 2, ( + f"Expected 2 replicas in rack B, got {rc.get('B', 0)}" + ) + assert rc.get(None, 0) == 0, ( + f"Expected 0 replicas on rackless node, got {rc.get(None, 0)}" + ) + + # ------------------------------------------------------------------ + # Test 9a: Spread within rack -- rack awareness ON + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_spread_within_rack_awareness_on(self): + """ + With rack awareness ON, each partition's RF=3 replicas land in + 3 distinct racks. Racks A and B are preferred (group 0), so they + should each receive ~60 replicas. Rack C is overflow (group 1) + and gets at most the remainder. Within A and B, replicas spread + across the 3 nodes per rack (no node > 2x average). + + 8 nodes: 3xA, 3xB, 2xC. 60 partitions, RF=3, pref="racks: A, B" + Total replicas = 180. + + With rack awareness ON, each partition gets one replica per rack. + Racks A and B are group 0 (preferred), rack C is group 1. + With 3 racks available and RF=3, ideal per partition = {A, B, C}. + So racks A and B each get ~60 replicas (one per partition), and + rack C gets ~60 as overflow. + + Due to internal topic partitions, actual counts may deviate slightly. + We allow JITTER_PERCENT deviation from the expected counts. + + Expected: A ~60, B ~60, C ~60 (bounded by jitter). + """ + self._start_cluster( + ["A", "A", "A", "B", "B", "B", "C", "C"], rack_awareness=True + ) + + num_partitions = 60 + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "spread-rack-on", + partitions=num_partitions, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B"}, + ) + + self._wait_no_reconfigurations() + + n2r = self._node_id_to_rack() + rack_counts = self._get_multi_partition_rack_counts( + "spread-rack-on", n2r, num_partitions + ) + node_counts = self._get_multi_partition_node_counts( + "spread-rack-on", num_partitions + ) + + self.logger.info(f"Rack counts: {rack_counts}") + self.logger.info(f"Node counts: {node_counts}") + + count_a = rack_counts.get("A", 0) + count_b = rack_counts.get("B", 0) + count_c = rack_counts.get("C", 0) + + # Ideal: each partition gets one replica per rack -> 60 per rack. + expected_per_rack = num_partitions # 60 + jitter = int(expected_per_rack * JITTER_PERCENT) + + assert count_a >= expected_per_rack - jitter, ( + f"Expected rack A >= {expected_per_rack - jitter} replicas, got {count_a}" + ) + assert count_b >= expected_per_rack - jitter, ( + f"Expected rack B >= {expected_per_rack - jitter} replicas, got {count_b}" + ) + assert count_c <= expected_per_rack + jitter, ( + f"Expected rack C <= {expected_per_rack + jitter} replicas, got {count_c}" + ) + + # Within A and B, no node has > 2x the average + for rack in ["A", "B"]: + rack_node_ids = [nid for nid, r in n2r.items() if r == rack] + if not rack_node_ids: + continue + counts = [node_counts.get(nid, 0) for nid in rack_node_ids] + avg = sum(counts) / len(counts) + if avg == 0: + continue + for nid, count in zip(rack_node_ids, counts): + assert count <= avg * 2.0, ( + f"Node {nid} in rack {rack} has {count} replicas, " + f"more than 2.0x the average ({avg:.1f})" + ) + + # ------------------------------------------------------------------ + # Test 9b: Spread within rack -- rack awareness OFF + # ------------------------------------------------------------------ + @cluster(num_nodes=8) + def test_spread_within_rack_awareness_off(self): + """ + With rack awareness OFF, the allocator ignores rack diversity. + Rack A has 4 nodes (capacity 4 in group 0), so nearly all 180 + replicas should land in rack A. Rack B (group 1) and C (group 2) + are overflow only. + + 7 nodes: 4xA, 2xB, 1xC. 60 partitions, RF=3, pref="racks: A, B" + Total replicas = 180. + + With rack awareness OFF, capacity of group 0 (rack A) = 4 nodes. + RF=3 fits entirely in group 0, so ideal = {A, A, A}. + Nearly all 180 replicas should land in rack A. B and C get only + residual replicas from internal topics. + + Expected: A ~180, B ~0, C ~0 (with jitter from internal topics). + """ + self._start_cluster(["A", "A", "A", "A", "B", "B", "C"], rack_awareness=False) + + num_partitions = 60 + rpk = RpkTool(self.redpanda) + rpk.create_topic( + "spread-rack-off", + partitions=num_partitions, + replicas=3, + config={REPLICAS_PREFERENCE_KEY: "racks: A, B"}, + ) + + self._wait_no_reconfigurations() + + n2r = self._node_id_to_rack() + rack_counts = self._get_multi_partition_rack_counts( + "spread-rack-off", n2r, num_partitions + ) + node_counts = self._get_multi_partition_node_counts( + "spread-rack-off", num_partitions + ) + + self.logger.info(f"Rack counts: {rack_counts}") + self.logger.info(f"Node counts: {node_counts}") + + total_replicas = num_partitions * 3 # 180 + count_a = rack_counts.get("A", 0) + count_b = rack_counts.get("B", 0) + count_c = rack_counts.get("C", 0) + + # Ideal: all 180 replicas in A. Internal topics add jitter. + jitter = int(total_replicas * JITTER_PERCENT) + + assert count_a >= total_replicas - jitter, ( + f"Expected rack A >= {total_replicas - jitter} of " + f"{total_replicas} replicas, got {count_a}" + ) + assert count_b <= jitter, f"Expected rack B <= {jitter} replicas, got {count_b}" + assert count_c <= jitter, f"Expected rack C <= {jitter} replicas, got {count_c}" + + # Within rack A, no node has > 2x the average + rack_a_node_ids = [nid for nid, r in n2r.items() if r == "A"] + counts_a = [node_counts.get(nid, 0) for nid in rack_a_node_ids] + avg_a = sum(counts_a) / len(counts_a) if counts_a else 0 + if avg_a > 0: + for nid, count in zip(rack_a_node_ids, counts_a): + assert count <= avg_a * 2.0, ( + f"Node {nid} in rack A has {count} replicas, " + f"more than 2.0x the average ({avg_a:.1f})" + ) + + # Every rack-A node should have meaningful load. + avg_per_node = total_replicas / len(rack_a_node_ids) + min_per_node = int(avg_per_node * (1.0 - JITTER_PERCENT)) + for nid, count in zip(rack_a_node_ids, counts_a): + assert count >= min_per_node, ( + f"Node {nid} in rack A has only {count} replicas, " + f"expected >= {min_per_node}" + )