Skip to content

Commit

Permalink
c/metadata_dissemination: removed old update_leadership_request
Browse files Browse the repository at this point in the history
Signed-off-by: Michal Maslanka <michal@redpanda.com>
  • Loading branch information
mmaslankaprv committed Mar 17, 2023
1 parent 4a332be commit 9563860
Show file tree
Hide file tree
Showing 8 changed files with 0 additions and 161 deletions.
30 changes: 0 additions & 30 deletions src/v/cluster/metadata_dissemination_handler.cc
Original file line number Diff line number Diff line change
Expand Up @@ -22,27 +22,6 @@
#include <algorithm>
#include <iterator>

namespace {
std::vector<cluster::ntp_leader_revision>
from_ntp_leaders(std::vector<cluster::ntp_leader> old_leaders) {
std::vector<cluster::ntp_leader_revision> leaders;
leaders.reserve(old_leaders.size());
std::transform(
old_leaders.begin(),
old_leaders.end(),
std::back_inserter(leaders),
[](cluster::ntp_leader& leader) {
return cluster::ntp_leader_revision(
std::move(leader.ntp),
leader.term,
leader.leader_id,
model::revision_id{}, /* explicitly default */
model::initial_revision_id{} /* explicitly default */
);
});
return leaders;
}
} // namespace
namespace cluster {
metadata_dissemination_handler::metadata_dissemination_handler(
ss::scheduling_group sg,
Expand All @@ -51,15 +30,6 @@ metadata_dissemination_handler::metadata_dissemination_handler(
: metadata_dissemination_rpc_service(sg, ssg)
, _leaders(leaders) {}

ss::future<update_leadership_reply>
metadata_dissemination_handler::update_leadership(
update_leadership_request&& req, rpc::streaming_context&) {
return ss::with_scheduling_group(
get_scheduling_group(), [this, req = std::move(req)]() mutable {
return do_update_leadership(from_ntp_leaders(std::move(req.leaders)));
});
}

ss::future<update_leadership_reply>
metadata_dissemination_handler::update_leadership_v2(
update_leadership_request_v2&& req, rpc::streaming_context&) {
Expand Down
3 changes: 0 additions & 3 deletions src/v/cluster/metadata_dissemination_handler.h
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,6 @@ class metadata_dissemination_handler
ss::smp_service_group,
ss::sharded<partition_leaders_table>&);

ss::future<update_leadership_reply> update_leadership(
update_leadership_request&&, rpc::streaming_context&) final;

ss::future<get_leadership_reply>
get_leadership(get_leadership_request&&, rpc::streaming_context&) final;

Expand Down
5 changes: 0 additions & 5 deletions src/v/cluster/metadata_dissemination_rpc.json
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,6 @@
"cluster/metadata_dissemination_types.h"
],
"methods": [
{
"name": "update_leadership",
"input_type": "update_leadership_request",
"output_type": "update_leadership_reply"
},
{
"name": "get_leadership",
"input_type": "get_leadership_request",
Expand Down
45 changes: 0 additions & 45 deletions src/v/cluster/metadata_dissemination_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -431,23 +431,6 @@ ss::future<> metadata_dissemination_service::update_leaders_with_health_report(
}
}

namespace {
std::vector<cluster::ntp_leader> from_ntp_leader_revision_vector(
std::vector<cluster::ntp_leader_revision> leaders) {
std::vector<cluster::ntp_leader> old_leaders;
old_leaders.reserve(leaders.size());
std::transform(
leaders.begin(),
leaders.end(),
std::back_inserter(old_leaders),
[](cluster::ntp_leader_revision& leader) {
return cluster::ntp_leader(
std::move(leader.ntp), leader.term, leader.leader_id);
});
return old_leaders;
}
} // namespace

ss::future<> metadata_dissemination_service::dispatch_one_update(
model::node_id target_id, update_retry_meta& meta) {
return _clients.local()
Expand All @@ -470,34 +453,6 @@ ss::future<> metadata_dissemination_service::dispatch_one_update(
_dissemination_interval + rpc::clock_type::now()))
.then(&rpc::get_ctx_data<update_leadership_reply>);
})
.then([this, target_id, &meta](result<update_leadership_reply> r) {
if (r.has_error() && r.error() == rpc::errc::method_not_found) {
// old version of redpanda, not yet having the v2 method,
// fallback to old request
return _clients.local()
.with_node_client<metadata_dissemination_rpc_client_protocol>(
_self.id(),
ss::this_shard_id(),
target_id,
_dissemination_interval,
[this, updates = meta.updates, target_id](
metadata_dissemination_rpc_client_protocol proto) mutable {
vlog(
clusterlog.trace,
"Falling back to old version to send {} metadata "
"updates to {}",
updates,
target_id);
return proto.update_leadership(
update_leadership_request(
from_ntp_leader_revision_vector(std::move(updates))),
rpc::client_opts(
_dissemination_interval + rpc::clock_type::now()));
})
.then(&rpc::get_ctx_data<update_leadership_reply>);
}
return ss::make_ready_future<result<update_leadership_reply>>(r);
})
.then([target_id, &meta](result<update_leadership_reply> r) {
if (r) {
vlog(
Expand Down
26 changes: 0 additions & 26 deletions src/v/cluster/metadata_dissemination_types.h
Original file line number Diff line number Diff line change
Expand Up @@ -100,32 +100,6 @@ struct ntp_leader_revision
}
};

struct update_leadership_request
: serde::envelope<
update_leadership_request,
serde::version<0>,
serde::compat_version<0>> {
using rpc_adl_exempt = std::true_type;
std::vector<ntp_leader> leaders;

update_leadership_request() noexcept = default;

explicit update_leadership_request(std::vector<ntp_leader> leaders)
: leaders(std::move(leaders)) {}

friend bool operator==(
const update_leadership_request&, const update_leadership_request&)
= default;

auto serde_fields() { return std::tie(leaders); }

friend std::ostream&
operator<<(std::ostream& o, const update_leadership_request& r) {
fmt::print(o, "leaders {}", r.leaders);
return o;
}
};

struct update_leadership_request_v2
: serde::envelope<
update_leadership_request_v2,
Expand Down
35 changes: 0 additions & 35 deletions src/v/compat/metadata_dissemination_compat.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,41 +18,6 @@

namespace compat {

/*
* cluster::update_leadership_request
*/
template<>
struct compat_check<cluster::update_leadership_request> {
static constexpr std::string_view name
= "cluster::update_leadership_request";

static std::vector<cluster::update_leadership_request> create_test_cases() {
return generate_instances<cluster::update_leadership_request>();
}

static void to_json(
cluster::update_leadership_request obj,
json::Writer<json::StringBuffer>& wr) {
json_write(leaders);
}

static cluster::update_leadership_request from_json(json::Value& rd) {
cluster::update_leadership_request obj;
json_read(leaders);
return obj;
}

static std::vector<compat_binary>
to_binary(cluster::update_leadership_request obj) {
return {compat_binary::serde(obj)};
}

static void
check(cluster::update_leadership_request obj, compat_binary test) {
verify_serde_only(obj, std::move(test));
}
};

/*
* cluster::update_leadership_request_v2
*/
Expand Down
16 changes: 0 additions & 16 deletions src/v/compat/metadata_dissemination_generator.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,22 +18,6 @@

namespace compat {

template<>
struct instance_generator<cluster::update_leadership_request> {
static cluster::update_leadership_request random() {
return cluster::update_leadership_request({
cluster::ntp_leader(
model::random_ntp(),
tests::random_named_int<model::term_id>(),
tests::random_named_int<model::node_id>()),
});
}

static std::vector<cluster::update_leadership_request> limits() {
return {};
}
};

template<>
struct instance_generator<cluster::update_leadership_request_v2> {
static cluster::update_leadership_request_v2 random() {
Expand Down
1 change: 0 additions & 1 deletion src/v/compat/run.cc
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@ using compat_checks = type_list<
cluster::finish_reallocation_reply,
cluster::set_maintenance_mode_request,
cluster::set_maintenance_mode_reply,
cluster::update_leadership_request,
cluster::config_status,
cluster::config_status_request,
cluster::config_status_reply,
Expand Down

0 comments on commit 9563860

Please sign in to comment.