Skip to content

Commit

Permalink
c/feature_manager: init node versions from cluster_discovery
Browse files Browse the repository at this point in the history
Peer seed nodes discovered throught cluster_discovery before the cluster
is bootstrapped are required to be at the same latest logical versions
as the local. Therefore as the cluster is initialized, versions of seed
nodes can safely be assumed to be at the latest. That enables the auto
node_id feature that is essential for cluster bootstrap.
  • Loading branch information
dlex committed Oct 13, 2022
1 parent 413a981 commit 402bf3a
Show file tree
Hide file tree
Showing 3 changed files with 41 additions and 4 deletions.
14 changes: 12 additions & 2 deletions src/v/cluster/controller.cc
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,14 @@ ss::future<> controller::start(
std::vector<model::broker>&& initial_raft0_brokers,
const std::optional<cluster_uuid>& stored_cluster_uuid) {
const bool local_node_is_seed_server = !initial_raft0_brokers.empty();
std::vector<model::node_id> seed_nodes;
seed_nodes.reserve(initial_raft0_brokers.size());
std::transform(
initial_raft0_brokers.cbegin(),
initial_raft0_brokers.cend(),
std::back_inserter(seed_nodes),
[](const model::broker& b) { return b.id(); });

return create_raft0(
_partition_manager,
_shard_table,
Expand Down Expand Up @@ -335,9 +343,11 @@ ss::future<> controller::start(
.then([this] {
return _hm_frontend.invoke_on_all(&health_monitor_frontend::start);
})
.then([this] {
.then([this, seed_nodes = std::move(seed_nodes)]() mutable {
return _feature_manager.invoke_on(
feature_manager::backend_shard, &feature_manager::start);
feature_manager::backend_shard,
&feature_manager::start,
std::move(seed_nodes));
})
.then([this] {
return _metrics_reporter.start_single(
Expand Down
22 changes: 21 additions & 1 deletion src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ feature_manager::feature_manager(

) {}

ss::future<> feature_manager::start() {
ss::future<> feature_manager::start(std::vector<model::node_id>&& seed_nodes) {
vlog(clusterlog.info, "Starting...");

// Register for node health change notifications
Expand Down Expand Up @@ -153,6 +153,10 @@ ss::future<> feature_manager::start() {
[this] { return maybe_log_license_check_info(); });
});

for (const model::node_id n : seed_nodes) {
set_node_to_latest_version(n);
}

co_return;
}
ss::future<> feature_manager::stop() {
Expand Down Expand Up @@ -510,4 +514,20 @@ feature_manager::write_action(cluster::feature_update_action action) {
}
}

void feature_manager::set_node_to_latest_version(const model::node_id node_id) {
const cluster_version latest
= features::feature_table::get_latest_logical_version();
if (const version_map::iterator i = _node_versions.find(node_id);
i != _node_versions.end()) {
if (i->second == latest) return; // already there
vlog(
clusterlog.warn,
"Overriding a previously set version for {} from {} to {}",
node_id,
i->second,
latest);
}
update_node_version(node_id, latest);
}

} // namespace cluster
9 changes: 8 additions & 1 deletion src/v/cluster/feature_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ class feature_manager {
ss::sharded<rpc::connection_cache>& connection_cache,
raft::group_id raft0_group);

ss::future<> start();
ss::future<> start(std::vector<model::node_id>&& seed_nodes);
ss::future<> stop();

ss::future<std::error_code> write_action(cluster::feature_update_action);
Expand Down Expand Up @@ -94,6 +94,13 @@ class feature_manager {
private:
void update_node_version(model::node_id, cluster_version v);

/**
* When cluster discovery has verified seed server latest versions are
* the same as local latest version, this function is used to initialize
* seed servers versions in feature manager.
*/
void set_node_to_latest_version(model::node_id);

ss::future<> do_maybe_update_active_version();
ss::future<> maybe_update_active_version();

Expand Down

0 comments on commit 402bf3a

Please sign in to comment.