Skip to content

Commit

Permalink
stm_manager: tighten interaction between bg application and snapshots
Browse files Browse the repository at this point in the history
wait for background applicators to finish before applying raft
snapshots, else there is a danger of next_to_apply offset moving
backwards with interleaved bg_apply and snapshot fibers.
  • Loading branch information
bharathv committed Jul 4, 2024
1 parent 413d0c7 commit c8290e5
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 3 deletions.
10 changes: 8 additions & 2 deletions src/v/raft/state_machine_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,10 @@ ss::future<> state_machine_manager::apply_raft_snapshot() {
}

auto fut = co_await ss::coroutine::as_future(
do_apply_raft_snapshot(std::move(snapshot->metadata), snapshot->reader));
acquire_background_apply_mutexes().then([&, this](auto units) mutable {
return do_apply_raft_snapshot(
std::move(snapshot->metadata), snapshot->reader, std::move(units));
}));
co_await snapshot->reader.close();
if (fut.failed()) {
const auto e = fut.get_exception();
Expand All @@ -228,7 +231,9 @@ ss::future<> state_machine_manager::apply_raft_snapshot() {
}

ss::future<> state_machine_manager::do_apply_raft_snapshot(
snapshot_metadata metadata, storage::snapshot_reader& reader) {
snapshot_metadata metadata,
storage::snapshot_reader& reader,
std::vector<ssx::semaphore_units> background_apply_units) {
const auto snapshot_file_sz = co_await reader.get_snapshot_size();
const auto last_offset = metadata.last_included_index;

Expand Down Expand Up @@ -276,6 +281,7 @@ ss::future<> state_machine_manager::do_apply_raft_snapshot(
});
}
_next = model::next_offset(metadata.last_included_index);
background_apply_units.clear();
}

ss::future<> state_machine_manager::apply_snapshot_to_stm(
Expand Down
4 changes: 3 additions & 1 deletion src/v/raft/state_machine_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,9 @@ class state_machine_manager final {

ss::future<> apply_raft_snapshot();
ss::future<> do_apply_raft_snapshot(
raft::snapshot_metadata metadata, storage::snapshot_reader& reader);
raft::snapshot_metadata metadata,
storage::snapshot_reader& reader,
std::vector<ssx::semaphore_units> background_apply_units);
ss::future<> apply();
ss::future<> try_apply_in_foreground();

Expand Down

0 comments on commit c8290e5

Please sign in to comment.