Skip to content

Commit

Permalink
Merge pull request #24216 from vbotbuildovich/backport-pr-24210-v24.2…
Browse files Browse the repository at this point in the history
….x-85

[v24.2.x] [CORE-8010] cluster: Fix race condition in the `archival_metadata_stm`
  • Loading branch information
piyushredpanda authored Nov 21, 2024
2 parents 27cae87 + e716ff1 commit 29b8a8e
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 9 deletions.
7 changes: 6 additions & 1 deletion src/v/archival/archival_metadata_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -460,6 +460,7 @@ ss::future<std::error_code> command_batch_builder::replicate() {
_as.check();

auto units = co_await _stm.get()._lock.get_units(_as);
auto holder = _stm.get()._gate.hold();

vlog(_stm.get()._logger.debug, "command_batch_builder::replicate called");
auto now = ss::lowres_clock::now();
Expand All @@ -474,7 +475,7 @@ ss::future<std::error_code> command_batch_builder::replicate() {
auto batch = std::move(_builder).build();
auto f = _stm.get()
.do_replicate_commands(std::move(batch), _as)
.finally([u = std::move(units), h = _stm.get()._gate.hold()] {});
.finally([u = std::move(units), h = std::move(holder)] {});

// The above do_replicate_commands call is not cancellable at every point
// due to the guarantees we need from the operation for linearizability. To
Expand Down Expand Up @@ -780,6 +781,7 @@ ss::future<std::optional<model::offset>> archival_metadata_stm::sync(

ss::future<bool> archival_metadata_stm::do_sync(
model::timeout_clock::duration timeout, ss::abort_source* as) {
auto holder = _gate.hold();
if (!co_await raft::persisted_stm<>::sync(timeout)) {
co_return false;
}
Expand Down Expand Up @@ -827,6 +829,8 @@ ss::future<std::error_code> archival_metadata_stm::do_replicate_commands(
// early allowing for concurrent sync and replicate calls which will lead
// to race conditions/corruption/undefined behavior.

auto holder = _gate.hold();

vassert(
!_lock.try_get_units().has_value(),
"Attempt to replicate STM command while not under lock");
Expand Down Expand Up @@ -948,6 +952,7 @@ ss::future<std::error_code> archival_metadata_stm::do_add_segments(
ss::lowres_clock::time_point deadline,
ss::abort_source& as,
segment_validated is_validated) {
auto holder = _gate.hold();
{
auto now = ss::lowres_clock::now();
auto timeout = now < deadline ? deadline - now : 0ms;
Expand Down
12 changes: 4 additions & 8 deletions src/v/archival/tests/archival_metadata_stm_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@
#include "cloud_storage/remote.h"
#include "cloud_storage/types.h"
#include "cluster/errc.h"
#include "features/feature_table.h"
#include "http/tests/http_imposter.h"
#include "model/fundamental.h"
#include "model/metadata.h"
Expand Down Expand Up @@ -91,7 +90,6 @@ struct archival_metadata_stm_base_fixture
archival_metadata_stm_base_fixture()
: http_imposter_fixture(4446) {
// Blank feature table to satisfy constructor interface
feature_table.start().get();
// Cloud storage config
cloud_cfg.start().get();
cloud_cfg
Expand Down Expand Up @@ -125,10 +123,8 @@ struct archival_metadata_stm_base_fixture
cloud_api.stop().get();
cloud_conn_pool.stop().get();
cloud_cfg.stop().get();
feature_table.stop().get();
}

ss::sharded<features::feature_table> feature_table;
ss::sharded<cloud_storage::configuration> cloud_cfg;
ss::sharded<cloud_storage_clients::client_pool> cloud_conn_pool;
ss::sharded<cloud_storage::remote> cloud_api;
Expand All @@ -142,7 +138,7 @@ struct archival_metadata_stm_fixture : archival_metadata_stm_base_fixture {
archival_stm = builder.create_stm<cluster::archival_metadata_stm>(
_raft.get(),
cloud_api.local(),
feature_table.local(),
_feature_table.local(),
logger,
std::nullopt,
std::nullopt);
Expand Down Expand Up @@ -358,7 +354,7 @@ FIXTURE_TEST(test_snapshot_loading, archival_metadata_stm_base_fixture) {
auto archival_stm = builder.create_stm<cluster::archival_metadata_stm>(
_raft.get(),
cloud_api.local(),
feature_table.local(),
_feature_table.local(),
logger,
std::nullopt,
std::nullopt);
Expand Down Expand Up @@ -458,7 +454,7 @@ FIXTURE_TEST(test_sname_derivation, archival_metadata_stm_base_fixture) {
auto archival_stm = builder.create_stm<cluster::archival_metadata_stm>(
_raft.get(),
cloud_api.local(),
feature_table.local(),
_feature_table.local(),
logger,
std::nullopt,
std::nullopt);
Expand Down Expand Up @@ -672,7 +668,7 @@ FIXTURE_TEST(
auto archival_stm = builder.create_stm<cluster::archival_metadata_stm>(
_raft.get(),
cloud_api.local(),
feature_table.local(),
_feature_table.local(),
logger,
std::nullopt,
std::nullopt);
Expand Down

0 comments on commit 29b8a8e

Please sign in to comment.