Skip to content

Commit

Permalink
c/controller_backend: use std::list to store deltas
Browse files Browse the repository at this point in the history
Deltas were previously stored as a vector per `ntp`. Deltas access
pattern (iteration, inserting and popping elements from back and from
the end) makes it perfect candidate for `std::list` usage. The
`std::list` doesn't use contiguous allocation so will not account for
the memory fragmentation.

Signed-off-by: Michal Maslanka <michal@redpanda.com>
  • Loading branch information
mmaslankaprv committed Jun 27, 2023
1 parent 6a566a9 commit 8e3d48e
Show file tree
Hide file tree
Showing 3 changed files with 20 additions and 12 deletions.
8 changes: 6 additions & 2 deletions src/v/cluster/controller_backend.cc
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
#include <absl/container/flat_hash_map.h>
#include <absl/container/flat_hash_set.h>
#include <absl/container/node_hash_map.h>
#include <fmt/ranges.h>

#include <algorithm>
#include <exception>
Expand Down Expand Up @@ -589,7 +590,6 @@ controller_backend::deltas_t calculate_bootstrap_deltas(

// *it is not included
auto start = it.base();
result_delta.reserve(std::distance(start, deltas.end()));
std::move(start, deltas.end(), std::back_inserter(result_delta));
return result_delta;
}
Expand Down Expand Up @@ -2023,7 +2023,11 @@ ss::future<> controller_backend::delete_partition(
std::vector<controller_backend::delta_metadata>
controller_backend::list_ntp_deltas(const model::ntp& ntp) const {
if (auto it = _topic_deltas.find(ntp); it != _topic_deltas.end()) {
return it->second;
std::vector<controller_backend::delta_metadata> ret;
ret.reserve(it->second.size());
std::copy(
it->second.begin(), it->second.end(), std::back_inserter(ret));
return ret;
}

return {};
Expand Down
6 changes: 3 additions & 3 deletions src/v/cluster/controller_backend.h
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ class controller_backend
friend std::ostream& operator<<(std::ostream&, const delta_metadata&);
};

using deltas_t = std::vector<delta_metadata>;
using deltas_t = std::list<delta_metadata>;
using results_t = std::vector<std::error_code>;
controller_backend(
ss::sharded<cluster::topic_table>&,
Expand Down Expand Up @@ -437,6 +437,6 @@ class controller_backend
ss::metrics::metric_groups _metrics;
};

std::vector<controller_backend::delta_metadata> calculate_bootstrap_deltas(
model::node_id self, const std::vector<controller_backend::delta_metadata>&);
std::list<controller_backend::delta_metadata> calculate_bootstrap_deltas(
model::node_id self, const std::list<controller_backend::delta_metadata>&);
} // namespace cluster
18 changes: 11 additions & 7 deletions src/v/cluster/tests/controller_backend_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ make_assignment(std::vector<model::broker_shard> replicas) {
using op_t = cluster::topic_table::delta::op_type;
using delta_t = cluster::topic_table::delta;
using meta_t = cluster::controller_backend::delta_metadata;
using deltas_t = std::vector<meta_t>;
using deltas_t = std::list<meta_t>;

meta_t make_delta(
int64_t o,
Expand Down Expand Up @@ -142,11 +142,14 @@ SEASTAR_THREAD_TEST_CASE(update_including_current_node) {
current_node, std::move(d_1));

BOOST_REQUIRE_EQUAL(deltas.size(), 3);
BOOST_REQUIRE_EQUAL(deltas[0].delta.offset, recreate_current.delta.offset);
BOOST_REQUIRE_EQUAL(
deltas[1].delta.offset, update_with_current.delta.offset);
deltas.begin()->delta.offset, recreate_current.delta.offset);
BOOST_REQUIRE_EQUAL(
deltas[2].delta.offset, finish_update_with_current.delta.offset);
std::next(deltas.begin())->delta.offset,
update_with_current.delta.offset);
BOOST_REQUIRE_EQUAL(
std::next(deltas.begin(), 2)->delta.offset,
finish_update_with_current.delta.offset);
}

SEASTAR_THREAD_TEST_CASE(update_excluding_current_node) {
Expand Down Expand Up @@ -196,9 +199,10 @@ SEASTAR_THREAD_TEST_CASE(move_back_to_current_node) {

BOOST_REQUIRE_EQUAL(deltas.size(), 2);
BOOST_REQUIRE_EQUAL(
deltas[0].delta.offset, update_with_current_2.delta.offset);
deltas.begin()->delta.offset, update_with_current_2.delta.offset);
BOOST_REQUIRE_EQUAL(
deltas[1].delta.offset, finish_update_with_current_2.delta.offset);
std::next(deltas.begin())->delta.offset,
finish_update_with_current_2.delta.offset);
}

SEASTAR_THREAD_TEST_CASE(move_back_to_current_node_not_finished) {
Expand All @@ -218,5 +222,5 @@ SEASTAR_THREAD_TEST_CASE(move_back_to_current_node_not_finished) {

BOOST_REQUIRE_EQUAL(deltas.size(), 1);
BOOST_REQUIRE_EQUAL(
deltas[0].delta.offset, update_with_current_2.delta.offset);
deltas.begin()->delta.offset, update_with_current_2.delta.offset);
}

0 comments on commit 8e3d48e

Please sign in to comment.