Skip to content

Commit

Permalink
tests: improve raft_fixture::retry_with_leader error processing
Browse files Browse the repository at this point in the history
Support std::error_code, raft::errc, cluster::errc and bool,
both individually and as part of a result<..., ...>
  • Loading branch information
bashtanov committed May 10, 2024
1 parent b52b4c3 commit 94e99f0
Showing 1 changed file with 70 additions and 9 deletions.
79 changes: 70 additions & 9 deletions src/v/raft/tests/raft_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
#include "base/vassert.h"
#include "bytes/iobuf.h"
#include "bytes/random.h"
#include "cluster/errc.h"
#include "config/property.h"
#include "features/feature_table.h"
#include "model/fundamental.h"
Expand Down Expand Up @@ -44,6 +45,9 @@
#include <boost/range/irange.hpp>

#include <optional>
#include <ranges>
#include <system_error>
#include <type_traits>

namespace raft {

Expand Down Expand Up @@ -397,6 +401,42 @@ class raft_fixture
[f = std::forward<Func>(f)](auto& pair) { return f(*pair.second); });
}

template<typename E>
struct retry_policy {
static_assert(
std::is_same_v<E, raft::errc> || std::is_same_v<E, cluster::errc>);
static E timeout_error() { return E::timeout; }
static bool should_retry(const E& err) {
return err == E::timeout || err == E::not_leader;
}
};
template<>
struct retry_policy<std::error_code> {
static std::error_code timeout_error() {
return raft::make_error_code(raft::errc::timeout);
}
static bool should_retry(const std::error_code& err) {
if (err.category() == raft::error_category()) {
return retry_policy<raft::errc>::should_retry(
raft::errc(err.value()));
} else if (err.category() == cluster::error_category()) {
return retry_policy<cluster::errc>::should_retry(
cluster::errc(err.value()));
} else {
vassert(
false,
"error category {} not supported",
err.category().name());
}
}
};

template<>
struct retry_policy<bool> {
static bool timeout_error() { return false; }
static bool should_retry(const bool& err) { return !err; }
};

template<typename Func>
auto retry_with_leader(
model::timeout_clock::time_point deadline,
Expand All @@ -405,9 +445,27 @@ class raft_fixture
using futurator
= ss::futurize<std::invoke_result_t<Func, raft_node_instance&>>;
using ret_t = futurator::value_type;
// some functions return bare error code, we'll cast to result anyway
using result_t = std::
conditional_t<is_result_v<ret_t>, const ret_t&, result<void, ret_t>>;
using error_t = std::remove_cvref_t<result_t>::error_type;
using policy = retry_policy<error_t>;

struct retry_state {
ret_t result = errc::timeout;
ret_t result = policy::timeout_error();
int retry = 0;

auto result_with_value() { return static_cast<result_t>(result); }

// assume, as result_t may be e.g. result<void, bool>
error_t get_error() { return result_with_value().assume_error(); }

bool ready() {
if (!result_with_value().has_error()) {
return true;
}
return !policy::should_retry(get_error());
}
};
return ss::do_with(
retry_state{},
Expand All @@ -416,7 +474,7 @@ class raft_fixture
return ss::do_until(
[&state, deadline] {
return model::timeout_clock::now() > deadline
|| state.result.has_value();
|| state.ready();
},
[this,
&state,
Expand All @@ -425,24 +483,27 @@ class raft_fixture
backoff]() mutable {
vlog(
_logger.info,
"Executing action with leader, current retry: {}",
"Executing action with leader, current retry: "
"{}",
state.retry);

return wait_for_leader(deadline).then(
[this, f = std::forward<Func>(f), &state, backoff](
model::node_id leader_id) {
return ss::futurize_invoke(f, node(leader_id))
.then([this, &state, backoff](
auto result) mutable {
// success
if (result) {
state.result = std::move(result);
ret_t result) mutable {
state.result = std::move(result);
// "success"
if (state.ready()) {
return ss::now();
}
vlog(
_logger.info,
"Leader action returned an error: {}",
result.error());
"Leader action returned an error: "
"{}",
state.get_error());
state.result = policy::timeout_error();
state.retry++;

return ss::sleep(backoff);
Expand Down

0 comments on commit 94e99f0

Please sign in to comment.