Skip to content

Commit 5d7bc14

Browse files
authored
Merge pull request #446 from rystsov/conditional-replicate2
r/consensus: adds conditional replicate
2 parents bc1c80a + 5b76e38 commit 5d7bc14

File tree

6 files changed

+179
-38
lines changed

6 files changed

+179
-38
lines changed

src/v/raft/consensus.cc

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -411,18 +411,32 @@ void consensus::dispatch_recovery(follower_index_metadata& idx) {
411411

412412
ss::future<result<replicate_result>>
413413
consensus::replicate(model::record_batch_reader&& rdr, replicate_options opts) {
414+
return do_replicate({}, std::move(rdr), opts);
415+
}
416+
417+
ss::future<result<replicate_result>> consensus::replicate(
418+
model::term_id expected_term,
419+
model::record_batch_reader&& rdr,
420+
replicate_options opts) {
421+
return do_replicate(expected_term, std::move(rdr), opts);
422+
}
423+
424+
ss::future<result<replicate_result>> consensus::do_replicate(
425+
std::optional<model::term_id> expected_term,
426+
model::record_batch_reader&& rdr,
427+
replicate_options opts) {
414428
if (!is_leader() || unlikely(_transferring_leadership)) {
415429
return seastar::make_ready_future<result<replicate_result>>(
416430
errc::not_leader);
417431
}
418432

419433
if (opts.consistency == consistency_level::quorum_ack) {
420434
_probe.replicate_requests_ack_all();
421-
return ss::with_gate(_bg, [this, rdr = std::move(rdr)]() mutable {
422-
return _batcher.replicate(std::move(rdr)).finally([this] {
423-
_probe.replicate_done();
424-
});
425-
});
435+
return ss::with_gate(
436+
_bg, [this, expected_term, rdr = std::move(rdr)]() mutable {
437+
return _batcher.replicate(expected_term, std::move(rdr))
438+
.finally([this] { _probe.replicate_done(); });
439+
});
426440
}
427441

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

456+
if (expected_term.has_value() && expected_term.value() != _term) {
457+
return seastar::make_ready_future<result<replicate_result>>(
458+
errc::not_leader);
459+
}
460+
442461
return disk_append(model::make_record_batch_reader<
443462
details::term_assigning_reader>(
444463
std::move(rdr), model::term_id(_term)))

src/v/raft/consensus.h

Lines changed: 32 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -162,6 +162,34 @@ class consensus {
162162
ss::future<result<replicate_result>>
163163
replicate(model::record_batch_reader&&, replicate_options);
164164

165+
/**
166+
* Replication happens only when expected_term matches the current _term
167+
* otherwise consensus returns not_leader. This feature is needed to keep
168+
* ingestion-time state machine in sync with the log. The conventional
169+
* state machines running on top on the log are optimistic: to execute a
170+
* command a user should add a command to a log (replicate) then continue
171+
* reading the commands from the log and executing them one after another.
172+
* When the commands are conditional the conventional approach is wasteful
173+
* because we even when a condition resolves to false we still pay the
174+
* replication costs. An alternative approach is to check the conditions
175+
* before replication but in this case there is a risk of divergence between
176+
* the log and the state (e.g. a leadership moves to an another broker, it
177+
* adds messages then the leadership moves back). The expected_term
178+
* prevents this situation. The expected use case is:
179+
* 1. when a cached term matches consensus.term() call replicate using
180+
* the cached term as expected_term
181+
* 2. otherwise:
182+
* a. abrt all incoming requests
183+
* b. call consensus meta() to get the latest offset and a term
184+
* c. wait until the state caches up with the latest offset
185+
* d. cache the term
186+
* e. continue with step #1
187+
*/
188+
ss::future<result<replicate_result>> replicate(
189+
model::term_id expected_term,
190+
model::record_batch_reader&&,
191+
replicate_options);
192+
165193
ss::future<model::record_batch_reader> make_reader(
166194
storage::log_reader_config,
167195
std::optional<clock_type::time_point> = std::nullopt);
@@ -284,8 +312,10 @@ class consensus {
284312
append_entries_reply
285313
make_append_entries_reply(vnode, storage::append_result);
286314

287-
ss::future<result<replicate_result>>
288-
do_replicate(model::record_batch_reader&&);
315+
ss::future<result<replicate_result>> do_replicate(
316+
std::optional<model::term_id>,
317+
model::record_batch_reader&&,
318+
replicate_options);
289319

290320
ss::future<storage::append_result>
291321
disk_append(model::record_batch_reader&&);

src/v/raft/replicate_batcher.cc

Lines changed: 44 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,9 @@ replicate_batcher::replicate_batcher(consensus* ptr, size_t cache_size)
3232
: _ptr(ptr)
3333
, _max_batch_size_sem(cache_size) {}
3434

35-
ss::future<result<replicate_result>>
36-
replicate_batcher::replicate(model::record_batch_reader&& r) {
37-
return do_cache(std::move(r)).then([this](item_ptr i) {
35+
ss::future<result<replicate_result>> replicate_batcher::replicate(
36+
std::optional<model::term_id> expected_term, model::record_batch_reader&& r) {
37+
return do_cache(expected_term, std::move(r)).then([this](item_ptr i) {
3838
return _lock.with([this] { return flush(); }).then([i] {
3939
return i->_promise.get_future();
4040
});
@@ -51,13 +51,14 @@ ss::future<> replicate_batcher::stop() {
5151
"finish replicating pending entries"));
5252
}
5353
_item_cache.clear();
54-
_data_cache.clear();
5554
});
5655
}
57-
ss::future<replicate_batcher::item_ptr>
58-
replicate_batcher::do_cache(model::record_batch_reader&& r) {
56+
57+
ss::future<replicate_batcher::item_ptr> replicate_batcher::do_cache(
58+
std::optional<model::term_id> expected_term, model::record_batch_reader&& r) {
5959
return model::consume_reader_to_memory(std::move(r), model::no_timeout)
60-
.then([this](ss::circular_buffer<model::record_batch> batches) {
60+
.then([this,
61+
expected_term](ss::circular_buffer<model::record_batch> batches) {
6162
ss::circular_buffer<model::record_batch> data;
6263
size_t bytes = std::accumulate(
6364
batches.cbegin(),
@@ -66,27 +67,31 @@ replicate_batcher::do_cache(model::record_batch_reader&& r) {
6667
[](size_t sum, const model::record_batch& b) {
6768
return sum + b.size_bytes();
6869
});
69-
return do_cache_with_backpressure(std::move(batches), bytes);
70+
return do_cache_with_backpressure(
71+
expected_term, std::move(batches), bytes);
7072
});
7173
}
7274

7375
ss::future<replicate_batcher::item_ptr>
7476
replicate_batcher::do_cache_with_backpressure(
75-
ss::circular_buffer<model::record_batch> batches, size_t bytes) {
77+
std::optional<model::term_id> expected_term,
78+
ss::circular_buffer<model::record_batch> batches,
79+
size_t bytes) {
7680
return ss::get_units(_max_batch_size_sem, bytes)
77-
.then([this, batches = std::move(batches)](
81+
.then([this, expected_term, batches = std::move(batches)](
7882

7983
ss::semaphore_units<> u) mutable {
8084
size_t record_count = 0;
85+
auto i = ss::make_lw_shared<item>();
8186
for (auto& b : batches) {
8287
record_count += b.record_count();
8388
if (b.header().ctx.owner_shard == ss::this_shard_id()) {
84-
_data_cache.push_back(std::move(b));
89+
i->data.push_back(std::move(b));
8590
} else {
86-
_data_cache.push_back(b.copy());
91+
i->data.push_back(b.copy());
8792
}
8893
}
89-
auto i = ss::make_lw_shared<item>();
94+
i->expected_term = expected_term;
9095
i->record_count = record_count;
9196
i->units = std::move(u);
9297
_item_cache.emplace_back(i);
@@ -95,20 +100,14 @@ replicate_batcher::do_cache_with_backpressure(
95100
}
96101

97102
ss::future<> replicate_batcher::flush() {
98-
auto notifications = std::exchange(_item_cache, {});
99-
auto data = std::exchange(_data_cache, {});
100-
if (notifications.empty()) {
103+
auto item_cache = std::exchange(_item_cache, {});
104+
if (item_cache.empty()) {
101105
return ss::now();
102106
}
103107
return ss::with_gate(
104-
_ptr->_bg,
105-
[this,
106-
data = std::move(data),
107-
notifications = std::move(notifications)]() mutable {
108+
_ptr->_bg, [this, item_cache = std::move(item_cache)]() mutable {
108109
return _ptr->_op_lock.get_units().then(
109-
[this,
110-
data = std::move(data),
111-
notifications = std::move(notifications)](
110+
[this, item_cache = std::move(item_cache)](
112111
ss::semaphore_units<> u) mutable {
113112
// we have to check if we are the leader
114113
// it is critical as term could have been updated already by
@@ -117,17 +116,35 @@ ss::future<> replicate_batcher::flush() {
117116
// this problem caused truncation failure.
118117

119118
if (!_ptr->is_leader()) {
120-
for (auto& n : notifications) {
119+
for (auto& n : item_cache) {
121120
n->_promise.set_value(errc::not_leader);
122121
}
123122
return ss::make_ready_future<>();
124123
}
125124

126125
auto meta = _ptr->meta();
127126
auto const term = model::term_id(meta.term);
128-
for (auto& b : data) {
129-
b.set_term(term);
127+
ss::circular_buffer<model::record_batch> data;
128+
std::vector<item_ptr> notifications;
129+
130+
for (auto& n : item_cache) {
131+
if (
132+
!n->expected_term.has_value()
133+
|| n->expected_term.value() == term) {
134+
for (auto& b : n->data) {
135+
b.set_term(term);
136+
data.push_back(std::move(b));
137+
}
138+
notifications.push_back(std::move(n));
139+
} else {
140+
n->_promise.set_value(errc::not_leader);
141+
}
142+
}
143+
144+
if (notifications.empty()) {
145+
return ss::now();
130146
}
147+
131148
auto seqs = _ptr->next_followers_request_seq();
132149
append_entries_request req(
133150
_ptr->_self,
@@ -141,6 +158,7 @@ ss::future<> replicate_batcher::flush() {
141158
});
142159
});
143160
}
161+
144162
static void propagate_result(
145163
result<replicate_result> r,
146164
std::vector<replicate_batcher::item_ptr>& notifications) {

src/v/raft/replicate_batcher.h

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,8 @@ class replicate_batcher {
2929
ss::promise<result<replicate_result>> _promise;
3030
replicate_result ret;
3131
size_t record_count;
32+
std::vector<model::record_batch> data;
33+
std::optional<model::term_id> expected_term;
3234
/**
3335
* Item keeps semaphore units until replicate batcher is done with
3436
* processing the request.
@@ -45,7 +47,7 @@ class replicate_batcher {
4547
~replicate_batcher() noexcept = default;
4648

4749
ss::future<result<replicate_result>>
48-
replicate(model::record_batch_reader&&);
50+
replicate(std::optional<model::term_id>, model::record_batch_reader&&);
4951

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

6062
private:
61-
ss::future<item_ptr> do_cache(model::record_batch_reader&&);
63+
ss::future<item_ptr>
64+
do_cache(std::optional<model::term_id>, model::record_batch_reader&&);
6265
ss::future<replicate_batcher::item_ptr> do_cache_with_backpressure(
63-
ss::circular_buffer<model::record_batch>, size_t);
66+
std::optional<model::term_id>,
67+
ss::circular_buffer<model::record_batch>,
68+
size_t);
6469

6570
consensus* _ptr;
6671
ss::semaphore _max_batch_size_sem;
6772

6873
std::vector<item_ptr> _item_cache;
69-
ss::circular_buffer<model::record_batch> _data_cache;
7074
mutex _lock;
7175
};
7276

src/v/raft/tests/append_entries_test.cc

Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,54 @@ FIXTURE_TEST(test_replicate_multiple_entries, raft_test_fixture) {
6666
"State is consistent");
6767
};
6868

69+
FIXTURE_TEST(test_replicate_with_expected_term_leader, raft_test_fixture) {
70+
raft_group gr = raft_group(raft::group_id(0), 3);
71+
gr.enable_all();
72+
auto leader_id = wait_for_group_leader(gr);
73+
auto leader_raft = gr.get_member(leader_id).consensus;
74+
auto term = leader_raft->term();
75+
bool success = replicate_random_batches(
76+
term, gr, 5, raft::consistency_level::leader_ack)
77+
.get0();
78+
BOOST_REQUIRE(success);
79+
};
80+
81+
FIXTURE_TEST(test_replicate_with_expected_term_quorum, raft_test_fixture) {
82+
raft_group gr = raft_group(raft::group_id(0), 3);
83+
gr.enable_all();
84+
auto leader_id = wait_for_group_leader(gr);
85+
auto leader_raft = gr.get_member(leader_id).consensus;
86+
auto term = leader_raft->term();
87+
bool success = replicate_random_batches(
88+
term, gr, 5, raft::consistency_level::quorum_ack)
89+
.get0();
90+
BOOST_REQUIRE(success);
91+
};
92+
93+
FIXTURE_TEST(test_replicate_violating_expected_term_leader, raft_test_fixture) {
94+
raft_group gr = raft_group(raft::group_id(0), 3);
95+
gr.enable_all();
96+
auto leader_id = wait_for_group_leader(gr);
97+
auto leader_raft = gr.get_member(leader_id).consensus;
98+
auto term = leader_raft->term() + model::term_id(100);
99+
bool success = replicate_random_batches(
100+
term, gr, 5, raft::consistency_level::leader_ack)
101+
.get0();
102+
BOOST_REQUIRE(!success);
103+
};
104+
105+
FIXTURE_TEST(test_replicate_violating_expected_term_quorum, raft_test_fixture) {
106+
raft_group gr = raft_group(raft::group_id(0), 3);
107+
gr.enable_all();
108+
auto leader_id = wait_for_group_leader(gr);
109+
auto leader_raft = gr.get_member(leader_id).consensus;
110+
auto term = leader_raft->term() + model::term_id(100);
111+
bool success = replicate_random_batches(
112+
term, gr, 5, raft::consistency_level::quorum_ack)
113+
.get0();
114+
BOOST_REQUIRE(!success);
115+
};
116+
69117
FIXTURE_TEST(test_single_node_recovery, raft_test_fixture) {
70118
raft_group gr = raft_group(raft::group_id(0), 3);
71119
gr.enable_all();

src/v/raft/tests/raft_group_fixture.h

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -690,6 +690,28 @@ static ss::future<bool> replicate_random_batches(
690690
});
691691
}
692692

693+
static ss::future<bool> replicate_random_batches(
694+
model::term_id expected_term,
695+
raft_group& gr,
696+
int count,
697+
raft::consistency_level c_lvl = raft::consistency_level::quorum_ack,
698+
model::timeout_clock::duration tout = 1s) {
699+
return retry_with_leader(
700+
gr, 5, tout, [count, expected_term, c_lvl](raft_node& leader_node) {
701+
auto rdr = random_batches_reader(count);
702+
raft::replicate_options opts(c_lvl);
703+
704+
return leader_node.consensus
705+
->replicate(expected_term, std::move(rdr), opts)
706+
.then([](result<raft::replicate_result> res) {
707+
if (!res) {
708+
return false;
709+
}
710+
return true;
711+
});
712+
});
713+
}
714+
693715
/**
694716
* Makes compactible batches, having one record per batch
695717
*/

0 commit comments

Comments
 (0)