Skip to content

Commit

Permalink
rm_stm: replace most std::vector instances with fragmented_vector
Browse files Browse the repository at this point in the history
  • Loading branch information
bharathv committed Mar 17, 2023
1 parent 1a5dde7 commit 89e2b17
Show file tree
Hide file tree
Showing 12 changed files with 103 additions and 98 deletions.
2 changes: 1 addition & 1 deletion src/v/archival/ntp_archiver_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -907,7 +907,7 @@ ss::future<cloud_storage::upload_result> ntp_archiver::upload_tx(

auto path = segment_path_for_candidate(candidate);

cloud_storage::tx_range_manifest manifest(path, tx_range);
cloud_storage::tx_range_manifest manifest(path, std::move(tx_range));

co_return co_await _remote.upload_manifest(
get_bucket_name(), manifest, fib, _tx_tags);
Expand Down
4 changes: 3 additions & 1 deletion src/v/cloud_storage/tests/tx_range_manifest_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,9 @@ SEASTAR_THREAD_TEST_CASE(empty_serialization_roundtrip_test) {
}

SEASTAR_THREAD_TEST_CASE(serialization_roundtrip_test) {
tx_range_manifest m(segment_path, ranges);
fragmented_vector<tx_range_t> tx_ranges;
tx_ranges = ranges;
tx_range_manifest m(segment_path, std::move(tx_ranges));
auto [is, size] = m.serialize().get();
iobuf buf;
auto os = make_iobuf_ref_output_stream(buf);
Expand Down
10 changes: 3 additions & 7 deletions src/v/cloud_storage/tx_range_manifest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,9 @@ remote_manifest_path generate_remote_tx_path(const remote_segment_path& path) {
}

tx_range_manifest::tx_range_manifest(
remote_segment_path spath, const std::vector<model::tx_range>& range)
: _path(std::move(spath)) {
for (const auto& tx : range) {
_ranges.push_back(tx);
}
_ranges.shrink_to_fit();
}
remote_segment_path spath, fragmented_vector<model::tx_range>&& range)
: _path(std::move(spath))
, _ranges(std::move(range)) {}

tx_range_manifest::tx_range_manifest(remote_segment_path spath)
: _path(std::move(spath)) {}
Expand Down
2 changes: 1 addition & 1 deletion src/v/cloud_storage/tx_range_manifest.h
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ class tx_range_manifest final : public base_manifest {
public:
/// Create manifest for specific ntp
explicit tx_range_manifest(
remote_segment_path spath, const std::vector<model::tx_range>& range);
remote_segment_path spath, fragmented_vector<model::tx_range>&& range);

/// Create empty manifest that supposed to be updated later
explicit tx_range_manifest(remote_segment_path spath);
Expand Down
7 changes: 4 additions & 3 deletions src/v/cluster/partition.h
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
#include "raft/types.h"
#include "storage/translating_reader.h"
#include "storage/types.h"
#include "utils/fragmented_vector.h"

#include <seastar/core/shared_ptr.hh>

Expand Down Expand Up @@ -211,11 +212,11 @@ class partition {

ss::shared_ptr<cluster::tm_stm> tm_stm() { return _tm_stm; }

ss::future<std::vector<rm_stm::tx_range>>
ss::future<fragmented_vector<rm_stm::tx_range>>
aborted_transactions(model::offset from, model::offset to) {
if (!_rm_stm) {
return ss::make_ready_future<std::vector<rm_stm::tx_range>>(
std::vector<rm_stm::tx_range>());
return ss::make_ready_future<fragmented_vector<rm_stm::tx_range>>(
fragmented_vector<rm_stm::tx_range>());
}
return _rm_stm->aborted_transactions(from, to);
}
Expand Down
4 changes: 2 additions & 2 deletions src/v/cluster/persisted_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -185,9 +185,9 @@ model::offset persisted_stm::max_collectible_offset() {
return model::offset::max();
}

ss::future<std::vector<model::tx_range>>
ss::future<fragmented_vector<model::tx_range>>
persisted_stm::aborted_tx_ranges(model::offset, model::offset) {
return ss::make_ready_future<std::vector<model::tx_range>>();
return ss::make_ready_future<fragmented_vector<model::tx_range>>();
}

ss::future<> persisted_stm::wait_offset_committed(
Expand Down
3 changes: 2 additions & 1 deletion src/v/cluster/persisted_stm.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include "storage/snapshot.h"
#include "storage/types.h"
#include "utils/expiring_promise.h"
#include "utils/fragmented_vector.h"
#include "utils/mutex.h"

#include <absl/container/flat_hash_map.h>
Expand Down Expand Up @@ -88,7 +89,7 @@ class persisted_stm
void make_snapshot_in_background() final;
ss::future<> ensure_snapshot_exists(model::offset) final;
model::offset max_collectible_offset() override;
ss::future<std::vector<model::tx_range>>
ss::future<fragmented_vector<model::tx_range>>
aborted_tx_ranges(model::offset, model::offset) override;
const ss::sstring& name() override { return _snapshot_mgr.name(); }

Expand Down
Loading

0 comments on commit 89e2b17

Please sign in to comment.