Skip to content

Commit

Permalink
Merge pull request #446 from rystsov/conditional-replicate2
Browse files Browse the repository at this point in the history
r/consensus: adds conditional replicate
  • Loading branch information
rystsov authored Jan 22, 2021
2 parents bc1c80a + 5b76e38 commit 5d7bc14
Show file tree
Hide file tree
Showing 6 changed files with 179 additions and 38 deletions.
31 changes: 25 additions & 6 deletions src/v/raft/consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -411,18 +411,32 @@ void consensus::dispatch_recovery(follower_index_metadata& idx) {

ss::future<result<replicate_result>>
consensus::replicate(model::record_batch_reader&& rdr, replicate_options opts) {
return do_replicate({}, std::move(rdr), opts);
}

ss::future<result<replicate_result>> consensus::replicate(
model::term_id expected_term,
model::record_batch_reader&& rdr,
replicate_options opts) {
return do_replicate(expected_term, std::move(rdr), opts);
}

ss::future<result<replicate_result>> consensus::do_replicate(
std::optional<model::term_id> expected_term,
model::record_batch_reader&& rdr,
replicate_options opts) {
if (!is_leader() || unlikely(_transferring_leadership)) {
return seastar::make_ready_future<result<replicate_result>>(
errc::not_leader);
}

if (opts.consistency == consistency_level::quorum_ack) {
_probe.replicate_requests_ack_all();
return ss::with_gate(_bg, [this, rdr = std::move(rdr)]() mutable {
return _batcher.replicate(std::move(rdr)).finally([this] {
_probe.replicate_done();
});
});
return ss::with_gate(
_bg, [this, expected_term, rdr = std::move(rdr)]() mutable {
return _batcher.replicate(expected_term, std::move(rdr))
.finally([this] { _probe.replicate_done(); });
});
}

if (opts.consistency == consistency_level::leader_ack) {
Expand All @@ -433,12 +447,17 @@ consensus::replicate(model::record_batch_reader&& rdr, replicate_options opts) {
// For relaxed consistency, append data to leader disk without flush
// asynchronous replication is provided by Raft protocol recovery mechanism.
return _op_lock
.with([this, rdr = std::move(rdr)]() mutable {
.with([this, expected_term, rdr = std::move(rdr)]() mutable {
if (!is_leader()) {
return seastar::make_ready_future<result<replicate_result>>(
errc::not_leader);
}

if (expected_term.has_value() && expected_term.value() != _term) {
return seastar::make_ready_future<result<replicate_result>>(
errc::not_leader);
}

return disk_append(model::make_record_batch_reader<
details::term_assigning_reader>(
std::move(rdr), model::term_id(_term)))
Expand Down
34 changes: 32 additions & 2 deletions src/v/raft/consensus.h
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,34 @@ class consensus {
ss::future<result<replicate_result>>
replicate(model::record_batch_reader&&, replicate_options);

/**
* Replication happens only when expected_term matches the current _term
* otherwise consensus returns not_leader. This feature is needed to keep
* ingestion-time state machine in sync with the log. The conventional
* state machines running on top on the log are optimistic: to execute a
* command a user should add a command to a log (replicate) then continue
* reading the commands from the log and executing them one after another.
* When the commands are conditional the conventional approach is wasteful
* because we even when a condition resolves to false we still pay the
* replication costs. An alternative approach is to check the conditions
* before replication but in this case there is a risk of divergence between
* the log and the state (e.g. a leadership moves to an another broker, it
* adds messages then the leadership moves back). The expected_term
* prevents this situation. The expected use case is:
* 1. when a cached term matches consensus.term() call replicate using
* the cached term as expected_term
* 2. otherwise:
* a. abrt all incoming requests
* b. call consensus meta() to get the latest offset and a term
* c. wait until the state caches up with the latest offset
* d. cache the term
* e. continue with step #1
*/
ss::future<result<replicate_result>> replicate(
model::term_id expected_term,
model::record_batch_reader&&,
replicate_options);

ss::future<model::record_batch_reader> make_reader(
storage::log_reader_config,
std::optional<clock_type::time_point> = std::nullopt);
Expand Down Expand Up @@ -284,8 +312,10 @@ class consensus {
append_entries_reply
make_append_entries_reply(vnode, storage::append_result);

ss::future<result<replicate_result>>
do_replicate(model::record_batch_reader&&);
ss::future<result<replicate_result>> do_replicate(
std::optional<model::term_id>,
model::record_batch_reader&&,
replicate_options);

ss::future<storage::append_result>
disk_append(model::record_batch_reader&&);
Expand Down
70 changes: 44 additions & 26 deletions src/v/raft/replicate_batcher.cc
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,9 @@ replicate_batcher::replicate_batcher(consensus* ptr, size_t cache_size)
: _ptr(ptr)
, _max_batch_size_sem(cache_size) {}

ss::future<result<replicate_result>>
replicate_batcher::replicate(model::record_batch_reader&& r) {
return do_cache(std::move(r)).then([this](item_ptr i) {
ss::future<result<replicate_result>> replicate_batcher::replicate(
std::optional<model::term_id> expected_term, model::record_batch_reader&& r) {
return do_cache(expected_term, std::move(r)).then([this](item_ptr i) {
return _lock.with([this] { return flush(); }).then([i] {
return i->_promise.get_future();
});
Expand All @@ -51,13 +51,14 @@ ss::future<> replicate_batcher::stop() {
"finish replicating pending entries"));
}
_item_cache.clear();
_data_cache.clear();
});
}
ss::future<replicate_batcher::item_ptr>
replicate_batcher::do_cache(model::record_batch_reader&& r) {

ss::future<replicate_batcher::item_ptr> replicate_batcher::do_cache(
std::optional<model::term_id> expected_term, model::record_batch_reader&& r) {
return model::consume_reader_to_memory(std::move(r), model::no_timeout)
.then([this](ss::circular_buffer<model::record_batch> batches) {
.then([this,
expected_term](ss::circular_buffer<model::record_batch> batches) {
ss::circular_buffer<model::record_batch> data;
size_t bytes = std::accumulate(
batches.cbegin(),
Expand All @@ -66,27 +67,31 @@ replicate_batcher::do_cache(model::record_batch_reader&& r) {
[](size_t sum, const model::record_batch& b) {
return sum + b.size_bytes();
});
return do_cache_with_backpressure(std::move(batches), bytes);
return do_cache_with_backpressure(
expected_term, std::move(batches), bytes);
});
}

ss::future<replicate_batcher::item_ptr>
replicate_batcher::do_cache_with_backpressure(
ss::circular_buffer<model::record_batch> batches, size_t bytes) {
std::optional<model::term_id> expected_term,
ss::circular_buffer<model::record_batch> batches,
size_t bytes) {
return ss::get_units(_max_batch_size_sem, bytes)
.then([this, batches = std::move(batches)](
.then([this, expected_term, batches = std::move(batches)](

ss::semaphore_units<> u) mutable {
size_t record_count = 0;
auto i = ss::make_lw_shared<item>();
for (auto& b : batches) {
record_count += b.record_count();
if (b.header().ctx.owner_shard == ss::this_shard_id()) {
_data_cache.push_back(std::move(b));
i->data.push_back(std::move(b));
} else {
_data_cache.push_back(b.copy());
i->data.push_back(b.copy());
}
}
auto i = ss::make_lw_shared<item>();
i->expected_term = expected_term;
i->record_count = record_count;
i->units = std::move(u);
_item_cache.emplace_back(i);
Expand All @@ -95,20 +100,14 @@ replicate_batcher::do_cache_with_backpressure(
}

ss::future<> replicate_batcher::flush() {
auto notifications = std::exchange(_item_cache, {});
auto data = std::exchange(_data_cache, {});
if (notifications.empty()) {
auto item_cache = std::exchange(_item_cache, {});
if (item_cache.empty()) {
return ss::now();
}
return ss::with_gate(
_ptr->_bg,
[this,
data = std::move(data),
notifications = std::move(notifications)]() mutable {
_ptr->_bg, [this, item_cache = std::move(item_cache)]() mutable {
return _ptr->_op_lock.get_units().then(
[this,
data = std::move(data),
notifications = std::move(notifications)](
[this, item_cache = std::move(item_cache)](
ss::semaphore_units<> u) mutable {
// we have to check if we are the leader
// it is critical as term could have been updated already by
Expand All @@ -117,17 +116,35 @@ ss::future<> replicate_batcher::flush() {
// this problem caused truncation failure.

if (!_ptr->is_leader()) {
for (auto& n : notifications) {
for (auto& n : item_cache) {
n->_promise.set_value(errc::not_leader);
}
return ss::make_ready_future<>();
}

auto meta = _ptr->meta();
auto const term = model::term_id(meta.term);
for (auto& b : data) {
b.set_term(term);
ss::circular_buffer<model::record_batch> data;
std::vector<item_ptr> notifications;

for (auto& n : item_cache) {
if (
!n->expected_term.has_value()
|| n->expected_term.value() == term) {
for (auto& b : n->data) {
b.set_term(term);
data.push_back(std::move(b));
}
notifications.push_back(std::move(n));
} else {
n->_promise.set_value(errc::not_leader);
}
}

if (notifications.empty()) {
return ss::now();
}

auto seqs = _ptr->next_followers_request_seq();
append_entries_request req(
_ptr->_self,
Expand All @@ -141,6 +158,7 @@ ss::future<> replicate_batcher::flush() {
});
});
}

static void propagate_result(
result<replicate_result> r,
std::vector<replicate_batcher::item_ptr>& notifications) {
Expand Down
12 changes: 8 additions & 4 deletions src/v/raft/replicate_batcher.h
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@ class replicate_batcher {
ss::promise<result<replicate_result>> _promise;
replicate_result ret;
size_t record_count;
std::vector<model::record_batch> data;
std::optional<model::term_id> expected_term;
/**
* Item keeps semaphore units until replicate batcher is done with
* processing the request.
Expand All @@ -45,7 +47,7 @@ class replicate_batcher {
~replicate_batcher() noexcept = default;

ss::future<result<replicate_result>>
replicate(model::record_batch_reader&&);
replicate(std::optional<model::term_id>, model::record_batch_reader&&);

ss::future<> flush();
ss::future<> stop();
Expand All @@ -58,15 +60,17 @@ class replicate_batcher {
absl::flat_hash_map<vnode, follower_req_seq>);

private:
ss::future<item_ptr> do_cache(model::record_batch_reader&&);
ss::future<item_ptr>
do_cache(std::optional<model::term_id>, model::record_batch_reader&&);
ss::future<replicate_batcher::item_ptr> do_cache_with_backpressure(
ss::circular_buffer<model::record_batch>, size_t);
std::optional<model::term_id>,
ss::circular_buffer<model::record_batch>,
size_t);

consensus* _ptr;
ss::semaphore _max_batch_size_sem;

std::vector<item_ptr> _item_cache;
ss::circular_buffer<model::record_batch> _data_cache;
mutex _lock;
};

Expand Down
48 changes: 48 additions & 0 deletions src/v/raft/tests/append_entries_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,54 @@ FIXTURE_TEST(test_replicate_multiple_entries, raft_test_fixture) {
"State is consistent");
};

FIXTURE_TEST(test_replicate_with_expected_term_leader, raft_test_fixture) {
raft_group gr = raft_group(raft::group_id(0), 3);
gr.enable_all();
auto leader_id = wait_for_group_leader(gr);
auto leader_raft = gr.get_member(leader_id).consensus;
auto term = leader_raft->term();
bool success = replicate_random_batches(
term, gr, 5, raft::consistency_level::leader_ack)
.get0();
BOOST_REQUIRE(success);
};

FIXTURE_TEST(test_replicate_with_expected_term_quorum, raft_test_fixture) {
raft_group gr = raft_group(raft::group_id(0), 3);
gr.enable_all();
auto leader_id = wait_for_group_leader(gr);
auto leader_raft = gr.get_member(leader_id).consensus;
auto term = leader_raft->term();
bool success = replicate_random_batches(
term, gr, 5, raft::consistency_level::quorum_ack)
.get0();
BOOST_REQUIRE(success);
};

FIXTURE_TEST(test_replicate_violating_expected_term_leader, raft_test_fixture) {
raft_group gr = raft_group(raft::group_id(0), 3);
gr.enable_all();
auto leader_id = wait_for_group_leader(gr);
auto leader_raft = gr.get_member(leader_id).consensus;
auto term = leader_raft->term() + model::term_id(100);
bool success = replicate_random_batches(
term, gr, 5, raft::consistency_level::leader_ack)
.get0();
BOOST_REQUIRE(!success);
};

FIXTURE_TEST(test_replicate_violating_expected_term_quorum, raft_test_fixture) {
raft_group gr = raft_group(raft::group_id(0), 3);
gr.enable_all();
auto leader_id = wait_for_group_leader(gr);
auto leader_raft = gr.get_member(leader_id).consensus;
auto term = leader_raft->term() + model::term_id(100);
bool success = replicate_random_batches(
term, gr, 5, raft::consistency_level::quorum_ack)
.get0();
BOOST_REQUIRE(!success);
};

FIXTURE_TEST(test_single_node_recovery, raft_test_fixture) {
raft_group gr = raft_group(raft::group_id(0), 3);
gr.enable_all();
Expand Down
22 changes: 22 additions & 0 deletions src/v/raft/tests/raft_group_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -690,6 +690,28 @@ static ss::future<bool> replicate_random_batches(
});
}

static ss::future<bool> replicate_random_batches(
model::term_id expected_term,
raft_group& gr,
int count,
raft::consistency_level c_lvl = raft::consistency_level::quorum_ack,
model::timeout_clock::duration tout = 1s) {
return retry_with_leader(
gr, 5, tout, [count, expected_term, c_lvl](raft_node& leader_node) {
auto rdr = random_batches_reader(count);
raft::replicate_options opts(c_lvl);

return leader_node.consensus
->replicate(expected_term, std::move(rdr), opts)
.then([](result<raft::replicate_result> res) {
if (!res) {
return false;
}
return true;
});
});
}

/**
* Makes compactible batches, having one record per batch
*/
Expand Down

0 comments on commit 5d7bc14

Please sign in to comment.