Skip to content

Commit

Permalink
Merge pull request redpanda-data#22972 from oleiman/nodeuuid/core-700…
Browse files Browse the repository at this point in the history
…0/poc

CORE-7000: Node ID/UUID Override
  • Loading branch information
michael-redpanda authored Sep 20, 2024
2 parents 937dd47 + ace0d61 commit a2bbe5a
Show file tree
Hide file tree
Showing 18 changed files with 842 additions and 27 deletions.
2 changes: 2 additions & 0 deletions src/v/config/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ redpanda_cc_library(
"client_group_byte_rate_quota.cc",
"configuration.cc",
"node_config.cc",
"node_overrides.cc",
"rest_authn_endpoint.cc",
"rjson_serialization.cc",
"throughput_control_group.cc",
Expand All @@ -29,6 +30,7 @@ redpanda_cc_library(
"fwd.h",
"mock_property.h",
"node_config.h",
"node_overrides.h",
"property.h",
"rest_authn_endpoint.h",
"rjson_serialization.h",
Expand Down
1 change: 1 addition & 0 deletions src/v/config/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ v_cc_library(
validators.cc
throughput_control_group.cc
tls_config.cc
node_overrides.cc
DEPS
v::json
v::model
Expand Down
22 changes: 22 additions & 0 deletions src/v/config/convert.h
Original file line number Diff line number Diff line change
Expand Up @@ -633,4 +633,26 @@ struct convert<config::tls_version> {
}
};

template<>
struct convert<model::node_uuid> {
using type = model::node_uuid;
static Node encode(const type& rhs) {
return Node(ssx::sformat("{}", rhs));
}
static bool decode(const Node& node, type& rhs) {
auto value = node.as<std::string>();
auto out = [&value]() -> std::optional<model::node_uuid> {
try {
return model::node_uuid(uuid_t::from_string(value));
} catch (const std::runtime_error& e) {
return std::nullopt;
}
}();
if (out.has_value()) {
rhs = out.value();
}
return out.has_value();
}
};

} // namespace YAML
9 changes: 9 additions & 0 deletions src/v/config/node_config.cc
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,15 @@ node_config::node_config() noexcept
"Path to the directory that contains the OpenSSL FIPS-compliant module.",
{.visibility = visibility::user},
std::nullopt)
, node_id_overrides(
*this,
"node_id_overrides",
"List of node ID and UUID overrides to be applied at broker startup. "
"Each entry includes the current UUID and desired ID and UUID. Each "
"entry applies to a given node if and only if 'current' matches that "
"node's current UUID.",
{.visibility = visibility::user},
{})
, _advertised_rpc_api(
*this,
"advertised_rpc_api",
Expand Down
4 changes: 4 additions & 0 deletions src/v/config/node_config.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,9 +14,11 @@
#include "config/broker_endpoint.h"
#include "config/convert.h"
#include "config/data_directory_path.h"
#include "config/node_overrides.h"
#include "config/property.h"
#include "config/seed_server.h"
#include "config_store.h"
#include "model/fundamental.h"

#include <algorithm>
#include <iterator>
Expand Down Expand Up @@ -99,6 +101,8 @@ struct node_config final : public config_store {
// Path to the directory that holds the OpenSSL FIPS module
property<std::optional<std::filesystem::path>> openssl_module_directory;

property<std::vector<config::node_id_override>> node_id_overrides;

// build pidfile path: `<data_directory>/pid.lock`
std::filesystem::path pidfile_path() const {
return data_directory().path / "pid.lock";
Expand Down
76 changes: 76 additions & 0 deletions src/v/config/node_overrides.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* Copyright 2024 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#include "config/node_overrides.h"

#include "re2/re2.h"

#include <iostream>
#include <string>

namespace config {

std::ostream& operator<<(std::ostream& os, const config::node_id_override& v) {
return os << ssx::sformat("{}:{}:{}", v.key, v.uuid, v.id);
}

std::istream& operator>>(std::istream& is, config::node_id_override& v) {
std::string s;
is >> s;

const static re2::RE2 pattern{R"(^([^:]+):([^:]+):([^:]+)$)"};
vassert(pattern.ok(), "Regex compilation failed");

std::string curr, uuid, id;

if (!re2::RE2::FullMatch(s, pattern, &curr, &uuid, &id)) {
throw std::runtime_error(fmt::format(
R"(Formatting error: '{}', must be of form '<uuid>:<uuid>:<id>')",
s));
}

v.key = boost::lexical_cast<model::node_uuid>(curr);
v.uuid = boost::lexical_cast<model::node_uuid>(uuid);
v.id = boost::lexical_cast<model::node_id>(id);

return is;
}

void node_override_store::maybe_set_overrides(
model::node_uuid node_uuid,
const std::vector<config::node_id_override>& overrides) {
vassert(ss::this_shard_id() == 0, "Only set overrides on shard 0");
for (const auto& o : overrides) {
if (o.key == node_uuid) {
if (_uuid_override.has_value() || _id_override.has_value()) {
throw std::runtime_error(
"Invalid node ID override: Limit one override per broker");
break;
}
_uuid_override.emplace(o.uuid);
_id_override.emplace(o.id);
}
}
}

const std::optional<model::node_uuid>&
node_override_store::node_uuid() const noexcept {
vassert(ss::this_shard_id() == 0, "Only get overrides on shard 0");
return _uuid_override;
}

const std::optional<model::node_id>&
node_override_store::node_id() const noexcept {
vassert(ss::this_shard_id() == 0, "Only get overrides on shard 0");
return _id_override;
}

} // namespace config
111 changes: 111 additions & 0 deletions src/v/config/node_overrides.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* Copyright 2024 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#pragma once

#include "config/convert.h"
#include "model/fundamental.h"
#include "ssx/sformat.h"

#include <yaml-cpp/yaml.h>

#include <iosfwd>
#include <vector>

namespace config {

/**
* In-memory representation of a Node ID/UUID override candidate.
*
* Semantics surrounding when and how to apply an override
* are not represented here and should be carefully considered at the
* point of usage.
*/
struct node_id_override {
node_id_override() = default;
node_id_override(
model::node_uuid key,
model::node_uuid uuid_value,
model::node_id id_value)
: key(key)
, uuid(uuid_value)
, id(id_value) {}
model::node_uuid key{};
model::node_uuid uuid{};
model::node_id id{};

private:
friend std::ostream&
operator<<(std::ostream& os, const node_id_override& v);
friend std::istream& operator>>(std::istream& is, node_id_override& v);
friend bool operator==(const node_id_override&, const node_id_override&)
= default;
};

/**
* Thin data structure to encapsulate 'node_id_override' filtering and
* short-term storage.
*
* Given an array of 'node_id_override's, select the appropriate override as:
*
* For some instance 'node_id_override O', 'O.uuid' and 'O.id' should
* be applied to some node if and only if its _current_ UUID matches
* 'O.key'.
*/
struct node_override_store {
node_override_store() = default;

/**
* From the provided vector, accept the override (if one exists) whose
* 'key' matches 'node_uuid'.
*
* Throws if multiple overrides match 'node_uuid'
*/
void maybe_set_overrides(
model::node_uuid node_uuid,
const std::vector<config::node_id_override>& overrides);

const std::optional<model::node_uuid>& node_uuid() const noexcept;
const std::optional<model::node_id>& node_id() const noexcept;

private:
std::optional<model::node_uuid> _uuid_override;
std::optional<model::node_id> _id_override;
};

} // namespace config

namespace YAML {

template<>
struct convert<config::node_id_override> {
using type = config::node_id_override;
static inline Node encode(const type& rhs) {
Node node;
node["current_uuid"] = ssx::sformat("{}", rhs.key);
node["new_uuid"] = ssx::sformat("{}", rhs.uuid);
node["new_id"] = ssx::sformat("{}", rhs.id);

return node;
}

static inline bool decode(const Node& node, type& rhs) {
if (!node["current_uuid"] || !node["new_uuid"] || !node["new_id"]) {
return false;
}
rhs.key = node["current_uuid"].as<model::node_uuid>();
rhs.uuid = node["new_uuid"].as<model::node_uuid>();
rhs.id = node["new_id"].as<model::node_id>();
return true;
}
};

} // namespace YAML
5 changes: 4 additions & 1 deletion src/v/config/property.h
Original file line number Diff line number Diff line change
Expand Up @@ -654,6 +654,10 @@ consteval std::string_view property_type_name() {
return "string";
} else if constexpr (std::is_same_v<type, config::tls_version>) {
return "string";
} else if constexpr (std::is_same_v<type, model::node_uuid>) {
return "string";
} else if constexpr (std::is_same_v<type, config::node_id_override>) {
return "node_id_override";
} else {
static_assert(
base::unsupported_type<T>::value, "Type name not defined");
Expand Down Expand Up @@ -928,5 +932,4 @@ class retention_duration_property final
}
}
};

}; // namespace config
29 changes: 29 additions & 0 deletions src/v/config/rjson_serialization.cc
Original file line number Diff line number Diff line change
Expand Up @@ -232,4 +232,33 @@ void rjson_serialize(
stringize(w, v);
}

void rjson_serialize(
json::Writer<json::StringBuffer>& w, const model::node_uuid& v) {
stringize(w, v);
}

void rjson_serialize(
json::Writer<json::StringBuffer>& w, const config::node_id_override& v) {
w.StartObject();

w.Key("current_uuid");
stringize(w, v.key);
w.Key("new_uuid");
stringize(w, v.uuid);
w.Key("new_id");
stringize(w, v.id);

w.EndObject();
}

void rjson_serialize(
json::Writer<json::StringBuffer>& w,
const std::vector<config::node_id_override>& v) {
w.StartArray();
for (const auto& e : v) {
rjson_serialize(w, e);
}
w.EndArray();
}

} // namespace json
11 changes: 11 additions & 0 deletions src/v/config/rjson_serialization.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "base/seastarx.h"
#include "config/data_directory_path.h"
#include "config/endpoint_tls_config.h"
#include "config/node_overrides.h"
#include "config/seed_server.h"
#include "config/tls_config.h"
#include "config/types.h"
Expand Down Expand Up @@ -122,4 +123,14 @@ void rjson_serialize(
void rjson_serialize(
json::Writer<json::StringBuffer>&, const config::tls_version& v);

void rjson_serialize(
json::Writer<json::StringBuffer>&, const model::node_uuid&);

void rjson_serialize(
json::Writer<json::StringBuffer>&, const config::node_id_override&);

void rjson_serialize(
json::Writer<json::StringBuffer>&,
const std::vector<config::node_id_override>&);

} // namespace json
3 changes: 2 additions & 1 deletion src/v/config/tests/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,8 @@ set(srcs
seed_server_property_test.cc
cloud_credentials_source_test.cc
validator_tests.cc
throughput_control_group_test.cc)
throughput_control_group_test.cc
node_override_test.cc)

rp_test(
UNIT_TEST
Expand Down
Loading

0 comments on commit a2bbe5a

Please sign in to comment.