Skip to content

Commit

Permalink
replication_monitor: fix missed notifications during race
Browse files Browse the repository at this point in the history
If the replication notification happens before wait_for_majority() is
called, the waiter is never resolved. Adds an additional check before
creating the waiter instance.
  • Loading branch information
bharathv committed Apr 16, 2024
1 parent b0ffde9 commit 53b82ba
Show file tree
Hide file tree
Showing 2 changed files with 25 additions and 0 deletions.
7 changes: 7 additions & 0 deletions src/v/raft/replication_monitor.cc
Original file line number Diff line number Diff line change
Expand Up @@ -101,11 +101,18 @@ ss::future<errc> replication_monitor::do_wait_until(

ss::future<errc>
replication_monitor::wait_until_committed(storage::append_result append) {
auto done = is_append_committed_or_truncated(append);
if (done) {
return ssx::now(done.value());
}
return do_wait_until(append, wait_type::commit);
}

ss::future<errc> replication_monitor::wait_until_majority_replicated(
storage::append_result append) {
if (is_append_replicated(append)) {
return ssx::now(errc::success);
}
return do_wait_until(append, wait_type::majority_replication);
}

Expand Down
18 changes: 18 additions & 0 deletions src/v/raft/tests/basic_raft_fixture_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,24 @@ TEST_P_CORO(all_acks_fixture, validate_replication) {
co_await assert_logs_equal();
}

TEST_P_CORO(all_acks_fixture, single_node_replication) {
co_await create_simple_group(1);

auto params = GetParam();
co_await set_write_caching(params.write_caching);

auto leader = co_await wait_for_leader(10s);
auto& leader_node = node(leader);

auto result = co_await leader_node.raft()->replicate(
make_batches({{"k_1", "v_1"}}), replicate_options(params.c_lvl));
ASSERT_TRUE_CORO(result.has_value());

// wait for committed offset to propagate
co_await wait_for_committed_offset(result.value().last_offset, 5s);
co_await assert_logs_equal();
}

TEST_P_CORO(all_acks_fixture, validate_recovery) {
co_await create_simple_group(5);
auto leader = co_await wait_for_leader(10s);
Expand Down

0 comments on commit 53b82ba

Please sign in to comment.