Skip to content

Commit

Permalink
raft: make raft requests abortable
Browse files Browse the repository at this point in the history
This patch adds an ability to pass abort_source to raft request APIs (
add_entry, modify_config) to make them abortable. A request issuer not
always want to wait for a request to complete. For instance because a
client disconnected or because it no longer interested in waiting
because of a timeout. After this patch it can now abort waiting for such
requests through an abort source. Note that aborting a request only
aborts the wait for it to complete, it does not mean that the request
will not be eventually executed.

Message-Id: <[email protected]>
  • Loading branch information
Gleb Natapov authored and tgrabiec committed Mar 16, 2022
1 parent a1d0f08 commit a1604aa
Show file tree
Hide file tree
Showing 8 changed files with 109 additions and 67 deletions.
4 changes: 2 additions & 2 deletions raft/fsm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -44,10 +44,10 @@ fsm::fsm(server_id id, term_t current_term, server_id voted_for, log log,
failure_detector& failure_detector, fsm_config config) :
fsm(id, current_term, voted_for, std::move(log), index_t{0}, failure_detector, config) {}

future<> fsm::wait_max_log_size() {
future<> fsm::wait_max_log_size(seastar::abort_source* as) {
check_is_leader();

return leader_state().log_limiter_semaphore->wait();
return as ? leader_state().log_limiter_semaphore->wait(*as) : leader_state().log_limiter_semaphore->wait();
}

const configuration& fsm::get_configuration() const {
Expand Down
2 changes: 1 addition & 1 deletion raft/fsm.hh
Original file line number Diff line number Diff line change
Expand Up @@ -385,7 +385,7 @@ public:

// Call this function to wait for the number of log entries to
// go below max_log_size.
future<> wait_max_log_size();
future<> wait_max_log_size(seastar::abort_source* as);

// Return current configuration. Throws if not a leader.
const configuration& get_configuration() const;
Expand Down
10 changes: 7 additions & 3 deletions raft/raft.hh
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,10 @@ struct no_other_voting_member : public error {
no_other_voting_member() : error("Cannot stepdown because there is no other voting member") {}
};

struct request_aborted : public error {
request_aborted() : error("Request is aborted by a caller") {}
};

// True if a failure to execute a Raft operation can be re-tried,
// perhaps with a different server.
inline bool is_transient_error(const std::exception& e) {
Expand Down Expand Up @@ -580,18 +584,18 @@ public:
virtual future<snapshot_reply> apply_snapshot(server_id from, install_snapshot snp) = 0;

// Try to execute read barrier, future resolves when the barrier is completed or error happens
virtual future<read_barrier_reply> execute_read_barrier(server_id from) = 0;
virtual future<read_barrier_reply> execute_read_barrier(server_id from, seastar::abort_source* as) = 0;

// An endpoint on the leader to add an entry to the raft log,
// as requested by a remote follower.
virtual future<add_entry_reply> execute_add_entry(server_id from, command cmd) = 0;
virtual future<add_entry_reply> execute_add_entry(server_id from, command cmd, seastar::abort_source* as) = 0;

// An endpoint on the leader to change configuration,
// as requested by a remote follower.
// If the future resolves successfully, a dummy entry was committed after the configuration change.
virtual future<add_entry_reply> execute_modify_config(server_id from,
std::vector<server_address> add,
std::vector<server_id> del) = 0;
std::vector<server_id> del, seastar::abort_source* as) = 0;

// Update RPC implementation with this client as
// the receiver of RPC input.
Expand Down
129 changes: 81 additions & 48 deletions raft/server.cc

Large diffs are not rendered by default.

13 changes: 9 additions & 4 deletions raft/server.hh
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
#pragma once
#include <seastar/core/abort_source.hh>
#include "raft.hh"

namespace raft {
Expand Down Expand Up @@ -55,7 +56,7 @@ public:
// this means that the entry is committed/applied locally (depending on the wait type).
// Applied locally means the local state machine replica applied this command;
// committed locally means simply that the commit index is beyond this entry's index.
virtual future<> add_entry(command command, wait_type type) = 0;
virtual future<> add_entry(command command, wait_type type, seastar::abort_source* as = nullptr) = 0;

// Set a new cluster configuration. If the configuration is
// identical to the previous one does nothing.
Expand All @@ -79,7 +80,9 @@ public:
// Note: committing a dummy entry extends the opportunity for
// uncertainty, thus commit_status_unknown exception may be
// returned even in case of a successful config change.
virtual future<> set_configuration(server_address_set c_new) = 0;
//
// A caller may pass a pointer to an abort_source to make operation abortable.
virtual future<> set_configuration(server_address_set c_new, seastar::abort_source* as = nullptr) = 0;

// A simplified wrapper around set_configuration() which adds
// and deletes servers. Unlike set_configuration(),
Expand All @@ -99,8 +102,9 @@ public:
// The local commit index is not necessarily up-to-date yet and the state of the local state machine
// replica may still come from before the configuration entry.
// (exception: if no server was actually added or removed, then nothing gets committed and the leader responds immediately).
// A caller may pass a pointer to an abort_source to make operation abortable.
virtual future<> modify_config(std::vector<server_address> add,
std::vector<server_id> del) = 0;
std::vector<server_id> del, seastar::abort_source* as = nullptr) = 0;

// Return the currently known configuration
virtual raft::configuration get_configuration() const = 0;
Expand All @@ -123,7 +127,8 @@ public:
// May be called before attempting a read from the local state
// machine. The read should proceed only after the returned
// future has resolved successfully.
virtual future<> read_barrier() = 0;
// A caller may pass a pointer to an abort_source to make operation abortable.
virtual future<> read_barrier(seastar::abort_source* as = nullptr) = 0;

// Initiate leader stepdown process.
// If the node is not a leader returns not_a_leader exception.
Expand Down
6 changes: 3 additions & 3 deletions service/raft/raft_rpc.cc
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ void raft_rpc::read_quorum_reply(raft::server_id from, raft::read_quorum_reply c
}

future<raft::read_barrier_reply> raft_rpc::execute_read_barrier(raft::server_id from) {
return _client->execute_read_barrier(from);
return _client->execute_read_barrier(from, nullptr);
}

future<raft::snapshot_reply> raft_rpc::apply_snapshot(raft::server_id from, raft::install_snapshot snp) {
Expand All @@ -190,13 +190,13 @@ future<raft::snapshot_reply> raft_rpc::apply_snapshot(raft::server_id from, raft
}

future<raft::add_entry_reply> raft_rpc::execute_add_entry(raft::server_id from, raft::command cmd) {
return _client->execute_add_entry(from, std::move(cmd));
return _client->execute_add_entry(from, std::move(cmd), nullptr);
}

future<raft::add_entry_reply> raft_rpc::execute_modify_config(raft::server_id from,
std::vector<raft::server_address> add,
std::vector<raft::server_id> del) {
return _client->execute_modify_config(from, std::move(add), std::move(del));
return _client->execute_modify_config(from, std::move(add), std::move(del), nullptr);
}

} // end of namespace service
6 changes: 3 additions & 3 deletions test/raft/randomized_nemesis_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -466,7 +466,7 @@ class rpc : public raft::rpc {
++_read_barrier_executions;
(void)[] (rpc& self, raft::server_id src, execute_barrier_on_leader m, gate::holder holder) -> future<> {
try {
auto reply = co_await self._client->execute_read_barrier(src);
auto reply = co_await self._client->execute_read_barrier(src, nullptr);

self._send(src, execute_barrier_on_leader_reply{
.reply = std::move(reply),
Expand Down Expand Up @@ -498,7 +498,7 @@ class rpc : public raft::rpc {
++_add_entry_executions;
(void)[] (rpc& self, raft::server_id src, add_entry_message m, gate::holder holder) -> future<> {
try {
auto reply = co_await self._client->execute_add_entry(src, std::move(m.cmd));
auto reply = co_await self._client->execute_add_entry(src, std::move(m.cmd), nullptr);

self._send(src, add_entry_reply_message{
.reply = std::move(reply),
Expand Down Expand Up @@ -530,7 +530,7 @@ class rpc : public raft::rpc {
++_modify_config_executions;
(void)[] (rpc& self, raft::server_id src, modify_config_message m, gate::holder holder) -> future<> {
try {
auto reply = co_await self._client->execute_modify_config(src, std::move(m.add), std::move(m.del));
auto reply = co_await self._client->execute_modify_config(src, std::move(m.add), std::move(m.del), nullptr);

self._send(src, add_entry_reply_message{
.reply = std::move(reply),
Expand Down
6 changes: 3 additions & 3 deletions test/raft/replication.hh
Original file line number Diff line number Diff line change
Expand Up @@ -707,7 +707,7 @@ public:
if (!(*_connected)(id, _id)) {
return make_exception_future<raft::read_barrier_reply>(std::runtime_error("cannot send append since nodes are disconnected"));
}
return _net[id]->_client->execute_read_barrier(_id);
return _net[id]->_client->execute_read_barrier(_id, nullptr);
}
void check_known_and_connected(raft::server_id id) {
if (!_net.count(id)) {
Expand All @@ -719,13 +719,13 @@ public:
}
future<raft::add_entry_reply> send_add_entry(raft::server_id id, const raft::command& cmd) override {
check_known_and_connected(id);
return _net[id]->_client->execute_add_entry(_id, cmd);
return _net[id]->_client->execute_add_entry(_id, cmd, nullptr);
}
future<raft::add_entry_reply> send_modify_config(raft::server_id id,
const std::vector<raft::server_address>& add,
const std::vector<raft::server_id>& del) override {
check_known_and_connected(id);
return _net[id]->_client->execute_modify_config(_id, add, del);
return _net[id]->_client->execute_modify_config(_id, add, del, nullptr);
}

void add_server(raft::server_id id, bytes node_info) override {
Expand Down

0 comments on commit a1604aa

Please sign in to comment.