Skip to content

Commit

Permalink
c/health_monitor: make node_health_report copies explicit
Browse files Browse the repository at this point in the history
  • Loading branch information
ztlpn committed Aug 5, 2024
1 parent 2f8f7f5 commit e0d9a05
Show file tree
Hide file tree
Showing 7 changed files with 28 additions and 45 deletions.
25 changes: 2 additions & 23 deletions src/v/cluster/health_monitor_types.cc
Original file line number Diff line number Diff line change
Expand Up @@ -84,29 +84,8 @@ node_health_report::node_health_report(
, topics(std::move(topics))
, drain_status(drain_status) {}

node_health_report::node_health_report(const node_health_report& other)
: id(other.id)
, local_state(other.local_state)
, topics()
, drain_status(other.drain_status) {
std::copy(
other.topics.cbegin(), other.topics.cend(), std::back_inserter(topics));
}

node_health_report&
node_health_report::operator=(const node_health_report& other) {
if (this == &other) {
return *this;
}
id = other.id;
local_state = other.local_state;
drain_status = other.drain_status;
chunked_vector<topic_status> t;
t.reserve(other.topics.size());
std::copy(
other.topics.cbegin(), other.topics.cend(), std::back_inserter(t));
topics = std::move(t);
return *this;
node_health_report node_health_report::copy() const {
return {id, local_state, topics.copy(), drain_status};
}

std::ostream& operator<<(std::ostream& o, const node_health_report& r) {
Expand Down
18 changes: 10 additions & 8 deletions src/v/cluster/health_monitor_types.h
Original file line number Diff line number Diff line change
Expand Up @@ -170,12 +170,7 @@ struct node_health_report
chunked_vector<topic_status>,
std::optional<drain_manager::drain_status>);

node_health_report(const node_health_report&);
node_health_report& operator=(const node_health_report&);

node_health_report(node_health_report&&) = default;
node_health_report& operator=(node_health_report&&) = default;
~node_health_report() = default;
node_health_report copy() const;

model::node_id id;
node::local_state local_state;
Expand Down Expand Up @@ -229,7 +224,7 @@ struct cluster_health_report
write(out, node_states);
write(out, static_cast<serde::serde_size_t>(node_reports.size()));
for (auto& nr : node_reports) {
co_await write_async(out, *nr);
co_await write_async(out, nr->copy());
}
write(out, bytes_in_cloud_storage);
}
Expand Down Expand Up @@ -267,7 +262,7 @@ struct cluster_health_report
write(out, node_states);
write(out, static_cast<serde::serde_size_t>(node_reports.size()));
for (auto& nr : node_reports) {
write(out, *nr);
write(out, nr->copy());
}
write(out, bytes_in_cloud_storage);
}
Expand Down Expand Up @@ -441,6 +436,13 @@ struct get_node_health_reply
operator==(const get_node_health_reply&, const get_node_health_reply&)
= default;

get_node_health_reply copy() const {
return {
.error = error,
.report = report ? std::optional{report->copy()} : std::nullopt,
};
}

friend std::ostream&
operator<<(std::ostream&, const get_node_health_reply&);

Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -522,7 +522,7 @@ service::do_collect_node_health_report(get_node_health_request) {
}
co_return get_node_health_reply{
.error = errc::success,
.report = *res.value(),
.report = res.value()->copy(),
};
}

Expand Down
8 changes: 4 additions & 4 deletions src/v/cluster/tests/partition_balancer_planner_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -470,8 +470,8 @@ FIXTURE_TEST(test_move_part_of_replicas, partition_balancer_planner_fixture) {
auto hr = create_health_report(full_nodes);

populate_node_status_table().get();
auto nr_1 = *hr.node_reports[1];
auto nr_2 = *hr.node_reports[2];
auto nr_1 = hr.node_reports[1]->copy();
auto nr_2 = hr.node_reports[2]->copy();
// Set order of full nodes

nr_1.local_state.log_data_size.value().data_current_size += 1_MiB;
Expand Down Expand Up @@ -522,7 +522,7 @@ FIXTURE_TEST(
std::set<size_t> full_nodes = {0, 1};
auto hr = create_health_report(full_nodes);
populate_node_status_table().get();
auto nr_0 = *hr.node_reports[0];
auto nr_0 = hr.node_reports[0]->copy();

// Set order of full nodes
nr_0.local_state.log_data_size.value().data_current_size += 1_MiB;
Expand Down Expand Up @@ -709,7 +709,7 @@ FIXTURE_TEST(test_rack_awareness, partition_balancer_planner_fixture) {
auto hr = create_health_report();
// Make node_4 disk free size less to make partition allocator disk usage
// constraint prefer node_3 rather than node_4
auto nr_4 = *hr.node_reports[4];
auto nr_4 = hr.node_reports[4]->copy();
nr_4.local_state.log_data_size.value().data_current_size
= hr.node_reports[3]->local_state.log_data_size.value().data_current_size
- 10_MiB;
Expand Down
3 changes: 2 additions & 1 deletion src/v/cluster/tests/partition_balancer_simulator_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -725,7 +725,8 @@ class partition_balancer_sim_fixture {
}

return ss::make_foreign(
ss::make_lw_shared<const cluster::node_health_report>(report));
ss::make_lw_shared<const cluster::node_health_report>(
std::move(report)));
}
};

Expand Down
15 changes: 8 additions & 7 deletions src/v/cluster/tests/serialization_rt_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1604,7 +1604,7 @@ SEASTAR_THREAD_TEST_CASE(serde_reflection_roundtrip) {
// test ADL roundtrip.
data.local_state.cache_disk = std::nullopt;

roundtrip_test(data);
roundtrip_test(std::move(data));
}
{
chunked_vector<cluster::topic_status> topics;
Expand All @@ -1620,14 +1620,15 @@ SEASTAR_THREAD_TEST_CASE(serde_reflection_roundtrip) {
// Squash to ADL-understood disk state
report.local_state.cache_disk = report.local_state.data_disk;

cluster::get_node_health_reply data{
.report = report,
};
roundtrip_test(data);
roundtrip_test(cluster::get_node_health_reply{
.report = report.copy(),
});
// try serde with non-default error code. adl doesn't encode error so
// this is a serde only test.
data.error = cluster::errc::error_collecting_health_report;
roundtrip_test(data);
roundtrip_test(cluster::get_node_health_reply{
.error = cluster::errc::error_collecting_health_report,
.report = report.copy(),
});
}
{
std::vector<model::node_id> nodes;
Expand Down
2 changes: 1 addition & 1 deletion src/v/compat/get_node_health_compat.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ struct compat_check<cluster::get_node_health_reply> {

static std::vector<compat_binary>
to_binary(cluster::get_node_health_reply obj) {
return {compat_binary::serde(obj)};
return {compat_binary::serde(std::move(obj))};
}

static void
Expand Down

0 comments on commit e0d9a05

Please sign in to comment.