[PATCH v1 10/56] gossiper: add get_endpoint_state_ptr() function that works on host id

3 views
Skip to first unread message

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:49 AMJan 13
to scylladb-dev@googlegroups.com
Will be used later to simplify code.
---
gms/gossiper.hh | 1 +
gms/gossiper.cc | 8 ++++++++
2 files changed, 9 insertions(+)

diff --git a/gms/gossiper.hh b/gms/gossiper.hh
index 2ff817f7000..595f7103ca6 100644
--- a/gms/gossiper.hh
+++ b/gms/gossiper.hh
@@ -435,6 +435,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
// The endpoint_state is immutable (except for its update_timestamp), guaranteed not to change while
// the endpoint_state_ptr is held.
endpoint_state_ptr get_endpoint_state_ptr(inet_address ep) const noexcept;
+ endpoint_state_ptr get_endpoint_state_ptr(locator::host_id ep) const noexcept;

// Return this node's endpoint_state_ptr
endpoint_state_ptr get_this_endpoint_state_ptr() const noexcept {
diff --git a/gms/gossiper.cc b/gms/gossiper.cc
index 19747be504f..c303230316e 100644
--- a/gms/gossiper.cc
+++ b/gms/gossiper.cc
@@ -1505,6 +1505,14 @@ endpoint_state_ptr gossiper::get_endpoint_state_ptr(inet_address ep) const noexc
}
}

+endpoint_state_ptr gossiper::get_endpoint_state_ptr(locator::host_id id) const noexcept {
+ auto ip = _address_map.find(id);
+ if (!ip) {
+ return nullptr;
+ }
+ return get_endpoint_state_ptr(*ip);
+}
+
void gossiper::update_timestamp(const endpoint_state_ptr& eps) noexcept {
const_cast<endpoint_state&>(*eps).update_timestamp();
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:49 AMJan 13
to scylladb-dev@googlegroups.com
Now that all topology related code uses host ids there is not point to
maintain ip to id (and back) mappings in the token metadata. After the
patch the mapping will be maintained in the gossiper only. The rest of
the system will use host ids and in rare cases where translation is
needed (mostly for UX compatibility reasons) the translation will be
done using gossiper.

Fixes: scylladb/scylla#21777

CI: https://jenkins.scylladb.com/job/scylla-master/job/scylla-ci/14388/

Also in scylla-dev gleb/drop-ip-from-tm

Gleb Natapov (56):
service: address_map: add lookup function that expects address to
exist
hints: simplify can_send() function
api: do not use token_metadata to retrieve ip to id mapping in
token_metadata RESTful endpoints
hits: move create_hint_sync_point function to host ids
hints: use gossiper to map ip to id in wait_for_sync_point
view: do not use get_endpoint_for_host_id_if_known to check if a node
is part of the topology
db/virtual_tables: use host id from the gossiper endpoint state in
cluster_status table
repair: use gossiper to map ip to host id instead of token_metadata
storage_proxy: used gossiper for map ip to host id in
connection_dropped callback
gossiper: add get_endpoint_state_ptr() function that works on host id
topology coordinator: drop get_endpoint_for_host_id_if_known usage
storage_service: force_remove_completion use address map to resolve id
to ip instead of token metadata
storage_service: use gossiper to map id to ip instead of
token_metadata in node_ops_cmd_handler
storage_service: use gossiper instead of token_metadata to map ip to
id in gossiper notifications
storage_service: use existing util function instead of re-iplementing
it
storage_service: move storage_service::get_natural_endpoints to use
host ids internally and translate to ips before returning
storage_service: move describe ring and get_range_to_endpoint_map to
use host ids inside and translate to ips at the last moment
test: drop inet_address usage from network_topology_strategy_test
locator: drop no longer used ip based functions
storage_service: do not use ip addresses from token_metadata in
handling of a normal state
messaging_service: drop get_raw_version and knows_version
gossiper: change get_live_members and all its users to work on host
ids
messaging_service: drop the usage of ip based token_metadata APIs
consistency_level: drop templates since the same types of ranges are
used by all the callers
locator: topology: drop no longer used ip based overloads
locator: topology: remove get_location overload that works on ip and
its last users
storage_proxy: drop is_alive that works on ip since it is not used any
more
locator: topology: drop is_me ip overload along with remaning users
locator: topology: change get_datacenter_endpoints and
get_datacenter_racks to return host ids and amend users
storage_service: change get_dc_rack_for() to work on host ids
hints: move id to ip translation into store_hint() function
node_ops, cdc: drop remaining
token_metadata::get_endpoint_for_host_id() usage
topology coordinator: remove gossiper entry only if host id matches
provided one
storage_service: fix logging
storage_service: ip_address_updater: check peers table instead of
token_metadata whether ip was changed
host_id_or_endpoint: use gossiper to resolve ip to id and back
mappings
token_metadata: drop no longer unused functions
storage_service: drop loops from node ops replace_prepare handling
since there can be only one replacing node
storage_service: fix indentation after the last patch
storage_service: use gossiper to map ip to id in node_ops operations
gossiper: drop get_unreachable_token_owners functions
locator: network_topology_strategy: use host_id based function to
check number of endpoints in dcs
locator: token_metadata: remove unused ip based functions
api: view_build_statuses: do not use IP from the topology, but
translate id to ip using address map instead
locator: drop unused function from tablet_effective_replication_map
cql3: report host id instead of ip in error during SELECT FROM
MUTATION_FRAGMENTS query
topology coordinator: change connection dropping code to work on host
ids
storage_proxy: translate id to ip using address map in tablets's
describe_ring code instead of taking one from the topology
storage_service: check for outdated ip in on_change notification in
the peers table
locator: topology: remove unused functions
storage_proxy: translate ips to ids in forward array using gossiper
storage_service: use host_id to look for a node in on_alive handler
repair: drop unneeded code
locator: topology: drop indexing by ips
locator: token_metadata: drop update_host_id() function that does
nothing now
hint manager: do not translate ip to id in case hint manager is
stopped already

api/api_init.hh | 6 +-
api/hinted_handoff.hh | 3 +-
api/storage_service.hh | 2 +-
api/token_metadata.hh | 3 +-
db/consistency_level.hh | 10 +-
db/hints/manager.hh | 4 +-
db/view/view_builder.hh | 2 +-
gms/gossiper.hh | 9 +-
locator/abstract_replication_strategy.hh | 5 -
locator/token_metadata.hh | 37 +--
locator/topology.hh | 58 +---
locator/util.hh | 5 +
message/messaging_service.hh | 10 +-
repair/repair.hh | 3 -
repair/task_manager_module.hh | 4 +-
service/address_map.hh | 9 +
service/storage_proxy.hh | 5 +-
service/storage_service.hh | 18 +-
alternator/server.cc | 5 +-
api/api.cc | 14 +-
api/hinted_handoff.cc | 13 +-
api/messaging_service.cc | 2 +-
api/storage_service.cc | 6 +-
api/token_metadata.cc | 45 ++-
cdc/generation.cc | 2 +-
cql3/statements/select_statement.cc | 2 +-
db/consistency_level.cc | 19 +-
db/hints/internal/hint_sender.cc | 33 +-
db/hints/manager.cc | 36 +-
db/system_keyspace.cc | 2 +-
db/view/view.cc | 8 +-
db/virtual_tables.cc | 19 +-
gms/gossiper.cc | 27 +-
locator/abstract_replication_strategy.cc | 23 --
locator/network_topology_strategy.cc | 4 +-
locator/tablets.cc | 17 -
locator/token_metadata.cc | 173 +---------
locator/topology.cc | 130 +-------
locator/util.cc | 20 +-
main.cc | 22 +-
message/messaging_service.cc | 31 +-
node_ops/node_ops_ctl.cc | 2 +-
repair/repair.cc | 32 +-
repair/row_level.cc | 2 +-
service/migration_manager.cc | 10 +-
service/qos/service_level_controller.cc | 2 +-
service/storage_proxy.cc | 48 ++-
service/storage_service.cc | 330 +++++++++----------
service/topology_coordinator.cc | 35 +-
test/boost/locator_topology_test.cc | 130 ++------
test/boost/network_topology_strategy_test.cc | 68 ++--
test/boost/tablets_test.cc | 68 +---
test/boost/token_metadata_test.cc | 38 ---
test/lib/cql_test_env.cc | 5 +-
test/manual/message.cc | 2 +-
test/perf/perf_sort_by_proximity.cc | 1 -
test/perf/tablet_load_balancing.cc | 1 -
57 files changed, 535 insertions(+), 1085 deletions(-)

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:50 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_proxy.cc | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index 7776c7fd988..d54b3bd7aab 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -1057,7 +1057,9 @@ class storage_proxy::remote {
void connection_dropped(gms::inet_address addr, std::optional<locator::host_id> id) {
slogger.debug("Drop hit rate info for {} because of disconnect", addr);
if (!id) {
- id = _sp.get_token_metadata_ptr()->get_host_id_if_known(addr);
+ try {
+ id = _gossiper.get_host_id(addr);
+ } catch (...) {}
}
if (!id) {
return;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:50 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
repair/task_manager_module.hh | 4 +++-
repair/repair.cc | 26 +++++++++++++++-----------
2 files changed, 18 insertions(+), 12 deletions(-)

diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh
index 01611468a7f..f941adb5e9a 100644
--- a/repair/task_manager_module.hh
+++ b/repair/task_manager_module.hh
@@ -50,8 +50,9 @@ class user_requested_repair_task_impl : public repair_task_impl {
std::unordered_set<locator::host_id> _ignore_nodes;
bool _small_table_optimization;
std::optional<int> _ranges_parallelism;
+ gms::gossiper& _gossiper;
public:
- user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism) noexcept
+ user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism, gms::gossiper& gossiper) noexcept
: repair_task_impl(module, id.uuid(), id.id, "keyspace", std::move(keyspace), "", std::move(entity), tasks::task_id::create_null_id(), streaming::stream_reason::repair)
, _germs(germs)
, _cfs(std::move(cfs))
@@ -61,6 +62,7 @@ class user_requested_repair_task_impl : public repair_task_impl {
, _ignore_nodes(std::move(ignore_nodes))
, _small_table_optimization(small_table_optimization)
, _ranges_parallelism(ranges_parallelism)
+ , _gossiper(gossiper)
{}

virtual tasks::is_abortable is_abortable() const noexcept override {
diff --git a/repair/repair.cc b/repair/repair.cc
index 67b68eb8064..51c0aa8d7db 100644
--- a/repair/repair.cc
+++ b/repair/repair.cc
@@ -226,6 +226,7 @@ repair_neighbors::repair_neighbors(std::vector<locator::host_id> nodes, std::vec

// Return all of the neighbors with whom we share the provided range.
static std::vector<locator::host_id> get_neighbors(
+ const gms::gossiper& gossiper,
const locator::effective_replication_map& erm,
const sstring& ksname, query::range<dht::token> range,
const std::vector<sstring>& data_centers,
@@ -282,17 +283,19 @@ static std::vector<locator::host_id> get_neighbors(
} catch(...) {
throw std::runtime_error(format("Unknown host specified: {}", host));
}
- auto endpoint = erm.get_token_metadata().get_host_id_if_known(ip);
- if (endpoint) {
+
+ try {
+ auto endpoint = gossiper.get_host_id(ip);
+
if (endpoint == my_address) {
found_me = true;
- } else if (neighbor_set.contains(*endpoint)) {
- ret.push_back(*endpoint);
+ } else if (neighbor_set.contains(endpoint)) {
+ ret.push_back(endpoint);
// If same host is listed twice, don't add it again later
- neighbor_set.erase(*endpoint);
- } else {
- rlogger.warn("Provided host ip {} has no corresponding host id", ip);
+ neighbor_set.erase(endpoint);
}
+ } catch (...) {
+ rlogger.warn("Provided host ip {} has no corresponding host id", ip);
}
// Nodes which aren't neighbors for this range are ignored.
// This allows the user to give a list of "good" nodes, where
@@ -329,6 +332,7 @@ static std::vector<locator::host_id> get_neighbors(
}

static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
+ const gms::gossiper& gossiper,
const locator::effective_replication_map& erm,
const sstring& ksname,
const dht::token_range_vector& ranges,
@@ -344,7 +348,7 @@ static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
participating_hosts.insert(my_address);

co_await do_for_each(ranges, [&] (const dht::token_range& range) {
- const auto nbs = get_neighbors(erm, ksname, range, data_centers, hosts, ignore_nodes);
+ const auto nbs = get_neighbors(gossiper, erm, ksname, range, data_centers, hosts, ignore_nodes);
for (const auto& nb : nbs) {
participating_hosts.insert(nb);
}
@@ -676,7 +680,7 @@ void repair::shard_repair_task_impl::check_in_abort_or_shutdown() {

repair_neighbors repair::shard_repair_task_impl::get_repair_neighbors(const dht::token_range& range) {
return neighbors.empty() ?
- repair_neighbors(get_neighbors(*erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
+ repair_neighbors(get_neighbors(gossiper, *erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
neighbors[range];
}

@@ -1306,7 +1310,7 @@ future<int> repair_service::do_repair_start(gms::gossip_address_map& addr_map, s
}

auto ranges_parallelism = options.ranges_parallelism == -1 ? std::nullopt : std::optional<int>(options.ranges_parallelism);
- auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism);
+ auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism, _gossiper.local());
co_return id.id;
}

@@ -1331,7 +1335,7 @@ future<> repair::user_requested_repair_task_impl::run() {
auto normal_nodes = germs->get().get_token_metadata().get_normal_token_owners();
participants = std::list<locator::host_id>(normal_nodes.begin(), normal_nodes.end());
} else {
- participants = get_hosts_participating_in_repair(germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
+ participants = get_hosts_participating_in_repair(_gossiper, germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
}
auto [hints_batchlog_flushed, flush_time] = rs.flush_hints(id, keyspace, cfs, ignore_nodes).get();

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:52 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 32 +++++++++++++++++++++-----------
1 file changed, 21 insertions(+), 11 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index d4627a85867..a0628a8e9ba 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2629,19 +2629,18 @@ future<> storage_service::on_join(gms::inet_address endpoint, gms::endpoint_stat

future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_state_ptr state, gms::permit_id pid) {
const auto& tm = get_token_metadata();
- const auto tm_host_id_opt = tm.get_host_id_if_known(endpoint);
- slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, tm_host_id_opt, pid);
+ const auto host_id = state->get_host_id();
+ slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid);
const auto* node = tm.get_topology().find_node(endpoint);
if (node && node->is_member()) {
co_await notify_up(endpoint);
} else if (raft_topology_change_enabled()) {
slogger.debug("ignore on_alive since topology changes are using raft and "
- "endpoint {}/{} is not a topology member", endpoint, tm_host_id_opt);
+ "endpoint {}/{} is not a topology member", endpoint, host_id);
} else {
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
const auto dc_rack = get_dc_rack_for(endpoint);
- const auto host_id = _gossiper.get_host_id(endpoint);
tmptr->update_host_id(host_id, endpoint);
tmptr->update_topology(host_id, dc_rack);
co_await replicate_to_all_cores(std::move(tmptr));
@@ -2732,15 +2731,26 @@ future<> storage_service::on_remove(gms::inet_address endpoint, gms::permit_id p
co_return;
}

+ locator::host_id host_id;
+
+ try {
+ // It seam gossiper does not check for endpoint existance before calling the callback
+ // so the lookpu may fail, but there is nothing to do in this case.
+ host_id = _gossiper.get_host_id(endpoint);
+ } catch (...) {
+ co_return;
+ }
+
+ // We should handle the case when the host id is mapped to a different address.
+ // This could happen when an address for the host id changes and the callback here is called
+ // with the old ip. We should just skip the remove in that case.
+ if (_address_map.get(host_id) != endpoint) {
+ co_return;
+ }
+
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
- // We should handle the case when we aren't able to find endpoint -> ip mapping in token_metadata.
- // This could happen e.g. when the new endpoint has bigger generation in handle_state_normal - the code
- // in handle_state_normal will remap host_id to the new IP and we won't find
- // old IP here. We should just skip the remove in that case.
- if (const auto host_id = tmptr->get_host_id_if_known(endpoint); host_id) {
- tmptr->remove_endpoint(*host_id);
- }
+ tmptr->remove_endpoint(host_id);
co_await update_topology_change_info(tmptr, ::format("on_remove {}", endpoint));
co_await replicate_to_all_cores(std::move(tmptr));
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:52 AMJan 13
to scylladb-dev@googlegroups.com
locator/util.hh already has get_range_to_address_map which is exactly
like the one in the storage_service. So remove the later one and use the
former instead.
---
locator/util.hh | 5 +++++
service/storage_service.hh | 11 -----------
locator/util.cc | 2 +-
service/storage_service.cc | 24 +-----------------------
4 files changed, 7 insertions(+), 35 deletions(-)

diff --git a/locator/util.hh b/locator/util.hh
index e85c04048ea..3a3360d3d94 100644
--- a/locator/util.hh
+++ b/locator/util.hh
@@ -9,6 +9,9 @@
#pragma once

#include "dht/token_range_endpoints.hh"
+#include "dht/i_partitioner_fwd.hh"
+#include "inet_address_vectors.hh"
+#include "locator/abstract_replication_strategy.hh"

namespace replica {
class database;
@@ -20,4 +23,6 @@ namespace gms {

namespace locator {
future<std::vector<dht::token_range_endpoints>> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false);
+ future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(
+ locator::effective_replication_map_ptr erm, const std::vector<token>& sorted_tokens);
}
\ No newline at end of file
diff --git a/service/storage_service.hh b/service/storage_service.hh
index edf279b497e..e72881cf1d4 100644
--- a/service/storage_service.hh
+++ b/service/storage_service.hh
@@ -427,8 +427,6 @@ class storage_service : public service::migration_listener, public gms::i_endpoi

public:
future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(locator::effective_replication_map_ptr erm) const;
- future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(locator::effective_replication_map_ptr erm,
- const std::vector<token>& sorted_tokens) const;

/**
* The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
@@ -465,15 +463,6 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
*/
future<std::map<token, inet_address>> get_tablet_to_endpoint_map(table_id table);

- /**
- * Construct the range to endpoint mapping based on the true view
- * of the world.
- * @param ranges
- * @return mapping of ranges to the replicas responsible for them.
- */
- future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> construct_range_to_endpoint_map(
- locator::effective_replication_map_ptr erm,
- const dht::token_range_vector& ranges) const;
public:
virtual future<> on_join(gms::inet_address endpoint, gms::endpoint_state_ptr ep_state, gms::permit_id) override;
/*
diff --git a/locator/util.cc b/locator/util.cc
index fc1cba53942..d7f8768cdae 100644
--- a/locator/util.cc
+++ b/locator/util.cc
@@ -47,7 +47,7 @@ get_all_ranges(const std::vector<token>& sorted_tokens) {
}

// Caller is responsible to hold token_metadata valid until the returned future is resolved
-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
get_range_to_address_map(locator::effective_replication_map_ptr erm,
const std::vector<token>& sorted_tokens) {
co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
diff --git a/service/storage_service.cc b/service/storage_service.cc
index a0628a8e9ba..1b8abd6366d 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2295,14 +2295,7 @@ future<> storage_service::bootstrap(std::unordered_set<token>& bootstrap_tokens,

future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const {
- return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
-}
-
-// Caller is responsible to hold token_metadata valid until the returned future is resolved
-future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
-storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm,
- const std::vector<token>& sorted_tokens) const {
- co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
+ return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
}

future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) {
@@ -5264,21 +5257,6 @@ storage_service::describe_ring_for_table(const sstring& keyspace_name, const sst
co_return ranges;
}

-future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
-storage_service::construct_range_to_endpoint_map(
- locator::effective_replication_map_ptr erm,
- const dht::token_range_vector& ranges) const {
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> res;
- res.reserve(ranges.size());
- for (auto r : ranges) {
- res[r] = erm->get_natural_endpoints(
- r.end() ? r.end()->value() : dht::maximum_token());
- co_await coroutine::maybe_yield();
- }
- co_return res;
-}
-
-
std::map<token, inet_address> storage_service::get_token_to_endpoint_map() {
const auto& tm = get_token_metadata();
std::map<token, inet_address> result;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:52 AMJan 13
to scylladb-dev@googlegroups.com
The state always has host id now, so there is no point to looks it up in
the token metadata.
---
db/virtual_tables.cc | 20 ++++++++------------
1 file changed, 8 insertions(+), 12 deletions(-)

diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc
index b42be729489..b9b3e0d5f0c 100644
--- a/db/virtual_tables.cc
+++ b/db/virtual_tables.cc
@@ -75,7 +75,7 @@ class cluster_status_table : public memtable_filling_virtual_table {
std::vector<frozen_mutation> muts;
muts.reserve(gossiper.num_endpoints());

- gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state&) {
+ gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state& eps) {
static thread_local auto s = build_schema();
mutation m(s, partition_key::from_single_value(*s, data_value(endpoint).serialize_nonnull()));
row& cr = m.partition().clustered_row(*schema(), clustering_key::make_empty()).cells();
@@ -86,24 +86,20 @@ class cluster_status_table : public memtable_filling_virtual_table {
}
set_cell(cr, "load", gossiper.get_application_state_value(endpoint, gms::application_state::LOAD));

- auto hostid = tm.get_host_id_if_known(endpoint);
- if (hostid) {
- if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) {
- set_cell(cr, "status", boost::to_upper_copy<std::string>(fmt::format("{}", ss.get_node_state(*hostid))));
- }
- set_cell(cr, "host_id", hostid->uuid());
+ auto hostid = eps.get_host_id();
+ if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) {
+ set_cell(cr, "status", boost::to_upper_copy<std::string>(fmt::format("{}", ss.get_node_state(hostid))));
}
+ set_cell(cr, "host_id", hostid.uuid());

- if (hostid) {
- sstring dc = tm.get_topology().get_location(endpoint).dc;
- set_cell(cr, "dc", dc);
- }
+ sstring dc = tm.get_topology().get_location(endpoint).dc;
+ set_cell(cr, "dc", dc);

if (ownership.contains(endpoint)) {
set_cell(cr, "owns", ownership[endpoint]);
}

- set_cell(cr, "tokens", int32_t(hostid ? tm.get_tokens(*hostid).size() : 0));
+ set_cell(cr, "tokens", int32_t(tm.get_tokens(hostid).size()));

muts.push_back(freeze(std::move(m)));
});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:53 AMJan 13
to scylladb-dev@googlegroups.com
Now that we have gossiper::get_endpoint_state_ptr that works on host ids
there is no need to translate id to ip at all.
---
service/topology_coordinator.cc | 35 ++++++++++++++-------------------
1 file changed, 15 insertions(+), 20 deletions(-)

diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc
index f2b3a5a68a1..e71aecbb9ed 100644
--- a/service/topology_coordinator.cc
+++ b/service/topology_coordinator.cc
@@ -3055,11 +3055,11 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
rtlogger.info("building initial raft topology state and CDC generation");
guard = co_await start_operation();

- auto get_application_state = [&] (locator::host_id host_id, gms::inet_address ep, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
+ auto get_application_state = [&] (locator::host_id host_id, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
const auto it = epmap.find(app_state);
if (it == epmap.end()) {
- throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{}: application state {} is missing in gossip",
- host_id, ep, app_state));
+ throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}: application state {} is missing in gossip",
+ host_id, app_state));
}
// it's versioned_value::value(), not std::optional::value() - it does not throw
return it->second.value();
@@ -3067,17 +3067,13 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {

// Create a new CDC generation
auto get_sharding_info_for_host_id = [&] (locator::host_id host_id) -> std::pair<size_t, uint8_t> {
- const auto ep = tmptr->get_endpoint_for_host_id_if_known(host_id);
- if (!ep) {
- throw std::runtime_error(format("IP of node with ID {} is not known", host_id));
- }
- const auto eptr = _gossiper.get_endpoint_state_ptr(*ep);
+ const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
if (!eptr) {
- throw std::runtime_error(format("no gossiper endpoint state for node {}/{}", host_id, *ep));
+ throw std::runtime_error(format("no gossiper endpoint state for node {}", host_id));
}
const auto& epmap = eptr->get_application_state_map();
- const auto shard_count = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::SHARD_COUNT));
- const auto ignore_msb = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::IGNORE_MSB_BITS));
+ const auto shard_count = std::stoi(get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT));
+ const auto ignore_msb = std::stoi(get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS));
return std::make_pair<size_t, uint8_t>(shard_count, ignore_msb);
};
auto [cdc_gen_uuid, guard_, mutation] = co_await prepare_and_broadcast_cdc_generation_data(tmptr, std::move(guard), std::nullopt, get_sharding_info_for_host_id);
@@ -3094,23 +3090,22 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
}

const auto& host_id = node.get().host_id();
- const auto& ep = node.get().endpoint();
- const auto eptr = _gossiper.get_endpoint_state_ptr(ep);
+ const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
if (!eptr) {
- throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{} as gossip contains no data for it", host_id, ep));
+ throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {} as gossip contains no data for it", host_id));
}

const auto& epmap = eptr->get_application_state_map();

- const auto datacenter = get_application_state(host_id, ep, epmap, gms::application_state::DC);
- const auto rack = get_application_state(host_id, ep, epmap, gms::application_state::RACK);
+ const auto datacenter = get_application_state(host_id, epmap, gms::application_state::DC);
+ const auto rack = get_application_state(host_id, epmap, gms::application_state::RACK);
const auto tokens_v = tmptr->get_tokens(host_id);
const std::unordered_set<dht::token> tokens(tokens_v.begin(), tokens_v.end());
- const auto release_version = get_application_state(host_id, ep, epmap, gms::application_state::RELEASE_VERSION);
+ const auto release_version = get_application_state(host_id, epmap, gms::application_state::RELEASE_VERSION);
const auto num_tokens = tokens.size();
- const auto shard_count = get_application_state(host_id, ep, epmap, gms::application_state::SHARD_COUNT);
- const auto ignore_msb = get_application_state(host_id, ep, epmap, gms::application_state::IGNORE_MSB_BITS);
- const auto supported_features_s = get_application_state(host_id, ep, epmap, gms::application_state::SUPPORTED_FEATURES);
+ const auto shard_count = get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT);
+ const auto ignore_msb = get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS);
+ const auto supported_features_s = get_application_state(host_id, epmap, gms::application_state::SUPPORTED_FEATURES);
const auto supported_features = gms::feature_service::to_feature_set(supported_features_s);

if (enabled_features.empty()) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:53 AMJan 13
to scylladb-dev@googlegroups.com
The function is called by RESful API so has to return ips for backwards
compatibility, but internally we can use host ids as long as possible
and convert to ips just before returning. This also drops usage of ip
based erm function which we want to get rid of.
---
service/storage_service.cc | 7 +++++--
1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 1b8abd6366d..81361a76a86 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -7314,10 +7314,13 @@ storage_service::get_natural_endpoints(const sstring& keyspace,
partition_key pk = partition_key::from_nodetool_style_string(schema, key);
dht::token token = schema->get_partitioner().get_token(*schema, pk.view());
const auto& ks = _db.local().find_keyspace(keyspace);
+ host_id_vector_replica_set replicas;
if (ks.uses_tablets()) {
- return table.get_effective_replication_map()->get_natural_endpoints(token);
+ replicas = table.get_effective_replication_map()->get_natural_replicas(token);
+ } else {
+ replicas = ks.get_vnode_effective_replication_map()->get_natural_replicas(token);
}
- return ks.get_vnode_effective_replication_map()->get_natural_endpoints(token);
+ return replicas | std::views::transform([&] (locator::host_id id) { return _address_map.get(id); }) | std::ranges::to<inet_address_vector_replica_set>();
}

future<> endpoint_lifecycle_notifier::notify_down(gms::inet_address endpoint) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:55 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 15 +++++++++------
1 file changed, 9 insertions(+), 6 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 512126ff2ec..d4627a85867 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -4455,11 +4455,11 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
// and waits for ring_delay. It's possible the node being decommissioned might
// die after it has sent this notification. If this happens, the node would
// have already been removed from this token_metadata, so we wouldn't find it here.
- const auto node_id = tmptr->get_host_id_if_known(node);
- slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
- if (node_id) {
- tmptr->del_leaving_endpoint(*node_id);
- }
+ try {
+ const auto node_id = _gossiper.get_host_id(node);
+ slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
+ tmptr->del_leaving_endpoint(node_id);
+ } catch (...) {}
}
return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes));
});
@@ -4475,7 +4475,10 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
check_again = false;
for (auto& node : req.leaving_nodes) {
auto tmptr = get_token_metadata_ptr();
- const auto host_id = tmptr->get_host_id_if_known(node);
+ std::optional<locator::host_id> host_id;
+ try {
+ host_id = _gossiper.get_host_id(node);
+ } catch(...) {};
if (host_id && tmptr->is_normal_token_owner(*host_id)) {
check_again = true;
if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:55 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index af82b21a524..512126ff2ec 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -7199,7 +7199,7 @@ future<> storage_service::force_remove_completion() {
auto leaving = tm.get_leaving_endpoints();
slogger.warn("Removal not confirmed, Leaving={}", leaving);
for (auto host_id : leaving) {
- const auto endpoint = tm.get_endpoint_for_host_id_if_known(host_id);
+ const auto endpoint = ss._address_map.find(host_id);
if (!endpoint) {
slogger.warn("No endpoint is found for host_id {}", host_id);
continue;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:56 AMJan 13
to scylladb-dev@googlegroups.com
Also use gossiper to translate instead of taken_metadata since we want
to get rid of ip base APIs there.
---
db/hints/manager.hh | 2 +-
db/hints/manager.cc | 3 ++-
service/storage_proxy.cc | 6 ++----
3 files changed, 5 insertions(+), 6 deletions(-)

diff --git a/db/hints/manager.hh b/db/hints/manager.hh
index afe7e96648f..7f075714f09 100644
--- a/db/hints/manager.hh
+++ b/db/hints/manager.hh
@@ -171,7 +171,7 @@ class manager {
void register_metrics(const sstring& group_name);
future<> start(shared_ptr<const gms::gossiper> gossiper_ptr);
future<> stop();
- bool store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
+ bool store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
tracing::trace_state_ptr tr_state) noexcept;

/// \brief Changes the host_filter currently used, stopping and starting endpoint_managers relevant to the new host_filter.
diff --git a/db/hints/manager.cc b/db/hints/manager.cc
index 7cdb515352b..015d492b788 100644
--- a/db/hints/manager.cc
+++ b/db/hints/manager.cc
@@ -430,9 +430,10 @@ bool manager::have_ep_manager(const std::variant<locator::host_id, gms::inet_add
return _hint_directory_manager.has_mapping(std::get<gms::inet_address>(ep));
}

-bool manager::store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
+bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
tracing::trace_state_ptr tr_state) noexcept
{
+ auto ip = _gossiper_anchor->get_address_map().get(host_id);
if (utils::get_local_injector().enter("reject_incoming_hints")) {
manager_logger.debug("Rejecting a hint to {} / {} due to an error injection", host_id, ip);
++_stats.dropped;
diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index 247283f153d..37676e64f01 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -1243,8 +1243,7 @@ class per_destination_mutation : public mutation_holder {
tracing::trace_state_ptr tr_state) override {
auto m = _mutations[hid];
if (m) {
- const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid);
- return hm.store_hint(hid, ep, _schema, std::move(m), tr_state);
+ return hm.store_hint(hid, _schema, std::move(m), tr_state);
} else {
return false;
}
@@ -1302,8 +1301,7 @@ class shared_mutation : public mutation_holder {
}
virtual bool store_hint(db::hints::manager& hm, locator::host_id hid, locator::effective_replication_map_ptr ermptr,
tracing::trace_state_ptr tr_state) override {
- const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid);
- return hm.store_hint(hid, ep, _schema, _mutation, tr_state);
+ return hm.store_hint(hid, _schema, _mutation, tr_state);
}
virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
tracing::trace_state_ptr tr_state, db::per_partition_rate_limit::info rate_limit_info,
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:56 AMJan 13
to scylladb-dev@googlegroups.com
Move the test to work on host ids. IPs will be dropped eventually.
---
test/boost/network_topology_strategy_test.cc | 32 ++++++++++----------
1 file changed, 16 insertions(+), 16 deletions(-)

diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc
index d80ddf27a87..142e0e4f598 100644
--- a/test/boost/network_topology_strategy_test.cc
+++ b/test/boost/network_topology_strategy_test.cc
@@ -49,7 +49,7 @@ struct ring_point {
host_id id = host_id::create_random_id();
};

-void print_natural_endpoints(double point, const inet_address_vector_replica_set v) {
+void print_natural_endpoints(double point, const host_id_vector_replica_set v) {
testlog.debug("Natural endpoints for a token {}:", point);
std::string str;
std::ostringstream strm(str);
@@ -103,11 +103,11 @@ void strategy_sanity_check(
void endpoints_check(
replication_strategy_ptr ars_ptr,
const token_metadata_ptr& tm,
- const inet_address_vector_replica_set& endpoints,
+ const host_id_vector_replica_set& endpoints,
const locator::topology& topo,
bool strict_dc_rf = false) {

- auto&& nodes_per_dc = tm->get_datacenter_token_owners_ips();
+ auto&& nodes_per_dc = tm->get_datacenter_token_owners();
const network_topology_strategy* nts_ptr =
dynamic_cast<const network_topology_strategy*>(ars_ptr.get());

@@ -122,7 +122,7 @@ void endpoints_check(
BOOST_CHECK_LE(total_rf, ars_ptr->get_replication_factor(*tm));

// Check the uniqueness
- std::unordered_set<inet_address> ep_set(endpoints.begin(), endpoints.end());
+ std::unordered_set<locator::host_id> ep_set(endpoints.begin(), endpoints.end());
BOOST_CHECK_EQUAL(endpoints.size(), ep_set.size());

// Check the per-DC RF
@@ -165,7 +165,7 @@ void full_ring_check(const std::vector<ring_point>& ring_points,
for (auto& rp : ring_points) {
double cur_point1 = rp.point - 0.5;
token t1(tests::d2t(cur_point1 / ring_points.size()));
- auto endpoints1 = erm->get_natural_endpoints(t1);
+ auto endpoints1 = erm->get_natural_replicas(t1);

endpoints_check(ars_ptr, tmptr, endpoints1, topo);

@@ -178,7 +178,7 @@ void full_ring_check(const std::vector<ring_point>& ring_points,
//
double cur_point2 = rp.point - 0.2;
token t2(tests::d2t(cur_point2 / ring_points.size()));
- auto endpoints2 = erm->get_natural_endpoints(t2);
+ auto endpoints2 = erm->get_natural_replicas(t2);

endpoints_check(ars_ptr, tmptr, endpoints2, topo);
check_ranges_are_sorted(erm, rp.id).get();
@@ -192,17 +192,17 @@ void full_ring_check(const tablet_map& tmap,
auto& tm = *tmptr;
const auto& topo = tm.get_topology();

- auto to_endpoint_set = [&] (const tablet_replica_set& replicas) {
- inet_address_vector_replica_set result;
+ auto to_replica_set = [&] (const tablet_replica_set& replicas) {
+ host_id_vector_replica_set result;
result.reserve(replicas.size());
for (auto&& replica : replicas) {
- result.emplace_back(tm.get_endpoint_for_host_id(replica.host));
+ result.emplace_back(replica.host);
}
return result;
};

for (tablet_id tb : tmap.tablet_ids()) {
- endpoints_check(rs_ptr, tmptr, to_endpoint_set(tmap.get_tablet_info(tb).replicas), topo, true);
+ endpoints_check(rs_ptr, tmptr, to_replica_set(tmap.get_tablet_info(tb).replicas), topo, true);
}
}

@@ -671,7 +671,7 @@ static size_t get_replication_factor(const sstring& dc,

static bool has_sufficient_replicas(const sstring& dc,
const std::unordered_map<sstring, std::unordered_set<host_id>>& dc_replicas,
- const std::unordered_map<sstring, std::unordered_set<inet_address>>& all_endpoints,
+ const std::unordered_map<sstring, std::unordered_set<host_id>>& all_endpoints,
const std::unordered_map<sstring, size_t>& datacenters) noexcept {
auto dc_replicas_it = dc_replicas.find(dc);
if (dc_replicas_it == dc_replicas.end()) {
@@ -689,7 +689,7 @@ static bool has_sufficient_replicas(const sstring& dc,

static bool has_sufficient_replicas(
const std::unordered_map<sstring, std::unordered_set<host_id>>& dc_replicas,
- const std::unordered_map<sstring, std::unordered_set<inet_address>>& all_endpoints,
+ const std::unordered_map<sstring, std::unordered_set<host_id>>& all_endpoints,
const std::unordered_map<sstring, size_t>& datacenters) noexcept {

for (auto& dc : datacenters | std::views::keys) {
@@ -740,16 +740,16 @@ static locator::host_id_set calculate_natural_endpoints(
// the token-owning members of a DC
//
const std::unordered_map<sstring,
- std::unordered_set<inet_address>>
- all_endpoints = tm.get_datacenter_token_owners_ips();
+ std::unordered_set<locator::host_id>>
+ all_endpoints = tm.get_datacenter_token_owners();
//
// all racks (with non-token owners filtered out) in a DC so we can check
// when we have exhausted all racks in a DC
//
const std::unordered_map<sstring,
std::unordered_map<sstring,
- std::unordered_set<inet_address>>>
- racks = tm.get_datacenter_racks_token_owners_ips();
+ std::unordered_set<host_id>>>
+ racks = tm.get_datacenter_racks_token_owners();

// not aware of any cluster members
SCYLLA_ASSERT(!all_endpoints.empty() && !racks.empty());
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:58 AMJan 13
to scylladb-dev@googlegroups.com
Replace operation is special though. In case of replacing with the same
IP the gossiper will not have the mapping, and node_ops RPC
unfortunately does not send host id of a replaced node. For replace we
consult peers table instead to find the old owner of the IP. A node that
is replacing (the coordinator of the replace) will not have it though,
but luckily it is not needed since it updates metadata during
join_topology() anyway. The only thing that is missing there is
add_replacing_endpoint() call which the patch adds.
---
service/storage_service.cc | 27 ++++++++++++++++++---------
1 file changed, 18 insertions(+), 9 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index ba29dbd5c8d..c0cc66feff4 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -1545,6 +1545,7 @@ future<> storage_service::join_topology(sharded<db::system_distributed_keyspace>
tmptr->update_topology(ri->host_id, std::move(ri->dc_rack), locator::node::state::being_replaced);
co_await tmptr->update_normal_tokens(bootstrap_tokens, ri->host_id);
tmptr->update_host_id(ri->host_id, *replace_address);
+ tmptr->add_replacing_endpoint(ri->host_id, tmptr->get_my_id());

replaced_host_id = ri->host_id;

@@ -4406,7 +4407,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
mutate_token_metadata([coordinator, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
for (auto& node : req.leaving_nodes) {
slogger.info("removenode[{}]: Added node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
- tmptr->add_leaving_endpoint(tmptr->get_host_id(node));
+ tmptr->add_leaving_endpoint(_gossiper.get_host_id(node));
}
return update_topology_change_info(tmptr, ::format("removenode {}", req.leaving_nodes));
}).get();
@@ -4414,7 +4415,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
return mutate_token_metadata([this, coordinator, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
for (auto& node : req.leaving_nodes) {
slogger.info("removenode[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
- tmptr->del_leaving_endpoint(tmptr->get_host_id(node));
+ tmptr->del_leaving_endpoint(_gossiper.get_host_id(node));
}
return update_topology_change_info(tmptr, ::format("removenode {}", req.leaving_nodes));
});
@@ -4438,7 +4439,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
_repair.local().removenode_with_repair(get_token_metadata_ptr(), _gossiper.get_host_id(node), ops).get();
} else {
slogger.info("removenode[{}]: Started to sync data for removing node={} using stream, coordinator={}", req.ops_uuid, node, coordinator);
- removenode_with_stream(get_token_metadata().get_host_id(node), topo_guard, as).get();
+ removenode_with_stream(_gossiper.get_host_id(node), topo_guard, as).get();
}
}
} else if (req.cmd == node_ops_cmd::removenode_abort) {
@@ -4454,7 +4455,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
mutate_token_metadata([coordinator, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
for (auto& node : req.leaving_nodes) {
slogger.info("decommission[{}]: Added node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
- tmptr->add_leaving_endpoint(tmptr->get_host_id(node));
+ tmptr->add_leaving_endpoint(_gossiper.get_host_id(node));
}
return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes));
}).get();
@@ -4531,8 +4532,13 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
}
auto existing_node = req.replace_nodes.begin()->first;
auto replacing_node = req.replace_nodes.begin()->second;
- mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
- const auto existing_node_id = tmptr->get_host_id(existing_node);
+ auto existing_node_id = _sys_ks.local().load_host_ids().get()[existing_node];
+ mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
+ if (is_me(*coordinator_host_id)) {
+ // A coordinor already updated token metadata in join_topology()
+ return make_ready_future<>();
+ }
+
const auto replacing_node_id = *coordinator_host_id;
slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
@@ -4553,9 +4559,12 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
return make_ready_future<>();
}).get();
auto ignore_nodes = std::move(req.ignore_nodes);
- node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable {
- return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
- const auto existing_node_id = tmptr->get_host_id(existing_node);
+ node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, req = std::move(req)] () mutable {
+ return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, existing_node_id, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
+ if (is_me(*coordinator_host_id)) {
+ // No need to cancel replace operation on a node replacing node since it will be killed anyway
+ return make_ready_future<>();
+ }
const auto replacing_node_id = *coordinator_host_id;
slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:58 AMJan 13
to scylladb-dev@googlegroups.com
When logger outputs a range it already does join, so no other join is
needed.
---
service/storage_service.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 6e687f9d03e..28bc19f2427 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -7429,7 +7429,7 @@ future<> storage_service::wait_for_normal_state_handled_on_boot() {
static auto fmt_nodes_with_statuses = [this] (const auto& eps) {
return eps | std::views::transform([this] (const auto& ep) {
return ::format("({}, status={})", ep, _gossiper.get_gossip_status(ep));
- }) | std::views::join_with(',');
+ });
};

slogger.info("Started waiting for normal state handlers to finish");
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:59 AMJan 13
to scylladb-dev@googlegroups.com
Currently the entry is removed only if ip is not used by any normal or
transitioning node. This is done to not remove a wrong entry that just
happen to use the same ip, but the same can be achieved by checking host
id in the entry.
---
service/storage_service.cc | 16 +---------------
1 file changed, 1 insertion(+), 15 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 374ea66cef6..6e687f9d03e 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -420,20 +420,6 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
}
};

- auto get_used_ips = [&, used_ips = std::optional<std::unordered_set<inet_address>>{}]() mutable
- -> const std::unordered_set<inet_address>&
- {
- if (!used_ips) {
- used_ips.emplace();
- for (const auto& [sid, rs]: boost::range::join(t.normal_nodes, t.transition_nodes)) {
- if (const auto used_ip = am.find(locator::host_id{sid.uuid()})) {
- used_ips->insert(*used_ip);
- }
- }
- }
- return *used_ips;
- };
-
using host_id_to_ip_map_t = std::unordered_map<locator::host_id, gms::inet_address>;
auto get_host_id_to_ip_map = [&, map = std::optional<host_id_to_ip_map_t>{}]() mutable -> future<const host_id_to_ip_map_t*> {
if (!map.has_value()) {
@@ -456,7 +442,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
auto remove_ip = [&](inet_address ip, locator::host_id host_id, bool notify) -> future<> {
sys_ks_futures.push_back(_sys_ks.local().remove_endpoint(ip));

- if (_gossiper.get_endpoint_state_ptr(ip) && !get_used_ips().contains(ip)) {
+ if (const auto ep = _gossiper.get_endpoint_state_ptr(ip); ep && ep->get_host_id() == host_id) {
co_await _gossiper.force_remove_endpoint(ip, gms::null_permit_id);
if (notify) {
nodes_to_notify.left.push_back({ip, host_id});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:59 AMJan 13
to scylladb-dev@googlegroups.com
Use address map to translate id to ip instead. We want to drop ips from token_metadata.
---
cdc/generation.cc | 2 +-
node_ops/node_ops_ctl.cc | 2 +-
service/storage_service.cc | 29 +++++++++++------------------
3 files changed, 13 insertions(+), 20 deletions(-)

diff --git a/cdc/generation.cc b/cdc/generation.cc
index 8ace3cea7e2..22645a7c27c 100644
--- a/cdc/generation.cc
+++ b/cdc/generation.cc
@@ -402,7 +402,7 @@ future<cdc::generation_id> generation_service::legacy_make_new_generation(const
throw std::runtime_error(
format("Can't find endpoint for token {}", end));
}
- const auto ep = tmptr->get_endpoint_for_host_id(*endpoint);
+ const auto ep = _gossiper.get_address_map().get(*endpoint);
auto sc = get_shard_count(ep, _gossiper);
return {sc > 0 ? sc : 1, get_sharding_ignore_msb(ep, _gossiper)};
}
diff --git a/node_ops/node_ops_ctl.cc b/node_ops/node_ops_ctl.cc
index 1eaaf2e333e..64a1ab36186 100644
--- a/node_ops/node_ops_ctl.cc
+++ b/node_ops/node_ops_ctl.cc
@@ -143,7 +143,7 @@ future<> node_ops_ctl::abort_on_error(node_ops_cmd cmd, std::exception_ptr ex) n
future<> node_ops_ctl::send_to_all(node_ops_cmd cmd) {
req.cmd = cmd;
req.ignore_nodes = ignore_nodes |
- std::views::transform([&] (locator::host_id id) { return tmptr->get_endpoint_for_host_id(id); }) |
+ std::views::transform([&] (locator::host_id id) { return ss.gossiper().get_address_map().get(id); }) |
std::ranges::to<std::list>();
sstring op_desc = ::format("{}", cmd);
nlogger.info("{}[{}]: Started {}", desc, uuid(), req);
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 74b1b1358c9..374ea66cef6 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -348,16 +348,6 @@ static future<> set_gossip_tokens(gms::gossiper& g,
);
}

-static std::unordered_map<token, gms::inet_address> get_token_to_endpoint(const locator::token_metadata& tm) {
- const auto& map = tm.get_token_to_endpoint();
- std::unordered_map<token, gms::inet_address> result;
- result.reserve(map.size());
- for (const auto [t, id]: map) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
- }
- return result;
-}
-
/*
* The helper waits for two things
* 1) for schema agreement
@@ -1956,7 +1946,7 @@ future<> storage_service::join_topology(sharded<db::system_distributed_keyspace>
for (auto token : bootstrap_tokens) {
auto existing = tmptr->get_endpoint(token);
if (existing) {
- auto eps = _gossiper.get_endpoint_state_ptr(tmptr->get_endpoint_for_host_id(*existing));
+ auto eps = _gossiper.get_endpoint_state_ptr(*existing);
if (eps && eps->get_update_timestamp() > gms::gossiper::clk::now() - delay) {
throw std::runtime_error("Cannot replace a live node...");
}
@@ -2460,7 +2450,10 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
// token_to_endpoint_map is used to track the current token owners for the purpose of removing replaced endpoints.
// when any token is replaced by a new owner, we track the existing owner in `candidates_for_removal`
// and eventually, if any candidate for removal ends up owning no tokens, it is removed from token_metadata.
- std::unordered_map<token, inet_address> token_to_endpoint_map = get_token_to_endpoint(get_token_metadata());
+ std::unordered_map<token, inet_address> token_to_endpoint_map = get_token_metadata().get_token_to_endpoint() |
+ std::views::transform([this] (auto& e) {
+ return std::make_pair(e.first, _address_map.get(e.second));
+ }) | std::ranges::to<std::unordered_map>();
std::unordered_set<inet_address> candidates_for_removal;

// Here we convert endpoint tokens from gossiper to owned_tokens, which will be assigned as a new
@@ -2579,7 +2572,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
const auto& tm = get_token_metadata();
auto ver = tm.get_ring_version();
for (auto& x : tm.get_token_to_endpoint()) {
- slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, tm.get_endpoint_for_host_id(x.second), x.second);
+ slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, _address_map.get(x.second), x.second);
}
}
_normal_state_handled_on_boot.insert(endpoint);
@@ -3464,7 +3457,7 @@ storage_service::prepare_replacement_info(std::unordered_set<gms::inet_address>
}

future<std::map<gms::inet_address, float>> storage_service::get_ownership() {
- return run_with_no_api_lock([] (storage_service& ss) {
+ return run_with_no_api_lock([this] (storage_service& ss) {
const auto& tm = ss.get_token_metadata();
auto token_map = dht::token::describe_ownership(tm.sorted_tokens());
// describeOwnership returns tokens in an unspecified order, let's re-order them
@@ -3472,7 +3465,7 @@ future<std::map<gms::inet_address, float>> storage_service::get_ownership() {
for (auto entry : token_map) {
locator::host_id id = tm.get_endpoint(entry.first).value();
auto token_ownership = entry.second;
- ownership[tm.get_endpoint_for_host_id(id)] += token_ownership;
+ ownership[_address_map.get(id)] += token_ownership;
}
return ownership;
});
@@ -5281,10 +5274,10 @@ std::map<token, inet_address> storage_service::get_token_to_endpoint_map() {
const auto& tm = get_token_metadata();
std::map<token, inet_address> result;
for (const auto [t, id]: tm.get_token_to_endpoint()) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
+ result.insert({t, _address_map.get(id)});
}
for (const auto [t, id]: tm.get_bootstrap_tokens()) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
+ result.insert({t, _address_map.get(id)});
}
return result;
}
@@ -5294,7 +5287,7 @@ future<std::map<token, inet_address>> storage_service::get_tablet_to_endpoint_ma
const auto& tmap = tm.tablets().get_tablet_map(table);
std::map<token, inet_address> result;
for (std::optional<locator::tablet_id> tid = tmap.first_tablet(); tid; tid = tmap.next_tablet(*tid)) {
- result.emplace(tmap.get_last_token(*tid), tm.get_endpoint_for_host_id(tmap.get_primary_replica(*tid).host));
+ result.emplace(tmap.get_last_token(*tid), _address_map.get(tmap.get_primary_replica(*tid).host));
co_await coroutine::maybe_yield();
}
co_return result;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:20:59 AMJan 13
to scylladb-dev@googlegroups.com
As part of changing IP address peers table is updated. If it has a new
address the update can be skipped.
---
service/storage_service.hh | 3 +++
service/storage_service.cc | 12 +++++++++++-
2 files changed, 14 insertions(+), 1 deletion(-)

diff --git a/service/storage_service.hh b/service/storage_service.hh
index 70a7eeba6ab..fff395375c1 100644
--- a/service/storage_service.hh
+++ b/service/storage_service.hh
@@ -603,6 +603,9 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
future<std::unordered_multimap<dht::token_range, locator::host_id>> get_changed_ranges_for_leaving(locator::vnode_effective_replication_map_ptr erm, locator::host_id endpoint);

future<> maybe_reconnect_to_preferred_ip(inet_address ep, inet_address local_ip);
+
+ // Return ip of the peers table entry with given host id
+ future<std::optional<gms::inet_address>> get_ip_from_peers_table(locator::host_id id);
public:

sstring get_release_version();
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 28bc19f2427..0795fc4357c 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -901,7 +901,9 @@ class storage_service::ip_address_updater: public gms::i_endpoint_state_change_s
locator::host_id id(utils::UUID(app_state_ptr->value()));
rslog.debug("ip_address_updater::on_endpoint_change({}) {} {}", ev, endpoint, id);

- auto prev_ip = _ss.get_token_metadata().get_endpoint_for_host_id_if_known(id);
+ // If id maps to different ip in peers table it needs to be updated which is done by sync_raft_topology_nodes below
+ auto peers = co_await _ss._sys_ks.local().load_host_ids();
+ std::optional<gms::inet_address> prev_ip = co_await _ss.get_ip_from_peers_table(id);
if (prev_ip == endpoint) {
co_return;
}
@@ -2639,6 +2641,14 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
}
}

+future<std::optional<gms::inet_address>> storage_service::get_ip_from_peers_table(locator::host_id id) {
+ auto peers = co_await _sys_ks.local().load_host_ids();
+ if (auto it = std::ranges::find_if(peers, [&id] (const auto& e) { return e.second == id; }); it != peers.end()) {
+ co_return it->first;
+ }
+ co_return std::nullopt;
+}
+
future<> storage_service::on_change(gms::inet_address endpoint, const gms::application_state_map& states_, gms::permit_id pid) {
// copy the states map locally since the coroutine may yield
auto states = states_;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:21:01 AMJan 13
to scylladb-dev@googlegroups.com
---
service/storage_service.cc | 54 +++++++++++++++++++-------------------
1 file changed, 27 insertions(+), 27 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index ee0310ce8a8..ba29dbd5c8d 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -4532,38 +4532,38 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
auto existing_node = req.replace_nodes.begin()->first;
auto replacing_node = req.replace_nodes.begin()->second;
mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
- const auto existing_node_id = tmptr->get_host_id(existing_node);
- const auto replacing_node_id = *coordinator_host_id;
- slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
- req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
-
- // In case of replace-with-same-ip we need to map both host_id-s
- // to the same IP. The locator::topology allows this specifically in case
- // where one node is being_replaced and another is replacing,
- // so here we adjust the state of the original node accordingly.
- // The host_id -> IP map works as usual, and IP -> host_id will map
- // IP to the being_replaced node - this is what is implied by the
- // current code. The IP will be placed in pending_endpoints and
- // excluded from normal_endpoints (maybe_remove_node_being_replaced function).
- // In handle_state_normal we'll remap the IP to the new host_id.
- tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced);
- tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing);
- tmptr->update_host_id(replacing_node_id, replacing_node);
- tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id);
+ const auto existing_node_id = tmptr->get_host_id(existing_node);
+ const auto replacing_node_id = *coordinator_host_id;
+ slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
+ req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
+
+ // In case of replace-with-same-ip we need to map both host_id-s
+ // to the same IP. The locator::topology allows this specifically in case
+ // where one node is being_replaced and another is replacing,
+ // so here we adjust the state of the original node accordingly.
+ // The host_id -> IP map works as usual, and IP -> host_id will map
+ // IP to the being_replaced node - this is what is implied by the
+ // current code. The IP will be placed in pending_endpoints and
+ // excluded from normal_endpoints (maybe_remove_node_being_replaced function).
+ // In handle_state_normal we'll remap the IP to the new host_id.
+ tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced);
+ tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing);
+ tmptr->update_host_id(replacing_node_id, replacing_node);
+ tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id);
return make_ready_future<>();
}).get();
auto ignore_nodes = std::move(req.ignore_nodes);
node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable {
return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
- const auto existing_node_id = tmptr->get_host_id(existing_node);
- const auto replacing_node_id = *coordinator_host_id;
- slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
- req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
-
- tmptr->del_replacing_endpoint(existing_node_id);
- const auto dc_rack = get_dc_rack_for(replacing_node_id);
- tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal);
- tmptr->remove_endpoint(replacing_node_id);
+ const auto existing_node_id = tmptr->get_host_id(existing_node);
+ const auto replacing_node_id = *coordinator_host_id;
+ slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
+ req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);
+
+ tmptr->del_replacing_endpoint(existing_node_id);
+ const auto dc_rack = get_dc_rack_for(replacing_node_id);
+ tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal);
+ tmptr->remove_endpoint(replacing_node_id);
return update_topology_change_info(tmptr, ::format("replace {}", req.replace_nodes));
});
});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:21:01 AMJan 13
to scylladb-dev@googlegroups.com
---
service/storage_service.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 39c1e1274a6..4013fc232c8 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2626,7 +2626,7 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
const auto& tm = get_token_metadata();
const auto host_id = state->get_host_id();
slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid);
- const auto* node = tm.get_topology().find_node(endpoint);
+ const auto* node = tm.get_topology().find_node(host_id);
if (node && node->is_member()) {
co_await notify_up(endpoint);
} else if (raft_topology_change_enabled()) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:28 AMJan 13
to scylladb-dev@googlegroups.com
We already use if to translate reply_to, so do it for consistency and to
drop ip based API usage.
---
service/storage_proxy.cc | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index 19e92fb0d39..9cd105b0438 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -104,13 +104,13 @@ namespace {


template<size_t N>
-utils::small_vector<locator::host_id, N> addr_vector_to_id(const locator::topology& topo, const utils::small_vector<gms::inet_address, N>& set) {
+utils::small_vector<locator::host_id, N> addr_vector_to_id(const gms::gossiper& g, const utils::small_vector<gms::inet_address, N>& set) {
return set | std::views::transform([&] (gms::inet_address ip) {
- auto* node = topo.find_node(ip);
- if (!node) {
+ try {
+ return g.get_host_id(ip);
+ } catch (...) {
on_internal_error(slogger, fmt::format("addr_vector_to_id cannot map {} to host id", ip));
}
- return node->host_id();
}) | std::ranges::to<utils::small_vector<locator::host_id, N>>();
}

@@ -586,7 +586,7 @@ class storage_proxy::remote {
}

auto reply_to_host_id = reply_to_id ? *reply_to_id : _gossiper.get_host_id(reply_to);
- auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_sp._shared_token_metadata.get()->get_topology(), forward);
+ auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_gossiper, forward);

if (reply_to_id) {
_gossiper.get_mutable_address_map().opt_add_entry(reply_to_host_id, reply_to);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:29 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/token_metadata.hh | 9 ---
locator/token_metadata.cc | 5 --
service/storage_service.cc | 26 ++------
test/boost/locator_topology_test.cc | 3 -
test/boost/network_topology_strategy_test.cc | 6 --
test/boost/tablets_test.cc | 62 +-------------------
test/boost/token_metadata_test.cc | 37 ------------
test/perf/tablet_load_balancing.cc | 1 -
8 files changed, 9 insertions(+), 140 deletions(-)

diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
index f2c27fcb14d..988734eef2b 100644
--- a/locator/token_metadata.hh
+++ b/locator/token_metadata.hh
@@ -227,15 +227,6 @@ class token_metadata final {
const topology& get_topology() const;
void debug_show() const;

- /**
- * Store an end-point to host ID mapping. Each ID must be unique, and
- * cannot be changed after the fact.
- *
- * @param hostId
- * @param endpoint
- */
- void update_host_id(const locator::host_id& host_id, inet_address endpoint);
-
/** Return the unique host ID for an end-point. */
host_id get_host_id(inet_address endpoint) const;

diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 519a6647549..e48c740db3b 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -949,11 +949,6 @@ token_metadata::debug_show() const {
_impl->debug_show();
}

-void
-token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) {
- // Do nothing for now. Remove later.
-}
-
std::unordered_set<host_id>
token_metadata::get_host_ids() const {
return _impl->get_host_ids();
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 4013fc232c8..b7a3c2168ba 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -412,12 +412,9 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
const auto& am =_address_map;
const auto& t = _topology_state_machine._topology;

- auto update_topology = [&] (locator::host_id id, std::optional<inet_address> ip, const replica_state& rs) {
+ auto update_topology = [&] (locator::host_id id, const replica_state& rs) {
tmptr->update_topology(id, locator::endpoint_dc_rack{rs.datacenter, rs.rack},
to_topology_node_state(rs.state), rs.shard_count);
- if (ip) {
- tmptr->update_host_id(id, *ip);
- }
};

using host_id_to_ip_map_t = std::unordered_map<locator::host_id, gms::inet_address>;
@@ -458,7 +455,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
}

if (t.left_nodes_rs.find(id) != t.left_nodes_rs.end()) {
- update_topology(host_id, std::nullopt, t.left_nodes_rs.at(id));
+ update_topology(host_id, t.left_nodes_rs.at(id));
}

// However if we do that, we need to also implement unbanning a node and do it if `removenode` is aborted.
@@ -516,7 +513,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
co_await remove_ip(it->second, host_id, false);
}
}
- update_topology(host_id, ip, rs);
+ update_topology(host_id, rs);
co_await tmptr->update_normal_tokens(rs.ring.value().tokens, host_id);
};

@@ -543,7 +540,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
// Save ip -> id mapping in peers table because we need it on restart, but do not save tokens until owned
sys_ks_futures.push_back(_sys_ks.local().update_peer_info(*ip, host_id, {}));
}
- update_topology(host_id, ip, rs);
+ update_topology(host_id, rs);
if (_topology_state_machine._topology.normal_nodes.empty()) {
// This is the first node in the cluster. Insert the tokens as normal to the token ring early
// so we can perform writes to regular 'distributed' tables during the bootstrap procedure
@@ -571,7 +568,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
co_await process_normal_node(id, rs);
break;
}
- update_topology(host_id, ip, rs);
+ update_topology(host_id, rs);
co_await tmptr->update_normal_tokens(rs.ring.value().tokens, host_id);
tmptr->add_leaving_endpoint(host_id);
co_await update_topology_change_info(tmptr, ::format("{} {}/{}", rs.state, id, ip));
@@ -584,7 +581,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
tmptr->update_topology(replaced_host_id, std::nullopt, locator::node::state::being_replaced);
tmptr->add_replacing_endpoint(replaced_host_id, host_id);
if (rs.ring.has_value()) {
- update_topology(host_id, ip, rs);
+ update_topology(host_id, rs);
co_await update_topology_change_info(tmptr, ::format("replacing {}/{} by {}/{}", replaced_id, existing_ip.value_or(gms::inet_address{}), id, ip));
}
}
@@ -1544,7 +1541,6 @@ future<> storage_service::join_topology(sharded<db::system_distributed_keyspace>
tmptr->update_topology(tmptr->get_my_id(), std::nullopt, locator::node::state::replacing);
tmptr->update_topology(ri->host_id, std::move(ri->dc_rack), locator::node::state::being_replaced);
co_await tmptr->update_normal_tokens(bootstrap_tokens, ri->host_id);
- tmptr->update_host_id(ri->host_id, *replace_address);
tmptr->add_replacing_endpoint(ri->host_id, tmptr->get_my_id());

replaced_host_id = ri->host_id;
@@ -1554,7 +1550,6 @@ future<> storage_service::join_topology(sharded<db::system_distributed_keyspace>
// therefore we need to "inject" their state here after we
// learn about them in the shadow round initiated in `prepare_replacement_info`.
for (const auto& [host_id, st] : ri->ignore_nodes) {
- tmptr->update_host_id(host_id, st.endpoint);
if (st.opt_dc_rack) {
tmptr->update_topology(host_id, st.opt_dc_rack, locator::node::state::normal);
}
@@ -2307,7 +2302,6 @@ future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::per
}
tmptr->update_topology(host_id, get_dc_rack_for(host_id), locator::node::state::bootstrapping);
tmptr->add_bootstrap_tokens(tokens, host_id);
- tmptr->update_host_id(host_id, endpoint);

co_await update_topology_change_info(tmptr, ::format("handle_state_bootstrap {}", endpoint));
co_await replicate_to_all_cores(std::move(tmptr));
@@ -2375,8 +2369,6 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit

slogger.warn("Host ID collision for {} between {} and {}; {} is the new owner", host_id, *existing, endpoint, endpoint);
do_remove_node(*existing);
- slogger.info("Set host_id={} to be owned by node={}, existing={}", host_id, endpoint, *existing);
- tmptr->update_host_id(host_id, endpoint);
} else {
// The new IP has smaller generation than the existing one,
// we are going to remove it, so we add it to the endpoints_to_remove.
@@ -2427,8 +2419,6 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
_normal_state_handled_on_boot.insert(endpoint);
co_return;
}
- slogger.info("Set host_id={} to be owned by node={}", host_id, endpoint);
- tmptr->update_host_id(host_id, endpoint);
}

// Tokens owned by the handled endpoint.
@@ -2636,7 +2626,6 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
const auto dc_rack = get_dc_rack_for(host_id);
- tmptr->update_host_id(host_id, endpoint);
tmptr->update_topology(host_id, dc_rack);
co_await replicate_to_all_cores(std::move(tmptr));
}
@@ -3040,7 +3029,6 @@ future<> storage_service::join_cluster(sharded<db::system_distributed_keyspace>&
slogger.debug("Loaded tokens: endpoint={}/{} dc={} rack={} tokens={}", host_id, st.endpoint, dc_rack.dc, dc_rack.rack, st.tokens);
tmptr->update_topology(host_id, dc_rack, locator::node::state::normal);
co_await tmptr->update_normal_tokens(st.tokens, host_id);
- tmptr->update_host_id(host_id, st.endpoint);
// gossiping hasn't started yet
// so no need to lock the endpoint
co_await _gossiper.add_saved_endpoint(host_id, st, gms::null_permit_id);
@@ -4554,7 +4542,6 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
// In handle_state_normal we'll remap the IP to the new host_id.
tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced);
tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing);
- tmptr->update_host_id(replacing_node_id, replacing_node);
tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id);
return make_ready_future<>();
}).get();
@@ -4618,7 +4605,6 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
const auto dc_rack = get_dc_rack_for(host_id);
slogger.info("bootstrap[{}]: Added node={}/{} as bootstrap, coordinator={}/{}",
req.ops_uuid, endpoint, host_id, coordinator, *coordinator_host_id);
- tmptr->update_host_id(host_id, endpoint);
tmptr->update_topology(host_id, dc_rack, locator::node::state::bootstrapping);
tmptr->add_bootstrap_tokens(tokens, host_id);
}
diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc
index f42dd743562..1256a33a493 100644
--- a/test/boost/locator_topology_test.cc
+++ b/test/boost/locator_topology_test.cc
@@ -215,9 +215,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_sketch) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, node1_shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, node2_shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, node3_shard_count);
diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc
index a175df290f7..1b904b3d9dc 100644
--- a/test/boost/network_topology_strategy_test.cc
+++ b/test/boost/network_topology_strategy_test.cc
@@ -484,7 +484,6 @@ SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_tablets_test) {
std::unordered_set<token> tokens;
tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
- tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
}).get();
@@ -576,7 +575,6 @@ static void test_random_balancing(sharded<snitch_ptr>& snitch, gms::inet_address
std::unordered_set<token> tokens;
tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
- tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
}).get();
@@ -1114,7 +1112,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {

auto host1 = host_id(utils::make_random_uuid());
auto host2 = host_id(utils::make_random_uuid());
- auto host3 = host_id(utils::make_random_uuid());

auto ip1_dc_rack = endpoint_dc_rack{ "dc1", "rack_ip1" };
auto ip1_dc_rack_v2 = endpoint_dc_rack{ "dc1", "rack_ip1_v2" };
@@ -1133,8 +1130,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack);

stm.mutate_token_metadata([&] (token_metadata& tm) {
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host1, ip1); // this_node added last on purpose
// Need to move to non left or none state in order to be indexed by ip
tm.update_topology(host1, {}, locator::node::state::normal);
tm.update_topology(host2, {}, locator::node::state::normal);
@@ -1158,7 +1153,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {

stm.mutate_token_metadata([&] (token_metadata& tm) {
tm.remove_endpoint(host1);
- tm.update_host_id(host3, ip3);
return make_ready_future<>();
}).get();

diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc
index e3e1d7c4d61..35ee10b9389 100644
--- a/test/boost/tablets_test.cc
+++ b/test/boost/tablets_test.cc
@@ -772,9 +772,6 @@ SEASTAR_TEST_CASE(test_get_shard) {
tablet_id tid1(0);

stm.mutate_token_metadata([&] (token_metadata& tm) {
- tm.update_host_id(h1, ip1);
- tm.update_host_id(h2, ip2);
- tm.update_host_id(h3, ip3);
tm.update_topology(h1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(h2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(h3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
@@ -1514,9 +1511,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_empty_node) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
@@ -1615,9 +1609,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_skiplist) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
@@ -1706,9 +1697,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rf_met) {
stm.mutate_token_metadata([&](token_metadata& tm) -> future<> {
const unsigned shard_count = 2;

- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned,
@@ -1806,10 +1794,6 @@ SEASTAR_THREAD_TEST_CASE(test_table_creation_during_decommission) {
const unsigned shard_count = 1;

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
- tm.update_host_id(host4, ip4);
tm.update_topology(host1, dcrack, node::state::normal, shard_count);
tm.update_topology(host2, dcrack, node::state::normal, shard_count);
tm.update_topology(host3, dcrack, node::state::being_decommissioned, shard_count);
@@ -1876,10 +1860,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_two_racks) {
stm.mutate_token_metadata([&](token_metadata& tm) -> future<> {
const unsigned shard_count = 1;

- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
- tm.update_host_id(host4, ip4);
tm.update_topology(host1, racks[0], node::state::normal, shard_count);
tm.update_topology(host2, racks[1], node::state::normal, shard_count);
tm.update_topology(host3, racks[0], node::state::normal, shard_count);
@@ -1983,10 +1963,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rack_load_failure) {
stm.mutate_token_metadata([&](token_metadata& tm) -> future<> {
const unsigned shard_count = 1;

- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
- tm.update_host_id(host4, ip4);
tm.update_topology(host1, racks[0], node::state::normal, shard_count);
tm.update_topology(host2, racks[0], node::state::normal, shard_count);
tm.update_topology(host3, racks[0], node::state::normal, shard_count);
@@ -2062,9 +2038,6 @@ SEASTAR_THREAD_TEST_CASE(test_decommission_rf_not_met) {
stm.mutate_token_metadata([&](token_metadata& tm) -> future<> {
const unsigned shard_count = 2;

- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned,
@@ -2121,9 +2094,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_works_with_in_progress_transitions)
});

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, 1);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 2);
@@ -2195,9 +2165,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancer_shuffle_mode) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, 1);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 2);
@@ -2262,10 +2229,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_two_empty_nodes) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
- tm.update_host_id(host4, ip4);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
@@ -2327,9 +2290,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_asymmetric_node_capacity) {
});

stm.mutate_token_metadata([&](token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::being_decommissioned, 8);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, 1);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, node::state::normal, 7);
@@ -2394,8 +2354,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancer_disabling) {
// host1 is loaded and host2 is empty, resulting in an imbalance.
// host1's shard 0 is loaded and shard 1 is empty, resulting in intra-node imbalance.
stm.mutate_token_metadata([&] (auto& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1);
@@ -2486,8 +2444,6 @@ SEASTAR_THREAD_TEST_CASE(test_drained_node_is_not_balanced_internally) {
});

stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1);
@@ -2536,7 +2492,6 @@ SEASTAR_THREAD_TEST_CASE(test_plan_fails_when_removing_last_replica) {
});

stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1);

@@ -2588,9 +2543,6 @@ SEASTAR_THREAD_TEST_CASE(test_skiplist_is_ignored_when_draining) {
});

stm.mutate_token_metadata([&] (locator::token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host3, ip3);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, locator::node::state::being_removed, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count);
tm.update_topology(host3, locator::endpoint_dc_rack::default_location, locator::node::state::normal, shard_count);
@@ -2698,10 +2650,8 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_with_random_load) {

int i = 0;
for (auto h : hosts) {
- auto ip = inet_address(format("192.168.0.{}", ++i));
auto shard_count = 2;
- tm.update_host_id(h, ip);
- auto rack = racks[i % racks.size()];
+ auto rack = racks[++i % racks.size()];
tm.update_topology(h, rack, node::state::normal, shard_count);
if (h != hosts[0]) {
// Leave the first host empty by making it invisible to allocation algorithm.
@@ -2870,13 +2820,11 @@ static void do_test_load_balancing_merge_colocation(cql_test_env& e, const int n

int i = 0;
for (auto h : hosts) {
- auto ip = inet_address(format("192.168.0.{}", ++i));
- tm.update_host_id(h, ip);
- auto rack = racks[i % racks.size()];
+ auto rack = racks[++i % racks.size()];
hosts_by_rack[rack.rack].push_back(h);
tm.update_topology(h, rack, node::state::normal, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(float(i) / hosts.size()))}, h);
- testlog.debug("adding host {}, ip {}, rack {}, token {}", h, ip, rack.rack, token(tests::d2t(1. / hosts.size())));
+ testlog.debug("adding host {}, rack {}, token {}", h, rack.rack, token(tests::d2t(1. / hosts.size())));
}

tablet_map tmap(initial_tablets);
@@ -3055,8 +3003,6 @@ SEASTAR_THREAD_TEST_CASE(test_load_balancing_resize_requests) {
});

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
- tm.update_host_id(host1, ip1);
- tm.update_host_id(host2, ip2);
tm.update_topology(host1, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
tm.update_topology(host2, locator::endpoint_dc_rack::default_location, node::state::normal, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(1. / 2))}, host1);
@@ -3340,7 +3286,6 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_
std::unordered_set<token> tokens;
tokens.insert(dht::token{tests::d2t(ring_point / test_config.ring_points.size())});
topo.add_or_update_endpoint(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1);
- tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
}).get();
@@ -3361,7 +3306,6 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_
stm.mutate_token_metadata([&] (token_metadata& tm) {
for (size_t i = 0; i < test_config.ring_points.size(); ++i) {
auto& [ring_point, endpoint, id] = test_config.ring_points[i];
- tm.update_host_id(id, endpoint);
tm.update_topology(id, make_endpoint_dc_rack(endpoint), node::state::normal, nodes_shard_count[i]);
}
return make_ready_future<>();
diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc
index 2aec13a1637..8faa23b7e46 100644
--- a/test/boost/token_metadata_test.cc
+++ b/test/boost/token_metadata_test.cc
@@ -50,16 +50,12 @@ namespace {
}

SEASTAR_THREAD_TEST_CASE(test_pending_and_read_endpoints_for_everywhere_strategy) {
- const auto e1 = inet_address("192.168.0.1");
- const auto e2 = inet_address("192.168.0.2");
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);
const auto t1 = dht::token::from_int64(10);
const auto t2 = dht::token::from_int64(20);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_normal_tokens({t1}, e1_id).get();
@@ -74,16 +70,12 @@ SEASTAR_THREAD_TEST_CASE(test_pending_and_read_endpoints_for_everywhere_strategy
}

SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_bootstrap_second_node) {
- const auto e1 = inet_address("192.168.0.1");
const auto t1 = dht::token::from_int64(1);
- const auto e2 = inet_address("192.168.0.2");
const auto t2 = dht::token::from_int64(100);
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_normal_tokens({t1}, e1_id).get();
@@ -107,17 +99,11 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_bootstrap_with_replicas) {
const auto t10 = dht::token::from_int64(10);
const auto t100 = dht::token::from_int64(100);
const auto t1000 = dht::token::from_int64(1000);
- const auto e1 = inet_address("192.168.0.1");
- const auto e2 = inet_address("192.168.0.2");
- const auto e3 = inet_address("192.168.0.3");
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);
const auto e3_id = gen_id(3);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
- token_metadata->update_host_id(e3_id, e3);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal);
@@ -143,17 +129,11 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_leave_with_replicas) {
const auto t10 = dht::token::from_int64(10);
const auto t100 = dht::token::from_int64(100);
const auto t1000 = dht::token::from_int64(1000);
- const auto e1 = inet_address("192.168.0.1");
- const auto e2 = inet_address("192.168.0.2");
- const auto e3 = inet_address("192.168.0.3");
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);
const auto e3_id = gen_id(3);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
- token_metadata->update_host_id(e3_id, e3);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal);
@@ -180,20 +160,12 @@ SEASTAR_THREAD_TEST_CASE(test_pending_endpoints_for_replace_with_replicas) {
const auto t10 = dht::token::from_int64(10);
const auto t100 = dht::token::from_int64(100);
const auto t1000 = dht::token::from_int64(1000);
- const auto e1 = inet_address("192.168.0.1");
- const auto e2 = inet_address("192.168.0.2");
- const auto e3 = inet_address("192.168.0.3");
- const auto e4 = inet_address("192.168.0.4");
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);
const auto e3_id = gen_id(3);
const auto e4_id = gen_id(4);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
- token_metadata->update_host_id(e3_id, e3);
- token_metadata->update_host_id(e4_id, e4);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal);
@@ -225,17 +197,11 @@ SEASTAR_THREAD_TEST_CASE(test_endpoints_for_reading_when_bootstrap_with_replicas
const auto t10 = dht::token::from_int64(10);
const auto t100 = dht::token::from_int64(100);
const auto t1000 = dht::token::from_int64(1000);
- const auto e1 = inet_address("192.168.0.1");
- const auto e2 = inet_address("192.168.0.2");
- const auto e3 = inet_address("192.168.0.3");
const auto e1_id = gen_id(1);
const auto e2_id = gen_id(2);
const auto e3_id = gen_id(3);

auto token_metadata = create_token_metadata(e1_id);
- token_metadata->update_host_id(e1_id, e1);
- token_metadata->update_host_id(e2_id, e2);
- token_metadata->update_host_id(e3_id, e3);
token_metadata->update_topology(e1_id, get_dc_rack(e1_id), node::state::normal);
token_metadata->update_topology(e2_id, get_dc_rack(e2_id), node::state::normal);
token_metadata->update_topology(e3_id, get_dc_rack(e3_id), node::state::normal);
@@ -285,17 +251,14 @@ SEASTAR_THREAD_TEST_CASE(test_endpoints_for_reading_when_bootstrap_with_replicas

SEASTAR_THREAD_TEST_CASE(test_replace_node_with_same_endpoint) {
const auto t1 = dht::token::from_int64(1);
- const auto e1 = inet_address("192.168.0.1");
const auto e1_id1 = gen_id(1);
const auto e1_id2 = gen_id(2);

auto token_metadata = create_token_metadata(e1_id2);
- token_metadata->update_host_id(e1_id1, e1);
token_metadata->update_topology(e1_id1, get_dc_rack(e1_id1), node::state::being_replaced);
token_metadata->update_normal_tokens({t1}, e1_id1).get();

token_metadata->update_topology(e1_id2, get_dc_rack(e1_id2), node::state::replacing);
- token_metadata->update_host_id(e1_id2, e1);

token_metadata->add_replacing_endpoint(e1_id1, e1_id2);

diff --git a/test/perf/tablet_load_balancing.cc b/test/perf/tablet_load_balancing.cc
index 9a9e5178cb9..c0625b0f318 100644
--- a/test/perf/tablet_load_balancing.cc
+++ b/test/perf/tablet_load_balancing.cc
@@ -243,7 +243,6 @@ future<results> test_load_balancing_with_many_tables(params p, bool tablet_aware
};

auto add_host_to_topology = [&] (token_metadata& tm, int i) -> future<> {
- tm.update_host_id(hosts[i], ips[i]);
tm.update_topology(hosts[i], rack1, node::state::normal, shard_count);
co_await tm.update_normal_tokens(std::unordered_set{token(tests::d2t(float(i) / hosts.size()))}, hosts[i]);
};
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:29 AMJan 13
to scylladb-dev@googlegroups.com
There is a code that creates a map from id to ip and then creates a
vector from the keys of the map. Create a vector directly instead.
---
repair/repair.hh | 3 ---
repair/repair.cc | 6 +++---
2 files changed, 3 insertions(+), 6 deletions(-)

diff --git a/repair/repair.hh b/repair/repair.hh
index a65c4acc929..942ecc435b0 100644
--- a/repair/repair.hh
+++ b/repair/repair.hh
@@ -143,9 +143,6 @@ class repair_neighbors {
explicit repair_neighbors(std::vector<locator::host_id> a)
: all(std::move(a)) {
}
- explicit repair_neighbors(const std::unordered_map<locator::host_id, gms::inet_address>& a)
- : all(a | std::views::keys | std::ranges::to<std::vector<locator::host_id>>()) {
- }
repair_neighbors(std::vector<locator::host_id> a, std::vector<locator::host_id> m)
: all(std::move(a))
, mandatory(std::move(m)) {
diff --git a/repair/repair.cc b/repair/repair.cc
index 51c0aa8d7db..bd54cb6d924 100644
--- a/repair/repair.cc
+++ b/repair/repair.cc
@@ -2150,11 +2150,11 @@ future<> repair_service::do_rebuild_replace_with_repair(std::unordered_map<sstri
return sync_nodes.contains(node);
}) | std::views::transform([&topology] (const auto& node) {
const auto& n = topology.get_node(node);
- return std::make_pair(n.host_id(), n.endpoint());
- }) | std::ranges::to<std::unordered_map>();
+ return n.host_id();
+ }) | std::ranges::to<std::vector>();
rlogger.debug("{}: keyspace={}, range={}, natural_enpoints={}, neighbors={}", op, keyspace_name, r, natural_eps, neighbors);
if (!neighbors.empty()) {
- range_sources[r] = repair_neighbors(neighbors);
+ range_sources[r] = repair_neighbors(std::move(neighbors));
++it;
} else {
// Skip the range with zero neighbors
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:30 AMJan 13
to scylladb-dev@googlegroups.com
Since we do not stop storage proxy on shutdown this code can be called
during shutdown when address map is no longer usable.
---
db/hints/manager.cc | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/db/hints/manager.cc b/db/hints/manager.cc
index 1f996d193fc..77ca698b2ae 100644
--- a/db/hints/manager.cc
+++ b/db/hints/manager.cc
@@ -433,9 +433,8 @@ bool manager::have_ep_manager(const std::variant<locator::host_id, gms::inet_add
bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
tracing::trace_state_ptr tr_state) noexcept
{
- auto ip = _gossiper_anchor->get_address_map().get(host_id);
if (utils::get_local_injector().enter("reject_incoming_hints")) {
- manager_logger.debug("Rejecting a hint to {} / {} due to an error injection", host_id, ip);
+ manager_logger.debug("Rejecting a hint to {} due to an error injection", host_id);
++_stats.dropped;
return false;
}
@@ -446,6 +445,8 @@ bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const
return false;
}

+ auto ip = _gossiper_anchor->get_address_map().get(host_id);
+
try {
manager_logger.trace("Going to store a hint to {}", host_id);
tracing::trace(tr_state, "Going to store a hint to {}", host_id);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:31 AMJan 13
to scylladb-dev@googlegroups.com
host_id_or_endpoint is a helper class that hold either id or ip and
translate one into another on demand. Use gossiper to do a translation
there instead of token_metadata since we want to drop ip based APIs from
the later.
---
locator/token_metadata.hh | 8 ++++++--
db/hints/manager.cc | 8 ++++----
locator/token_metadata.cc | 14 +++++++-------
service/storage_service.cc | 2 +-
4 files changed, 18 insertions(+), 14 deletions(-)

diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
index 541c91f086e..b8420761242 100644
--- a/locator/token_metadata.hh
+++ b/locator/token_metadata.hh
@@ -37,6 +37,10 @@ namespace replica {
class keyspace;
}

+namespace gms {
+class gossiper;
+}
+
namespace locator {

class abstract_replication_strategy;
@@ -75,8 +79,8 @@ struct host_id_or_endpoint {

// Map the host_id to endpoint or vice verse, using the token_metadata.
// Throws runtime error if failed to resolve.
- host_id resolve_id(const token_metadata&) const;
- gms::inet_address resolve_endpoint(const token_metadata&) const;
+ host_id resolve_id(const gms::gossiper&) const;
+ gms::inet_address resolve_endpoint(const gms::gossiper&) const;
};

using host_id_or_endpoint_list = std::vector<host_id_or_endpoint>;
diff --git a/db/hints/manager.cc b/db/hints/manager.cc
index 015d492b788..1f996d193fc 100644
--- a/db/hints/manager.cc
+++ b/db/hints/manager.cc
@@ -594,9 +594,9 @@ future<> manager::change_host_filter(host_filter filter) {
// been created by mistake and they're invalid. The same for pre-host-ID hinted handoff
// -- hint directories representing host IDs are NOT valid.
if (hid_or_ep.has_host_id() && _uses_host_id) {
- return std::make_optional(pair_type{hid_or_ep.id(), hid_or_ep.resolve_endpoint(*tmptr)});
+ return std::make_optional(pair_type{hid_or_ep.id(), hid_or_ep.resolve_endpoint(*_gossiper_anchor)});
} else if (hid_or_ep.has_endpoint() && !_uses_host_id) {
- return std::make_optional(pair_type{hid_or_ep.resolve_id(*tmptr), hid_or_ep.endpoint()});
+ return std::make_optional(pair_type{hid_or_ep.resolve_id(*_gossiper_anchor), hid_or_ep.endpoint()});
} else {
return std::nullopt;
}
@@ -817,7 +817,7 @@ future<> manager::initialize_endpoint_managers() {

const auto maybe_host_id = std::invoke([&] () -> std::optional<locator::host_id> {
try {
- return maybe_host_id_or_ep->resolve_id(*tmptr);
+ return maybe_host_id_or_ep->resolve_id(*_gossiper_anchor);
} catch (...) {
return std::nullopt;
}
@@ -869,7 +869,7 @@ future<> manager::migrate_ip_directories() {
continue;
}

- const locator::host_id host_id = hid_or_ep.resolve_id(*tmptr);
+ const locator::host_id host_id = hid_or_ep.resolve_id(*_gossiper_anchor);
dirs_to_rename.push_back({.current_name = std::move(directory), .new_name = host_id.to_sstring()});
} catch (...) {
// We cannot map the IP to the corresponding host ID either because
diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 3c369e44ce5..75603c58e6a 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -22,7 +22,7 @@
#include <seastar/core/smp.hh>
#include "utils/assert.hh"
#include "utils/stall_free.hh"
-
+#include "gms/gossiper.hh"
namespace locator {

static logging::logger tlogger("token_metadata");
@@ -1391,22 +1391,22 @@ host_id_or_endpoint::host_id_or_endpoint(const sstring& s, param_type restrict)
}
}

-host_id host_id_or_endpoint::resolve_id(const token_metadata& tm) const {
+host_id host_id_or_endpoint::resolve_id(const gms::gossiper& g) const {
if (has_host_id()) {
return id();
}
- auto opt_id = tm.get_host_id_if_known(endpoint());
- if (!opt_id) {
+ try {
+ return g.get_host_id(endpoint());
+ } catch (...) {
throw std::runtime_error(format("Host inet address {} not found in the cluster", endpoint()));
}
- return *opt_id;
}

-gms::inet_address host_id_or_endpoint::resolve_endpoint(const token_metadata& tm) const {
+gms::inet_address host_id_or_endpoint::resolve_endpoint(const gms::gossiper& g) const {
if (has_endpoint()) {
return endpoint();
}
- auto endpoint_opt = tm.get_endpoint_for_host_id_if_known(id());
+ auto endpoint_opt = g.get_address_map().find(id());
if (!endpoint_opt) {
throw std::runtime_error(format("Host ID {} not found in the cluster", id()));
}
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 0795fc4357c..0f21854a516 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -4178,7 +4178,7 @@ future<> storage_service::removenode(locator::host_id host_id, locator::host_id_
}

for (auto& hoep : ignore_nodes_params) {
- ctl.ignore_nodes.insert(hoep.resolve_id(*tmptr));
+ ctl.ignore_nodes.insert(hoep.resolve_id(ss._gossiper));
}

if (!removed_from_token_ring) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:31 AMJan 13
to scylladb-dev@googlegroups.com
Do not track id to ip mapping in the topology class any longer. There
are no remaining users.
---
locator/topology.hh | 27 +-----
locator/token_metadata.cc | 17 +---
locator/topology.cc | 97 +++-----------------
main.cc | 3 +-
test/boost/locator_topology_test.cc | 97 ++++----------------
test/boost/network_topology_strategy_test.cc | 30 ++----
test/boost/tablets_test.cc | 6 +-
test/lib/cql_test_env.cc | 3 +-
test/perf/perf_sort_by_proximity.cc | 1 -
9 files changed, 52 insertions(+), 229 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index f27c7428321..c87e8cc2790 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -60,7 +60,6 @@ class node {
private:
const locator::topology* _topology;
locator::host_id _host_id;
- inet_address _endpoint;
endpoint_dc_rack _dc_rack;
state _state;
shard_id _shard_count = 0;
@@ -73,7 +72,6 @@ class node {
public:
node(const locator::topology* topology,
locator::host_id id,
- inet_address endpoint,
endpoint_dc_rack dc_rack,
state state,
shard_id shard_count = 0,
@@ -95,10 +93,6 @@ class node {
return _host_id;
}

- const inet_address& endpoint() const noexcept {
- return _endpoint;
- }
-
const endpoint_dc_rack& dc_rack() const noexcept {
return _dc_rack;
}
@@ -165,7 +159,6 @@ class node {
private:
static node_holder make(const locator::topology* topology,
locator::host_id id,
- inet_address endpoint,
endpoint_dc_rack dc_rack,
state state,
shard_id shard_count = 0,
@@ -212,7 +205,7 @@ class topology {
}

// Adds a node with given host_id, endpoint, and DC/rack.
- const node& add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state,
+ const node& add_node(host_id id, const endpoint_dc_rack& dr, node::state state,
shard_id shard_count = 0);

// Optionally updates node's current host_id, endpoint, or DC/rack.
@@ -220,7 +213,6 @@ class topology {
// or a peer node host_id may be updated when the node is replaced with another node using the same ip address.
void update_node(node& node,
std::optional<host_id> opt_id,
- std::optional<inet_address> opt_ep,
std::optional<endpoint_dc_rack> opt_dr,
std::optional<node::state> opt_st,
std::optional<shard_id> opt_shard_count = std::nullopt);
@@ -242,10 +234,6 @@ class topology {
return *n;
};

- // Looks up a node by its inet_address.
- // Returns a pointer to the node if found, or nullptr otherwise.
- const node* find_node(const inet_address& ep) const noexcept;
-
// Finds a node by its index
// Returns a pointer to the node if found, or nullptr otherwise.
const node* find_node(node::idx_type idx) const noexcept;
@@ -258,8 +246,7 @@ class topology {
*
* Adds or updates a node with given endpoint
*/
- const node& add_or_update_endpoint(host_id id, std::optional<inet_address> opt_ep,
- std::optional<endpoint_dc_rack> opt_dr = std::nullopt,
+ const node& add_or_update_endpoint(host_id id, std::optional<endpoint_dc_rack> opt_dr = std::nullopt,
std::optional<node::state> opt_st = std::nullopt,
std::optional<shard_id> shard_count = std::nullopt);

@@ -412,7 +399,6 @@ class topology {
const node* _this_node = nullptr;
std::vector<node_holder> _nodes;
std::unordered_map<host_id, std::reference_wrapper<const node>> _nodes_by_host_id;
- std::unordered_map<inet_address, std::reference_wrapper<const node>> _nodes_by_endpoint;

std::unordered_map<sstring, std::unordered_set<std::reference_wrapper<const node>>> _dc_nodes;
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<std::reference_wrapper<const node>>>> _dc_rack_nodes;
@@ -435,10 +421,6 @@ class topology {

void calculate_datacenters();

- const std::unordered_map<inet_address, std::reference_wrapper<const node>>& get_nodes_by_endpoint() const noexcept {
- return _nodes_by_endpoint;
- };
-
mutable random_engine_type _random_engine;

friend class token_metadata_impl;
@@ -492,12 +474,11 @@ struct fmt::formatter<locator::node> : fmt::formatter<string_view> {
template <typename FormatContext>
auto format(const locator::node& node, FormatContext& ctx) const {
if (!verbose) {
- return fmt::format_to(ctx.out(), "{}/{}", node.host_id(), node.endpoint());
+ return fmt::format_to(ctx.out(), "{}", node.host_id());
} else {
- return fmt::format_to(ctx.out(), " idx={} host_id={} endpoint={} dc={} rack={} state={} shards={} this_node={}",
+ return fmt::format_to(ctx.out(), " idx={} host_id={} dc={} rack={} state={} shards={} this_node={}",
node.idx(),
node.host_id(),
- node.endpoint(),
node.dc_rack().dc,
node.dc_rack().rack,
locator::node::to_string(node.get_state()),
diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 27b8e991285..519a6647549 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -116,7 +116,7 @@ class token_metadata_impl final {
}

void update_topology(host_id id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count = std::nullopt) {
- _topology.add_or_update_endpoint(id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
+ _topology.add_or_update_endpoint(id, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
}

/**
@@ -141,15 +141,6 @@ class token_metadata_impl final {

void debug_show() const;

- /**
- * Store an end-point to host ID mapping. Each ID must be unique, and
- * cannot be changed after the fact.
- *
- * @param hostId
- * @param endpoint
- */
- void update_host_id(const host_id& host_id, inet_address endpoint);
-
/** @return a copy of host id set for read-only operations */
std::unordered_set<host_id> get_host_ids() const;

@@ -516,10 +507,6 @@ void token_metadata_impl::debug_show() const {
reporter->arm_periodic(std::chrono::seconds(1));
}

-void token_metadata_impl::update_host_id(const host_id& host_id, inet_address endpoint) {
- _topology.add_or_update_endpoint(host_id, endpoint);
-}
-
std::unordered_set<host_id> token_metadata_impl::get_host_ids() const {
return _topology.get_nodes() |
std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) |
@@ -964,7 +951,7 @@ token_metadata::debug_show() const {

void
token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) {
- _impl->update_host_id(host_id, endpoint);
+ // Do nothing for now. Remove later.
}

std::unordered_set<host_id>
diff --git a/locator/topology.cc b/locator/topology.cc
index 195c512a11a..8c03ac33b12 100644
--- a/locator/topology.cc
+++ b/locator/topology.cc
@@ -54,10 +54,9 @@ thread_local const endpoint_dc_rack endpoint_dc_rack::default_location = {
.rack = locator::production_snitch_base::default_rack,
};

-node::node(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx)
+node::node(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx)
: _topology(topology)
, _host_id(id)
- , _endpoint(endpoint)
, _dc_rack(std::move(dc_rack))
, _state(state)
, _shard_count(std::move(shard_count))
@@ -65,12 +64,12 @@ node::node(const locator::topology* topology, locator::host_id id, inet_address
, _idx(idx)
{}

-node_holder node::make(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) {
- return std::make_unique<node>(topology, std::move(id), std::move(endpoint), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx);
+node_holder node::make(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) {
+ return std::make_unique<node>(topology, std::move(id), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx);
}

node_holder node::clone() const {
- return make(nullptr, host_id(), endpoint(), dc_rack(), get_state(), get_shard_count(), is_this_node());
+ return make(nullptr, host_id(), dc_rack(), get_state(), get_shard_count(), is_this_node());
}

std::string node::to_string(node::state s) {
@@ -94,7 +93,6 @@ future<> topology::clear_gently() noexcept {
_datacenters.clear();
_dc_rack_nodes.clear();
_dc_nodes.clear();
- _nodes_by_endpoint.clear();
_nodes_by_host_id.clear();
co_await utils::clear_gently(_nodes);
}
@@ -115,7 +113,7 @@ topology::topology(config cfg)
{
tlogger.trace("topology[{}]: constructing using config: endpoint={} id={} dc={} rack={}", fmt::ptr(this),
cfg.this_endpoint, cfg.this_host_id, cfg.local_dc_rack.dc, cfg.local_dc_rack.rack);
- add_node(cfg.this_host_id, cfg.this_endpoint, cfg.local_dc_rack, node::state::none);
+ add_node(cfg.this_host_id, cfg.local_dc_rack, node::state::none);
}

topology::topology(topology&& o) noexcept
@@ -124,7 +122,6 @@ topology::topology(topology&& o) noexcept
, _this_node(std::exchange(o._this_node, nullptr))
, _nodes(std::move(o._nodes))
, _nodes_by_host_id(std::move(o._nodes_by_host_id))
- , _nodes_by_endpoint(std::move(o._nodes_by_endpoint))
, _dc_nodes(std::move(o._dc_nodes))
, _dc_rack_nodes(std::move(o._dc_rack_nodes))
, _dc_endpoints(std::move(o._dc_endpoints))
@@ -170,7 +167,7 @@ void topology::set_host_id_cfg(host_id this_host_id) {
tlogger.trace("topology[{}]: set host id to {}", fmt::ptr(this), this_host_id);

_cfg.this_host_id = this_host_id;
- add_or_update_endpoint(this_host_id, _cfg.this_endpoint);
+ add_or_update_endpoint(this_host_id);
}

future<topology> topology::clone_gently() const {
@@ -187,21 +184,15 @@ future<topology> topology::clone_gently() const {
co_return ret;
}

-const node& topology::add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) {
+const node& topology::add_node(host_id id, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) {
if (dr.dc.empty() || dr.rack.empty()) {
on_internal_error(tlogger, "Node must have valid dc and rack");
}
- return add_node(node::make(this, id, ep, dr, state, shard_count));
+ return add_node(node::make(this, id, dr, state, shard_count));
}

bool topology::is_configured_this_node(const node& n) const {
- if (_cfg.this_host_id && n.host_id()) { // Selection by host_id
- return _cfg.this_host_id == n.host_id();
- }
- if (_cfg.this_endpoint != inet_address()) { // Selection by endpoint
- return _cfg.this_endpoint == n.endpoint();
- }
- return false; // No selection;
+ return _cfg.this_host_id == n.host_id();
}

const node& topology::add_node(node_holder nptr) {
@@ -244,10 +235,9 @@ const node& topology::add_node(node_holder nptr) {
return *node;
}

-void topology::update_node(node& node, std::optional<host_id> opt_id, std::optional<inet_address> opt_ep, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> opt_shard_count) {
- tlogger.debug("topology[{}]: update_node: {}: to: host_id={} endpoint={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node),
+void topology::update_node(node& node, std::optional<host_id> opt_id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> opt_shard_count) {
+ tlogger.debug("topology[{}]: update_node: {}: to: host_id={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node),
opt_id ? format("{}", *opt_id) : "unchanged",
- opt_ep ? format("{}", *opt_ep) : "unchanged",
opt_dr ? format("{}", opt_dr->dc) : "unchanged",
opt_dr ? format("{}", opt_dr->rack) : "unchanged",
opt_st ? format("{}", *opt_st) : "unchanged",
@@ -271,16 +261,6 @@ void topology::update_node(node& node, std::optional<host_id> opt_id, std::optio
opt_id.reset();
}
}
- if (opt_ep) {
- if (*opt_ep != node.endpoint()) {
- if (*opt_ep == inet_address{}) {
- on_internal_error(tlogger, seastar::format("Updating node endpoint to null is disallowed: {}: new endpoint={}", node_printer(&node), *opt_ep));
- }
- changed = true;
- } else {
- opt_ep.reset();
- }
- }
if (opt_dr) {
if (opt_dr->dc.empty() || opt_dr->dc == production_snitch_base::default_dc) {
opt_dr->dc = node.dc_rack().dc;
@@ -311,9 +291,6 @@ void topology::update_node(node& node, std::optional<host_id> opt_id, std::optio
if (opt_id) {
node._host_id = *opt_id;
}
- if (opt_ep) {
- node._endpoint = *opt_ep;
- }
if (opt_dr) {
node._dc_rack = std::move(*opt_dr);
}
@@ -359,32 +336,6 @@ void topology::index_node(const node& node) {
if (!inserted_host_id) {
on_internal_error(tlogger, seastar::format("topology[{}]: {}: node already exists", fmt::ptr(this), node_printer(&node)));
}
- if (node.endpoint() != inet_address{}) {
- auto eit = _nodes_by_endpoint.find(node.endpoint());
- if (eit != _nodes_by_endpoint.end()) {
- if (eit->second.get().get_state() == node::state::none && eit->second.get().is_this_node()) {
- // eit->second is default entry created for local node and it is replaced by existing node with the same ip
- // it means this node is going to replace the existing node with the same ip, but it does not know it yet
- // map ip to the old node
- _nodes_by_endpoint.erase(node.endpoint());
- } else if (eit->second.get().get_state() == node::state::replacing && node.get_state() == node::state::being_replaced) {
- // replace-with-same-ip, map ip to the old node
- _nodes_by_endpoint.erase(node.endpoint());
- } else if (eit->second.get().get_state() == node::state::being_replaced && node.get_state() == node::state::replacing) {
- // replace-with-same-ip, map ip to the old node, do nothing if it's already the case
- } else if (eit->second.get().is_leaving() || eit->second.get().left()) {
- _nodes_by_endpoint.erase(node.endpoint());
- } else if (!node.is_leaving() && !node.left()) {
- if (node.host_id()) {
- _nodes_by_host_id.erase(node.host_id());
- }
- on_internal_error(tlogger, seastar::format("topology[{}]: {}: node endpoint already mapped to {}", fmt::ptr(this), node_printer(&node), node_printer(&eit->second.get())));
- }
- }
- if (!node.left() && !node.is_none()) {
- _nodes_by_endpoint.try_emplace(node.endpoint(), std::cref(node));
- }
- }

// We keep location of left nodes because they may still appear in tablet replica sets
// and algorithms expect to know which dc they belonged to. View replica pairing needs stable
@@ -441,10 +392,6 @@ void topology::unindex_node(const node& node) {
if (host_it != _nodes_by_host_id.end() && host_it->second == node) {
_nodes_by_host_id.erase(host_it);
}
- auto ep_it = _nodes_by_endpoint.find(node.endpoint());
- if (ep_it != _nodes_by_endpoint.end() && ep_it->second.get() == node) {
- _nodes_by_endpoint.erase(ep_it);
- }
if (_this_node == &node) {
_this_node = nullptr;
}
@@ -483,16 +430,6 @@ node* topology::find_node(host_id id) noexcept {
return make_mutable(const_cast<const topology*>(this)->find_node(id));
}

-// Finds a node by its endpoint
-// Returns nullptr if not found
-const node* topology::find_node(const inet_address& ep) const noexcept {
- auto it = _nodes_by_endpoint.find(ep);
- if (it != _nodes_by_endpoint.end()) {
- return &it->second.get();
- }
- return nullptr;
-}
-
// Finds a node by its index
// Returns nullptr if not found
const node* topology::find_node(node::idx_type idx) const noexcept {
@@ -502,23 +439,19 @@ const node* topology::find_node(node::idx_type idx) const noexcept {
return _nodes.at(idx).get();
}

-const node& topology::add_or_update_endpoint(host_id id, std::optional<inet_address> opt_ep, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count)
+const node& topology::add_or_update_endpoint(host_id id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count)
{
- tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} ep={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this),
- id, opt_ep, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count,
+ tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this),
+ id, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count,
lazy_backtrace());

auto* n = find_node(id);
if (n) {
- update_node(*n, std::nullopt, opt_ep, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
- return *n;
- } else if (opt_ep && (n = make_mutable(find_node(*opt_ep)))) {
- update_node(*n, id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
+ update_node(*n, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
return *n;
}

return add_node(id,
- opt_ep.value_or(inet_address{}),
opt_dr.value_or(endpoint_dc_rack::default_location),
opt_st.value_or(node::state::none),
shard_count.value_or(0));
diff --git a/main.cc b/main.cc
index 9d394363ca3..263a41ee160 100644
--- a/main.cc
+++ b/main.cc
@@ -1437,12 +1437,11 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
const auto listen_address = utils::resolve(cfg->listen_address, family).get();
const auto host_id = initialize_local_info_thread(sys_ks, snitch, listen_address, *cfg, broadcast_addr, broadcast_rpc_addr);

- shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id, endpoint = broadcast_addr] (locator::token_metadata& tm) {
+ shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id] (locator::token_metadata& tm) {
// Makes local host id available in topology cfg as soon as possible.
// Raft topology discard the endpoint-to-id map, so the local id can
// still be found in the config.
tm.get_topology().set_host_id_cfg(host_id);
- tm.get_topology().add_or_update_endpoint(host_id, endpoint);
return make_ready_future<>();
}).get();

diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc
index ccb321a11d7..f42dd743562 100644
--- a/test/boost/locator_topology_test.cc
+++ b/test/boost/locator_topology_test.cc
@@ -32,9 +32,7 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) {
auto id1 = host_id::create_random_id();
auto ep1 = gms::inet_address("127.0.0.1");
auto id2 = host_id::create_random_id();
- auto ep2 = gms::inet_address("127.0.0.2");
auto id3 = host_id::create_random_id();
- auto ep3 = gms::inet_address("127.0.0.3");

topology::config cfg = {
.this_endpoint = ep1,
@@ -51,16 +49,13 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) {

std::unordered_set<std::reference_wrapper<const locator::node>> nodes;

- nodes.insert(std::cref(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal)));
- nodes.insert(std::cref(topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal)));
+ nodes.insert(std::cref(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal)));
+ nodes.insert(std::cref(topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal)));

- BOOST_REQUIRE_THROW(topo.add_node(id1, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
- BOOST_REQUIRE_THROW(topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
- BOOST_REQUIRE_THROW(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
- BOOST_REQUIRE_THROW(topo.add_node(id2, ep3, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
- BOOST_REQUIRE_THROW(topo.add_node(id3, ep3, endpoint_dc_rack{}, node::state::normal), std::runtime_error);
+ BOOST_REQUIRE_THROW(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
+ BOOST_REQUIRE_THROW(topo.add_node(id3, endpoint_dc_rack{}, node::state::normal), std::runtime_error);

- nodes.insert(std::cref(topo.add_node(id3, ep3, endpoint_dc_rack::default_location, node::state::normal)));
+ nodes.insert(std::cref(topo.add_node(id3, endpoint_dc_rack::default_location, node::state::normal)));

topo.for_each_node([&] (const locator::node& node) {
BOOST_REQUIRE(nodes.erase(std::cref(node)));
@@ -82,7 +77,7 @@ SEASTAR_THREAD_TEST_CASE(test_moving) {

auto topo = topology(cfg);

- topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal);
+ topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal);

BOOST_REQUIRE(topo.this_node()->topology() == &topo);

@@ -101,8 +96,6 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
auto id1 = host_id::create_random_id();
auto ep1 = gms::inet_address("127.0.0.1");
auto id2 = host_id::create_random_id();
- auto ep2 = gms::inet_address("127.0.0.2");
- auto ep3 = gms::inet_address("127.0.0.3");

topology::config cfg = {
.this_endpoint = ep1,
@@ -117,93 +110,42 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
set_abort_on_internal_error(true);
});

- topo.add_or_update_endpoint(id1, std::nullopt, endpoint_dc_rack::default_location, node::state::normal);
+ topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal);

auto node = const_cast<class node*>(topo.this_node());

- topo.update_node(*node, std::nullopt, ep1, std::nullopt, std::nullopt);
+ topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt);

BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);

- BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt, std::nullopt), std::runtime_error);
- BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt, std::nullopt), std::runtime_error);
+ BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt), std::runtime_error);
+ BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt), std::runtime_error);
BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
BOOST_REQUIRE_EQUAL(topo.find_node(id2), nullptr);

- topo.update_node(*node, std::nullopt, ep2, std::nullopt, std::nullopt);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep2), node);
-
auto dc_rack1 = endpoint_dc_rack{"DC1", "RACK1"};
- topo.update_node(*node, std::nullopt, std::nullopt, dc_rack1, std::nullopt);
+ topo.update_node(*node, std::nullopt, dc_rack1, std::nullopt);

BOOST_REQUIRE(topo.get_location(id1) == dc_rack1);

auto dc_rack2 = endpoint_dc_rack{"DC2", "RACK2"};
- topo.update_node(*node, std::nullopt, std::nullopt, dc_rack2, std::nullopt);
+ topo.update_node(*node, std::nullopt, dc_rack2, std::nullopt);

BOOST_REQUIRE(topo.get_location(id1) == dc_rack2);

BOOST_REQUIRE_NE(node->get_state(), locator::node::state::being_decommissioned);
- topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::being_decommissioned);
+ topo.update_node(*node, std::nullopt, std::nullopt, locator::node::state::being_decommissioned);

BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);

auto dc_rack3 = endpoint_dc_rack{"DC3", "RACK3"};
// Note: engage state option, but keep node::state value the same
// to reproduce #13502
- topo.update_node(*node, std::nullopt, ep3, dc_rack3, locator::node::state::being_decommissioned);
+ topo.update_node(*node, std::nullopt, dc_rack3, locator::node::state::being_decommissioned);

BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep3), node);
BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);
-
- // In state::left the node will remain indexed only by its host_id
- topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::left);
- BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
- BOOST_REQUIRE_EQUAL(topo.find_node(ep3), nullptr);
- BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
- BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::left);
-}
-
-SEASTAR_THREAD_TEST_CASE(test_add_or_update_by_host_id) {
- auto id1 = host_id::create_random_id();
- auto id2 = host_id::create_random_id();
- auto ep1 = gms::inet_address("127.0.0.1");
-
- // In this test we check that add_or_update_endpoint searches by host_id first.
- // We create two nodes, one matches by id, another - by ip,
- // and SCYLLA_ASSERT that add_or_update_endpoint updates the first.
- // We need to make the second node 'being_decommissioned', so that
- // it gets removed from ip index and we don't get the non-unique IP error.
-
- auto topo = topology({
- .this_host_id = id1,
- .local_dc_rack = endpoint_dc_rack::default_location,
- });
-
- topo.add_or_update_endpoint(id1, gms::inet_address{}, endpoint_dc_rack::default_location, node::state::normal);
- topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::being_decommissioned);
-
- topo.add_or_update_endpoint(id1, ep1, std::nullopt, node::state::bootstrapping);
-
- auto* n = topo.find_node(id1);
- BOOST_REQUIRE_EQUAL(n->get_state(), node::state::bootstrapping);
- BOOST_REQUIRE_EQUAL(n->host_id(), id1);
- BOOST_REQUIRE_EQUAL(n->endpoint(), ep1);
-
- auto* n2 = topo.find_node(ep1);
- BOOST_REQUIRE_EQUAL(n, n2);
-
- auto* n3 = topo.find_node(id2);
- BOOST_REQUIRE_EQUAL(n3->get_state(), node::state::being_decommissioned);
- BOOST_REQUIRE_EQUAL(n3->host_id(), id2);
- BOOST_REQUIRE_EQUAL(n3->endpoint(), ep1);
}

SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
@@ -214,7 +156,6 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
const auto id1 = host_id::create_random_id();
const auto ep1 = gms::inet_address("127.0.0.1");
const auto id2 = host_id::create_random_id();
- const auto ep2 = gms::inet_address("127.0.0.2");
const auto dc_rack1 = endpoint_dc_rack {
.dc = "dc1",
.rack = "rack1"
@@ -232,8 +173,8 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {

auto topo = topology(cfg);

- topo.add_or_update_endpoint(id1, ep1, dc_rack1, node::state::normal);
- topo.add_node(id2, ep2, dc_rack2, node::state::normal);
+ topo.add_or_update_endpoint(id1, dc_rack1, node::state::normal);
+ topo.add_node(id2, dc_rack2, node::state::normal);

BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1, id2}}}));
BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}, {"rack2", {id2}}}}}));
@@ -374,9 +315,7 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) {
auto id1 = host_id::create_random_id();
auto ep1 = gms::inet_address("127.0.0.1");
auto id2 = host_id::create_random_id();
- auto ep2 = gms::inet_address("127.0.0.2");
auto id3 = host_id::create_random_id();
- auto ep3 = gms::inet_address("127.0.0.3");

const auto dc_rack1 = endpoint_dc_rack {
.dc = "dc1",
@@ -397,8 +336,8 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) {

std::unordered_set<std::reference_wrapper<const locator::node>> nodes;

- nodes.insert(std::cref(topo.add_node(id2, ep2, dc_rack1, node::state::normal)));
- nodes.insert(std::cref(topo.add_node(id3, ep3, dc_rack1, node::state::left)));
+ nodes.insert(std::cref(topo.add_node(id2, dc_rack1, node::state::normal)));
+ nodes.insert(std::cref(topo.add_node(id3, dc_rack1, node::state::left)));

topo.for_each_node([&] (const locator::node& node) {
BOOST_REQUIRE(node.host_id() != id3);
diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc
index 142e0e4f598..a175df290f7 100644
--- a/test/boost/network_topology_strategy_test.cc
+++ b/test/boost/network_topology_strategy_test.cc
@@ -303,7 +303,7 @@ void simple_test() {
for (const auto& [ring_point, endpoint, id] : ring_points) {
std::unordered_set<token> tokens;
tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
- topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
+ topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
}).get();
@@ -411,7 +411,7 @@ void heavy_origin_test() {
stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
auto& topo = tm.get_topology();
for (const auto& [ring_point, endpoint, id] : ring_points) {
- topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
+ topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
co_await tm.update_normal_tokens(tokens[endpoint], id);
}
}).get();
@@ -483,7 +483,7 @@ SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_tablets_test) {
for (const auto& [ring_point, endpoint, id] : ring_points) {
std::unordered_set<token> tokens;
tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
- topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
+ topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
@@ -575,7 +575,7 @@ static void test_random_balancing(sharded<snitch_ptr>& snitch, gms::inet_address
for (const auto& [ring_point, endpoint, id] : ring_points) {
std::unordered_set<token> tokens;
tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
- topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
+ topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
@@ -864,12 +864,11 @@ void generate_topology(topology& topo, const std::unordered_map<sstring, size_t>
out = std::fill_n(out, rf, std::cref(dc));
}

- unsigned i = 0;
for (auto& node : nodes) {
const sstring& dc = dcs[udist(0, dcs.size() - 1)(e1)];
auto rc = racks_per_dc.at(dc);
auto r = udist(0, rc)(e1);
- topo.add_or_update_endpoint(node, inet_address((127u << 24) | ++i), endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal);
+ topo.add_or_update_endpoint(node, endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal);
}
}

@@ -1144,21 +1143,15 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {

const node* n1 = stm.get()->get_topology().find_node(host1);
BOOST_REQUIRE(n1);
- n1 = stm.get()->get_topology().find_node(ip1);
- BOOST_REQUIRE(n1);
BOOST_REQUIRE(bool(n1->is_this_node()));
BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
- BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack);
BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack);

const node* n2 = stm.get()->get_topology().find_node(host2);
BOOST_REQUIRE(n2);
- n2 = stm.get()->get_topology().find_node(ip2);
- BOOST_REQUIRE(n2);
BOOST_REQUIRE(!bool(n2->is_this_node()));
BOOST_REQUIRE_EQUAL(n2->host_id(), host2);
- BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2);
BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location);

// Local node cannot be removed
@@ -1171,8 +1164,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {

n1 = stm.get()->get_topology().find_node(host1);
BOOST_REQUIRE(n1);
- n1 = stm.get()->get_topology().find_node(ip1);
- BOOST_REQUIRE(n1);

// Removing node with no local node

@@ -1183,22 +1174,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {

n2 = stm.get()->get_topology().find_node(host2);
BOOST_REQUIRE(!n2);
- n2 = stm.get()->get_topology().find_node(ip2);
- BOOST_REQUIRE(!n2);

// Repopulate after clear_gently()

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
co_await tm.clear_gently();
- tm.update_host_id(host2, ip2);
- tm.update_host_id(host1, ip1); // this_node added last on purpose
+ tm.update_topology(host2, std::nullopt, std::nullopt);
+ tm.update_topology(host1, std::nullopt, std::nullopt); // this_node added last on purpose
}).get();

n1 = stm.get()->get_topology().find_node(host1);
BOOST_REQUIRE(n1);
BOOST_REQUIRE(bool(n1->is_this_node()));
BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
- BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack);
BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack);

@@ -1206,21 +1194,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
BOOST_REQUIRE(n2);
BOOST_REQUIRE(!bool(n2->is_this_node()));
BOOST_REQUIRE_EQUAL(n2->host_id(), host2);
- BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2);
BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location);

// get_location() should pick up endpoint_dc_rack from node info

stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
co_await tm.clear_gently();
- tm.get_topology().add_or_update_endpoint(host1, ip1, ip1_dc_rack_v2, node::state::being_decommissioned);
+ tm.get_topology().add_or_update_endpoint(host1, ip1_dc_rack_v2, node::state::being_decommissioned);
}).get();

n1 = stm.get()->get_topology().find_node(host1);
BOOST_REQUIRE(n1);
BOOST_REQUIRE(bool(n1->is_this_node()));
BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
- BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack_v2);
BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack_v2);
}
diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc
index 07ceaf04cac..e3e1d7c4d61 100644
--- a/test/boost/tablets_test.cc
+++ b/test/boost/tablets_test.cc
@@ -1019,7 +1019,7 @@ SEASTAR_TEST_CASE(test_sharder) {
auto table1 = table_id(utils::UUID_gen::get_time_UUID());

token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } });
- tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address());
+ tokm.get_topology().add_or_update_endpoint(h1);

std::vector<tablet_id> tablet_ids;
{
@@ -1234,7 +1234,7 @@ SEASTAR_TEST_CASE(test_intranode_sharding) {
auto table1 = table_id(utils::UUID_gen::get_time_UUID());

token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } });
- tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address());
+ tokm.get_topology().add_or_update_endpoint(h1);

auto leaving_replica = tablet_replica{h1, 5};
auto pending_replica = tablet_replica{h1, 7};
@@ -3339,7 +3339,7 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_
for (const auto& [ring_point, endpoint, id] : test_config.ring_points) {
std::unordered_set<token> tokens;
tokens.insert(dht::token{tests::d2t(ring_point / test_config.ring_points.size())});
- topo.add_or_update_endpoint(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1);
+ topo.add_or_update_endpoint(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1);
tm.update_host_id(id, endpoint);
co_await tm.update_normal_tokens(std::move(tokens), id);
}
diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc
index b73cf303438..f44423bc1e2 100644
--- a/test/lib/cql_test_env.cc
+++ b/test/lib/cql_test_env.cc
@@ -685,11 +685,10 @@ class single_node_cql_env : public cql_test_env {
host_id = linfo.host_id;
_sys_ks.local().save_local_info(std::move(linfo), _snitch.local()->get_location(), my_address, my_address).get();
}
- locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id, &cfg_in] (locator::token_metadata& tm) {
+ locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id] (locator::token_metadata& tm) {
auto& topo = tm.get_topology();
topo.set_host_id_cfg(hostid);
topo.add_or_update_endpoint(hostid,
- cfg_in.broadcast_address,
std::nullopt,
locator::node::state::normal,
smp::count);
diff --git a/test/perf/perf_sort_by_proximity.cc b/test/perf/perf_sort_by_proximity.cc
index 4d86bcea1df..438bf19ebd2 100644
--- a/test/perf/perf_sort_by_proximity.cc
+++ b/test/perf/perf_sort_by_proximity.cc
@@ -53,7 +53,6 @@ struct sort_by_proximity_topology {
auto id = locator::host_id{utils::UUID(0, i)};
nodes[dc][rack].emplace_back(id);
topology.add_or_update_endpoint(id,
- gms::inet_address((127u << 24) | i),
locator::endpoint_dc_rack{format("dc{}", dc), format("rack{}", rack)},
locator::node::state::normal);
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:31 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/topology.hh | 3 ---
db/virtual_tables.cc | 7 +++++--
locator/topology.cc | 18 ------------------
test/boost/locator_topology_test.cc | 12 +++---------
4 files changed, 8 insertions(+), 32 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index 7c1dda4081f..577bf315bc5 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -311,9 +311,6 @@ class topology {
const endpoint_dc_rack& get_location(host_id id) const {
return find_node(id)->dc_rack();
}
- // Get dc/rack location of a node identified by endpoint
- // The specified node must exist.
- const endpoint_dc_rack& get_location(const inet_address& ep) const;

// Get datacenter of this node
const sstring& get_datacenter() const noexcept {
diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc
index b9b3e0d5f0c..95cb7523da2 100644
--- a/db/virtual_tables.cc
+++ b/db/virtual_tables.cc
@@ -92,8 +92,11 @@ class cluster_status_table : public memtable_filling_virtual_table {
}
set_cell(cr, "host_id", hostid.uuid());

- sstring dc = tm.get_topology().get_location(endpoint).dc;
- set_cell(cr, "dc", dc);
+ if (tm.get_topology().has_node(hostid)) {
+ // Not all entries in gossiper are present in the topology
+ sstring dc = tm.get_topology().get_location(hostid).dc;
+ set_cell(cr, "dc", dc);
+ }

if (ownership.contains(endpoint)) {
set_cell(cr, "owns", ownership[endpoint]);
diff --git a/locator/topology.cc b/locator/topology.cc
index 7371c10df0a..c64a5be8513 100644
--- a/locator/topology.cc
+++ b/locator/topology.cc
@@ -554,24 +554,6 @@ bool topology::has_endpoint(inet_address ep) const
return has_node(ep);
}

-const endpoint_dc_rack& topology::get_location(const inet_address& ep) const {
- if (auto node = find_node(ep)) {
- return node->dc_rack();
- }
- // We should do the following check after lookup in nodes.
- // In tests, there may be no config for local node, so fall back to get_location()
- // only if no mapping is found. Otherwise, get_location() will return empty location
- // from config or random node, neither of which is correct.
- if (ep == _cfg.this_endpoint) {
- return get_location();
- }
- // FIXME -- this shouldn't happen. After topology is stable and is
- // correctly populated with endpoints, this should be replaced with
- // on_internal_error()
- tlogger.warn("Requested location for node {} not in topology. backtrace {}", ep, lazy_backtrace());
- return endpoint_dc_rack::default_location;
-}
-
void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const {
if (can_sort_by_proximity()) {
do_sort_by_proximity(address, addresses);
diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc
index c41ded826c8..99e815e8471 100644
--- a/test/boost/locator_topology_test.cc
+++ b/test/boost/locator_topology_test.cc
@@ -122,7 +122,7 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
auto node = const_cast<class node*>(topo.this_node());

topo.update_node(*node, std::nullopt, ep1, std::nullopt, std::nullopt);
-
+
BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);

BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt, std::nullopt), std::runtime_error);
@@ -138,31 +138,27 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
topo.update_node(*node, std::nullopt, std::nullopt, dc_rack1, std::nullopt);

BOOST_REQUIRE(topo.get_location(id1) == dc_rack1);
- BOOST_REQUIRE(topo.get_location(ep1) == dc_rack1);

auto dc_rack2 = endpoint_dc_rack{"DC2", "RACK2"};
topo.update_node(*node, std::nullopt, std::nullopt, dc_rack2, std::nullopt);

BOOST_REQUIRE(topo.get_location(id1) == dc_rack2);
- BOOST_REQUIRE(topo.get_location(ep1) == dc_rack2);

BOOST_REQUIRE_NE(node->get_state(), locator::node::state::being_decommissioned);
topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::being_decommissioned);
-
+
BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);

auto dc_rack3 = endpoint_dc_rack{"DC3", "RACK3"};
// Note: engage state option, but keep node::state value the same
// to reproduce #13502
topo.update_node(*node, std::nullopt, ep3, dc_rack3, locator::node::state::being_decommissioned);
-
+
BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
BOOST_REQUIRE_EQUAL(topo.find_node(ep3), node);
BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
- BOOST_REQUIRE(topo.get_location(ep2) == endpoint_dc_rack::default_location);
- BOOST_REQUIRE(topo.get_location(ep3) == dc_rack3);
BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);

// In state::left the node will remain indexed only by its host_id
@@ -172,8 +168,6 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
BOOST_REQUIRE_EQUAL(topo.find_node(ep3), nullptr);
BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
- BOOST_REQUIRE(topo.get_location(ep2) == endpoint_dc_rack::default_location);
- BOOST_REQUIRE(topo.get_location(ep3) == endpoint_dc_rack::default_location);
BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::left);
}

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:32 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/token_metadata.hh | 7 -------
locator/token_metadata.cc | 32 --------------------------------
2 files changed, 39 deletions(-)

diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
index b8420761242..4ac6a2143b4 100644
--- a/locator/token_metadata.hh
+++ b/locator/token_metadata.hh
@@ -239,13 +239,6 @@ class token_metadata final {
/** Return the unique host ID for an end-point. */
host_id get_host_id(inet_address endpoint) const;

- /// Return the unique host ID for an end-point or nullopt if not found.
- std::optional<host_id> get_host_id_if_known(inet_address endpoint) const;
-
- /** Return the end-point for a unique host ID or nullopt if not found. */
- std::optional<inet_address> get_endpoint_for_host_id_if_known(locator::host_id host_id) const;
-
- /** Return the end-point for a unique host ID */
inet_address get_endpoint_for_host_id(locator::host_id host_id) const;

/** @return a copy of the endpoint-to-id map for read-only operations */
diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 75603c58e6a..3320e37ed14 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -153,12 +153,6 @@ class token_metadata_impl final {
/** Return the unique host ID for an end-point. */
host_id get_host_id(inet_address endpoint) const;

- /// Return the unique host ID for an end-point or nullopt if not found.
- std::optional<host_id> get_host_id_if_known(inet_address endpoint) const;
-
- /** Return the end-point for a unique host ID or nullopt if not found.*/
- std::optional<inet_address> get_endpoint_for_host_id_if_known(host_id) const;
-
/** Return the end-point for a unique host ID.*/
inet_address get_endpoint_for_host_id(host_id) const;

@@ -543,22 +537,6 @@ host_id token_metadata_impl::get_host_id(inet_address endpoint) const {
}
}

-std::optional<host_id> token_metadata_impl::get_host_id_if_known(inet_address endpoint) const {
- if (const auto* node = _topology.find_node(endpoint)) [[likely]] {
- return node->host_id();
- } else {
- return std::nullopt;
- }
-}
-
-std::optional<inet_address> token_metadata_impl::get_endpoint_for_host_id_if_known(host_id host_id) const {
- if (const auto* node = _topology.find_node(host_id)) [[likely]] {
- return node->endpoint();
- } else {
- return std::nullopt;
- }
-}
-
inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) const {
if (const auto* node = _topology.find_node(host_id)) [[likely]] {
return node->endpoint();
@@ -1042,16 +1020,6 @@ token_metadata::get_host_id(inet_address endpoint) const {
return _impl->get_host_id(endpoint);
}

-std::optional<host_id>
-token_metadata::get_host_id_if_known(inet_address endpoint) const {
- return _impl->get_host_id_if_known(endpoint);
-}
-
-std::optional<token_metadata::inet_address>
-token_metadata::get_endpoint_for_host_id_if_known(host_id host_id) const {
- return _impl->get_endpoint_for_host_id_if_known(host_id);
-}
-
token_metadata::inet_address
token_metadata::get_endpoint_for_host_id(host_id host_id) const {
return _impl->get_endpoint_for_host_id(host_id);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:32 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/topology.hh | 4 ----
service/storage_service.hh | 2 +-
db/system_keyspace.cc | 2 +-
service/qos/service_level_controller.cc | 2 +-
4 files changed, 3 insertions(+), 7 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index 577bf315bc5..91af733a9f1 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -394,10 +394,6 @@ class topology {
return id == my_host_id();
}

- bool is_me(const inet_address& addr) const noexcept {
- return addr == my_address();
- }
-
private:
using random_engine_type = std::mt19937_64;

diff --git a/service/storage_service.hh b/service/storage_service.hh
index e72881cf1d4..9a15130ef0e 100644
--- a/service/storage_service.hh
+++ b/service/storage_service.hh
@@ -326,7 +326,7 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
return get_token_metadata_ptr()->get_topology().my_host_id();
}
bool is_me(inet_address addr) const noexcept {
- return get_token_metadata_ptr()->get_topology().is_me(addr);
+ return addr == get_broadcast_address();
}
bool is_me(locator::host_id id) const noexcept {
return get_token_metadata_ptr()->get_topology().is_me(id);
diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc
index 419e201d769..01edce0b7eb 100644
--- a/db/system_keyspace.cc
+++ b/db/system_keyspace.cc
@@ -2080,7 +2080,7 @@ future<> system_keyspace::update_peer_info(gms::inet_address ep, locator::host_i
if (!hid) {
on_internal_error(slogger, format("update_peer_info called with empty host_id, ep {}", ep));
}
- if (_db.get_token_metadata().get_topology().is_me(ep)) {
+ if (_db.get_token_metadata().get_topology().is_me(hid)) {
on_internal_error(slogger, format("update_peer_info called for this node: {}", ep));
}

diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc
index 9c1ebeb6c6d..bdcbaac2d58 100644
--- a/service/qos/service_level_controller.cc
+++ b/service/qos/service_level_controller.cc
@@ -894,7 +894,7 @@ future<> service_level_controller::do_remove_service_level(sstring name, bool re
void service_level_controller::on_join_cluster(const gms::inet_address& endpoint) { }

void service_level_controller::on_leave_cluster(const gms::inet_address& endpoint, const locator::host_id& hid) {
- if (this_shard_id() == global_controller && _token_metadata.get()->get_topology().is_me(endpoint)) {
+ if (this_shard_id() == global_controller && _token_metadata.get()->get_topology().is_me(hid)) {
_global_controller_db->dist_data_update_aborter.request_abort();
_global_controller_db->group0_aborter.request_abort();
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:33 AMJan 13
to scylladb-dev@googlegroups.com
The call already throw an error if there are more than one. Throw is
there are zero as well and drop the loops.
---
service/storage_service.cc | 21 ++++++++++-----------
1 file changed, 10 insertions(+), 11 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 0f21854a516..ee0310ce8a8 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -4521,13 +4521,17 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
slogger.warn("{}", msg);
throw std::runtime_error(msg);
}
+ if (req.replace_nodes.size() == 0) {
+ auto msg = ::format("replace[{}]: Replacing node was not specified", req.ops_uuid);
+ slogger.warn("{}", msg);
+ throw std::runtime_error(msg);
+ }
if (!coordinator_host_id) {
throw std::runtime_error("Coordinator host_id not found");
}
- mutate_token_metadata([coordinator, coordinator_host_id, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
- for (auto& x: req.replace_nodes) {
- auto existing_node = x.first;
- auto replacing_node = x.second;
+ auto existing_node = req.replace_nodes.begin()->first;
+ auto replacing_node = req.replace_nodes.begin()->second;
+ mutate_token_metadata([coordinator, coordinator_host_id, existing_node, replacing_node, &req, this] (mutable_token_metadata_ptr tmptr) mutable {
const auto existing_node_id = tmptr->get_host_id(existing_node);
const auto replacing_node_id = *coordinator_host_id;
slogger.info("replace[{}]: Added replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
@@ -4546,15 +4550,11 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing);
tmptr->update_host_id(replacing_node_id, replacing_node);
tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id);
- }
return make_ready_future<>();
}).get();
auto ignore_nodes = std::move(req.ignore_nodes);
- node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, req = std::move(req)] () mutable {
- return mutate_token_metadata([this, coordinator, coordinator_host_id, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
- for (auto& x: req.replace_nodes) {
- auto existing_node = x.first;
- auto replacing_node = x.second;
+ node_ops_insert(ops_uuid, coordinator, std::move(ignore_nodes), [this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] () mutable {
+ return mutate_token_metadata([this, coordinator, coordinator_host_id, existing_node, replacing_node, req = std::move(req)] (mutable_token_metadata_ptr tmptr) mutable {
const auto existing_node_id = tmptr->get_host_id(existing_node);
const auto replacing_node_id = *coordinator_host_id;
slogger.info("replace[{}]: Removed replacing_node={}/{} to replace existing_node={}/{}, coordinator={}/{}",
@@ -4564,7 +4564,6 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
const auto dc_rack = get_dc_rack_for(replacing_node_id);
tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal);
tmptr->remove_endpoint(replacing_node_id);
- }

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:33 AMJan 13
to scylladb-dev@googlegroups.com
---
service/storage_proxy.hh | 3 +--
service/storage_proxy.cc | 22 +++++++---------------
2 files changed, 8 insertions(+), 17 deletions(-)

diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh
index 43ec188f5bf..319f836ae52 100644
--- a/service/storage_proxy.hh
+++ b/service/storage_proxy.hh
@@ -360,8 +360,7 @@ class storage_proxy : public seastar::async_sharded_service<storage_proxy>, publ
host_id_vector_replica_set filter_replicas_for_read(db::consistency_level, const locator::effective_replication_map&, host_id_vector_replica_set live_endpoints, const host_id_vector_replica_set& preferred_endpoints, db::read_repair_decision, std::optional<locator::host_id>* extra, replica::column_family*) const;
// As above with read_repair_decision=NONE, extra=nullptr.
host_id_vector_replica_set filter_replicas_for_read(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set& live_endpoints, const host_id_vector_replica_set& preferred_endpoints, replica::column_family*) const;
- bool is_alive(const gms::inet_address&) const;
- bool is_alive_id(const locator::effective_replication_map& erm, const locator::host_id&) const;
+ bool is_alive(const locator::effective_replication_map& erm, const locator::host_id&) const;
result<::shared_ptr<abstract_read_executor>> get_read_executor(lw_shared_ptr<query::read_command> cmd,
locator::effective_replication_map_ptr ermp,
schema_ptr schema,
diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index d54b3bd7aab..247283f153d 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -185,10 +185,6 @@ locator::host_id storage_proxy::my_host_id(const locator::effective_replication_
return erm.get_topology().my_host_id();
}

-bool storage_proxy::is_me(gms::inet_address addr) const noexcept {
- return local_db().get_token_metadata().get_topology().is_me(addr);
-}
-
bool storage_proxy::is_me(const locator::effective_replication_map& erm, locator::host_id id) const noexcept {
return erm.get_topology().is_me(id);
}
@@ -3391,7 +3387,7 @@ storage_proxy::create_write_response_handler_helper(schema_ptr s, const dht::tok
live_endpoints.reserve(all.size());
dead_endpoints.reserve(all.size());
std::partition_copy(all.begin(), all.end(), std::back_inserter(live_endpoints),
- std::back_inserter(dead_endpoints), std::bind_front(&storage_proxy::is_alive_id, this, std::cref(*erm)));
+ std::back_inserter(dead_endpoints), std::bind_front(&storage_proxy::is_alive, this, std::cref(*erm)));

db::per_partition_rate_limit::info rate_limit_info;
if (allow_limit && _db.local().can_apply_per_partition_rate_limit(*s, db::operation_type::write)) {
@@ -3768,7 +3764,7 @@ storage_proxy::get_paxos_participants(const sstring& ks_name, const locator::eff

auto all_as_spans = std::array{std::span(natural_endpoints), std::span(pending_endpoints)};
std::ranges::copy(all_as_spans | std::views::join |
- std::views::filter(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm))), std::back_inserter(live_endpoints));
+ std::views::filter(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm))), std::back_inserter(live_endpoints));

if (live_endpoints.size() < required_participants) {
throw exceptions::unavailable_exception(cl_for_paxos, required_participants, live_endpoints.size());
@@ -4018,7 +4014,7 @@ storage_proxy::mutate_atomically_result(std::vector<mutation> mutations, db::con
std::ranges::to<std::unordered_set<locator::host_id>>());
}
auto local_rack = topology.get_rack();
- auto chosen_endpoints = endpoint_filter(std::bind_front(&storage_proxy::is_alive_id, &_p, std::cref(*_ermp)), local_addr,
+ auto chosen_endpoints = endpoint_filter(std::bind_front(&storage_proxy::is_alive, &_p, std::cref(*_ermp)), local_addr,
local_rack, local_token_owners);

if (chosen_endpoints.empty()) {
@@ -4172,7 +4168,7 @@ future<> storage_proxy::send_to_endpoint(
std::array{std::span(pending_endpoints), std::span(target.begin(), target.end())} | std::views::join,
std::inserter(targets, targets.begin()),
std::back_inserter(dead_endpoints),
- std::bind_front(&storage_proxy::is_alive_id, this, std::cref(*erm)));
+ std::bind_front(&storage_proxy::is_alive, this, std::cref(*erm)));
slogger.trace("Creating write handler with live: {}; dead: {}", targets, dead_endpoints);
db::assure_sufficient_live_nodes(cl, *erm, targets, pending_endpoints);
return create_write_response_handler(
@@ -6601,7 +6597,7 @@ future<bool> storage_proxy::cas(schema_ptr schema, shared_ptr<cas_request> reque

host_id_vector_replica_set storage_proxy::get_live_endpoints(const locator::effective_replication_map& erm, const dht::token& token) const {
host_id_vector_replica_set eps = erm.get_natural_replicas(token);
- auto itend = std::ranges::remove_if(eps, std::not_fn(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm)))).begin();
+ auto itend = std::ranges::remove_if(eps, std::not_fn(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm)))).begin();
eps.erase(itend, eps.end());
return eps;
}
@@ -6626,7 +6622,7 @@ void storage_proxy::sort_endpoints_by_proximity(const locator::effective_replica
host_id_vector_replica_set storage_proxy::get_endpoints_for_reading(const sstring& ks_name, const locator::effective_replication_map& erm, const dht::token& token) const {
auto endpoints = erm.get_replicas_for_reading(token);
validate_read_replicas(erm, endpoints);
- auto it = std::ranges::remove_if(endpoints, std::not_fn(std::bind_front(&storage_proxy::is_alive_id, this, std::cref(erm)))).begin();
+ auto it = std::ranges::remove_if(endpoints, std::not_fn(std::bind_front(&storage_proxy::is_alive, this, std::cref(erm)))).begin();
endpoints.erase(it, endpoints.end());
sort_endpoints_by_proximity(erm, endpoints);
return endpoints;
@@ -6663,11 +6659,7 @@ storage_proxy::filter_replicas_for_read(
return filter_replicas_for_read(cl, erm, live_endpoints, preferred_endpoints, db::read_repair_decision::NONE, nullptr, cf);
}

-bool storage_proxy::is_alive(const gms::inet_address& ep) const {
- return _remote ? _remote->is_alive(ep) : is_me(ep);
-}
-
-bool storage_proxy::is_alive_id(const locator::effective_replication_map& erm, const locator::host_id& ep) const {
+bool storage_proxy::is_alive(const locator::effective_replication_map& erm, const locator::host_id& ep) const {
return is_me(erm, ep) || (_remote ? _remote->is_alive(ep) : false);
}

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:35 AMJan 13
to scylladb-dev@googlegroups.com
---
gms/gossiper.hh | 3 ++-
gms/gossiper.cc | 8 ++++++--
service/migration_manager.cc | 6 ++----
service/storage_service.cc | 12 ++++++------
4 files changed, 16 insertions(+), 13 deletions(-)

diff --git a/gms/gossiper.hh b/gms/gossiper.hh
index 595f7103ca6..1b647370ff3 100644
--- a/gms/gossiper.hh
+++ b/gms/gossiper.hh
@@ -296,7 +296,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
*/
future<> unregister_(shared_ptr<i_endpoint_state_change_subscriber> subscriber);

- std::set<inet_address> get_live_members() const;
+ std::set<locator::host_id> get_live_members() const;

std::set<locator::host_id> get_live_token_owners() const;

@@ -528,6 +528,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
future<> wait_alive(std::vector<gms::inet_address> nodes, std::chrono::milliseconds timeout);
future<> wait_alive(std::vector<locator::host_id> nodes, std::chrono::milliseconds timeout);
future<> wait_alive(noncopyable_function<std::vector<locator::host_id>()> get_nodes, std::chrono::milliseconds timeout);
+ std::set<inet_address> get_live_members_helper() const;

// Wait for `n` live nodes to show up in gossip (including ourself).
future<> wait_for_live_nodes_to_show_up(size_t n);
diff --git a/gms/gossiper.cc b/gms/gossiper.cc
index c303230316e..a8ec00dc816 100644
--- a/gms/gossiper.cc
+++ b/gms/gossiper.cc
@@ -924,7 +924,7 @@ future<std::set<inet_address>> gossiper::get_live_members_synchronized() {
return container().invoke_on(0, [] (gms::gossiper& g) -> future<std::set<inet_address>> {
// Make sure the value we return is synchronized on all shards
auto lock = co_await g.lock_endpoint_update_semaphore();
- co_return g.get_live_members();
+ co_return g.get_live_members_helper();
});
}

@@ -1175,7 +1175,7 @@ future<> gossiper::unregister_(shared_ptr<i_endpoint_state_change_subscriber> su
return _subscribers.remove(subscriber);
}

-std::set<inet_address> gossiper::get_live_members() const {
+std::set<inet_address> gossiper::get_live_members_helper() const {
std::set<inet_address> live_members(_live_endpoints.begin(), _live_endpoints.end());
auto myip = get_broadcast_address();
logger.debug("live_members before={}", live_members);
@@ -1186,6 +1186,10 @@ std::set<inet_address> gossiper::get_live_members() const {
return live_members;
}

+std::set<locator::host_id> gossiper::get_live_members() const {
+ return get_live_members_helper() | std::views::transform([this] (inet_address ip) { return get_host_id(ip); }) | std::ranges::to<std::set>();
+}
+
std::set<locator::host_id> gossiper::get_live_token_owners() const {
std::set<locator::host_id> token_owners;
auto normal_token_owners = get_token_metadata_ptr()->get_normal_token_owners();
diff --git a/service/migration_manager.cc b/service/migration_manager.cc
index a4474034be4..568f2061b05 100644
--- a/service/migration_manager.cc
+++ b/service/migration_manager.cc
@@ -919,13 +919,11 @@ future<> migration_manager::announce_without_raft(std::vector<mutation> schema,
try {
using namespace std::placeholders;
auto all_live = _gossiper.get_live_members();
- auto live_members = all_live | std::views::filter([this, my_address = _messaging.broadcast_address()] (const gms::inet_address& endpoint) {
+ auto live_members = all_live | std::views::filter([my_address = _gossiper.my_host_id()] (const locator::host_id& endpoint) {
// only push schema to nodes with known and equal versions
return endpoint != my_address;
});
- // FIXME: gossiper should return host id set
- auto live_host_ids = live_members | std::views::transform([&] (const gms::inet_address& ip) { return _gossiper.get_host_id(ip); });
- co_await coroutine::parallel_for_each(live_host_ids,
+ co_await coroutine::parallel_for_each(live_members,
std::bind(std::mem_fn(&migration_manager::push_schema_mutation), this, std::placeholders::_1, schema));
} catch (...) {
mlogger.error("failed to announce migration to all nodes: {}", std::current_exception());
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 4c1b4d91415..16c23fc8a20 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2727,7 +2727,7 @@ future<> storage_service::maybe_reconnect_to_preferred_ip(inet_address ep, inet_
}

const auto& topo = get_token_metadata().get_topology();
- if (topo.get_datacenter() == topo.get_datacenter(ep) && _messaging.local().get_preferred_ip(ep) != local_ip) {
+ if (topo.get_datacenter() == topo.get_datacenter(_gossiper.get_host_id(ep)) && _messaging.local().get_preferred_ip(ep) != local_ip) {
slogger.debug("Initiated reconnect to an Internal IP {} for the {}", local_ip, ep);
co_await _messaging.invoke_on_all([ep, local_ip] (auto& local_ms) {
local_ms.cache_preferred_ip(ep, local_ip);
@@ -3601,17 +3601,17 @@ future<std::unordered_map<sstring, std::vector<sstring>>> storage_service::descr
std::unordered_map<sstring, std::vector<sstring>> results;
netw::messaging_service& ms = _messaging.local();
return map_reduce(std::move(live_hosts), [&ms, as = abort_source()] (auto host) mutable {
- auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, netw::msg_addr{ host, 0 }, as);
+ auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, host, as);
return std::move(f0).then_wrapped([host] (auto f) {
if (f.failed()) {
f.ignore_ready_future();
- return std::pair<gms::inet_address, std::optional<table_schema_version>>(host, std::nullopt);
+ return std::pair<locator::host_id, std::optional<table_schema_version>>(host, std::nullopt);
}
- return std::pair<gms::inet_address, std::optional<table_schema_version>>(host, f.get());
+ return std::pair<locator::host_id, std::optional<table_schema_version>>(host, f.get());
});
- }, std::move(results), [] (auto results, auto host_and_version) {
+ }, std::move(results), [this] (auto results, auto host_and_version) {
auto version = host_and_version.second ? host_and_version.second->to_sstring() : UNREACHABLE;
- results.try_emplace(version).first->second.emplace_back(fmt::to_string(host_and_version.first));
+ results.try_emplace(version).first->second.emplace_back(fmt::to_string(_address_map.get(host_and_version.first)));
return results;
}).then([this] (auto results) {
// we're done: the results map is ready to return to the client. the rest is just debug logging:
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:35 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/topology.hh | 10 ----------
1 file changed, 10 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index db5921e761f..7c1dda4081f 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -324,11 +324,6 @@ class topology {
const sstring& get_datacenter(host_id id) const {
return get_location(id).dc;
}
- // Get datacenter of a node identified by endpoint
- // The specified node must exist.
- const sstring& get_datacenter(inet_address ep) const {
- return get_location(ep).dc;
- }

// Get rack of this node
const sstring& get_rack() const noexcept {
@@ -339,11 +334,6 @@ class topology {
const sstring& get_rack(host_id id) const {
return get_location(id).rack;
}
- // Get rack of a node identified by endpoint
- // The specified node must exist.
- const sstring& get_rack(inet_address ep) const {
- return get_location(ep).rack;
- }

auto get_local_dc_filter() const noexcept {
return [ this, local_dc = get_datacenter() ] (auto ep) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:35 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to use host id based
counterparts. Messaging service gets a function that maps from ips to id
when is starts listening.
---
message/messaging_service.hh | 6 ++++--
main.cc | 13 ++++++++++++-
message/messaging_service.cc | 21 +++++++++++++--------
test/lib/cql_test_env.cc | 2 +-
test/manual/message.cc | 2 +-
5 files changed, 31 insertions(+), 13 deletions(-)

diff --git a/message/messaging_service.hh b/message/messaging_service.hh
index 873aa83c903..083caee6eab 100644
--- a/message/messaging_service.hh
+++ b/message/messaging_service.hh
@@ -337,6 +337,8 @@ class messaging_service : public seastar::async_sharded_service<messaging_servic
private:
config _cfg;
locator::shared_token_metadata* _token_metadata = nullptr;
+ // a function that maps from ip to host id if known (returns default constructable host_id if there is no mapping)
+ std::function<locator::host_id(gms::inet_address)> _address_to_host_id_mapper;
// map: Node broadcast address -> Node internal IP, and the reversed mapping, for communication within the same data center
std::unordered_map<gms::inet_address, gms::inet_address> _preferred_ip_cache, _preferred_to_endpoint;
std::unique_ptr<rpc_protocol_wrapper> _rpc;
@@ -378,7 +380,7 @@ class messaging_service : public seastar::async_sharded_service<messaging_servic
~messaging_service();

future<> start();
- future<> start_listen(locator::shared_token_metadata& stm);
+ future<> start_listen(locator::shared_token_metadata& stm, std::function<locator::host_id(gms::inet_address)> address_to_host_id_mapper);
uint16_t port() const noexcept {
return _cfg.port;
}
@@ -455,7 +457,7 @@ class messaging_service : public seastar::async_sharded_service<messaging_servic

bool is_host_banned(locator::host_id);

- sstring client_metrics_domain(unsigned idx, inet_address addr) const;
+ sstring client_metrics_domain(unsigned idx, inet_address addr, std::optional<locator::host_id> id) const;

public:
// Return rpc::protocol::client for a shard which is a ip + cpuid pair.
diff --git a/main.cc b/main.cc
index 164cdc47167..0483e957a40 100644
--- a/main.cc
+++ b/main.cc
@@ -2090,7 +2090,18 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
group0_service.setup_group0_if_exist(sys_ks.local(), ss.local(), qp.local(), mm.local()).get();

with_scheduling_group(maintenance_scheduling_group, [&] {
- return messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata));
+ return messaging.invoke_on_all([&] (auto& ms) {
+ return ms.start_listen(token_metadata.local(), [&gossiper] (gms::inet_address ip) {
+ if (ip == gossiper.local().get_broadcast_address()) {
+ return gossiper.local().my_host_id();
+ }
+ try {
+ return gossiper.local().get_host_id(ip);
+ } catch (...) {
+ return locator::host_id{};
+ }
+ });
+ });
}).get();

const auto generation_number = gms::generation_type(sys_ks.local().increment_and_get_generation().get());
diff --git a/message/messaging_service.cc b/message/messaging_service.cc
index 22b6e35eabd..8dc7ce6e9e0 100644
--- a/message/messaging_service.cc
+++ b/message/messaging_service.cc
@@ -299,8 +299,9 @@ future<> messaging_service::start() {
return make_ready_future<>();
}

-future<> messaging_service::start_listen(locator::shared_token_metadata& stm) {
+future<> messaging_service::start_listen(locator::shared_token_metadata& stm, std::function<locator::host_id(gms::inet_address)> address_to_host_id_mapper) {
_token_metadata = &stm;
+ _address_to_host_id_mapper = std::move(address_to_host_id_mapper);
do_start_listen();
return make_ready_future<>();
}
@@ -308,20 +309,21 @@ future<> messaging_service::start_listen(locator::shared_token_metadata& stm) {
bool messaging_service::topology_known_for(inet_address addr) const {
// The token metadata pointer is nullptr before
// the service is start_listen()-ed and after it's being shutdown()-ed.
+ const locator::node* node;
return _token_metadata
- && _token_metadata->get()->get_topology().has_endpoint(addr);
+ && (node = _token_metadata->get()->get_topology().find_node(_address_to_host_id_mapper(addr))) && !node->is_none();
}

// Precondition: `topology_known_for(addr)`.
bool messaging_service::is_same_dc(inet_address addr) const {
const auto& topo = _token_metadata->get()->get_topology();
- return topo.get_datacenter(addr) == topo.get_datacenter();
+ return topo.get_datacenter(_address_to_host_id_mapper(addr)) == topo.get_datacenter();
}

// Precondition: `topology_known_for(addr)`.
bool messaging_service::is_same_rack(inet_address addr) const {
const auto& topo = _token_metadata->get()->get_topology();
- return topo.get_rack(addr) == topo.get_rack();
+ return topo.get_rack(_address_to_host_id_mapper(addr)) == topo.get_rack();
}

// The socket metrics domain defines the way RPC metrics are grouped
@@ -334,12 +336,15 @@ bool messaging_service::is_same_rack(inet_address addr) const {
// that the isolation cookie suits very well, because these cookies
// are different for different indices and are more informative than
// plain numbers
-sstring messaging_service::client_metrics_domain(unsigned idx, inet_address addr) const {
+sstring messaging_service::client_metrics_domain(unsigned idx, inet_address addr, std::optional<locator::host_id> id) const {
sstring ret = _scheduling_info_for_connection_index[idx].isolation_cookie;
+ if (!id) {
+ id = _address_to_host_id_mapper(addr);
+ }
if (_token_metadata) {
const auto& topo = _token_metadata->get()->get_topology();
- if (topo.has_endpoint(addr)) {
- ret += ":" + topo.get_datacenter(addr);
+ if (topo.has_node(*id)) {
+ ret += ":" + topo.get_datacenter(*id);
}
}
return ret;
@@ -1106,7 +1111,7 @@ shared_ptr<messaging_service::rpc_protocol_client_wrapper> messaging_service::ge
opts.tcp_nodelay = must_tcp_nodelay;
opts.reuseaddr = true;
opts.isolation_cookie = _scheduling_info_for_connection_index[idx].isolation_cookie;
- opts.metrics_domain = client_metrics_domain(idx, id.addr); // not just `addr` as the latter may be internal IP
+ opts.metrics_domain = client_metrics_domain(idx, id.addr, host_id); // not just `addr` as the latter may be internal IP

SCYLLA_ASSERT(!must_encrypt || _credentials);

diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc
index c1c906859ec..b73cf303438 100644
--- a/test/lib/cql_test_env.cc
+++ b/test/lib/cql_test_env.cc
@@ -742,7 +742,7 @@ class single_node_cql_env : public cql_test_env {
// Once the seastar issue is fixed, we can just keep the tmp socket aliva across
// the listen invoke below.
tmp = {};
- _ms.invoke_on_all(&netw::messaging_service::start_listen, std::ref(_token_metadata)).get();
+ _ms.invoke_on_all(&netw::messaging_service::start_listen, std::ref(_token_metadata), [host_id] (gms::inet_address ip) {return host_id; }).get();
}
} catch (std::system_error& e) {
// if we still hit a used port (quick other process), just shut down ms and try again.
diff --git a/test/manual/message.cc b/test/manual/message.cc
index 7e153edd3e0..d32a9891a8d 100644
--- a/test/manual/message.cc
+++ b/test/manual/message.cc
@@ -224,7 +224,7 @@ int main(int ac, char ** av) {
auto deinit_testers = deferred_action([&testers] {
testers.invoke_on_all(&tester::deinit_handler).get();
});
- messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata)).get();
+ messaging.invoke_on_all(&netw::messaging_service::start_listen, std::ref(token_metadata), [] (gms::inet_address ip){ return locator::host_id{}; }).get();
if (config.contains("server")) {
auto ip = config["server"].as<std::string>();
auto cpuid = config["cpuid"].as<uint32_t>();
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:36 AMJan 13
to scylladb-dev@googlegroups.com
---
service/storage_service.hh | 2 +-
service/storage_service.cc | 20 ++++++++++----------
2 files changed, 11 insertions(+), 11 deletions(-)

diff --git a/service/storage_service.hh b/service/storage_service.hh
index 9a15130ef0e..70a7eeba6ab 100644
--- a/service/storage_service.hh
+++ b/service/storage_service.hh
@@ -529,7 +529,7 @@ class storage_service : public service::migration_listener, public gms::i_endpoi

std::unordered_set<token> get_tokens_for(inet_address endpoint);
std::optional<locator::endpoint_dc_rack> get_dc_rack_for(const gms::endpoint_state& ep_state);
- std::optional<locator::endpoint_dc_rack> get_dc_rack_for(inet_address endpoint);
+ std::optional<locator::endpoint_dc_rack> get_dc_rack_for(locator::host_id endpoint);
private:
// Should be serialized under token_metadata_lock.
future<> replicate_to_all_cores(mutable_token_metadata_ptr tmptr) noexcept;
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 16c23fc8a20..74b1b1358c9 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2326,7 +2326,7 @@ future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::per
}
tmptr->remove_endpoint(host_id);
}
- tmptr->update_topology(host_id, get_dc_rack_for(endpoint), locator::node::state::bootstrapping);
+ tmptr->update_topology(host_id, get_dc_rack_for(host_id), locator::node::state::bootstrapping);
tmptr->add_bootstrap_tokens(tokens, host_id);
tmptr->update_host_id(host_id, endpoint);

@@ -2536,7 +2536,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
do_notify_joined = true;
}

- const auto dc_rack = get_dc_rack_for(endpoint);
+ const auto dc_rack = get_dc_rack_for(host_id);
tmptr->update_topology(host_id, dc_rack, locator::node::state::normal);
co_await tmptr->update_normal_tokens(owned_tokens, host_id);
if (replaced_id) {
@@ -2653,7 +2653,7 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
} else {
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
- const auto dc_rack = get_dc_rack_for(endpoint);
+ const auto dc_rack = get_dc_rack_for(host_id);
tmptr->update_host_id(host_id, endpoint);
tmptr->update_topology(host_id, dc_rack);
co_await replicate_to_all_cores(std::move(tmptr));
@@ -2873,7 +2873,7 @@ std::optional<locator::endpoint_dc_rack> storage_service::get_dc_rack_for(const
};
}

-std::optional<locator::endpoint_dc_rack> storage_service::get_dc_rack_for(inet_address endpoint) {
+std::optional<locator::endpoint_dc_rack> storage_service::get_dc_rack_for(locator::host_id endpoint) {
auto eps = _gossiper.get_endpoint_state_ptr(endpoint);
if (!eps) {
return std::nullopt;
@@ -3402,12 +3402,12 @@ storage_service::prepare_replacement_info(std::unordered_set<gms::inet_address>
}
}

- auto dc_rack = get_dc_rack_for(replace_address).value_or(locator::endpoint_dc_rack::default_location);
-
if (!replace_host_id) {
replace_host_id = _gossiper.get_host_id(replace_address);
}

+ auto dc_rack = get_dc_rack_for(replace_host_id).value_or(locator::endpoint_dc_rack::default_location);
+
auto ri = replacement_info {
.tokens = std::move(tokens),
.dc_rack = std::move(dc_rack),
@@ -4554,7 +4554,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
// excluded from normal_endpoints (maybe_remove_node_being_replaced function).
// In handle_state_normal we'll remap the IP to the new host_id.
tmptr->update_topology(existing_node_id, std::nullopt, locator::node::state::being_replaced);
- tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node), locator::node::state::replacing);
+ tmptr->update_topology(replacing_node_id, get_dc_rack_for(replacing_node_id), locator::node::state::replacing);
tmptr->update_host_id(replacing_node_id, replacing_node);
tmptr->add_replacing_endpoint(existing_node_id, replacing_node_id);
}
@@ -4572,7 +4572,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
req.ops_uuid, replacing_node, replacing_node_id, existing_node, existing_node_id, coordinator, *coordinator_host_id);

tmptr->del_replacing_endpoint(existing_node_id);
- const auto dc_rack = get_dc_rack_for(replacing_node);
+ const auto dc_rack = get_dc_rack_for(replacing_node_id);
tmptr->update_topology(existing_node_id, dc_rack, locator::node::state::normal);
tmptr->remove_endpoint(replacing_node_id);
}
@@ -4618,7 +4618,7 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
auto& endpoint = x.first;
auto tokens = std::unordered_set<dht::token>(x.second.begin(), x.second.end());
const auto host_id = *coordinator_host_id;
- const auto dc_rack = get_dc_rack_for(endpoint);
+ const auto dc_rack = get_dc_rack_for(host_id);
slogger.info("bootstrap[{}]: Added node={}/{} as bootstrap, coordinator={}/{}",
req.ops_uuid, endpoint, host_id, coordinator, *coordinator_host_id);
tmptr->update_host_id(host_id, endpoint);
@@ -5350,7 +5350,7 @@ future<> storage_service::update_topology_change_info(mutable_token_metadata_ptr
return std::nullopt;
}

- return get_dc_rack_for(tm.get_endpoint_for_host_id(host_id));
+ return get_dc_rack_for(host_id);
});
co_await tmptr->update_topology_change_info(get_dc_rack_by_host_id);
} catch (...) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:36 AMJan 13
to scylladb-dev@googlegroups.com
Instead use gossiper and peers table to retrieve same information.
Token_metadata is created from the mix of those two anyway. The goal is
to drop ips from token_metadata entirely.
---
service/storage_service.cc | 28 ++++++++++++++++++++++------
1 file changed, 22 insertions(+), 6 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 10241a40f70..4c1b4d91415 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2362,11 +2362,26 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
if (tmptr->is_normal_token_owner(host_id)) {
slogger.info("handle_state_normal: node {}/{} was already a normal token owner", endpoint, host_id);
}
- auto existing = tmptr->get_endpoint_for_host_id_if_known(host_id);

// Old node in replace-with-same-IP scenario.
std::optional<locator::host_id> replaced_id;

+ auto ips = _gossiper.get_nodes_with_host_id(host_id);
+
+ std::optional<inet_address> existing;
+
+ if (tmptr->get_topology().find_node(host_id)) {
+ // If node is not in the topology there is no existsing address
+ // If there are two addresses for the same id the "other" one is existing
+ // If there is only one it is existing
+ if (ips.size() == 2) {
+ if (ips.erase(endpoint) == 0) {
+ on_internal_error(slogger, fmt::format("Did not find endpoint {} in gossiper {}", endpoint, ips));
+ }
+ }
+ existing = *ips.begin();
+ }
+
if (existing && *existing != endpoint) {
// This branch in taken when a node changes its IP address.

@@ -2412,11 +2427,12 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
// We do this after update_normal_tokens, allowing for tokens to be properly
// migrated to the new host_id.

- slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
- if (const auto old_host_id = tmptr->get_host_id_if_known(endpoint); old_host_id && *old_host_id != host_id) {
- // Replace with same IP scenario
- slogger.info("The IP {} previously owned host ID {}", endpoint, *old_host_id);
- replaced_id = *old_host_id;
+ auto peers = co_await _sys_ks.local().load_host_ids();
+ if (peers.contains(endpoint) && peers[endpoint] != host_id) {
+ replaced_id = peers[endpoint];
+ slogger.info("The IP {} previously owned host ID {}", endpoint, *replaced_id);
+ } else {
+ slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
}
} else {
// This branch is taken if this node wasn't involved in node_ops
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:37 AMJan 13
to scylladb-dev@googlegroups.com
---
db/consistency_level.hh | 10 ++--------
db/consistency_level.cc | 19 ++++++-------------
2 files changed, 8 insertions(+), 21 deletions(-)

diff --git a/db/consistency_level.hh b/db/consistency_level.hh
index 787702b1273..f9619a97c4e 100644
--- a/db/consistency_level.hh
+++ b/db/consistency_level.hh
@@ -61,15 +61,9 @@ is_sufficient_live_nodes(consistency_level cl,
const locator::effective_replication_map& erm,
const host_id_vector_replica_set& live_endpoints);

-template<typename Range, typename PendingRange = std::array<gms::inet_address, 0>>
void assure_sufficient_live_nodes(
consistency_level cl,
const locator::effective_replication_map& erm,
- const Range& live_endpoints,
- const PendingRange& pending_endpoints = std::array<gms::inet_address, 0>());
-
-extern template void assure_sufficient_live_nodes(consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
-extern template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
-extern template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set&, const host_id_vector_topology_change&);
-
+ const host_id_vector_replica_set& live_endpoints,
+ const host_id_vector_topology_change& pending_endpoints = host_id_vector_topology_change());
}
diff --git a/db/consistency_level.cc b/db/consistency_level.cc
index 1ecefa837eb..7bdaaaa046e 100644
--- a/db/consistency_level.cc
+++ b/db/consistency_level.cc
@@ -113,11 +113,10 @@ bool is_datacenter_local(consistency_level l) {
return l == consistency_level::LOCAL_ONE || l == consistency_level::LOCAL_QUORUM;
}

-template <typename Range, typename PendingRange = std::array<gms::inet_address, 0>>
std::unordered_map<sstring, dc_node_count> count_per_dc_endpoints(
const locator::effective_replication_map& erm,
- const Range& live_endpoints,
- const PendingRange& pending_endpoints = std::array<gms::inet_address, 0>()) {
+ const host_id_vector_replica_set& live_endpoints,
+ const host_id_vector_topology_change& pending_endpoints = host_id_vector_topology_change()) {
using namespace locator;

auto& rs = erm.get_replication_strategy();
@@ -148,12 +147,11 @@ std::unordered_map<sstring, dc_node_count> count_per_dc_endpoints(
return dc_endpoints;
}

-template<typename Range, typename PendingRange>
bool assure_sufficient_live_nodes_each_quorum(
consistency_level cl,
const locator::effective_replication_map& erm,
- const Range& live_endpoints,
- const PendingRange& pending_endpoints) {
+ const host_id_vector_replica_set& live_endpoints,
+ const host_id_vector_topology_change& pending_endpoints) {
using namespace locator;

auto& rs = erm.get_replication_strategy();
@@ -175,12 +173,11 @@ bool assure_sufficient_live_nodes_each_quorum(
return false;
}

-template<typename Range, typename PendingRange>
void assure_sufficient_live_nodes(
consistency_level cl,
const locator::effective_replication_map& erm,
- const Range& live_endpoints,
- const PendingRange& pending_endpoints) {
+ const host_id_vector_replica_set& live_endpoints,
+ const host_id_vector_topology_change& pending_endpoints) {
size_t need = block_for(erm, cl);

auto adjust_live_for_error = [] (size_t live, size_t pending) {
@@ -228,10 +225,6 @@ void assure_sufficient_live_nodes(
}
}

-template void assure_sufficient_live_nodes(consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const std::array<gms::inet_address, 0>&);
-template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const inet_address_vector_replica_set&, const utils::small_vector<gms::inet_address, 1ul>&);
-template void assure_sufficient_live_nodes(db::consistency_level, const locator::effective_replication_map&, const host_id_vector_replica_set&, const host_id_vector_topology_change&);
-
host_id_vector_replica_set
filter_for_query(consistency_level cl,
const locator::effective_replication_map& erm,
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:37 AMJan 13
to scylladb-dev@googlegroups.com
Do not use ip from topology::node, but look it up in address map
instead. We want to drop ip from the topology::node.
---
service/storage_service.cc | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index c0cc66feff4..a5f66f525f9 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -772,9 +772,9 @@ future<> storage_service::topology_state_load(state_change_hint hint) {
{
std::vector<future<>> futures;
get_token_metadata_ptr()->get_topology().for_each_node([&](const locator::node& n) {
- const auto ep = n.endpoint();
- if (ep != inet_address{} && !saved_tmpr->get_topology().has_endpoint(ep)) {
- futures.push_back(remove_rpc_client_with_ignored_topology(ep, n.host_id()));
+ const auto ep = n.host_id();
+ if (auto ip_opt = _address_map.find(ep); ip_opt && !saved_tmpr->get_topology().has_node(ep)) {
+ futures.push_back(remove_rpc_client_with_ignored_topology(*ip_opt, n.host_id()));
}
});
co_await when_all_succeed(futures.begin(), futures.end()).discard_result();
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:39 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ip from the locator::node.
---
service/storage_service.cc | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index a5f66f525f9..dc806124483 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -5261,10 +5261,11 @@ storage_service::describe_ring_for_table(const sstring& keyspace_name, const sst
for (auto& r : replicas) {
dht::endpoint_details details;
const auto& node = topology.get_node(r.host);
+ const auto ip = _address_map.get(r.host);
details._datacenter = node.dc_rack().dc;
details._rack = node.dc_rack().rack;
- details._host = node.endpoint();
- tr._rpc_endpoints.push_back(_gossiper.get_rpc_address(node.endpoint()));
+ details._host = ip;
+ tr._rpc_endpoints.push_back(_gossiper.get_rpc_address(ip));
tr._endpoints.push_back(fmt::to_string(details._host));
tr._endpoint_details.push_back(std::move(details));
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:39 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/topology.hh | 6 ------
locator/topology.cc | 11 -----------
2 files changed, 17 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index c13674d6506..f27c7428321 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -252,7 +252,6 @@ class topology {

// Returns true if a node with given host_id is found
bool has_node(host_id id) const noexcept;
- bool has_node(inet_address id) const noexcept;

/**
* Stores current DC/rack assignment for ep
@@ -266,11 +265,6 @@ class topology {

bool remove_endpoint(locator::host_id ep);

- /**
- * Returns true iff contains given endpoint.
- */
- bool has_endpoint(inet_address) const;
-
const std::unordered_map<sstring,
std::unordered_set<host_id>>&
get_datacenter_endpoints() const {
diff --git a/locator/topology.cc b/locator/topology.cc
index 5ad63094897..195c512a11a 100644
--- a/locator/topology.cc
+++ b/locator/topology.cc
@@ -543,17 +543,6 @@ bool topology::has_node(host_id id) const noexcept {
return bool(node);
}

-bool topology::has_node(inet_address ep) const noexcept {
- auto node = find_node(ep);
- tlogger.trace("topology[{}]: has_node: endpoint={}: node={}", fmt::ptr(this), ep, node_printer(node));
- return bool(node);
-}
-
-bool topology::has_endpoint(inet_address ep) const
-{
- return has_node(ep);
-}
-
void topology::sort_by_proximity(locator::host_id address, host_id_vector_replica_set& addresses) const {
if (can_sort_by_proximity()) {
do_sort_by_proximity(address, addresses);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:39 AMJan 13
to scylladb-dev@googlegroups.com
The code checks that it does not run for an ip address that is no longer
in use (after ip address change). To check that we can use peers table
and see if the host id is mapped to the address. If yes, this is the
latest address for this host id otherwise this is an outdated entry.
---
service/storage_service.cc | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index dc806124483..39c1e1274a6 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2687,14 +2687,14 @@ future<> storage_service::on_change(gms::inet_address endpoint, const gms::appli
const auto host_id = _gossiper.get_host_id(endpoint);
const auto& tm = get_token_metadata();
const auto* node = tm.get_topology().find_node(host_id);
- // The check node->endpoint() == endpoint is needed when a node changes
+ // The check peers[host_id] == endpoint is needed when a node changes
// its IP - on_change can be called by the gossiper for old IP as part
// of its removal, after handle_state_normal has already been called for
// the new one. Without the check, the do_update_system_peers_table call
// overwrites the IP back to its old value.
// In essence, the code under the 'if' should fire if the given IP belongs
// to a cluster member.
- if (node && node->is_member() && node->endpoint() == endpoint) {
+ if (node && node->is_member() && (co_await get_ip_from_peers_table(host_id)) == endpoint) {
if (!is_me(endpoint)) {
slogger.debug("endpoint={}/{} on_change: updating system.peers table", endpoint, host_id);
if (auto info = get_peer_info_for_update(endpoint, states)) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:40 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/topology.hh | 8 ++++----
alternator/server.cc | 5 +++--
locator/topology.cc | 4 ++--
test/boost/locator_topology_test.cc | 18 +++++++++---------
4 files changed, 18 insertions(+), 17 deletions(-)

diff --git a/locator/topology.hh b/locator/topology.hh
index 91af733a9f1..c13674d6506 100644
--- a/locator/topology.hh
+++ b/locator/topology.hh
@@ -272,7 +272,7 @@ class topology {
bool has_endpoint(inet_address) const;

const std::unordered_map<sstring,
- std::unordered_set<inet_address>>&
+ std::unordered_set<host_id>>&
get_datacenter_endpoints() const {
return _dc_endpoints;
}
@@ -293,7 +293,7 @@ class topology {

const std::unordered_map<sstring,
std::unordered_map<sstring,
- std::unordered_set<inet_address>>>&
+ std::unordered_set<host_id>>>&
get_datacenter_racks() const {
return _dc_racks;
}
@@ -425,13 +425,13 @@ class topology {

/** multi-map: DC -> endpoints in that DC */
std::unordered_map<sstring,
- std::unordered_set<inet_address>>
+ std::unordered_set<host_id>>
_dc_endpoints;

/** map: DC -> (multi-map: rack -> endpoints in that rack) */
std::unordered_map<sstring,
std::unordered_map<sstring,
- std::unordered_set<inet_address>>>
+ std::unordered_set<host_id>>>
_dc_racks;

bool _sort_by_proximity = true;
diff --git a/alternator/server.cc b/alternator/server.cc
index f20f52786c6..8a9a7d6884b 100644
--- a/alternator/server.cc
+++ b/alternator/server.cc
@@ -217,7 +217,7 @@ class local_nodelist_handler : public gated_handler {
// If the DC does not exist, we return an empty list - not an error.
sstring query_dc = req->get_query_param("dc");
sstring local_dc = query_dc.empty() ? topology.get_datacenter() : query_dc;
- std::unordered_set<gms::inet_address> local_dc_nodes;
+ std::unordered_set<locator::host_id> local_dc_nodes;
const auto& endpoints = topology.get_datacenter_endpoints();
auto dc_it = endpoints.find(local_dc);
if (dc_it != endpoints.end()) {
@@ -227,7 +227,8 @@ class local_nodelist_handler : public gated_handler {
// DC, unless a single rack is selected by the "rack" query option.
// If the rack does not exist, we return an empty list - not an error.
sstring query_rack = req->get_query_param("rack");
- for (auto& ip : local_dc_nodes) {
+ for (auto& id : local_dc_nodes) {
+ auto ip = _gossiper.get_address_map().get(id);
if (!query_rack.empty()) {
auto rack = _gossiper.get_application_state_value(ip, gms::application_state::RACK);
if (rack != query_rack) {
diff --git a/locator/topology.cc b/locator/topology.cc
index c64a5be8513..5ad63094897 100644
--- a/locator/topology.cc
+++ b/locator/topology.cc
@@ -393,7 +393,7 @@ void topology::index_node(const node& node) {
if (!node.left() && !node.is_none()) {
const auto& dc = node.dc_rack().dc;
const auto& rack = node.dc_rack().rack;
- const auto& endpoint = node.endpoint();
+ const auto& endpoint = node.host_id();
_dc_nodes[dc].emplace(std::cref(node));
_dc_rack_nodes[dc][rack].emplace(std::cref(node));
_dc_endpoints[dc].insert(endpoint);
@@ -415,7 +415,7 @@ void topology::unindex_node(const node& node) {
bool found = _dc_nodes.at(dc).erase(std::cref(node));
if (found) {
if (auto dit = _dc_endpoints.find(dc); dit != _dc_endpoints.end()) {
- const auto& ep = node.endpoint();
+ const auto& ep = node.host_id();
auto& eps = dit->second;
eps.erase(ep);
if (eps.empty()) {
diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc
index 99e815e8471..ccb321a11d7 100644
--- a/test/boost/locator_topology_test.cc
+++ b/test/boost/locator_topology_test.cc
@@ -207,8 +207,8 @@ SEASTAR_THREAD_TEST_CASE(test_add_or_update_by_host_id) {
}

SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
- using dc_endpoints_t = std::unordered_map<sstring, std::unordered_set<inet_address>>;
- using dc_racks_t = std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>>;
+ using dc_endpoints_t = std::unordered_map<sstring, std::unordered_set<locator::host_id>>;
+ using dc_racks_t = std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<locator::host_id>>>;
using dcs_t = std::unordered_set<sstring>;

const auto id1 = host_id::create_random_id();
@@ -235,19 +235,19 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
topo.add_or_update_endpoint(id1, ep1, dc_rack1, node::state::normal);
topo.add_node(id2, ep2, dc_rack2, node::state::normal);

- BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1, ep2}}}));
- BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}, {"rack2", {ep2}}}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1, id2}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}, {"rack2", {id2}}}}}));
BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"}));

topo.remove_endpoint(id2);
- BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1}}}));
- BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}}}}));
BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"}));

// Local endpoint cannot be removed
topo.remove_endpoint(id1);
- BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {ep1}}}));
- BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {ep1}}}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1}}}));
+ BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}}}}));
BOOST_REQUIRE_EQUAL(topo.get_datacenters(), (dcs_t{"dc1"}));
}

@@ -411,7 +411,7 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) {
}

// left nodes are not members.
- BOOST_REQUIRE(!topo.get_datacenter_endpoints().at(dc_rack1.dc).contains(ep3));
+ BOOST_REQUIRE(!topo.get_datacenter_endpoints().at(dc_rack1.dc).contains(id3));

BOOST_REQUIRE(topo.get_datacenter(id3) == dc_rack1.dc);
BOOST_REQUIRE(topo.get_rack(id3) == dc_rack1.rack);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:28:42 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/token_metadata.hh | 11 +----
locator/token_metadata.cc | 77 -------------------------------
test/boost/token_metadata_test.cc | 1 -
3 files changed, 2 insertions(+), 87 deletions(-)

diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
index 4ac6a2143b4..f2c27fcb14d 100644
--- a/locator/token_metadata.hh
+++ b/locator/token_metadata.hh
@@ -239,8 +239,6 @@ class token_metadata final {
/** Return the unique host ID for an end-point. */
host_id get_host_id(inet_address endpoint) const;

- inet_address get_endpoint_for_host_id(locator::host_id host_id) const;
-
/** @return a copy of the endpoint-to-id map for read-only operations */
std::unordered_set<host_id> get_host_ids() const;

@@ -324,23 +322,18 @@ class token_metadata final {

const std::unordered_set<host_id>& get_normal_token_owners() const;

- std::unordered_set<gms::inet_address> get_normal_token_owners_ips() const;
-
void for_each_token_owner(std::function<void(const node&)> func) const;

/* Returns the number of different endpoints that own tokens in the ring.
* Bootstrapping tokens are not taken into account. */
size_t count_normal_token_owners() const;

- // Returns the map: DC -> addresses of token owners in that DC.
+ // Returns the map: DC -> host_id of token owners in that DC.
// If there are no token owners in a DC, it is not present in the result.
- std::unordered_map<sstring, std::unordered_set<inet_address>> get_datacenter_token_owners_ips() const;
std::unordered_map<sstring, std::unordered_set<host_id>> get_datacenter_token_owners() const;

- // Returns the map: DC -> (map: rack -> addresses of token owners in that rack).
+ // Returns the map: DC -> (map: rack -> host_id of token owners in that rack).
// If there are no token owners in a DC/rack, it is not present in the result.
- std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>>
- get_datacenter_racks_token_owners_ips() const;
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<host_id>>>
get_datacenter_racks_token_owners() const;

diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 3320e37ed14..27b8e991285 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -150,12 +150,6 @@ class token_metadata_impl final {
*/
void update_host_id(const host_id& host_id, inet_address endpoint);

- /** Return the unique host ID for an end-point. */
- host_id get_host_id(inet_address endpoint) const;
-
- /** Return the end-point for a unique host ID.*/
- inet_address get_endpoint_for_host_id(host_id) const;
-
/** @return a copy of host id set for read-only operations */
std::unordered_set<host_id> get_host_ids() const;

@@ -254,11 +248,8 @@ class token_metadata_impl final {
* Bootstrapping tokens are not taken into account. */
size_t count_normal_token_owners() const;

- std::unordered_map<sstring, std::unordered_set<inet_address>> get_datacenter_token_owners_ips() const;
std::unordered_map<sstring, std::unordered_set<host_id>> get_datacenter_token_owners() const;

- std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>>
- get_datacenter_racks_token_owners_ips() const;
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<host_id>>>
get_datacenter_racks_token_owners() const;

@@ -529,22 +520,6 @@ void token_metadata_impl::update_host_id(const host_id& host_id, inet_address en
_topology.add_or_update_endpoint(host_id, endpoint);
}

-host_id token_metadata_impl::get_host_id(inet_address endpoint) const {
- if (const auto* node = _topology.find_node(endpoint)) [[likely]] {
- return node->host_id();
- } else {
- on_internal_error(tlogger, format("host_id for endpoint {} is not found", endpoint));
- }
-}
-
-inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) const {
- if (const auto* node = _topology.find_node(host_id)) [[likely]] {
- return node->endpoint();
- } else {
- on_internal_error(tlogger, format("endpoint for host_id {} is not found", host_id));
- }
-}
-
std::unordered_set<host_id> token_metadata_impl::get_host_ids() const {
return _topology.get_nodes() |
std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) |
@@ -773,17 +748,6 @@ void token_metadata_impl::for_each_token_owner(std::function<void(const node&)>
});
}

-std::unordered_map<sstring, std::unordered_set<inet_address>>
-token_metadata_impl::get_datacenter_token_owners_ips() const {
- std::unordered_map<sstring, std::unordered_set<inet_address>> datacenter_token_owners;
- _topology.for_each_node([&] (const node& n) {
- if (is_normal_token_owner(n.host_id())) {
- datacenter_token_owners[n.dc_rack().dc].insert(n.endpoint());
- }
- });
- return datacenter_token_owners;
-}
-
std::unordered_map<sstring, std::unordered_set<host_id>>
token_metadata_impl::get_datacenter_token_owners() const {
std::unordered_map<sstring, std::unordered_set<host_id>> datacenter_token_owners;
@@ -795,18 +759,6 @@ token_metadata_impl::get_datacenter_token_owners() const {
return datacenter_token_owners;
}

-std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>>
-token_metadata_impl::get_datacenter_racks_token_owners_ips() const {
- std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>> dc_racks_token_owners;
- _topology.for_each_node([&] (const node& n) {
- const auto& dc_rack = n.dc_rack();
- if (is_normal_token_owner(n.host_id())) {
- dc_racks_token_owners[dc_rack.dc][dc_rack.rack].insert(n.endpoint());
- }
- });
- return dc_racks_token_owners;
-}
-
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<host_id>>>
token_metadata_impl::get_datacenter_racks_token_owners() const {
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<host_id>>> dc_racks_token_owners;
@@ -1015,16 +967,6 @@ token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) {
_impl->update_host_id(host_id, endpoint);
}

-host_id
-token_metadata::get_host_id(inet_address endpoint) const {
- return _impl->get_host_id(endpoint);
-}
-
-token_metadata::inet_address
-token_metadata::get_endpoint_for_host_id(host_id host_id) const {
- return _impl->get_endpoint_for_host_id(host_id);
-}
-
std::unordered_set<host_id>
token_metadata::get_host_ids() const {
return _impl->get_host_ids();
@@ -1147,16 +1089,6 @@ token_metadata::get_normal_token_owners() const {
return _impl->get_normal_token_owners();
}

-std::unordered_set<gms::inet_address> token_metadata::get_normal_token_owners_ips() const {
- const auto& host_ids = _impl->get_normal_token_owners();
- std::unordered_set<gms::inet_address> result;
- result.reserve(host_ids.size());
- for (const auto& id: host_ids) {
- result.insert(_impl->get_endpoint_for_host_id(id));
- }
- return result;
-}
-
void token_metadata::for_each_token_owner(std::function<void(const node&)> func) const {
return _impl->for_each_token_owner(func);
}
@@ -1166,19 +1098,10 @@ token_metadata::count_normal_token_owners() const {
return _impl->count_normal_token_owners();
}

-std::unordered_map<sstring, std::unordered_set<inet_address>> token_metadata::get_datacenter_token_owners_ips() const {
- return _impl->get_datacenter_token_owners_ips();
-}
-
std::unordered_map<sstring, std::unordered_set<host_id>> token_metadata::get_datacenter_token_owners() const {
return _impl->get_datacenter_token_owners();
}

-std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<inet_address>>>
-token_metadata::get_datacenter_racks_token_owners_ips() const {
- return _impl->get_datacenter_racks_token_owners_ips();
-}
-
std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<host_id>>>
token_metadata::get_datacenter_racks_token_owners() const {
return _impl->get_datacenter_racks_token_owners();
diff --git a/test/boost/token_metadata_test.cc b/test/boost/token_metadata_test.cc
index 6f62ad4a157..2aec13a1637 100644
--- a/test/boost/token_metadata_test.cc
+++ b/test/boost/token_metadata_test.cc
@@ -300,7 +300,6 @@ SEASTAR_THREAD_TEST_CASE(test_replace_node_with_same_endpoint) {
token_metadata->add_replacing_endpoint(e1_id1, e1_id2);

auto erm = create_erm<simple_strategy>(token_metadata, {{"replication_factor", "2"}});
- BOOST_REQUIRE_EQUAL(token_metadata->get_host_id(e1), e1_id1);
BOOST_REQUIRE_EQUAL(erm->get_pending_replicas(dht::token::from_int64(1)),
host_id_vector_topology_change{e1_id2});
BOOST_REQUIRE_EQUAL(erm->get_natural_replicas(dht::token::from_int64(1)),
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:45 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/abstract_replication_strategy.hh | 5 -----
locator/abstract_replication_strategy.cc | 23 -----------------------
locator/tablets.cc | 4 ----
3 files changed, 32 deletions(-)

diff --git a/locator/abstract_replication_strategy.hh b/locator/abstract_replication_strategy.hh
index 46f469f80ea..5358139c72b 100644
--- a/locator/abstract_replication_strategy.hh
+++ b/locator/abstract_replication_strategy.hh
@@ -233,9 +233,6 @@ class effective_replication_map {
/// new replica.
///
/// The returned addresses are present in the topology object associated with this instance.
- virtual inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const = 0;
-
- /// Same as above but returns host ids instead of addresses
virtual host_id_vector_replica_set get_natural_replicas(const token& search_token) const = 0;

/// Same as above but returns host ids instead of addresses
@@ -335,7 +332,6 @@ class vnode_effective_replication_map : public enable_shared_from_this<vnode_eff
friend class abstract_replication_strategy;
friend class effective_replication_map_factory;
public: // effective_replication_map
- inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const override;
host_id_vector_replica_set get_natural_replicas(const token& search_token) const override;
host_id_vector_topology_change get_pending_replicas(const token& search_token) const override;
host_id_vector_replica_set get_replicas_for_reading(const token& token) const override;
@@ -399,7 +395,6 @@ class vnode_effective_replication_map : public enable_shared_from_this<vnode_eff

private:
future<dht::token_range_vector> do_get_ranges(noncopyable_function<stop_iteration(bool& add_range, const host_id& natural_endpoint)> consider_range_for_endpoint) const;
- inet_address_vector_replica_set do_get_natural_endpoints(const token& tok, bool is_vnode) const;
host_id_vector_replica_set do_get_replicas(const token& tok, bool is_vnode) const;
stop_iteration for_each_natural_endpoint_until(const token& vnode_tok, const noncopyable_function<stop_iteration(const host_id&)>& func) const;

diff --git a/locator/abstract_replication_strategy.cc b/locator/abstract_replication_strategy.cc
index 468426e8ffa..c48a0a80acd 100644
--- a/locator/abstract_replication_strategy.cc
+++ b/locator/abstract_replication_strategy.cc
@@ -56,19 +56,6 @@ auto& ring_mapping::operator*() const {
return std::as_const(_impl->map);
}

-template <typename ResultSet, typename SourceSet>
-static ResultSet resolve_endpoints(const SourceSet& host_ids, const token_metadata& tm) {
- ResultSet result{};
- result.reserve(host_ids.size());
- for (const auto& host_id: host_ids) {
- // Empty host_id is used as a marker for local address.
- // The reason for this hack is that we need local_strategy to
- // work before the local host_id is loaded from the system.local table.
- result.push_back(host_id ? tm.get_endpoint_for_host_id(host_id) : tm.get_topology().my_address());
- }
- return result;
-}
-
logging::logger rslogger("replication_strategy");

abstract_replication_strategy::abstract_replication_strategy(
@@ -507,20 +494,10 @@ host_id_vector_replica_set vnode_effective_replication_map::do_get_replicas(cons
return it->second;
}

-inet_address_vector_replica_set vnode_effective_replication_map::do_get_natural_endpoints(const token& tok,
- bool is_vnode) const
-{
- return resolve_endpoints<inet_address_vector_replica_set>(do_get_replicas(tok, is_vnode), *_tmptr);
-}
-
host_id_vector_replica_set vnode_effective_replication_map::get_replicas(const token& tok) const {
return do_get_replicas(tok, false);
}

-inet_address_vector_replica_set vnode_effective_replication_map::get_natural_endpoints(const token& search_token) const {
- return do_get_natural_endpoints(search_token, false);
-}
-
host_id_vector_replica_set vnode_effective_replication_map::get_natural_replicas(const token& search_token) const {
return get_replicas(search_token);
}
diff --git a/locator/tablets.cc b/locator/tablets.cc
index cafe0fe22aa..935a171f88f 100644
--- a/locator/tablets.cc
+++ b/locator/tablets.cc
@@ -879,10 +879,6 @@ class tablet_effective_replication_map : public effective_replication_map {
return to_host_set(get_replicas_for_write(search_token));
}

- virtual inet_address_vector_replica_set get_natural_endpoints(const token& search_token) const override {
- return to_replica_set(get_replicas_for_write(search_token));
- }
-
virtual host_id_vector_replica_set get_natural_replicas(const token& search_token) const override {
return to_host_set(get_replicas_for_write(search_token));
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:46 AMJan 13
to scylladb-dev@googlegroups.com
The are unused. The version is always fixed.
---
message/messaging_service.hh | 4 ----
api/messaging_service.cc | 2 +-
message/messaging_service.cc | 10 ----------
service/migration_manager.cc | 4 +---
4 files changed, 2 insertions(+), 18 deletions(-)

diff --git a/message/messaging_service.hh b/message/messaging_service.hh
index eb5c52613ca..873aa83c903 100644
--- a/message/messaging_service.hh
+++ b/message/messaging_service.hh
@@ -272,10 +272,6 @@ class messaging_service : public seastar::async_sharded_service<messaging_servic

const uint64_t* get_dropped_messages() const;

- int32_t get_raw_version(const gms::inet_address& endpoint) const;
-
- bool knows_version(const gms::inet_address& endpoint) const;
-
enum class encrypt_what {
none,
rack,
diff --git a/api/messaging_service.cc b/api/messaging_service.cc
index b1665c8c5c2..3fc38b127c0 100644
--- a/api/messaging_service.cc
+++ b/api/messaging_service.cc
@@ -114,7 +114,7 @@ void set_messaging_service(http_context& ctx, routes& r, sharded<netw::messaging
}));

get_version.set(r, [&ms](const_req req) {
- return ms.local().get_raw_version(gms::inet_address(req.get_query_param("addr")));
+ return ms.local().current_version;
});

get_dropped_messages_by_ver.set(r, [&ms](std::unique_ptr<request> req) {
diff --git a/message/messaging_service.cc b/message/messaging_service.cc
index eb7b814d8fc..22b6e35eabd 100644
--- a/message/messaging_service.cc
+++ b/message/messaging_service.cc
@@ -257,16 +257,6 @@ const uint64_t* messaging_service::get_dropped_messages() const {
return _dropped_messages;
}

-int32_t messaging_service::get_raw_version(const gms::inet_address& endpoint) const {
- // FIXME: messaging service versioning
- return current_version;
-}
-
-bool messaging_service::knows_version(const gms::inet_address& endpoint) const {
- // FIXME: messaging service versioning
- return true;
-}
-
future<> messaging_service::unregister_handler(messaging_verb verb) {
return _rpc->unregister_handler(verb);
}
diff --git a/service/migration_manager.cc b/service/migration_manager.cc
index 21d0e730c5b..a4474034be4 100644
--- a/service/migration_manager.cc
+++ b/service/migration_manager.cc
@@ -921,9 +921,7 @@ future<> migration_manager::announce_without_raft(std::vector<mutation> schema,
auto all_live = _gossiper.get_live_members();
auto live_members = all_live | std::views::filter([this, my_address = _messaging.broadcast_address()] (const gms::inet_address& endpoint) {
// only push schema to nodes with known and equal versions
- return endpoint != my_address &&
- _messaging.knows_version(endpoint) &&
- _messaging.get_raw_version(endpoint) == netw::messaging_service::current_version;
+ return endpoint != my_address;
});
// FIXME: gossiper should return host id set
auto live_host_ids = live_members | std::views::transform([&] (const gms::inet_address& ip) { return _gossiper.get_host_id(ip); });
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:47 AMJan 13
to scylladb-dev@googlegroups.com
We want to drop ip from the topology::node.
---
cql3/statements/select_statement.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc
index 25d40eaf53f..662bdeaf70a 100644
--- a/cql3/statements/select_statement.cc
+++ b/cql3/statements/select_statement.cc
@@ -1820,7 +1820,7 @@ mutation_fragments_select_statement::do_execute(query_processor& qp, service::qu
throw exceptions::invalid_request_exception(seastar::format(
"Moving between coordinators is not allowed in SELECT FROM MUTATION_FRAGMENTS() statements, last page's coordinator was {}{}",
last_host,
- last_node ? fmt::format("({})", last_node->endpoint()) : ""));
+ last_node ? fmt::format("({})", last_node->host_id()) : ""));
}
}
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:47 AMJan 13
to scylladb-dev@googlegroups.com
It is used by truncate code only and even there it only check if the
returned set is not empty. Check for dead token owners in the truncation
code directly.
---
gms/gossiper.hh | 5 -----
gms/gossiper.cc | 11 -----------
service/storage_proxy.cc | 4 +++-
3 files changed, 3 insertions(+), 17 deletions(-)

diff --git a/gms/gossiper.hh b/gms/gossiper.hh
index 1b647370ff3..f3d5dd631cb 100644
--- a/gms/gossiper.hh
+++ b/gms/gossiper.hh
@@ -306,11 +306,6 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
std::set<inet_address> get_unreachable_members() const;
std::set<locator::host_id> get_unreachable_host_ids() const;

- /**
- * @return a list of unreachable token owners
- */
- std::set<inet_address> get_unreachable_token_owners() const;
-
/**
* @return a list of unreachable nodes
*/
diff --git a/gms/gossiper.cc b/gms/gossiper.cc
index a8ec00dc816..0521080f8c4 100644
--- a/gms/gossiper.cc
+++ b/gms/gossiper.cc
@@ -1201,17 +1201,6 @@ std::set<locator::host_id> gossiper::get_live_token_owners() const {
return token_owners;
}

-std::set<inet_address> gossiper::get_unreachable_token_owners() const {
- std::set<inet_address> token_owners;
- auto normal_token_owners = get_token_metadata_ptr()->get_normal_token_owners_ips();
- for (auto& node: normal_token_owners) {
- if (!is_alive(node)) {
- token_owners.insert(node);
- }
- }
- return token_owners;
-}
-
std::set<locator::host_id> gossiper::get_unreachable_nodes() const {
std::set<locator::host_id> unreachable_nodes;
auto nodes = get_token_metadata_ptr()->get_topology().get_all_host_ids();
diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index 37676e64f01..19e92fb0d39 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -470,7 +470,9 @@ class storage_proxy::remote {
}

future<> send_truncate_blocking(sstring keyspace, sstring cfname, std::chrono::milliseconds timeout_in_ms) {
- if (!_gossiper.get_unreachable_token_owners().empty()) {
+ auto s = _sp.local_db().find_schema(keyspace, cfname);
+ auto erm_ptr = s->table().get_effective_replication_map();
+ if (!std::ranges::all_of(erm_ptr->get_token_metadata().get_normal_token_owners(), std::bind_front(&storage_proxy::is_alive, &_sp, std::cref(*erm_ptr)))) {
slogger.info("Cannot perform truncate, some hosts are down");
// Since the truncate operation is so aggressive and is typically only
// invoked by an admin, for simplicity we require that all nodes are up
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:48 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/tablets.cc | 13 -------------
1 file changed, 13 deletions(-)

diff --git a/locator/tablets.cc b/locator/tablets.cc
index 935a171f88f..bdddae1ae7d 100644
--- a/locator/tablets.cc
+++ b/locator/tablets.cc
@@ -765,19 +765,6 @@ class tablet_effective_replication_map : public effective_replication_map {
tablet_sharder _sharder;
mutable const tablet_map* _tmap = nullptr;
private:
- inet_address_vector_replica_set to_replica_set(const tablet_replica_set& replicas) const {
- inet_address_vector_replica_set result;
- result.reserve(replicas.size());
- auto& topo = _tmptr->get_topology();
- for (auto&& replica : replicas) {
- auto* node = topo.find_node(replica.host);
- if (node && !node->left()) {
- result.emplace_back(node->endpoint());
- }
- }
- return result;
- }
-
host_id_vector_replica_set to_host_set(const tablet_replica_set& replicas) const {
host_id_vector_replica_set result;
result.reserve(replicas.size());
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:48 AMJan 13
to scylladb-dev@googlegroups.com
The functions are called from RESful API so has to return ips for backwards
compatibility, but internally we can use host ids as long as possible
and convert to ips just before returning. This also drops usage of ip
based erm function which we want to get rid of.
---
locator/util.hh | 2 +-
locator/util.cc | 20 ++++++++++----------
service/storage_service.cc | 6 +++++-
3 files changed, 16 insertions(+), 12 deletions(-)

diff --git a/locator/util.hh b/locator/util.hh
index 3a3360d3d94..c66e400680a 100644
--- a/locator/util.hh
+++ b/locator/util.hh
@@ -23,6 +23,6 @@ namespace gms {

namespace locator {
future<std::vector<dht::token_range_endpoints>> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false);
- future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(
+ future<std::unordered_map<dht::token_range, host_id_vector_replica_set>> get_range_to_address_map(
locator::effective_replication_map_ptr erm, const std::vector<token>& sorted_tokens);
}
\ No newline at end of file
diff --git a/locator/util.cc b/locator/util.cc
index d7f8768cdae..55c0e0d4911 100644
--- a/locator/util.cc
+++ b/locator/util.cc
@@ -12,14 +12,14 @@

namespace locator {

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+static future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
construct_range_to_endpoint_map(
locator::effective_replication_map_ptr erm,
const dht::token_range_vector& ranges) {
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> res;
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> res;
res.reserve(ranges.size());
for (auto r : ranges) {
- res[r] = erm->get_natural_endpoints(
+ res[r] = erm->get_natural_replicas(
r.end() ? r.end()->value() : dht::maximum_token());
co_await coroutine::maybe_yield();
}
@@ -47,7 +47,7 @@ get_all_ranges(const std::vector<token>& sorted_tokens) {
}

// Caller is responsible to hold token_metadata valid until the returned future is resolved
-future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map(locator::effective_replication_map_ptr erm,
const std::vector<token>& sorted_tokens) {
co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
@@ -67,12 +67,12 @@ get_tokens_in_local_dc(const locator::token_metadata& tm) {
co_return filtered_tokens;
}

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+static future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map_in_local_dc(
locator::effective_replication_map_ptr erm) {
auto tmptr = erm->get_token_metadata_ptr();
auto orig_map = co_await get_range_to_address_map(erm, co_await get_tokens_in_local_dc(*tmptr));
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> filtered_map;
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> filtered_map;
filtered_map.reserve(orig_map.size());
auto local_dc_filter = tmptr->get_topology().get_local_dc_filter();
for (auto entry : orig_map) {
@@ -90,7 +90,7 @@ get_range_to_address_map_in_local_dc(
// return get_range_to_address_map(db.find_keyspace(keyspace).get_vnode_effective_replication_map());
// }

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map(locator::effective_replication_map_ptr erm) {
return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
}
@@ -100,7 +100,7 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const
std::vector<dht::token_range_endpoints> ranges;

auto erm = db.find_keyspace(keyspace).get_vnode_effective_replication_map();
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> range_to_address_map = co_await (
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> range_to_address_map = co_await (
include_only_local_dc
? get_range_to_address_map_in_local_dc(erm)
: get_range_to_address_map(erm)
@@ -119,10 +119,10 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const
}
for (auto endpoint : addresses) {
dht::endpoint_details details;
- details._host = endpoint;
+ details._host = gossiper.get_address_map().get(endpoint);
details._datacenter = topology.get_datacenter(endpoint);
details._rack = topology.get_rack(endpoint);
- tr._rpc_endpoints.push_back(gossiper.get_rpc_address(endpoint));
+ tr._rpc_endpoints.push_back(gossiper.get_rpc_address(details._host));
tr._endpoints.push_back(fmt::to_string(details._host));
tr._endpoint_details.push_back(details);
}
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 81361a76a86..10241a40f70 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -13,6 +13,7 @@
#include "compaction/task_manager_module.hh"
#include "gc_clock.hh"
#include "raft/raft.hh"
+#include <ranges>
#include <seastar/core/sleep.hh>
#include "service/qos/raft_service_level_distributed_data_accessor.hh"
#include "service/qos/service_level_controller.hh"
@@ -2295,7 +2296,10 @@ future<> storage_service::bootstrap(std::unordered_set<token>& bootstrap_tokens,

future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const {
- return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
+ co_return (co_await locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens())) |
+ std::views::transform([&] (auto tid) { return std::make_pair(tid.first,
+ tid.second | std::views::transform([&] (auto id) { return _address_map.get(id); }) | std::ranges::to<inet_address_vector_replica_set>()); }) |
+ std::ranges::to<std::unordered_map>();
}

future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:49 AMJan 13
to scylladb-dev@googlegroups.com
---
api/api_init.hh | 2 +-
api/storage_service.hh | 2 +-
db/view/view_builder.hh | 2 +-
api/api.cc | 4 ++--
api/storage_service.cc | 6 +++---
db/view/view.cc | 4 ++--
main.cc | 2 +-
7 files changed, 11 insertions(+), 11 deletions(-)

diff --git a/api/api_init.hh b/api/api_init.hh
index fe9e770b089..8ce02e50d80 100644
--- a/api/api_init.hh
+++ b/api/api_init.hh
@@ -100,7 +100,7 @@ future<> set_server_storage_service(http_context& ctx, sharded<service::storage_
future<> unset_server_storage_service(http_context& ctx);
future<> set_server_sstables_loader(http_context& ctx, sharded<sstables_loader>& sst_loader);
future<> unset_server_sstables_loader(http_context& ctx);
-future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb);
+future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb, sharded<gms::gossiper>& g);
future<> unset_server_view_builder(http_context& ctx);
future<> set_server_repair(http_context& ctx, sharded<repair_service>& repair, sharded<gms::gossip_address_map>& am);
future<> unset_server_repair(http_context& ctx);
diff --git a/api/storage_service.hh b/api/storage_service.hh
index 339bf8f37a6..b26795a21bb 100644
--- a/api/storage_service.hh
+++ b/api/storage_service.hh
@@ -75,7 +75,7 @@ void set_storage_service(http_context& ctx, httpd::routes& r, sharded<service::s
void unset_storage_service(http_context& ctx, httpd::routes& r);
void set_sstables_loader(http_context& ctx, httpd::routes& r, sharded<sstables_loader>& sst_loader);
void unset_sstables_loader(http_context& ctx, httpd::routes& r);
-void set_view_builder(http_context& ctx, httpd::routes& r, sharded<db::view::view_builder>& vb);
+void set_view_builder(http_context& ctx, httpd::routes& r, sharded<db::view::view_builder>& vb, sharded<gms::gossiper>& g);
void unset_view_builder(http_context& ctx, httpd::routes& r);
void set_repair(http_context& ctx, httpd::routes& r, sharded<repair_service>& repair, sharded<gms::gossip_address_map>& am);
void unset_repair(http_context& ctx, httpd::routes& r);
diff --git a/db/view/view_builder.hh b/db/view/view_builder.hh
index 89dfddc070f..6b26a6eaa4a 100644
--- a/db/view/view_builder.hh
+++ b/db/view/view_builder.hh
@@ -235,7 +235,7 @@ class view_builder final : public service::migration_listener::only_view_notific
// For tests
future<> wait_until_built(const sstring& ks_name, const sstring& view_name);

- future<std::unordered_map<sstring, sstring>> view_build_statuses(sstring keyspace, sstring view_name) const;
+ future<std::unordered_map<sstring, sstring>> view_build_statuses(sstring keyspace, sstring view_name, const gms::gossiper& g) const;

// Can only be called on shard-0
future<> mark_existing_views_as_built();
diff --git a/api/api.cc b/api/api.cc
index b0924cf6f26..880cb6edbd9 100644
--- a/api/api.cc
+++ b/api/api.cc
@@ -153,8 +153,8 @@ future<> unset_server_sstables_loader(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_sstables_loader(ctx, r); });
}

-future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb) {
- return ctx.http_server.set_routes([&ctx, &vb] (routes& r) { set_view_builder(ctx, r, vb); });
+future<> set_server_view_builder(http_context& ctx, sharded<db::view::view_builder>& vb, sharded<gms::gossiper>& g) {
+ return ctx.http_server.set_routes([&ctx, &vb, &g] (routes& r) { set_view_builder(ctx, r, vb, g); });
}

future<> unset_server_view_builder(http_context& ctx) {
diff --git a/api/storage_service.cc b/api/storage_service.cc
index 0c245dbdce9..332ed14d553 100644
--- a/api/storage_service.cc
+++ b/api/storage_service.cc
@@ -533,11 +533,11 @@ void unset_sstables_loader(http_context& ctx, routes& r) {
ss::start_restore.unset(r);
}

-void set_view_builder(http_context& ctx, routes& r, sharded<db::view::view_builder>& vb) {
- ss::view_build_statuses.set(r, [&ctx, &vb] (std::unique_ptr<http::request> req) {
+void set_view_builder(http_context& ctx, routes& r, sharded<db::view::view_builder>& vb, sharded<gms::gossiper>& g) {
+ ss::view_build_statuses.set(r, [&ctx, &vb, &g] (std::unique_ptr<http::request> req) {
auto keyspace = validate_keyspace(ctx, req);
auto view = req->get_path_param("view");
- return vb.local().view_build_statuses(std::move(keyspace), std::move(view)).then([] (std::unordered_map<sstring, sstring> status) {
+ return vb.local().view_build_statuses(std::move(keyspace), std::move(view), g.local()).then([] (std::unordered_map<sstring, sstring> status) {
std::vector<storage_service_json::mapper> res;
return make_ready_future<json::json_return_type>(map_to_key_value(std::move(status), res));
});
diff --git a/db/view/view.cc b/db/view/view.cc
index 48b1091ab29..5b492c26588 100644
--- a/db/view/view.cc
+++ b/db/view/view.cc
@@ -2434,14 +2434,14 @@ future<std::unordered_map<locator::host_id, sstring>> view_builder::view_status(
}

future<std::unordered_map<sstring, sstring>>
-view_builder::view_build_statuses(sstring keyspace, sstring view_name) const {
+view_builder::view_build_statuses(sstring keyspace, sstring view_name, const gms::gossiper& gossiper) const {
std::unordered_map<locator::host_id, sstring> status = co_await view_status(std::move(keyspace), std::move(view_name));
std::unordered_map<sstring, sstring> status_map;
const auto& topo = _db.get_token_metadata().get_topology();
topo.for_each_node([&] (const locator::node& node) {
auto it = status.find(node.host_id());
auto s = it != status.end() ? std::move(it->second) : "UNKNOWN";
- status_map.emplace(fmt::to_string(node.endpoint()), std::move(s));
+ status_map.emplace(fmt::to_string(gossiper.get_address_map().get(node.host_id())), std::move(s));
});
co_return status_map;
}
diff --git a/main.cc b/main.cc
index 0483e957a40..9d394363ca3 100644
--- a/main.cc
+++ b/main.cc
@@ -2261,7 +2261,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
view_builder.invoke_on_all(&db::view::view_builder::start, std::ref(mm), utils::cross_shard_barrier()).get();
}

- api::set_server_view_builder(ctx, view_builder).get();
+ api::set_server_view_builder(ctx, view_builder, gossiper).get();
auto stop_vb_api = defer_verbose_shutdown("view builder API", [&ctx] {
api::unset_server_view_builder(ctx).get();
});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:32:49 AMJan 13
to scylladb-dev@googlegroups.com
---
locator/network_topology_strategy.cc | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/locator/network_topology_strategy.cc b/locator/network_topology_strategy.cc
index e59581a32ee..776778b81cc 100644
--- a/locator/network_topology_strategy.cc
+++ b/locator/network_topology_strategy.cc
@@ -235,7 +235,7 @@ class natural_endpoints_tracker {
}

static void check_enough_endpoints(const token_metadata& tm, const std::unordered_map<sstring, size_t>& dc_rf) {
- auto dc_endpoints = tm.get_datacenter_token_owners_ips();
+ auto dc_endpoints = tm.get_datacenter_token_owners();
auto endpoints_in = [&dc_endpoints](sstring dc) {
auto i = dc_endpoints.find(dc);
return i != dc_endpoints.end() ? i->second.size() : size_t(0);
@@ -309,7 +309,7 @@ effective_replication_map_ptr network_topology_strategy::make_replication_map(ta

static unsigned calculate_initial_tablets_from_topology(const schema& s, token_metadata_ptr tm, const std::unordered_map<sstring, size_t>& rf) {
unsigned initial_tablets = std::numeric_limits<unsigned>::min();
- for (const auto& dc : tm->get_datacenter_token_owners_ips()) {
+ for (const auto& dc : tm->get_datacenter_token_owners()) {
unsigned shards_in_dc = 0;
unsigned rf_in_dc = 1;

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 13, 2025, 3:48:57 AMJan 13
to scylladb-dev@googlegroups.com
This is on top of "Load peers table into the gossiper on boot" series.

On Mon, Jan 13, 2025 at 10:06:47AM +0200, Gleb Natapov wrote:
> Now that all topology related code uses host ids there is not point to
> maintain ip to id (and back) mappings in the token metadata. After the
> patch the mapping will be maintained in the gossiper only. The rest of
> the system will use host ids and in rare cases where translation is
> needed (mostly for UX compatibility reasons) the translation will be
> done using gossiper.
>
> Fixes: scylladb/scylla#21777
>
> CI: https://jenkins.scylladb.com/job/scylla-master/job/scylla-ci/14388/
>
> Also in scylla-dev gleb/drop-ip-from-tm
>
> Gleb Natapov (56):
> service: address_map: add lookup function that expects address to
> exist
> hints: simplify can_send() function
> api: do not use token_metadata to retrieve ip to id mapping in
> token_metadata RESTful endpoints
> hits: move create_hint_sync_point function to host ids
> hints: use gossiper to map ip to id in wait_for_sync_point
> view: do not use get_endpoint_for_host_id_if_known to check if a node
> is part of the topology
> db/virtual_tables: use host id from the gossiper endpoint state in
> cluster_status table
> repair: use gossiper to map ip to host id instead of token_metadata
> storage_proxy: used gossiper for map ip to host id in
> connection_dropped callback
> gossiper: add get_endpoint_state_ptr() function that works on host id
> topology coordinator: drop get_endpoint_for_host_id_if_known usage
> storage_service: force_remove_completion use address map to resolve id
> to ip instead of token metadata
> storage_service: use gossiper to map id to ip instead of
> token_metadata in node_ops_cmd_handler
> storage_service: use gossiper instead of token_metadata to map ip to
> id in gossiper notifications
> storage_service: use existing util function instead of re-iplementing
> it
> storage_service: move storage_service::get_natural_endpoints to use
> host ids internally and translate to ips before returning
> storage_service: move describe ring and get_range_to_endpoint_map to
> use host ids inside and translate to ips at the last moment
> test: drop inet_address usage from network_topology_strategy_test
> locator: drop no longer used ip based functions
> storage_service: do not use ip addresses from token_metadata in
> handling of a normal state
> messaging_service: drop get_raw_version and knows_version
> gossiper: change get_live_members and all its users to work on host
> ids
> messaging_service: drop the usage of ip based token_metadata APIs
> consistency_level: drop templates since the same types of ranges are
> used by all the callers
> locator: topology: drop no longer used ip based overloads
> locator: topology: remove get_location overload that works on ip and
> its last users
> storage_proxy: drop is_alive that works on ip since it is not used any
> more
> locator: topology: drop is_me ip overload along with remaning users
> locator: topology: change get_datacenter_endpoints and
> get_datacenter_racks to return host ids and amend users
> storage_service: change get_dc_rack_for() to work on host ids
> hints: move id to ip translation into store_hint() function
> node_ops, cdc: drop remaining
> token_metadata::get_endpoint_for_host_id() usage
> topology coordinator: remove gossiper entry only if host id matches
> provided one
> storage_service: fix logging
> storage_service: ip_address_updater: check peers table instead of
> token_metadata whether ip was changed
> host_id_or_endpoint: use gossiper to resolve ip to id and back
> mappings
> token_metadata: drop no longer unused functions
> storage_service: drop loops from node ops replace_prepare handling
> since there can be only one replacing node
> storage_service: fix indentation after the last patch
> storage_service: use gossiper to map ip to id in node_ops operations
> gossiper: drop get_unreachable_token_owners functions
> locator: network_topology_strategy: use host_id based function to
> check number of endpoints in dcs
> locator: token_metadata: remove unused ip based functions
> api: view_build_statuses: do not use IP from the topology, but
> translate id to ip using address map instead
> locator: drop unused function from tablet_effective_replication_map
> cql3: report host id instead of ip in error during SELECT FROM
> MUTATION_FRAGMENTS query
> topology coordinator: change connection dropping code to work on host
> ids
> storage_proxy: translate id to ip using address map in tablets's
> describe_ring code instead of taking one from the topology
> storage_service: check for outdated ip in on_change notification in
> the peers table
> locator: topology: remove unused functions
> storage_proxy: translate ips to ids in forward array using gossiper
> storage_service: use host_id to look for a node in on_alive handler
> repair: drop unneeded code
> locator: topology: drop indexing by ips
> locator: token_metadata: drop update_host_id() function that does
> nothing now
> hint manager: do not translate ip to id in case hint manager is
> stopped already
>
> api/api_init.hh | 6 +-
> api/hinted_handoff.hh | 3 +-
> api/storage_service.hh | 2 +-
> api/token_metadata.hh | 3 +-
> db/consistency_level.hh | 10 +-
> db/hints/manager.hh | 4 +-
> db/view/view_builder.hh | 2 +-
> gms/gossiper.hh | 9 +-
> locator/abstract_replication_strategy.hh | 5 -
> locator/token_metadata.hh | 37 +--
> locator/topology.hh | 58 +---
> locator/util.hh | 5 +
> message/messaging_service.hh | 10 +-
> repair/repair.hh | 3 -
> repair/task_manager_module.hh | 4 +-
> service/address_map.hh | 9 +
> service/storage_proxy.hh | 5 +-
> service/storage_service.hh | 18 +-
> alternator/server.cc | 5 +-
> api/api.cc | 14 +-
> api/hinted_handoff.cc | 13 +-
> api/messaging_service.cc | 2 +-
> api/storage_service.cc | 6 +-
> api/token_metadata.cc | 45 ++-
> cdc/generation.cc | 2 +-
> cql3/statements/select_statement.cc | 2 +-
> db/consistency_level.cc | 19 +-
> db/hints/internal/hint_sender.cc | 33 +-
> db/hints/manager.cc | 36 +-
> db/system_keyspace.cc | 2 +-
> db/view/view.cc | 8 +-
> db/virtual_tables.cc | 19 +-
> gms/gossiper.cc | 27 +-
> locator/abstract_replication_strategy.cc | 23 --
> locator/network_topology_strategy.cc | 4 +-
> locator/tablets.cc | 17 -
> locator/token_metadata.cc | 173 +---------
> locator/topology.cc | 130 +-------
> locator/util.cc | 20 +-
> main.cc | 22 +-
> message/messaging_service.cc | 31 +-
> node_ops/node_ops_ctl.cc | 2 +-
> repair/repair.cc | 32 +-
> repair/row_level.cc | 2 +-
> service/migration_manager.cc | 10 +-
> service/qos/service_level_controller.cc | 2 +-
> service/storage_proxy.cc | 48 ++-
> service/storage_service.cc | 330 +++++++++----------
> service/topology_coordinator.cc | 35 +-
> test/boost/locator_topology_test.cc | 130 ++------
> test/boost/network_topology_strategy_test.cc | 68 ++--
> test/boost/tablets_test.cc | 68 +---
> test/boost/token_metadata_test.cc | 38 ---
> test/lib/cql_test_env.cc | 5 +-
> test/manual/message.cc | 2 +-
> test/perf/perf_sort_by_proximity.cc | 1 -
> test/perf/tablet_load_balancing.cc | 1 -
> 57 files changed, 535 insertions(+), 1085 deletions(-)
>
> --
> 2.47.1
>

--
Gleb.

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 13, 2025, 10:32:10 AMJan 13
to Gleb Natapov, scylladb-dev@googlegroups.com, bdenes@scylladb.com, Asias He
Botond or Asias, please review

On 1/13/25 9:06 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> We want to drop ips from token_metadata so move to different API to map
> ip to id.
> ---
> repair/task_manager_module.hh | 4 +++-
> repair/repair.cc | 26 +++++++++++++++-----------
> 2 files changed, 18 insertions(+), 12 deletions(-)
>
> diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh
> index 01611468a7f..f941adb5e9a 100644
> --- a/repair/task_manager_module.hh
> +++ b/repair/task_manager_module.hh
> @@ -50,8 +50,9 @@ class user_requested_repair_task_impl : public repair_task_impl {
> std::unordered_set<locator::host_id> _ignore_nodes;
> bool _small_table_optimization;
> std::optional<int> _ranges_parallelism;
> + gms::gossiper& _gossiper;
> public:
> - user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism) noexcept
> + user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism, gms::gossiper& gossiper) noexcept
> : repair_task_impl(module, id.uuid(), id.id, "keyspace", std::move(keyspace), "", std::move(entity), tasks::task_id::create_null_id(), streaming::stream_reason::repair)
> , _germs(germs)
> , _cfs(std::move(cfs))
> @@ -61,6 +62,7 @@ class user_requested_repair_task_impl : public repair_task_impl {
> , _ignore_nodes(std::move(ignore_nodes))
> , _small_table_optimization(small_table_optimization)
> , _ranges_parallelism(ranges_parallelism)
> + , _gossiper(gossiper)
> {}
>
> virtual tasks::is_abortable is_abortable() const noexcept override {
> diff --git a/repair/repair.cc b/repair/repair.cc
> index 67b68eb8064..51c0aa8d7db 100644
> --- a/repair/repair.cc
> +++ b/repair/repair.cc
> @@ -226,6 +226,7 @@ repair_neighbors::repair_neighbors(std::vector<locator::host_id> nodes, std::vec
>
> // Return all of the neighbors with whom we share the provided range.
> static std::vector<locator::host_id> get_neighbors(
> + const gms::gossiper& gossiper,
> const locator::effective_replication_map& erm,
> const sstring& ksname, query::range<dht::token> range,
> const std::vector<sstring>& data_centers,
> @@ -282,17 +283,19 @@ static std::vector<locator::host_id> get_neighbors(
> } catch(...) {
> throw std::runtime_error(format("Unknown host specified: {}", host));
> }
> - auto endpoint = erm.get_token_metadata().get_host_id_if_known(ip);
> - if (endpoint) {
> +
> + try {
> + auto endpoint = gossiper.get_host_id(ip);
> +
> if (endpoint == my_address) {
> found_me = true;
> - } else if (neighbor_set.contains(*endpoint)) {
> - ret.push_back(*endpoint);
> + } else if (neighbor_set.contains(endpoint)) {
> + ret.push_back(endpoint);
> // If same host is listed twice, don't add it again later
> - neighbor_set.erase(*endpoint);
> - } else {
> - rlogger.warn("Provided host ip {} has no corresponding host id", ip);
> + neighbor_set.erase(endpoint);
> }
> + } catch (...) {
> + rlogger.warn("Provided host ip {} has no corresponding host id", ip);
> }
> // Nodes which aren't neighbors for this range are ignored.
> // This allows the user to give a list of "good" nodes, where
> @@ -329,6 +332,7 @@ static std::vector<locator::host_id> get_neighbors(
> }
>
> static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
> + const gms::gossiper& gossiper,
> const locator::effective_replication_map& erm,
> const sstring& ksname,
> const dht::token_range_vector& ranges,
> @@ -344,7 +348,7 @@ static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
> participating_hosts.insert(my_address);
>
> co_await do_for_each(ranges, [&] (const dht::token_range& range) {
> - const auto nbs = get_neighbors(erm, ksname, range, data_centers, hosts, ignore_nodes);
> + const auto nbs = get_neighbors(gossiper, erm, ksname, range, data_centers, hosts, ignore_nodes);
> for (const auto& nb : nbs) {
> participating_hosts.insert(nb);
> }
> @@ -676,7 +680,7 @@ void repair::shard_repair_task_impl::check_in_abort_or_shutdown() {
>
> repair_neighbors repair::shard_repair_task_impl::get_repair_neighbors(const dht::token_range& range) {
> return neighbors.empty() ?
> - repair_neighbors(get_neighbors(*erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
> + repair_neighbors(get_neighbors(gossiper, *erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
> neighbors[range];
> }
>
> @@ -1306,7 +1310,7 @@ future<int> repair_service::do_repair_start(gms::gossip_address_map& addr_map, s
> }
>
> auto ranges_parallelism = options.ranges_parallelism == -1 ? std::nullopt : std::optional<int>(options.ranges_parallelism);
> - auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism);
> + auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism, _gossiper.local());
> co_return id.id;
> }
>
> @@ -1331,7 +1335,7 @@ future<> repair::user_requested_repair_task_impl::run() {
> auto normal_nodes = germs->get().get_token_metadata().get_normal_token_owners();
> participants = std::list<locator::host_id>(normal_nodes.begin(), normal_nodes.end());
> } else {
> - participants = get_hosts_participating_in_repair(germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
> + participants = get_hosts_participating_in_repair(_gossiper, germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
> }
> auto [hints_batchlog_flushed, flush_time] = rs.flush_hints(id, keyspace, cfs, ignore_nodes).get();
>

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 13, 2025, 10:32:18 AMJan 13
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:06 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> Now that we have gossiper::get_endpoint_state_ptr that works on host ids
> there is no need to translate id to ip at all.
This error message tricks the reader into thinking no translation goes
on, when the translation was just moved inside get_endpoint_state_ptr

> ---
> service/topology_coordinator.cc | 35 ++++++++++++++-------------------
> 1 file changed, 15 insertions(+), 20 deletions(-)
>
> diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc
> index f2b3a5a68a1..e71aecbb9ed 100644
> --- a/service/topology_coordinator.cc
> +++ b/service/topology_coordinator.cc
> @@ -3055,11 +3055,11 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
> rtlogger.info("building initial raft topology state and CDC generation");
> guard = co_await start_operation();
>
> - auto get_application_state = [&] (locator::host_id host_id, gms::inet_address ep, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
> + auto get_application_state = [&] (locator::host_id host_id, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
> const auto it = epmap.find(app_state);
> if (it == epmap.end()) {
> - throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{}: application state {} is missing in gossip",
> - host_id, ep, app_state));
> + throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}: application state {} is missing in gossip",
I think providing fewer details in error messages is always for the worse.

Same for all the other messages you changed like this in the commit.

> + host_id, app_state));
> }
> // it's versioned_value::value(), not std::optional::value() - it does not throw
> return it->second.value();
> @@ -3067,17 +3067,13 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
>
> // Create a new CDC generation
> auto get_sharding_info_for_host_id = [&] (locator::host_id host_id) -> std::pair<size_t, uint8_t> {
> - const auto ep = tmptr->get_endpoint_for_host_id_if_known(host_id);
> - if (!ep) {
> - throw std::runtime_error(format("IP of node with ID {} is not known", host_id));
> - }
> - const auto eptr = _gossiper.get_endpoint_state_ptr(*ep);
> + const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
> if (!eptr) {
> - throw std::runtime_error(format("no gossiper endpoint state for node {}/{}", host_id, *ep));
> + throw std::runtime_error(format("no gossiper endpoint state for node {}", host_id));
Previous code would distinguish the case where host ID was present but
IP was missing, this merges the two cases :(

Maybe there should be a version of get_endpoint_state_ptr which throws
an error (instead of returning nullptr) if mapping is missing?
Or return variant or std::expected


> }
> const auto& epmap = eptr->get_application_state_map();
> - const auto shard_count = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::SHARD_COUNT));
> - const auto ignore_msb = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::IGNORE_MSB_BITS));
> + const auto shard_count = std::stoi(get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT));
> + const auto ignore_msb = std::stoi(get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS));
> return std::make_pair<size_t, uint8_t>(shard_count, ignore_msb);
> };
> auto [cdc_gen_uuid, guard_, mutation] = co_await prepare_and_broadcast_cdc_generation_data(tmptr, std::move(guard), std::nullopt, get_sharding_info_for_host_id);
> @@ -3094,23 +3090,22 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
> }
>
> const auto& host_id = node.get().host_id();
> - const auto& ep = node.get().endpoint();
> - const auto eptr = _gossiper.get_endpoint_state_ptr(ep);
> + const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
> if (!eptr) {
> - throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{} as gossip contains no data for it", host_id, ep));
> + throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {} as gossip contains no data for it", host_id));
> }
>
> const auto& epmap = eptr->get_application_state_map();
>
> - const auto datacenter = get_application_state(host_id, ep, epmap, gms::application_state::DC);
> - const auto rack = get_application_state(host_id, ep, epmap, gms::application_state::RACK);
> + const auto datacenter = get_application_state(host_id, epmap, gms::application_state::DC);
> + const auto rack = get_application_state(host_id, epmap, gms::application_state::RACK);
> const auto tokens_v = tmptr->get_tokens(host_id);
> const std::unordered_set<dht::token> tokens(tokens_v.begin(), tokens_v.end());
> - const auto release_version = get_application_state(host_id, ep, epmap, gms::application_state::RELEASE_VERSION);
> + const auto release_version = get_application_state(host_id, epmap, gms::application_state::RELEASE_VERSION);
> const auto num_tokens = tokens.size();
> - const auto shard_count = get_application_state(host_id, ep, epmap, gms::application_state::SHARD_COUNT);
> - const auto ignore_msb = get_application_state(host_id, ep, epmap, gms::application_state::IGNORE_MSB_BITS);
> - const auto supported_features_s = get_application_state(host_id, ep, epmap, gms::application_state::SUPPORTED_FEATURES);
> + const auto shard_count = get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT);
> + const auto ignore_msb = get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS);
> + const auto supported_features_s = get_application_state(host_id, epmap, gms::application_state::SUPPORTED_FEATURES);
> const auto supported_features = gms::feature_service::to_feature_set(supported_features_s);
>
> if (enabled_features.empty()) {

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 13, 2025, 10:32:26 AMJan 13
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> We want to drop ips from token_metadata so move to different API to map
> ip to id.
> ---
> service/storage_service.cc | 15 +++++++++------
> 1 file changed, 9 insertions(+), 6 deletions(-)
>
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index 512126ff2ec..d4627a85867 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -4455,11 +4455,11 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
> // and waits for ring_delay. It's possible the node being decommissioned might
> // die after it has sent this notification. If this happens, the node would
> // have already been removed from this token_metadata, so we wouldn't find it here.
> - const auto node_id = tmptr->get_host_id_if_known(node);
> - slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
> - if (node_id) {
> - tmptr->del_leaving_endpoint(*node_id);
> - }
> + try {
> + const auto node_id = _gossiper.get_host_id(node);
> + slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
> + tmptr->del_leaving_endpoint(node_id);
Could we skip mapping and delete by host ID instead?

> + } catch (...) {}
> }
> return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes));
> });
> @@ -4475,7 +4475,10 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
Maybe we should provide node_ops RPC with host ID (if it doesn't already
send it) and skip the mapping

> check_again = false;
> for (auto& node : req.leaving_nodes) {
> auto tmptr = get_token_metadata_ptr();
> - const auto host_id = tmptr->get_host_id_if_known(node);
> + std::optional<locator::host_id> host_id;
> + try {
> + host_id = _gossiper.get_host_id(node);
> + } catch(...) {};
> if (host_id && tmptr->is_normal_token_owner(*host_id)) {
> check_again = true;
> if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) {

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 13, 2025, 10:32:35 AMJan 13
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> We want to drop ips from token_metadata so move to different API to map
> ip to id.
> ---
> service/storage_service.cc | 32 +++++++++++++++++++++-----------
> 1 file changed, 21 insertions(+), 11 deletions(-)
>
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index d4627a85867..a0628a8e9ba 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -2629,19 +2629,18 @@ future<> storage_service::on_join(gms::inet_address endpoint, gms::endpoint_stat
>
> future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_state_ptr state, gms::permit_id pid) {
> const auto& tm = get_token_metadata();
> - const auto tm_host_id_opt = tm.get_host_id_if_known(endpoint);
> - slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, tm_host_id_opt, pid);
> + const auto host_id = state->get_host_id();
> + slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid);
> const auto* node = tm.get_topology().find_node(endpoint);
> if (node && node->is_member()) {
> co_await notify_up(endpoint);
> } else if (raft_topology_change_enabled()) {
> slogger.debug("ignore on_alive since topology changes are using raft and "
> - "endpoint {}/{} is not a topology member", endpoint, tm_host_id_opt);
> + "endpoint {}/{} is not a topology member", endpoint, host_id);
> } else {
> auto tmlock = co_await get_token_metadata_lock();
> auto tmptr = co_await get_mutable_token_metadata_ptr();
> const auto dc_rack = get_dc_rack_for(endpoint);
> - const auto host_id = _gossiper.get_host_id(endpoint);
> tmptr->update_host_id(host_id, endpoint);
> tmptr->update_topology(host_id, dc_rack);
> co_await replicate_to_all_cores(std::move(tmptr));
> @@ -2732,15 +2731,26 @@ future<> storage_service::on_remove(gms::inet_address endpoint, gms::permit_id p
> co_return;
> }
>
> + locator::host_id host_id;
> +
> + try {
> + // It seam gossiper does not check for endpoint existance before calling the callback
> + // so the lookpu may fail, but there is nothing to do in this case.
typos typos typos
> + host_id = _gossiper.get_host_id(endpoint);
> + } catch (...) {
> + co_return;
> + }
> +
> + // We should handle the case when the host id is mapped to a different address.
> + // This could happen when an address for the host id changes and the callback here is called
> + // with the old ip. We should just skip the remove in that case.
Aren't we supposed to purge the old mapping?
Didn't previous version do that?

> + if (_address_map.get(host_id) != endpoint) {
> + co_return;
> + }
> +
> auto tmlock = co_await get_token_metadata_lock();
> auto tmptr = co_await get_mutable_token_metadata_ptr();
> - // We should handle the case when we aren't able to find endpoint -> ip mapping in token_metadata.
> - // This could happen e.g. when the new endpoint has bigger generation in handle_state_normal - the code
> - // in handle_state_normal will remap host_id to the new IP and we won't find
> - // old IP here. We should just skip the remove in that case.
> - if (const auto host_id = tmptr->get_host_id_if_known(endpoint); host_id) {
> - tmptr->remove_endpoint(*host_id);
> - }
> + tmptr->remove_endpoint(host_id);
> co_await update_topology_change_info(tmptr, ::format("on_remove {}", endpoint));
> co_await replicate_to_all_cores(std::move(tmptr));
> }

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 3:45:55 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
On Mon, Jan 13, 2025 at 04:32:13PM +0100, Kamil Braun wrote:
>
>
> On 1/13/25 9:06 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> > Now that we have gossiper::get_endpoint_state_ptr that works on host ids
> > there is no need to translate id to ip at all.
> This error message tricks the reader into thinking no translation goes on,
> when the translation was just moved inside get_endpoint_state_ptr
>
A user of the API should not be concerned with implementation details
which may change later anyway.

> > ---
> > service/topology_coordinator.cc | 35 ++++++++++++++-------------------
> > 1 file changed, 15 insertions(+), 20 deletions(-)
> >
> > diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc
> > index f2b3a5a68a1..e71aecbb9ed 100644
> > --- a/service/topology_coordinator.cc
> > +++ b/service/topology_coordinator.cc
> > @@ -3055,11 +3055,11 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
> > rtlogger.info("building initial raft topology state and CDC generation");
> > guard = co_await start_operation();
> > - auto get_application_state = [&] (locator::host_id host_id, gms::inet_address ep, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
> > + auto get_application_state = [&] (locator::host_id host_id, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
> > const auto it = epmap.find(app_state);
> > if (it == epmap.end()) {
> > - throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{}: application state {} is missing in gossip",
> > - host_id, ep, app_state));
> > + throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}: application state {} is missing in gossip",
> I think providing fewer details in error messages is always for the worse.

We cannot provide details we no longer have and the idea is to make most
of the code non ip address aware.

>
> Same for all the other messages you changed like this in the commit.
>
> > + host_id, app_state));
> > }
> > // it's versioned_value::value(), not std::optional::value() - it does not throw
> > return it->second.value();
> > @@ -3067,17 +3067,13 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
> > // Create a new CDC generation
> > auto get_sharding_info_for_host_id = [&] (locator::host_id host_id) -> std::pair<size_t, uint8_t> {
> > - const auto ep = tmptr->get_endpoint_for_host_id_if_known(host_id);
> > - if (!ep) {
> > - throw std::runtime_error(format("IP of node with ID {} is not known", host_id));
> > - }
> > - const auto eptr = _gossiper.get_endpoint_state_ptr(*ep);
> > + const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
> > if (!eptr) {
> > - throw std::runtime_error(format("no gossiper endpoint state for node {}/{}", host_id, *ep));
> > + throw std::runtime_error(format("no gossiper endpoint state for node {}", host_id));
> Previous code would distinguish the case where host ID was present but IP
> was missing, this merges the two cases :(
>
There cannot be such case now. Gossiper endpoint always has both.
--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 3:49:01 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
I wish. This requires changing node ops API to sent host ids and still
have the mapping here to support mixed versions.

> > + } catch (...) {}
> > }
> > return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes));
> > });
> > @@ -4475,7 +4475,10 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
> Maybe we should provide node_ops RPC with host ID (if it doesn't already
> send it) and skip the mapping
>
This is legacy code that does not worth the afford and will have to map
anyway (see above).

> > check_again = false;
> > for (auto& node : req.leaving_nodes) {
> > auto tmptr = get_token_metadata_ptr();
> > - const auto host_id = tmptr->get_host_id_if_known(node);
> > + std::optional<locator::host_id> host_id;
> > + try {
> > + host_id = _gossiper.get_host_id(node);
> > + } catch(...) {};
> > if (host_id && tmptr->is_normal_token_owner(*host_id)) {
> > check_again = true;
> > if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) {
>

--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 4:21:27 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
No. Look at the code that is removed below. It checks the same thing but
using get_host_id_if_known instead. For the address map the remapping
happens in gossiper though. We should not remove a node from
token_metadata just because its address has changed.

> > + if (_address_map.get(host_id) != endpoint) {
> > + co_return;
> > + }
> > +
> > auto tmlock = co_await get_token_metadata_lock();
> > auto tmptr = co_await get_mutable_token_metadata_ptr();
> > - // We should handle the case when we aren't able to find endpoint -> ip mapping in token_metadata.
> > - // This could happen e.g. when the new endpoint has bigger generation in handle_state_normal - the code
> > - // in handle_state_normal will remap host_id to the new IP and we won't find
> > - // old IP here. We should just skip the remove in that case.
> > - if (const auto host_id = tmptr->get_host_id_if_known(endpoint); host_id) {
> > - tmptr->remove_endpoint(*host_id);
> > - }
> > + tmptr->remove_endpoint(host_id);
> > co_await update_topology_change_info(tmptr, ::format("on_remove {}", endpoint));
> > co_await replicate_to_all_cores(std::move(tmptr));
> > }
>

--
Gleb.

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:56:24 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> locator/util.hh already has get_range_to_address_map which is exactly
> like the one in the storage_service. So remove the later one and use the
> former instead.
> ---
> locator/util.hh | 5 +++++
> service/storage_service.hh | 11 -----------
> locator/util.cc | 2 +-
> service/storage_service.cc | 24 +-----------------------
> 4 files changed, 7 insertions(+), 35 deletions(-)
>
> diff --git a/locator/util.hh b/locator/util.hh
> index e85c04048ea..3a3360d3d94 100644
> --- a/locator/util.hh
> +++ b/locator/util.hh
> @@ -9,6 +9,9 @@
> #pragma once
>
> #include "dht/token_range_endpoints.hh"
> +#include "dht/i_partitioner_fwd.hh"
> +#include "inet_address_vectors.hh"
> +#include "locator/abstract_replication_strategy.hh"
>
> namespace replica {
> class database;
> @@ -20,4 +23,6 @@ namespace gms {
>
> namespace locator {
> future<std::vector<dht::token_range_endpoints>> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false);
> + future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(
> + locator::effective_replication_map_ptr erm, const std::vector<token>& sorted_tokens);
> }
> \ No newline at end of file
> diff --git a/service/storage_service.hh b/service/storage_service.hh
> index edf279b497e..e72881cf1d4 100644
> --- a/service/storage_service.hh
> +++ b/service/storage_service.hh
> @@ -427,8 +427,6 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
>
> public:
> future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(locator::effective_replication_map_ptr erm) const;
> - future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(locator::effective_replication_map_ptr erm,
> - const std::vector<token>& sorted_tokens) const;
>
> /**
> * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
> @@ -465,15 +463,6 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
> */
> future<std::map<token, inet_address>> get_tablet_to_endpoint_map(table_id table);
>
> - /**
> - * Construct the range to endpoint mapping based on the true view
> - * of the world.
> - * @param ranges
> - * @return mapping of ranges to the replicas responsible for them.
> - */
> - future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> construct_range_to_endpoint_map(
> - locator::effective_replication_map_ptr erm,
> - const dht::token_range_vector& ranges) const;
> public:
> virtual future<> on_join(gms::inet_address endpoint, gms::endpoint_state_ptr ep_state, gms::permit_id) override;
> /*
> diff --git a/locator/util.cc b/locator/util.cc
> index fc1cba53942..d7f8768cdae 100644
> --- a/locator/util.cc
> +++ b/locator/util.cc
> @@ -47,7 +47,7 @@ get_all_ranges(const std::vector<token>& sorted_tokens) {
> }
>
> // Caller is responsible to hold token_metadata valid until the returned future is resolved
> -static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
> +future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
> get_range_to_address_map(locator::effective_replication_map_ptr erm,
> const std::vector<token>& sorted_tokens) {
> co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
Looks like similar copy-pasta happens for get_all_ranges; possibly others
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index a0628a8e9ba..1b8abd6366d 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -2295,14 +2295,7 @@ future<> storage_service::bootstrap(std::unordered_set<token>& bootstrap_tokens,
>
> future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
> storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const {
> - return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
> -}
> -
> -// Caller is responsible to hold token_metadata valid until the returned future is resolved
> -future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
> -storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm,
> - const std::vector<token>& sorted_tokens) const {
> - co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
> + return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
> }
>
> future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) {
> @@ -5264,21 +5257,6 @@ storage_service::describe_ring_for_table(const sstring& keyspace_name, const sst
> co_return ranges;
> }
>
> -future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
> -storage_service::construct_range_to_endpoint_map(
> - locator::effective_replication_map_ptr erm,
> - const dht::token_range_vector& ranges) const {
> - std::unordered_map<dht::token_range, inet_address_vector_replica_set> res;
> - res.reserve(ranges.size());
> - for (auto r : ranges) {
> - res[r] = erm->get_natural_endpoints(
> - r.end() ? r.end()->value() : dht::maximum_token());
> - co_await coroutine::maybe_yield();
> - }
> - co_return res;
> -}
> -
> -
> std::map<token, inet_address> storage_service::get_token_to_endpoint_map() {
> const auto& tm = get_token_metadata();
> std::map<token, inet_address> result;

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:57:08 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> ---
> locator/abstract_replication_strategy.hh | 5 -----
> locator/abstract_replication_strategy.cc | 23 -----------------------
> locator/tablets.cc | 4 ----
> 3 files changed, 32 deletions(-)
>
> diff --git a/locator/abstract_replication_strategy.hh b/locator/abstract_replication_strategy.hh
> index 46f469f80ea..5358139c72b 100644
> --- a/locator/abstract_replication_strategy.hh
> +++ b/locator/abstract_replication_strategy.hh
> @@ -233,9 +233,6 @@ class effective_replication_map {
> /// new replica.
> ///
> /// The returned addresses are present in the topology object associated with this instance.
The other comments need to be updated (every mention of "address" should
be changed to "host ID" etc.)

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:57:23 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> Instead use gossiper and peers table to retrieve same information.
> Token_metadata is created from the mix of those two anyway. The goal is
> to drop ips from token_metadata entirely.
> ---
> service/storage_service.cc | 28 ++++++++++++++++++++++------
> 1 file changed, 22 insertions(+), 6 deletions(-)
>
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index 10241a40f70..4c1b4d91415 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -2362,11 +2362,26 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
> if (tmptr->is_normal_token_owner(host_id)) {
> slogger.info("handle_state_normal: node {}/{} was already a normal token owner", endpoint, host_id);
> }
> - auto existing = tmptr->get_endpoint_for_host_id_if_known(host_id);
>
> // Old node in replace-with-same-IP scenario.
> std::optional<locator::host_id> replaced_id;
>
> + auto ips = _gossiper.get_nodes_with_host_id(host_id);
> +
> + std::optional<inet_address> existing;
> +
> + if (tmptr->get_topology().find_node(host_id)) {
> + // If node is not in the topology there is no existsing address
> + // If there are two addresses for the same id the "other" one is existing
> + // If there is only one it is existing
> + if (ips.size() == 2) {
> + if (ips.erase(endpoint) == 0) {
> + on_internal_error(slogger, fmt::format("Did not find endpoint {} in gossiper {}", endpoint, ips));
"Did not find endpoint X in gossiper Y Z" -- error hard to read and
uninformative (how are X, Y, Z related)? Should provide more context (at
least how hard would it be to print the host_id as well)

> + }
> + }
> + existing = *ips.begin();
what if `ips` is empty?

> + }
> +
> if (existing && *existing != endpoint) {
> // This branch in taken when a node changes its IP address.
>
> @@ -2412,11 +2427,12 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
> // We do this after update_normal_tokens, allowing for tokens to be properly
> // migrated to the new host_id.
>
> - slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
> - if (const auto old_host_id = tmptr->get_host_id_if_known(endpoint); old_host_id && *old_host_id != host_id) {
> - // Replace with same IP scenario
> - slogger.info("The IP {} previously owned host ID {}", endpoint, *old_host_id);
> - replaced_id = *old_host_id;
> + auto peers = co_await _sys_ks.local().load_host_ids();
> + if (peers.contains(endpoint) && peers[endpoint] != host_id) {
> + replaced_id = peers[endpoint];
> + slogger.info("The IP {} previously owned host ID {}", endpoint, *replaced_id);
> + } else {
> + slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
> }
> } else {
> // This branch is taken if this node wasn't involved in node_ops

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:57:30 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
what is this, some leftover from Cassandra?

On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:57:58 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> ---
> gms/gossiper.hh | 3 ++-
> gms/gossiper.cc | 8 ++++++--
> service/migration_manager.cc | 6 ++----
> service/storage_service.cc | 12 ++++++------
> 4 files changed, 16 insertions(+), 13 deletions(-)
>
> diff --git a/gms/gossiper.hh b/gms/gossiper.hh
> index 595f7103ca6..1b647370ff3 100644
> --- a/gms/gossiper.hh
> +++ b/gms/gossiper.hh
> @@ -296,7 +296,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
> */
> future<> unregister_(shared_ptr<i_endpoint_state_change_subscriber> subscriber);
>
> - std::set<inet_address> get_live_members() const;
> + std::set<locator::host_id> get_live_members() const;
>
> std::set<locator::host_id> get_live_token_owners() const;
>
> @@ -528,6 +528,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
> future<> wait_alive(std::vector<gms::inet_address> nodes, std::chrono::milliseconds timeout);
> future<> wait_alive(std::vector<locator::host_id> nodes, std::chrono::milliseconds timeout);
> future<> wait_alive(noncopyable_function<std::vector<locator::host_id>()> get_nodes, std::chrono::milliseconds timeout);
> + std::set<inet_address> get_live_members_helper() const;
>
> // Wait for `n` live nodes to show up in gossip (including ourself).
> future<> wait_for_live_nodes_to_show_up(size_t n);
> diff --git a/gms/gossiper.cc b/gms/gossiper.cc
> index c303230316e..a8ec00dc816 100644
> --- a/gms/gossiper.cc
> +++ b/gms/gossiper.cc
> @@ -924,7 +924,7 @@ future<std::set<inet_address>> gossiper::get_live_members_synchronized() {
> return container().invoke_on(0, [] (gms::gossiper& g) -> future<std::set<inet_address>> {
> // Make sure the value we return is synchronized on all shards
> auto lock = co_await g.lock_endpoint_update_semaphore();
> - co_return g.get_live_members();
> + co_return g.get_live_members_helper();
> });
> }
>
> @@ -1175,7 +1175,7 @@ future<> gossiper::unregister_(shared_ptr<i_endpoint_state_change_subscriber> su
> return _subscribers.remove(subscriber);
> }
>
> -std::set<inet_address> gossiper::get_live_members() const {
> +std::set<inet_address> gossiper::get_live_members_helper() const {
> std::set<inet_address> live_members(_live_endpoints.begin(), _live_endpoints.end());
> auto myip = get_broadcast_address();
> logger.debug("live_members before={}", live_members);
> @@ -1186,6 +1186,10 @@ std::set<inet_address> gossiper::get_live_members() const {
> return live_members;
> }
>
> +std::set<locator::host_id> gossiper::get_live_members() const {
> + return get_live_members_helper() | std::views::transform([this] (inet_address ip) { return get_host_id(ip); }) | std::ranges::to<std::set>();
> +}
> +
> std::set<locator::host_id> gossiper::get_live_token_owners() const {
> std::set<locator::host_id> token_owners;
> auto normal_token_owners = get_token_metadata_ptr()->get_normal_token_owners();
> diff --git a/service/migration_manager.cc b/service/migration_manager.cc
> index a4474034be4..568f2061b05 100644
> --- a/service/migration_manager.cc
> +++ b/service/migration_manager.cc
> @@ -919,13 +919,11 @@ future<> migration_manager::announce_without_raft(std::vector<mutation> schema,
> try {
> using namespace std::placeholders;
> auto all_live = _gossiper.get_live_members();
> - auto live_members = all_live | std::views::filter([this, my_address = _messaging.broadcast_address()] (const gms::inet_address& endpoint) {
> + auto live_members = all_live | std::views::filter([my_address = _gossiper.my_host_id()] (const locator::host_id& endpoint) {
> // only push schema to nodes with known and equal versions
> return endpoint != my_address;
> });
> - // FIXME: gossiper should return host id set
> - auto live_host_ids = live_members | std::views::transform([&] (const gms::inet_address& ip) { return _gossiper.get_host_id(ip); });
> - co_await coroutine::parallel_for_each(live_host_ids,
> + co_await coroutine::parallel_for_each(live_members,
> std::bind(std::mem_fn(&migration_manager::push_schema_mutation), this, std::placeholders::_1, schema));
> } catch (...) {
> mlogger.error("failed to announce migration to all nodes: {}", std::current_exception());
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index 4c1b4d91415..16c23fc8a20 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -2727,7 +2727,7 @@ future<> storage_service::maybe_reconnect_to_preferred_ip(inet_address ep, inet_
> }
>
> const auto& topo = get_token_metadata().get_topology();
> - if (topo.get_datacenter() == topo.get_datacenter(ep) && _messaging.local().get_preferred_ip(ep) != local_ip) {
> + if (topo.get_datacenter() == topo.get_datacenter(_gossiper.get_host_id(ep)) && _messaging.local().get_preferred_ip(ep) != local_ip) {
nit: this change looks unrelated to the get_live_members change? Maybe
it belongs in another commit

> slogger.debug("Initiated reconnect to an Internal IP {} for the {}", local_ip, ep);
> co_await _messaging.invoke_on_all([ep, local_ip] (auto& local_ms) {
> local_ms.cache_preferred_ip(ep, local_ip);
> @@ -3601,17 +3601,17 @@ future<std::unordered_map<sstring, std::vector<sstring>>> storage_service::descr
> std::unordered_map<sstring, std::vector<sstring>> results;
> netw::messaging_service& ms = _messaging.local();
> return map_reduce(std::move(live_hosts), [&ms, as = abort_source()] (auto host) mutable {
> - auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, netw::msg_addr{ host, 0 }, as);
> + auto f0 = ser::migration_manager_rpc_verbs::send_schema_check(&ms, host, as);
> return std::move(f0).then_wrapped([host] (auto f) {
> if (f.failed()) {
> f.ignore_ready_future();
> - return std::pair<gms::inet_address, std::optional<table_schema_version>>(host, std::nullopt);
> + return std::pair<locator::host_id, std::optional<table_schema_version>>(host, std::nullopt);
> }
> - return std::pair<gms::inet_address, std::optional<table_schema_version>>(host, f.get());
> + return std::pair<locator::host_id, std::optional<table_schema_version>>(host, f.get());
> });
> - }, std::move(results), [] (auto results, auto host_and_version) {
> + }, std::move(results), [this] (auto results, auto host_and_version) {
> auto version = host_and_version.second ? host_and_version.second->to_sstring() : UNREACHABLE;
> - results.try_emplace(version).first->second.emplace_back(fmt::to_string(host_and_version.first));
> + results.try_emplace(version).first->second.emplace_back(fmt::to_string(_address_map.get(host_and_version.first)));
Looks like a change for the worse:
first get_live_members() translates from IPs (from _helper()) to host IDs
then here you translate back to IP


TBH I don't see the motivation for changing get_live_members(), it would
make sense if we stored _live_members in gossiper using host IDs,
without that you introduce more translations which looks undesirable
> return results;
> }).then([this] (auto results) {
> // we're done: the results map is ready to return to the client. the rest is just debug logging:

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:58:09 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> Also use gossiper to translate instead of taken_metadata since we want
> to get rid of ip base APIs there.
typos
> ---
> db/hints/manager.hh | 2 +-
> db/hints/manager.cc | 3 ++-
> service/storage_proxy.cc | 6 ++----
> 3 files changed, 5 insertions(+), 6 deletions(-)
>
> diff --git a/db/hints/manager.hh b/db/hints/manager.hh
> index afe7e96648f..7f075714f09 100644
> --- a/db/hints/manager.hh
> +++ b/db/hints/manager.hh
> @@ -171,7 +171,7 @@ class manager {
> void register_metrics(const sstring& group_name);
> future<> start(shared_ptr<const gms::gossiper> gossiper_ptr);
> future<> stop();
> - bool store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
> + bool store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
> tracing::trace_state_ptr tr_state) noexcept;
>
> /// \brief Changes the host_filter currently used, stopping and starting endpoint_managers relevant to the new host_filter.
> diff --git a/db/hints/manager.cc b/db/hints/manager.cc
> index 7cdb515352b..015d492b788 100644
> --- a/db/hints/manager.cc
> +++ b/db/hints/manager.cc
> @@ -430,9 +430,10 @@ bool manager::have_ep_manager(const std::variant<locator::host_id, gms::inet_add
> return _hint_directory_manager.has_mapping(std::get<gms::inet_address>(ep));
> }
>
> -bool manager::store_hint(endpoint_id host_id, gms::inet_address ip, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
> +bool manager::store_hint(endpoint_id host_id, schema_ptr s, lw_shared_ptr<const frozen_mutation> fm,
> tracing::trace_state_ptr tr_state) noexcept
> {
> + auto ip = _gossiper_anchor->get_address_map().get(host_id);
> if (utils::get_local_injector().enter("reject_incoming_hints")) {
> manager_logger.debug("Rejecting a hint to {} / {} due to an error injection", host_id, ip);
> ++_stats.dropped;
> diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
> index 247283f153d..37676e64f01 100644
> --- a/service/storage_proxy.cc
> +++ b/service/storage_proxy.cc
> @@ -1243,8 +1243,7 @@ class per_destination_mutation : public mutation_holder {
> tracing::trace_state_ptr tr_state) override {
> auto m = _mutations[hid];
> if (m) {
> - const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid);
> - return hm.store_hint(hid, ep, _schema, std::move(m), tr_state);
> + return hm.store_hint(hid, _schema, std::move(m), tr_state);
> } else {
> return false;
> }
> @@ -1302,8 +1301,7 @@ class shared_mutation : public mutation_holder {
> }
> virtual bool store_hint(db::hints::manager& hm, locator::host_id hid, locator::effective_replication_map_ptr ermptr,
> tracing::trace_state_ptr tr_state) override {
> - const auto ep = ermptr->get_token_metadata().get_endpoint_for_host_id(hid);
> - return hm.store_hint(hid, ep, _schema, _mutation, tr_state);
> + return hm.store_hint(hid, _schema, _mutation, tr_state);
> }
> virtual future<> apply_locally(storage_proxy& sp, storage_proxy::clock_type::time_point timeout,
> tracing::trace_state_ptr tr_state, db::per_partition_rate_limit::info rate_limit_info,

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:58:16 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> As part of changing IP address peers table is updated. If it has a new
> address the update can be skipped.
> ---
> service/storage_service.hh | 3 +++
> service/storage_service.cc | 12 +++++++++++-
> 2 files changed, 14 insertions(+), 1 deletion(-)
>
> diff --git a/service/storage_service.hh b/service/storage_service.hh
> index 70a7eeba6ab..fff395375c1 100644
> --- a/service/storage_service.hh
> +++ b/service/storage_service.hh
> @@ -603,6 +603,9 @@ class storage_service : public service::migration_listener, public gms::i_endpoi
> future<std::unordered_multimap<dht::token_range, locator::host_id>> get_changed_ranges_for_leaving(locator::vnode_effective_replication_map_ptr erm, locator::host_id endpoint);
>
> future<> maybe_reconnect_to_preferred_ip(inet_address ep, inet_address local_ip);
> +
> + // Return ip of the peers table entry with given host id
> + future<std::optional<gms::inet_address>> get_ip_from_peers_table(locator::host_id id);
> public:
>
> sstring get_release_version();
> diff --git a/service/storage_service.cc b/service/storage_service.cc
> index 28bc19f2427..0795fc4357c 100644
> --- a/service/storage_service.cc
> +++ b/service/storage_service.cc
> @@ -901,7 +901,9 @@ class storage_service::ip_address_updater: public gms::i_endpoint_state_change_s
> locator::host_id id(utils::UUID(app_state_ptr->value()));
> rslog.debug("ip_address_updater::on_endpoint_change({}) {} {}", ev, endpoint, id);
>
> - auto prev_ip = _ss.get_token_metadata().get_endpoint_for_host_id_if_known(id);
> + // If id maps to different ip in peers table it needs to be updated which is done by sync_raft_topology_nodes below
> + auto peers = co_await _ss._sys_ks.local().load_host_ids();
redundant loading of peers?

> + std::optional<gms::inet_address> prev_ip = co_await _ss.get_ip_from_peers_table(id);
> if (prev_ip == endpoint) {
> co_return;
> }
> @@ -2639,6 +2641,14 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
> }
> }
>
> +future<std::optional<gms::inet_address>> storage_service::get_ip_from_peers_table(locator::host_id id) {
> + auto peers = co_await _sys_ks.local().load_host_ids();
> + if (auto it = std::ranges::find_if(peers, [&id] (const auto& e) { return e.second == id; }); it != peers.end()) {
> + co_return it->first;
> + }
> + co_return std::nullopt;
> +}
> +
> future<> storage_service::on_change(gms::inet_address endpoint, const gms::application_state_map& states_, gms::permit_id pid) {
> // copy the states map locally since the coroutine may yield
> auto states = states_;

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:58:25 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com


On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> We already use if to translate reply_to, so do it for consistency and to
typo
> drop ip based API usage.
> ---
> service/storage_proxy.cc | 10 +++++-----
> 1 file changed, 5 insertions(+), 5 deletions(-)
>
> diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
> index 19e92fb0d39..9cd105b0438 100644
> --- a/service/storage_proxy.cc
> +++ b/service/storage_proxy.cc
> @@ -104,13 +104,13 @@ namespace {
>
>
> template<size_t N>
> -utils::small_vector<locator::host_id, N> addr_vector_to_id(const locator::topology& topo, const utils::small_vector<gms::inet_address, N>& set) {
> +utils::small_vector<locator::host_id, N> addr_vector_to_id(const gms::gossiper& g, const utils::small_vector<gms::inet_address, N>& set) {
> return set | std::views::transform([&] (gms::inet_address ip) {
> - auto* node = topo.find_node(ip);
> - if (!node) {
> + try {
> + return g.get_host_id(ip);
> + } catch (...) {
> on_internal_error(slogger, fmt::format("addr_vector_to_id cannot map {} to host id", ip));
> }
> - return node->host_id();
> }) | std::ranges::to<utils::small_vector<locator::host_id, N>>();
> }
>
> @@ -586,7 +586,7 @@ class storage_proxy::remote {
> }
>
> auto reply_to_host_id = reply_to_id ? *reply_to_id : _gossiper.get_host_id(reply_to);
> - auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_sp._shared_token_metadata.get()->get_topology(), forward);
> + auto forward_host_id = forward_id ? std::move(*forward_id) : addr_vector_to_id(_gossiper, forward);
>
> if (reply_to_id) {
> _gossiper.get_mutable_address_map().opt_add_entry(reply_to_host_id, reply_to);

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:58:31 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com, Patryk Jędrzejczak
Patryk, please review this patch

On 1/13/25 9:07 AM, 'Gleb Natapov' via ScyllaDB development wrote:
> Do not track id to ip mapping in the topology class any longer. There
> are no remaining users.
> ---
> locator/topology.hh | 27 +-----
> locator/token_metadata.cc | 17 +---
> locator/topology.cc | 97 +++-----------------
> main.cc | 3 +-
> test/boost/locator_topology_test.cc | 97 ++++----------------
> test/boost/network_topology_strategy_test.cc | 30 ++----
> test/boost/tablets_test.cc | 6 +-
> test/lib/cql_test_env.cc | 3 +-
> test/perf/perf_sort_by_proximity.cc | 1 -
> 9 files changed, 52 insertions(+), 229 deletions(-)
>
> diff --git a/locator/topology.hh b/locator/topology.hh
> index f27c7428321..c87e8cc2790 100644
> --- a/locator/topology.hh
> +++ b/locator/topology.hh
> @@ -60,7 +60,6 @@ class node {
> private:
> const locator::topology* _topology;
> locator::host_id _host_id;
> - inet_address _endpoint;
> endpoint_dc_rack _dc_rack;
> state _state;
> shard_id _shard_count = 0;
> @@ -73,7 +72,6 @@ class node {
> public:
> node(const locator::topology* topology,
> locator::host_id id,
> - inet_address endpoint,
> endpoint_dc_rack dc_rack,
> state state,
> shard_id shard_count = 0,
> @@ -95,10 +93,6 @@ class node {
> return _host_id;
> }
>
> - const inet_address& endpoint() const noexcept {
> - return _endpoint;
> - }
> -
> const endpoint_dc_rack& dc_rack() const noexcept {
> return _dc_rack;
> }
> @@ -165,7 +159,6 @@ class node {
> private:
> static node_holder make(const locator::topology* topology,
> locator::host_id id,
> - inet_address endpoint,
> endpoint_dc_rack dc_rack,
> state state,
> shard_id shard_count = 0,
> @@ -212,7 +205,7 @@ class topology {
> }
>
> // Adds a node with given host_id, endpoint, and DC/rack.
> - const node& add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state,
> + const node& add_node(host_id id, const endpoint_dc_rack& dr, node::state state,
> shard_id shard_count = 0);
>
> // Optionally updates node's current host_id, endpoint, or DC/rack.
> @@ -220,7 +213,6 @@ class topology {
> // or a peer node host_id may be updated when the node is replaced with another node using the same ip address.
> void update_node(node& node,
> std::optional<host_id> opt_id,
> - std::optional<inet_address> opt_ep,
> std::optional<endpoint_dc_rack> opt_dr,
> std::optional<node::state> opt_st,
> std::optional<shard_id> opt_shard_count = std::nullopt);
> @@ -242,10 +234,6 @@ class topology {
> return *n;
> };
>
> - // Looks up a node by its inet_address.
> - // Returns a pointer to the node if found, or nullptr otherwise.
> - const node* find_node(const inet_address& ep) const noexcept;
> -
> // Finds a node by its index
> // Returns a pointer to the node if found, or nullptr otherwise.
> const node* find_node(node::idx_type idx) const noexcept;
> @@ -258,8 +246,7 @@ class topology {
> *
> * Adds or updates a node with given endpoint
> */
> - const node& add_or_update_endpoint(host_id id, std::optional<inet_address> opt_ep,
> - std::optional<endpoint_dc_rack> opt_dr = std::nullopt,
> + const node& add_or_update_endpoint(host_id id, std::optional<endpoint_dc_rack> opt_dr = std::nullopt,
> std::optional<node::state> opt_st = std::nullopt,
> std::optional<shard_id> shard_count = std::nullopt);
>
> @@ -412,7 +399,6 @@ class topology {
> const node* _this_node = nullptr;
> std::vector<node_holder> _nodes;
> std::unordered_map<host_id, std::reference_wrapper<const node>> _nodes_by_host_id;
> - std::unordered_map<inet_address, std::reference_wrapper<const node>> _nodes_by_endpoint;
>
> std::unordered_map<sstring, std::unordered_set<std::reference_wrapper<const node>>> _dc_nodes;
> std::unordered_map<sstring, std::unordered_map<sstring, std::unordered_set<std::reference_wrapper<const node>>>> _dc_rack_nodes;
> @@ -435,10 +421,6 @@ class topology {
>
> void calculate_datacenters();
>
> - const std::unordered_map<inet_address, std::reference_wrapper<const node>>& get_nodes_by_endpoint() const noexcept {
> - return _nodes_by_endpoint;
> - };
> -
> mutable random_engine_type _random_engine;
>
> friend class token_metadata_impl;
> @@ -492,12 +474,11 @@ struct fmt::formatter<locator::node> : fmt::formatter<string_view> {
> template <typename FormatContext>
> auto format(const locator::node& node, FormatContext& ctx) const {
> if (!verbose) {
> - return fmt::format_to(ctx.out(), "{}/{}", node.host_id(), node.endpoint());
> + return fmt::format_to(ctx.out(), "{}", node.host_id());
> } else {
> - return fmt::format_to(ctx.out(), " idx={} host_id={} endpoint={} dc={} rack={} state={} shards={} this_node={}",
> + return fmt::format_to(ctx.out(), " idx={} host_id={} dc={} rack={} state={} shards={} this_node={}",
> node.idx(),
> node.host_id(),
> - node.endpoint(),
> node.dc_rack().dc,
> node.dc_rack().rack,
> locator::node::to_string(node.get_state()),
> diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
> index 27b8e991285..519a6647549 100644
> --- a/locator/token_metadata.cc
> +++ b/locator/token_metadata.cc
> @@ -116,7 +116,7 @@ class token_metadata_impl final {
> }
>
> void update_topology(host_id id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count = std::nullopt) {
> - _topology.add_or_update_endpoint(id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
> + _topology.add_or_update_endpoint(id, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
> }
>
> /**
> @@ -141,15 +141,6 @@ class token_metadata_impl final {
>
> void debug_show() const;
>
> - /**
> - * Store an end-point to host ID mapping. Each ID must be unique, and
> - * cannot be changed after the fact.
> - *
> - * @param hostId
> - * @param endpoint
> - */
> - void update_host_id(const host_id& host_id, inet_address endpoint);
> -
> /** @return a copy of host id set for read-only operations */
> std::unordered_set<host_id> get_host_ids() const;
>
> @@ -516,10 +507,6 @@ void token_metadata_impl::debug_show() const {
> reporter->arm_periodic(std::chrono::seconds(1));
> }
>
> -void token_metadata_impl::update_host_id(const host_id& host_id, inet_address endpoint) {
> - _topology.add_or_update_endpoint(host_id, endpoint);
> -}
> -
> std::unordered_set<host_id> token_metadata_impl::get_host_ids() const {
> return _topology.get_nodes() |
> std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) |
> @@ -964,7 +951,7 @@ token_metadata::debug_show() const {
>
> void
> token_metadata::update_host_id(const host_id& host_id, inet_address endpoint) {
> - _impl->update_host_id(host_id, endpoint);
> + // Do nothing for now. Remove later.
> }
>
> std::unordered_set<host_id>
> diff --git a/locator/topology.cc b/locator/topology.cc
> index 195c512a11a..8c03ac33b12 100644
> --- a/locator/topology.cc
> +++ b/locator/topology.cc
> @@ -54,10 +54,9 @@ thread_local const endpoint_dc_rack endpoint_dc_rack::default_location = {
> .rack = locator::production_snitch_base::default_rack,
> };
>
> -node::node(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx)
> +node::node(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, this_node is_this_node, node::idx_type idx)
> : _topology(topology)
> , _host_id(id)
> - , _endpoint(endpoint)
> , _dc_rack(std::move(dc_rack))
> , _state(state)
> , _shard_count(std::move(shard_count))
> @@ -65,12 +64,12 @@ node::node(const locator::topology* topology, locator::host_id id, inet_address
> , _idx(idx)
> {}
>
> -node_holder node::make(const locator::topology* topology, locator::host_id id, inet_address endpoint, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) {
> - return std::make_unique<node>(topology, std::move(id), std::move(endpoint), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx);
> +node_holder node::make(const locator::topology* topology, locator::host_id id, endpoint_dc_rack dc_rack, state state, shard_id shard_count, node::this_node is_this_node, node::idx_type idx) {
> + return std::make_unique<node>(topology, std::move(id), std::move(dc_rack), std::move(state), shard_count, is_this_node, idx);
> }
>
> node_holder node::clone() const {
> - return make(nullptr, host_id(), endpoint(), dc_rack(), get_state(), get_shard_count(), is_this_node());
> + return make(nullptr, host_id(), dc_rack(), get_state(), get_shard_count(), is_this_node());
> }
>
> std::string node::to_string(node::state s) {
> @@ -94,7 +93,6 @@ future<> topology::clear_gently() noexcept {
> _datacenters.clear();
> _dc_rack_nodes.clear();
> _dc_nodes.clear();
> - _nodes_by_endpoint.clear();
> _nodes_by_host_id.clear();
> co_await utils::clear_gently(_nodes);
> }
> @@ -115,7 +113,7 @@ topology::topology(config cfg)
> {
> tlogger.trace("topology[{}]: constructing using config: endpoint={} id={} dc={} rack={}", fmt::ptr(this),
> cfg.this_endpoint, cfg.this_host_id, cfg.local_dc_rack.dc, cfg.local_dc_rack.rack);
> - add_node(cfg.this_host_id, cfg.this_endpoint, cfg.local_dc_rack, node::state::none);
> + add_node(cfg.this_host_id, cfg.local_dc_rack, node::state::none);
> }
>
> topology::topology(topology&& o) noexcept
> @@ -124,7 +122,6 @@ topology::topology(topology&& o) noexcept
> , _this_node(std::exchange(o._this_node, nullptr))
> , _nodes(std::move(o._nodes))
> , _nodes_by_host_id(std::move(o._nodes_by_host_id))
> - , _nodes_by_endpoint(std::move(o._nodes_by_endpoint))
> , _dc_nodes(std::move(o._dc_nodes))
> , _dc_rack_nodes(std::move(o._dc_rack_nodes))
> , _dc_endpoints(std::move(o._dc_endpoints))
> @@ -170,7 +167,7 @@ void topology::set_host_id_cfg(host_id this_host_id) {
> tlogger.trace("topology[{}]: set host id to {}", fmt::ptr(this), this_host_id);
>
> _cfg.this_host_id = this_host_id;
> - add_or_update_endpoint(this_host_id, _cfg.this_endpoint);
> + add_or_update_endpoint(this_host_id);
> }
>
> future<topology> topology::clone_gently() const {
> @@ -187,21 +184,15 @@ future<topology> topology::clone_gently() const {
> co_return ret;
> }
>
> -const node& topology::add_node(host_id id, const inet_address& ep, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) {
> +const node& topology::add_node(host_id id, const endpoint_dc_rack& dr, node::state state, shard_id shard_count) {
> if (dr.dc.empty() || dr.rack.empty()) {
> on_internal_error(tlogger, "Node must have valid dc and rack");
> }
> - return add_node(node::make(this, id, ep, dr, state, shard_count));
> + return add_node(node::make(this, id, dr, state, shard_count));
> }
>
> bool topology::is_configured_this_node(const node& n) const {
> - if (_cfg.this_host_id && n.host_id()) { // Selection by host_id
> - return _cfg.this_host_id == n.host_id();
> - }
> - if (_cfg.this_endpoint != inet_address()) { // Selection by endpoint
> - return _cfg.this_endpoint == n.endpoint();
> - }
> - return false; // No selection;
> + return _cfg.this_host_id == n.host_id();
> }
>
> const node& topology::add_node(node_holder nptr) {
> @@ -244,10 +235,9 @@ const node& topology::add_node(node_holder nptr) {
> return *node;
> }
>
> -void topology::update_node(node& node, std::optional<host_id> opt_id, std::optional<inet_address> opt_ep, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> opt_shard_count) {
> - tlogger.debug("topology[{}]: update_node: {}: to: host_id={} endpoint={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node),
> +void topology::update_node(node& node, std::optional<host_id> opt_id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> opt_shard_count) {
> + tlogger.debug("topology[{}]: update_node: {}: to: host_id={} dc={} rack={} state={} shard_count={}, at {}", fmt::ptr(this), node_printer(&node),
> opt_id ? format("{}", *opt_id) : "unchanged",
> - opt_ep ? format("{}", *opt_ep) : "unchanged",
> opt_dr ? format("{}", opt_dr->dc) : "unchanged",
> opt_dr ? format("{}", opt_dr->rack) : "unchanged",
> opt_st ? format("{}", *opt_st) : "unchanged",
> @@ -271,16 +261,6 @@ void topology::update_node(node& node, std::optional<host_id> opt_id, std::optio
> opt_id.reset();
> }
> }
> - if (opt_ep) {
> - if (*opt_ep != node.endpoint()) {
> - if (*opt_ep == inet_address{}) {
> - on_internal_error(tlogger, seastar::format("Updating node endpoint to null is disallowed: {}: new endpoint={}", node_printer(&node), *opt_ep));
> - }
> - changed = true;
> - } else {
> - opt_ep.reset();
> - }
> - }
> if (opt_dr) {
> if (opt_dr->dc.empty() || opt_dr->dc == production_snitch_base::default_dc) {
> opt_dr->dc = node.dc_rack().dc;
> @@ -311,9 +291,6 @@ void topology::update_node(node& node, std::optional<host_id> opt_id, std::optio
> if (opt_id) {
> node._host_id = *opt_id;
> }
> - if (opt_ep) {
> - node._endpoint = *opt_ep;
> - }
> if (opt_dr) {
> node._dc_rack = std::move(*opt_dr);
> }
> @@ -359,32 +336,6 @@ void topology::index_node(const node& node) {
> if (!inserted_host_id) {
> on_internal_error(tlogger, seastar::format("topology[{}]: {}: node already exists", fmt::ptr(this), node_printer(&node)));
> }
> - if (node.endpoint() != inet_address{}) {
> - auto eit = _nodes_by_endpoint.find(node.endpoint());
> - if (eit != _nodes_by_endpoint.end()) {
> - if (eit->second.get().get_state() == node::state::none && eit->second.get().is_this_node()) {
> - // eit->second is default entry created for local node and it is replaced by existing node with the same ip
> - // it means this node is going to replace the existing node with the same ip, but it does not know it yet
> - // map ip to the old node
> - _nodes_by_endpoint.erase(node.endpoint());
> - } else if (eit->second.get().get_state() == node::state::replacing && node.get_state() == node::state::being_replaced) {
> - // replace-with-same-ip, map ip to the old node
> - _nodes_by_endpoint.erase(node.endpoint());
> - } else if (eit->second.get().get_state() == node::state::being_replaced && node.get_state() == node::state::replacing) {
> - // replace-with-same-ip, map ip to the old node, do nothing if it's already the case
> - } else if (eit->second.get().is_leaving() || eit->second.get().left()) {
> - _nodes_by_endpoint.erase(node.endpoint());
> - } else if (!node.is_leaving() && !node.left()) {
> - if (node.host_id()) {
> - _nodes_by_host_id.erase(node.host_id());
> - }
> - on_internal_error(tlogger, seastar::format("topology[{}]: {}: node endpoint already mapped to {}", fmt::ptr(this), node_printer(&node), node_printer(&eit->second.get())));
> - }
> - }
> - if (!node.left() && !node.is_none()) {
> - _nodes_by_endpoint.try_emplace(node.endpoint(), std::cref(node));
> - }
> - }
>
> // We keep location of left nodes because they may still appear in tablet replica sets
> // and algorithms expect to know which dc they belonged to. View replica pairing needs stable
> @@ -441,10 +392,6 @@ void topology::unindex_node(const node& node) {
> if (host_it != _nodes_by_host_id.end() && host_it->second == node) {
> _nodes_by_host_id.erase(host_it);
> }
> - auto ep_it = _nodes_by_endpoint.find(node.endpoint());
> - if (ep_it != _nodes_by_endpoint.end() && ep_it->second.get() == node) {
> - _nodes_by_endpoint.erase(ep_it);
> - }
> if (_this_node == &node) {
> _this_node = nullptr;
> }
> @@ -483,16 +430,6 @@ node* topology::find_node(host_id id) noexcept {
> return make_mutable(const_cast<const topology*>(this)->find_node(id));
> }
>
> -// Finds a node by its endpoint
> -// Returns nullptr if not found
> -const node* topology::find_node(const inet_address& ep) const noexcept {
> - auto it = _nodes_by_endpoint.find(ep);
> - if (it != _nodes_by_endpoint.end()) {
> - return &it->second.get();
> - }
> - return nullptr;
> -}
> -
> // Finds a node by its index
> // Returns nullptr if not found
> const node* topology::find_node(node::idx_type idx) const noexcept {
> @@ -502,23 +439,19 @@ const node* topology::find_node(node::idx_type idx) const noexcept {
> return _nodes.at(idx).get();
> }
>
> -const node& topology::add_or_update_endpoint(host_id id, std::optional<inet_address> opt_ep, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count)
> +const node& topology::add_or_update_endpoint(host_id id, std::optional<endpoint_dc_rack> opt_dr, std::optional<node::state> opt_st, std::optional<shard_id> shard_count)
> {
> - tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} ep={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this),
> - id, opt_ep, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count,
> + tlogger.trace("topology[{}]: add_or_update_endpoint: host_id={} dc={} rack={} state={} shards={}, at {}", fmt::ptr(this),
> + id, opt_dr.value_or(endpoint_dc_rack{}).dc, opt_dr.value_or(endpoint_dc_rack{}).rack, opt_st.value_or(node::state::none), shard_count,
> lazy_backtrace());
>
> auto* n = find_node(id);
> if (n) {
> - update_node(*n, std::nullopt, opt_ep, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
> - return *n;
> - } else if (opt_ep && (n = make_mutable(find_node(*opt_ep)))) {
> - update_node(*n, id, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
> + update_node(*n, std::nullopt, std::move(opt_dr), std::move(opt_st), std::move(shard_count));
> return *n;
> }
>
> return add_node(id,
> - opt_ep.value_or(inet_address{}),
> opt_dr.value_or(endpoint_dc_rack::default_location),
> opt_st.value_or(node::state::none),
> shard_count.value_or(0));
> diff --git a/main.cc b/main.cc
> index 9d394363ca3..263a41ee160 100644
> --- a/main.cc
> +++ b/main.cc
> @@ -1437,12 +1437,11 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
> const auto listen_address = utils::resolve(cfg->listen_address, family).get();
> const auto host_id = initialize_local_info_thread(sys_ks, snitch, listen_address, *cfg, broadcast_addr, broadcast_rpc_addr);
>
> - shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id, endpoint = broadcast_addr] (locator::token_metadata& tm) {
> + shared_token_metadata::mutate_on_all_shards(token_metadata, [host_id] (locator::token_metadata& tm) {
> // Makes local host id available in topology cfg as soon as possible.
> // Raft topology discard the endpoint-to-id map, so the local id can
> // still be found in the config.
> tm.get_topology().set_host_id_cfg(host_id);
> - tm.get_topology().add_or_update_endpoint(host_id, endpoint);
> return make_ready_future<>();
> }).get();
>
> diff --git a/test/boost/locator_topology_test.cc b/test/boost/locator_topology_test.cc
> index ccb321a11d7..f42dd743562 100644
> --- a/test/boost/locator_topology_test.cc
> +++ b/test/boost/locator_topology_test.cc
> @@ -32,9 +32,7 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) {
> auto id1 = host_id::create_random_id();
> auto ep1 = gms::inet_address("127.0.0.1");
> auto id2 = host_id::create_random_id();
> - auto ep2 = gms::inet_address("127.0.0.2");
> auto id3 = host_id::create_random_id();
> - auto ep3 = gms::inet_address("127.0.0.3");
>
> topology::config cfg = {
> .this_endpoint = ep1,
> @@ -51,16 +49,13 @@ SEASTAR_THREAD_TEST_CASE(test_add_node) {
>
> std::unordered_set<std::reference_wrapper<const locator::node>> nodes;
>
> - nodes.insert(std::cref(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal)));
> - nodes.insert(std::cref(topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal)));
> + nodes.insert(std::cref(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal)));
> + nodes.insert(std::cref(topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal)));
>
> - BOOST_REQUIRE_THROW(topo.add_node(id1, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
> - BOOST_REQUIRE_THROW(topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
> - BOOST_REQUIRE_THROW(topo.add_node(id2, ep2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
> - BOOST_REQUIRE_THROW(topo.add_node(id2, ep3, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
> - BOOST_REQUIRE_THROW(topo.add_node(id3, ep3, endpoint_dc_rack{}, node::state::normal), std::runtime_error);
> + BOOST_REQUIRE_THROW(topo.add_node(id2, endpoint_dc_rack::default_location, node::state::normal), std::runtime_error);
> + BOOST_REQUIRE_THROW(topo.add_node(id3, endpoint_dc_rack{}, node::state::normal), std::runtime_error);
>
> - nodes.insert(std::cref(topo.add_node(id3, ep3, endpoint_dc_rack::default_location, node::state::normal)));
> + nodes.insert(std::cref(topo.add_node(id3, endpoint_dc_rack::default_location, node::state::normal)));
>
> topo.for_each_node([&] (const locator::node& node) {
> BOOST_REQUIRE(nodes.erase(std::cref(node)));
> @@ -82,7 +77,7 @@ SEASTAR_THREAD_TEST_CASE(test_moving) {
>
> auto topo = topology(cfg);
>
> - topo.add_or_update_endpoint(id1, ep1, endpoint_dc_rack::default_location, node::state::normal);
> + topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal);
>
> BOOST_REQUIRE(topo.this_node()->topology() == &topo);
>
> @@ -101,8 +96,6 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
> auto id1 = host_id::create_random_id();
> auto ep1 = gms::inet_address("127.0.0.1");
> auto id2 = host_id::create_random_id();
> - auto ep2 = gms::inet_address("127.0.0.2");
> - auto ep3 = gms::inet_address("127.0.0.3");
>
> topology::config cfg = {
> .this_endpoint = ep1,
> @@ -117,93 +110,42 @@ SEASTAR_THREAD_TEST_CASE(test_update_node) {
> set_abort_on_internal_error(true);
> });
>
> - topo.add_or_update_endpoint(id1, std::nullopt, endpoint_dc_rack::default_location, node::state::normal);
> + topo.add_or_update_endpoint(id1, endpoint_dc_rack::default_location, node::state::normal);
>
> auto node = const_cast<class node*>(topo.this_node());
>
> - topo.update_node(*node, std::nullopt, ep1, std::nullopt, std::nullopt);
> + topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt);
>
> BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
>
> - BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt, std::nullopt), std::runtime_error);
> - BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt, std::nullopt), std::runtime_error);
> + BOOST_REQUIRE_THROW(topo.update_node(*node, host_id::create_null_id(), std::nullopt, std::nullopt), std::runtime_error);
> + BOOST_REQUIRE_THROW(topo.update_node(*node, id2, std::nullopt, std::nullopt), std::runtime_error);
> BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
> BOOST_REQUIRE_EQUAL(topo.find_node(id2), nullptr);
>
> - topo.update_node(*node, std::nullopt, ep2, std::nullopt, std::nullopt);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), node);
> -
> auto dc_rack1 = endpoint_dc_rack{"DC1", "RACK1"};
> - topo.update_node(*node, std::nullopt, std::nullopt, dc_rack1, std::nullopt);
> + topo.update_node(*node, std::nullopt, dc_rack1, std::nullopt);
>
> BOOST_REQUIRE(topo.get_location(id1) == dc_rack1);
>
> auto dc_rack2 = endpoint_dc_rack{"DC2", "RACK2"};
> - topo.update_node(*node, std::nullopt, std::nullopt, dc_rack2, std::nullopt);
> + topo.update_node(*node, std::nullopt, dc_rack2, std::nullopt);
>
> BOOST_REQUIRE(topo.get_location(id1) == dc_rack2);
>
> BOOST_REQUIRE_NE(node->get_state(), locator::node::state::being_decommissioned);
> - topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::being_decommissioned);
> + topo.update_node(*node, std::nullopt, std::nullopt, locator::node::state::being_decommissioned);
>
> BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);
>
> auto dc_rack3 = endpoint_dc_rack{"DC3", "RACK3"};
> // Note: engage state option, but keep node::state value the same
> // to reproduce #13502
> - topo.update_node(*node, std::nullopt, ep3, dc_rack3, locator::node::state::being_decommissioned);
> + topo.update_node(*node, std::nullopt, dc_rack3, locator::node::state::being_decommissioned);
>
> BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep3), node);
> BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
> BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::being_decommissioned);
> -
> - // In state::left the node will remain indexed only by its host_id
> - topo.update_node(*node, std::nullopt, std::nullopt, std::nullopt, locator::node::state::left);
> - BOOST_REQUIRE_EQUAL(topo.find_node(id1), node);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep1), nullptr);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep2), nullptr);
> - BOOST_REQUIRE_EQUAL(topo.find_node(ep3), nullptr);
> - BOOST_REQUIRE(topo.get_location(id1) == dc_rack3);
> - BOOST_REQUIRE_EQUAL(node->get_state(), locator::node::state::left);
> -}
> -
> -SEASTAR_THREAD_TEST_CASE(test_add_or_update_by_host_id) {
> - auto id1 = host_id::create_random_id();
> - auto id2 = host_id::create_random_id();
> - auto ep1 = gms::inet_address("127.0.0.1");
> -
> - // In this test we check that add_or_update_endpoint searches by host_id first.
> - // We create two nodes, one matches by id, another - by ip,
> - // and SCYLLA_ASSERT that add_or_update_endpoint updates the first.
> - // We need to make the second node 'being_decommissioned', so that
> - // it gets removed from ip index and we don't get the non-unique IP error.
> -
> - auto topo = topology({
> - .this_host_id = id1,
> - .local_dc_rack = endpoint_dc_rack::default_location,
> - });
> -
> - topo.add_or_update_endpoint(id1, gms::inet_address{}, endpoint_dc_rack::default_location, node::state::normal);
> - topo.add_node(id2, ep1, endpoint_dc_rack::default_location, node::state::being_decommissioned);
> -
> - topo.add_or_update_endpoint(id1, ep1, std::nullopt, node::state::bootstrapping);
> -
> - auto* n = topo.find_node(id1);
> - BOOST_REQUIRE_EQUAL(n->get_state(), node::state::bootstrapping);
> - BOOST_REQUIRE_EQUAL(n->host_id(), id1);
> - BOOST_REQUIRE_EQUAL(n->endpoint(), ep1);
> -
> - auto* n2 = topo.find_node(ep1);
> - BOOST_REQUIRE_EQUAL(n, n2);
> -
> - auto* n3 = topo.find_node(id2);
> - BOOST_REQUIRE_EQUAL(n3->get_state(), node::state::being_decommissioned);
> - BOOST_REQUIRE_EQUAL(n3->host_id(), id2);
> - BOOST_REQUIRE_EQUAL(n3->endpoint(), ep1);
> }
>
> SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
> @@ -214,7 +156,6 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
> const auto id1 = host_id::create_random_id();
> const auto ep1 = gms::inet_address("127.0.0.1");
> const auto id2 = host_id::create_random_id();
> - const auto ep2 = gms::inet_address("127.0.0.2");
> const auto dc_rack1 = endpoint_dc_rack {
> .dc = "dc1",
> .rack = "rack1"
> @@ -232,8 +173,8 @@ SEASTAR_THREAD_TEST_CASE(test_remove_endpoint) {
>
> auto topo = topology(cfg);
>
> - topo.add_or_update_endpoint(id1, ep1, dc_rack1, node::state::normal);
> - topo.add_node(id2, ep2, dc_rack2, node::state::normal);
> + topo.add_or_update_endpoint(id1, dc_rack1, node::state::normal);
> + topo.add_node(id2, dc_rack2, node::state::normal);
>
> BOOST_REQUIRE_EQUAL(topo.get_datacenter_endpoints(), (dc_endpoints_t{{"dc1", {id1, id2}}}));
> BOOST_REQUIRE_EQUAL(topo.get_datacenter_racks(), (dc_racks_t{{"dc1", {{"rack1", {id1}}, {"rack2", {id2}}}}}));
> @@ -374,9 +315,7 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) {
> auto id1 = host_id::create_random_id();
> auto ep1 = gms::inet_address("127.0.0.1");
> auto id2 = host_id::create_random_id();
> - auto ep2 = gms::inet_address("127.0.0.2");
> auto id3 = host_id::create_random_id();
> - auto ep3 = gms::inet_address("127.0.0.3");
>
> const auto dc_rack1 = endpoint_dc_rack {
> .dc = "dc1",
> @@ -397,8 +336,8 @@ SEASTAR_THREAD_TEST_CASE(test_left_node_is_kept_outside_dc) {
>
> std::unordered_set<std::reference_wrapper<const locator::node>> nodes;
>
> - nodes.insert(std::cref(topo.add_node(id2, ep2, dc_rack1, node::state::normal)));
> - nodes.insert(std::cref(topo.add_node(id3, ep3, dc_rack1, node::state::left)));
> + nodes.insert(std::cref(topo.add_node(id2, dc_rack1, node::state::normal)));
> + nodes.insert(std::cref(topo.add_node(id3, dc_rack1, node::state::left)));
>
> topo.for_each_node([&] (const locator::node& node) {
> BOOST_REQUIRE(node.host_id() != id3);
> diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc
> index 142e0e4f598..a175df290f7 100644
> --- a/test/boost/network_topology_strategy_test.cc
> +++ b/test/boost/network_topology_strategy_test.cc
> @@ -303,7 +303,7 @@ void simple_test() {
> for (const auto& [ring_point, endpoint, id] : ring_points) {
> std::unordered_set<token> tokens;
> tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
> - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
> + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
> co_await tm.update_normal_tokens(std::move(tokens), id);
> }
> }).get();
> @@ -411,7 +411,7 @@ void heavy_origin_test() {
> stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
> auto& topo = tm.get_topology();
> for (const auto& [ring_point, endpoint, id] : ring_points) {
> - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
> + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal);
> co_await tm.update_normal_tokens(tokens[endpoint], id);
> }
> }).get();
> @@ -483,7 +483,7 @@ SEASTAR_THREAD_TEST_CASE(NetworkTopologyStrategy_tablets_test) {
> for (const auto& [ring_point, endpoint, id] : ring_points) {
> std::unordered_set<token> tokens;
> tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
> - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
> + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
> tm.update_host_id(id, endpoint);
> co_await tm.update_normal_tokens(std::move(tokens), id);
> }
> @@ -575,7 +575,7 @@ static void test_random_balancing(sharded<snitch_ptr>& snitch, gms::inet_address
> for (const auto& [ring_point, endpoint, id] : ring_points) {
> std::unordered_set<token> tokens;
> tokens.insert(token{tests::d2t(ring_point / ring_points.size())});
> - topo.add_node(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
> + topo.add_node(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, shard_count);
> tm.update_host_id(id, endpoint);
> co_await tm.update_normal_tokens(std::move(tokens), id);
> }
> @@ -864,12 +864,11 @@ void generate_topology(topology& topo, const std::unordered_map<sstring, size_t>
> out = std::fill_n(out, rf, std::cref(dc));
> }
>
> - unsigned i = 0;
> for (auto& node : nodes) {
> const sstring& dc = dcs[udist(0, dcs.size() - 1)(e1)];
> auto rc = racks_per_dc.at(dc);
> auto r = udist(0, rc)(e1);
> - topo.add_or_update_endpoint(node, inet_address((127u << 24) | ++i), endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal);
> + topo.add_or_update_endpoint(node, endpoint_dc_rack{dc, to_sstring(r)}, locator::node::state::normal);
> }
> }
>
> @@ -1144,21 +1143,15 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
>
> const node* n1 = stm.get()->get_topology().find_node(host1);
> BOOST_REQUIRE(n1);
> - n1 = stm.get()->get_topology().find_node(ip1);
> - BOOST_REQUIRE(n1);
> BOOST_REQUIRE(bool(n1->is_this_node()));
> BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
> - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
> BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack);
> BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack);
>
> const node* n2 = stm.get()->get_topology().find_node(host2);
> BOOST_REQUIRE(n2);
> - n2 = stm.get()->get_topology().find_node(ip2);
> - BOOST_REQUIRE(n2);
> BOOST_REQUIRE(!bool(n2->is_this_node()));
> BOOST_REQUIRE_EQUAL(n2->host_id(), host2);
> - BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2);
> BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location);
>
> // Local node cannot be removed
> @@ -1171,8 +1164,6 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
>
> n1 = stm.get()->get_topology().find_node(host1);
> BOOST_REQUIRE(n1);
> - n1 = stm.get()->get_topology().find_node(ip1);
> - BOOST_REQUIRE(n1);
>
> // Removing node with no local node
>
> @@ -1183,22 +1174,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
>
> n2 = stm.get()->get_topology().find_node(host2);
> BOOST_REQUIRE(!n2);
> - n2 = stm.get()->get_topology().find_node(ip2);
> - BOOST_REQUIRE(!n2);
>
> // Repopulate after clear_gently()
>
> stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
> co_await tm.clear_gently();
> - tm.update_host_id(host2, ip2);
> - tm.update_host_id(host1, ip1); // this_node added last on purpose
> + tm.update_topology(host2, std::nullopt, std::nullopt);
> + tm.update_topology(host1, std::nullopt, std::nullopt); // this_node added last on purpose
> }).get();
>
> n1 = stm.get()->get_topology().find_node(host1);
> BOOST_REQUIRE(n1);
> BOOST_REQUIRE(bool(n1->is_this_node()));
> BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
> - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
> BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack);
> BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack);
>
> @@ -1206,21 +1194,19 @@ SEASTAR_THREAD_TEST_CASE(test_topology_tracks_local_node) {
> BOOST_REQUIRE(n2);
> BOOST_REQUIRE(!bool(n2->is_this_node()));
> BOOST_REQUIRE_EQUAL(n2->host_id(), host2);
> - BOOST_REQUIRE_EQUAL(n2->endpoint(), ip2);
> BOOST_REQUIRE(n2->dc_rack() == endpoint_dc_rack::default_location);
>
> // get_location() should pick up endpoint_dc_rack from node info
>
> stm.mutate_token_metadata([&] (token_metadata& tm) -> future<> {
> co_await tm.clear_gently();
> - tm.get_topology().add_or_update_endpoint(host1, ip1, ip1_dc_rack_v2, node::state::being_decommissioned);
> + tm.get_topology().add_or_update_endpoint(host1, ip1_dc_rack_v2, node::state::being_decommissioned);
> }).get();
>
> n1 = stm.get()->get_topology().find_node(host1);
> BOOST_REQUIRE(n1);
> BOOST_REQUIRE(bool(n1->is_this_node()));
> BOOST_REQUIRE_EQUAL(n1->host_id(), host1);
> - BOOST_REQUIRE_EQUAL(n1->endpoint(), ip1);
> BOOST_REQUIRE(n1->dc_rack() == ip1_dc_rack_v2);
> BOOST_REQUIRE(stm.get()->get_topology().get_location() == ip1_dc_rack_v2);
> }
> diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc
> index 07ceaf04cac..e3e1d7c4d61 100644
> --- a/test/boost/tablets_test.cc
> +++ b/test/boost/tablets_test.cc
> @@ -1019,7 +1019,7 @@ SEASTAR_TEST_CASE(test_sharder) {
> auto table1 = table_id(utils::UUID_gen::get_time_UUID());
>
> token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } });
> - tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address());
> + tokm.get_topology().add_or_update_endpoint(h1);
>
> std::vector<tablet_id> tablet_ids;
> {
> @@ -1234,7 +1234,7 @@ SEASTAR_TEST_CASE(test_intranode_sharding) {
> auto table1 = table_id(utils::UUID_gen::get_time_UUID());
>
> token_metadata tokm(token_metadata::config{ .topo_cfg{ .this_host_id = h1, .local_dc_rack = locator::endpoint_dc_rack::default_location } });
> - tokm.get_topology().add_or_update_endpoint(h1, tokm.get_topology().my_address());
> + tokm.get_topology().add_or_update_endpoint(h1);
>
> auto leaving_replica = tablet_replica{h1, 5};
> auto pending_replica = tablet_replica{h1, 7};
> @@ -3339,7 +3339,7 @@ static void execute_tablet_for_new_rf_test(calculate_tablet_replicas_for_new_rf_
> for (const auto& [ring_point, endpoint, id] : test_config.ring_points) {
> std::unordered_set<token> tokens;
> tokens.insert(dht::token{tests::d2t(ring_point / test_config.ring_points.size())});
> - topo.add_or_update_endpoint(id, endpoint, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1);
> + topo.add_or_update_endpoint(id, make_endpoint_dc_rack(endpoint), locator::node::state::normal, 1);
> tm.update_host_id(id, endpoint);
> co_await tm.update_normal_tokens(std::move(tokens), id);
> }
> diff --git a/test/lib/cql_test_env.cc b/test/lib/cql_test_env.cc
> index b73cf303438..f44423bc1e2 100644
> --- a/test/lib/cql_test_env.cc
> +++ b/test/lib/cql_test_env.cc
> @@ -685,11 +685,10 @@ class single_node_cql_env : public cql_test_env {
> host_id = linfo.host_id;
> _sys_ks.local().save_local_info(std::move(linfo), _snitch.local()->get_location(), my_address, my_address).get();
> }
> - locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id, &cfg_in] (locator::token_metadata& tm) {
> + locator::shared_token_metadata::mutate_on_all_shards(_token_metadata, [hostid = host_id] (locator::token_metadata& tm) {
> auto& topo = tm.get_topology();
> topo.set_host_id_cfg(hostid);
> topo.add_or_update_endpoint(hostid,
> - cfg_in.broadcast_address,
> std::nullopt,
> locator::node::state::normal,
> smp::count);
> diff --git a/test/perf/perf_sort_by_proximity.cc b/test/perf/perf_sort_by_proximity.cc
> index 4d86bcea1df..438bf19ebd2 100644
> --- a/test/perf/perf_sort_by_proximity.cc
> +++ b/test/perf/perf_sort_by_proximity.cc
> @@ -53,7 +53,6 @@ struct sort_by_proximity_topology {
> auto id = locator::host_id{utils::UUID(0, i)};
> nodes[dc][rack].emplace_back(id);
> topology.add_or_update_endpoint(id,
> - gms::inet_address((127u << 24) | i),
> locator::endpoint_dc_rack{format("dc{}", dc), format("rack{}", rack)},
> locator::node::state::normal);
> }

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 8:59:25 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
If you haven't, please run full dtest suite in all 3 modes (no raft,
only raft for schema, topology on raft) on this series

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:05:43 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
It cannot be. We are handling change state here with an id provided by
the gossiper.

> > + }
> > +
> > if (existing && *existing != endpoint) {
> > // This branch in taken when a node changes its IP address.
> > @@ -2412,11 +2427,12 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
> > // We do this after update_normal_tokens, allowing for tokens to be properly
> > // migrated to the new host_id.
> > - slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
> > - if (const auto old_host_id = tmptr->get_host_id_if_known(endpoint); old_host_id && *old_host_id != host_id) {
> > - // Replace with same IP scenario
> > - slogger.info("The IP {} previously owned host ID {}", endpoint, *old_host_id);
> > - replaced_id = *old_host_id;
> > + auto peers = co_await _sys_ks.local().load_host_ids();
> > + if (peers.contains(endpoint) && peers[endpoint] != host_id) {
> > + replaced_id = peers[endpoint];
> > + slogger.info("The IP {} previously owned host ID {}", endpoint, *replaced_id);
> > + } else {
> > + slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
> > }
> > } else {
> > // This branch is taken if this node wasn't involved in node_ops
>

--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:06:20 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
Looks like it indeed. That's for another cleanup.
--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:07:12 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
On Tue, Jan 14, 2025 at 02:57:26PM +0100, Kamil Braun wrote:
> what is this, some leftover from Cassandra?
>
Yes. Cassandra uses (at least in the past) this instead of features.
--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:25:47 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
May be should move to a separate patch. All other get_datacenter callers
converted naturally.
First of all we want to send by host id, not ip so this is either that
or convert from ips to ids to send,
>
> TBH I don't see the motivation for changing get_live_members(), it would
> make sense if we stored _live_members in gossiper using host IDs, without
> that you introduce more translations which looks undesirable

And second we do want to store _live_members by id eventually. It will
simplify gossiper::is_alive(host_id) as well.

> > return results;
> > }).then([this] (auto results) {
> > // we're done: the results map is ready to return to the client. the rest is just debug logging:
>

--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:42:08 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
Looks like leftover before rework. Wonder why compiler did not fail on
unused variable.

>
> > + std::optional<gms::inet_address> prev_ip = co_await _ss.get_ip_from_peers_table(id);
> > if (prev_ip == endpoint) {
> > co_return;
> > }
> > @@ -2639,6 +2641,14 @@ future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_sta
> > }
> > }
> > +future<std::optional<gms::inet_address>> storage_service::get_ip_from_peers_table(locator::host_id id) {
> > + auto peers = co_await _sys_ks.local().load_host_ids();
> > + if (auto it = std::ranges::find_if(peers, [&id] (const auto& e) { return e.second == id; }); it != peers.end()) {
> > + co_return it->first;
> > + }
> > + co_return std::nullopt;
> > +}
> > +
> > future<> storage_service::on_change(gms::inet_address endpoint, const gms::application_state_map& states_, gms::permit_id pid) {
> > // copy the states map locally since the coroutine may yield
> > auto states = states_;
>

--
Gleb.

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:43:58 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
On Tue, Jan 14, 2025 at 02:59:21PM +0100, Kamil Braun wrote:
> If you haven't, please run full dtest suite in all 3 modes (no raft, only
> raft for schema, topology on raft) on this series
>
Do we support only gossiper in this version at all? Although do we pass
all the test there?
--
Gleb.

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 9:47:25 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
On Tue, Jan 14, 2025 at 3:43 PM Gleb Natapov <gl...@scylladb.com> wrote:
On Tue, Jan 14, 2025 at 02:59:21PM +0100, Kamil Braun wrote:
> If you haven't, please run full dtest suite in all 3 modes (no raft, only
> raft for schema, topology on raft) on this series
>
Do we support only gossiper in this version at all? Although do we pass
all the test there?
Right sorry, we don't support pure gossiper. But schema on raft only should be tested still, since there is no mechanism to force user to upgrade to raft topology.

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 9:48:02 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
So I actually meant the following 3 modes:

- schema on raft
- topology on raft but no tablets
- topology on raft + tablets

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 9:48:24 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
(this is the set of 3 modes we're running on nightly dtest full runs)

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 14, 2025, 9:53:06 AMJan 14
to Kamil Braun, scylladb-dev@googlegroups.com
On Tue, Jan 14, 2025 at 03:47:49PM +0100, Kamil Braun wrote:
> So I actually meant the following 3 modes:
>
> - schema on raft
> - topology on raft but no tablets
> - topology on raft + tablets
>
I do not see why we need topology on raft with tablets. There is
(almost) nothing here about tablets and they can only improve the
situation.
--
Gleb.

Kamil Braun

<kbraun@scylladb.com>
unread,
Jan 14, 2025, 9:55:56 AMJan 14
to Gleb Natapov, scylladb-dev@googlegroups.com
Better be safe than sorry. You can run all them concurrently anyway, so it's not like you lose more time waiting

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 15, 2025, 9:24:42 AMJan 15
to Kamil Braun, scylladb-dev@googlegroups.com
On Tue, Jan 14, 2025 at 03:47:49PM +0100, Kamil Braun wrote:
> So I actually meant the following 3 modes:
>
> - schema on raft
https://jenkins.scylladb.com/job/scylla-master/job/byo/job/byo_build_tests_dtest/2682/#showFailuresLink

> - topology on raft but no tablets
https://jenkins.scylladb.com/job/scylla-master/job/byo/job/byo_build_tests_dtest/2681/#showFailuresLink

> - topology on raft + tablets
https://jenkins.scylladb.com/job/scylla-master/job/byo/job/byo_build_tests_dtest/2680/#showFailuresLink

There are failures, but all of them known:

test_schema_and_data_on_mixed_versions_cluster: https://github.com/scylladb/scylladb/issues/22183
test_add_remove_node: https://github.com/scylladb/scylladb/issues/22323
paging_test.TestPagingData.*: https://github.com/scylladb/scylladb/issues/22267
test_add_many_nodes_under_load: https://github.com/scylladb/scylladb/issues/22244
test_mv_resurrected_rows_after_decommission_interrupt: https://github.com/scylladb/scylladb/issues/21534
test_mv_populating_from_existing_data_during_node_decommission: https://github.com/scylladb/scylladb/issues/17038
--
Gleb.

Konstantin Osipov

<kostja@scylladb.com>
unread,
Jan 15, 2025, 3:27:45 PMJan 15
to Gleb Natapov, scylladb-dev@googlegroups.com
* 'Gleb Natapov' via ScyllaDB development <scylla...@googlegroups.com> [25/01/13 13:26]:

No longer unused -> no longer used

> ---
> locator/token_metadata.hh | 7 -------
> locator/token_metadata.cc | 32 --------------------------------
> 2 files changed, 39 deletions(-)
>
> diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
> index b8420761242..4ac6a2143b4 100644
> --- a/locator/token_metadata.hh
> +++ b/locator/token_metadata.hh
> @@ -239,13 +239,6 @@ class token_metadata final {
> /** Return the unique host ID for an end-point. */
> host_id get_host_id(inet_address endpoint) const;
>
> - /// Return the unique host ID for an end-point or nullopt if not found.
> - std::optional<host_id> get_host_id_if_known(inet_address endpoint) const;
> -
> - /** Return the end-point for a unique host ID or nullopt if not found. */
> - std::optional<inet_address> get_endpoint_for_host_id_if_known(locator::host_id host_id) const;
> -
> - /** Return the end-point for a unique host ID */
> inet_address get_endpoint_for_host_id(locator::host_id host_id) const;
>
> /** @return a copy of the endpoint-to-id map for read-only operations */
> diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
> index 75603c58e6a..3320e37ed14 100644
> --- a/locator/token_metadata.cc
> +++ b/locator/token_metadata.cc
> @@ -153,12 +153,6 @@ class token_metadata_impl final {
> /** Return the unique host ID for an end-point. */
> host_id get_host_id(inet_address endpoint) const;
>
> - /// Return the unique host ID for an end-point or nullopt if not found.
> - std::optional<host_id> get_host_id_if_known(inet_address endpoint) const;
> -
> - /** Return the end-point for a unique host ID or nullopt if not found.*/
> - std::optional<inet_address> get_endpoint_for_host_id_if_known(host_id) const;
> -
> /** Return the end-point for a unique host ID.*/
> inet_address get_endpoint_for_host_id(host_id) const;
>
> @@ -543,22 +537,6 @@ host_id token_metadata_impl::get_host_id(inet_address endpoint) const {
> }
> }
>
> -std::optional<host_id> token_metadata_impl::get_host_id_if_known(inet_address endpoint) const {
> - if (const auto* node = _topology.find_node(endpoint)) [[likely]] {
> - return node->host_id();
> - } else {
> - return std::nullopt;
> - }
> -}
> -
> -std::optional<inet_address> token_metadata_impl::get_endpoint_for_host_id_if_known(host_id host_id) const {
> - if (const auto* node = _topology.find_node(host_id)) [[likely]] {
> - return node->endpoint();
> - } else {
> - return std::nullopt;
> - }
> -}
> -
> inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) const {
> if (const auto* node = _topology.find_node(host_id)) [[likely]] {
> return node->endpoint();
> @@ -1042,16 +1020,6 @@ token_metadata::get_host_id(inet_address endpoint) const {
> return _impl->get_host_id(endpoint);
> }
>
> -std::optional<host_id>
> -token_metadata::get_host_id_if_known(inet_address endpoint) const {
> - return _impl->get_host_id_if_known(endpoint);
> -}
> -
> -std::optional<token_metadata::inet_address>
> -token_metadata::get_endpoint_for_host_id_if_known(host_id host_id) const {
> - return _impl->get_endpoint_for_host_id_if_known(host_id);
> -}
> -
> token_metadata::inet_address
> token_metadata::get_endpoint_for_host_id(host_id host_id) const {
> return _impl->get_endpoint_for_host_id(host_id);
> --
> 2.47.1
>
> --
> You received this message because you are subscribed to the Google Groups "ScyllaDB development" group.
> To unsubscribe from this group and stop receiving emails from it, send an email to scylladb-dev...@googlegroups.com.
> To view this discussion visit https://groups.google.com/d/msgid/scylladb-dev/20250113082037.577242-38-gleb%40scylladb.com.

--
Konstantin Osipov, Moscow, Russia

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:29 AMJan 16
to scylladb-dev@googlegroups.com
One of its caller is in the RESTful API which gets ips from the user, so
we convert ips to ids inside the API handler using gossiper before
calling the function. We need to deprecate ip based API and move to host
id based.
---
api/api_init.hh | 2 +-
api/hinted_handoff.hh | 3 ++-
db/hints/manager.hh | 2 +-
service/storage_proxy.hh | 2 +-
api/api.cc | 6 +++---
api/hinted_handoff.cc | 13 +++++++------
db/hints/manager.cc | 13 +++----------
main.cc | 2 +-
repair/row_level.cc | 2 +-
service/storage_proxy.cc | 2 +-
10 files changed, 21 insertions(+), 26 deletions(-)

diff --git a/api/api_init.hh b/api/api_init.hh
index 9cd9193645e..205090c1cb4 100644
--- a/api/api_init.hh
+++ b/api/api_init.hh
@@ -124,7 +124,7 @@ future<> set_server_storage_proxy(http_context& ctx, sharded<service::storage_pr
future<> unset_server_storage_proxy(http_context& ctx);
future<> set_server_stream_manager(http_context& ctx, sharded<streaming::stream_manager>& sm);
future<> unset_server_stream_manager(http_context& ctx);
-future<> set_hinted_handoff(http_context& ctx, sharded<service::storage_proxy>& p);
+future<> set_hinted_handoff(http_context& ctx, sharded<service::storage_proxy>& p, sharded<gms::gossiper>& g);
future<> unset_hinted_handoff(http_context& ctx);
future<> set_server_cache(http_context& ctx);
future<> unset_server_cache(http_context& ctx);
diff --git a/api/hinted_handoff.hh b/api/hinted_handoff.hh
index 7a83daa30d5..13b9f5acb65 100644
--- a/api/hinted_handoff.hh
+++ b/api/hinted_handoff.hh
@@ -10,12 +10,13 @@

#include <seastar/core/sharded.hh>
#include "api/api_init.hh"
+#include "gms/gossiper.hh"

namespace service { class storage_proxy; }

namespace api {

-void set_hinted_handoff(http_context& ctx, httpd::routes& r, sharded<service::storage_proxy>& p);
+void set_hinted_handoff(http_context& ctx, httpd::routes& r, sharded<service::storage_proxy>& p, sharded<gms::gossiper>& g);
void unset_hinted_handoff(http_context& ctx, httpd::routes& r);

}
diff --git a/db/hints/manager.hh b/db/hints/manager.hh
index 69d18199ed5..afe7e96648f 100644
--- a/db/hints/manager.hh
+++ b/db/hints/manager.hh
@@ -278,7 +278,7 @@ class manager {
///
/// \param target_eps The list of endpoints the sync point should correspond to. When empty, the function assumes all endpoints.
/// \return Sync point corresponding to the specified endpoints.
- sync_point::shard_rps calculate_current_sync_point(std::span<const gms::inet_address> target_eps) const;
+ sync_point::shard_rps calculate_current_sync_point(std::span<const locator::host_id> target_eps) const;

/// \brief Waits until hint replay reach replay positions described in `rps`.
future<> wait_for_sync_point(abort_source& as, const sync_point::shard_rps& rps);
diff --git a/service/storage_proxy.hh b/service/storage_proxy.hh
index 85756c3a093..6c1b0812df5 100644
--- a/service/storage_proxy.hh
+++ b/service/storage_proxy.hh
@@ -710,7 +710,7 @@ class storage_proxy : public seastar::async_sharded_service<storage_proxy>, publ
future<> change_hints_host_filter(db::hints::host_filter new_filter);
const db::hints::host_filter& get_hints_host_filter() const;

- future<db::hints::sync_point> create_hint_sync_point(std::vector<gms::inet_address> target_hosts) const;
+ future<db::hints::sync_point> create_hint_sync_point(std::vector<locator::host_id> target_hosts) const;
future<> wait_for_hint_sync_point(const db::hints::sync_point spoint, clock_type::time_point deadline);

const stats& get_stats() const {
diff --git a/api/api.cc b/api/api.cc
index 1753153f87d..670750e6843 100644
--- a/api/api.cc
+++ b/api/api.cc
@@ -273,10 +273,10 @@ future<> unset_server_cache(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_cache_service(ctx, r); });
}

-future<> set_hinted_handoff(http_context& ctx, sharded<service::storage_proxy>& proxy) {
+future<> set_hinted_handoff(http_context& ctx, sharded<service::storage_proxy>& proxy, sharded<gms::gossiper>& g) {
return register_api(ctx, "hinted_handoff",
- "The hinted handoff API", [&proxy] (http_context& ctx, routes& r) {
- set_hinted_handoff(ctx, r, proxy);
+ "The hinted handoff API", [&proxy, &g] (http_context& ctx, routes& r) {
+ set_hinted_handoff(ctx, r, proxy, g);
});
}

diff --git a/api/hinted_handoff.cc b/api/hinted_handoff.cc
index 73cd41bafff..00fc513bf21 100644
--- a/api/hinted_handoff.cc
+++ b/api/hinted_handoff.cc
@@ -14,6 +14,7 @@

#include "gms/inet_address.hh"
#include "service/storage_proxy.hh"
+#include "gms/gossiper.hh"

namespace api {

@@ -21,18 +22,18 @@ using namespace json;
using namespace seastar::httpd;
namespace hh = httpd::hinted_handoff_json;

-void set_hinted_handoff(http_context& ctx, routes& r, sharded<service::storage_proxy>& proxy) {
- hh::create_hints_sync_point.set(r, [&proxy] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
- auto parse_hosts_list = [] (sstring arg) {
+void set_hinted_handoff(http_context& ctx, routes& r, sharded<service::storage_proxy>& proxy, sharded<gms::gossiper>& g) {
+ hh::create_hints_sync_point.set(r, [&proxy, &g] (std::unique_ptr<http::request> req) -> future<json::json_return_type> {
+ auto parse_hosts_list = [&g] (sstring arg) {
std::vector<sstring> hosts_str = split(arg, ",");
- std::vector<gms::inet_address> hosts;
+ std::vector<locator::host_id> hosts;
hosts.reserve(hosts_str.size());

for (const auto& host_str : hosts_str) {
try {
gms::inet_address host;
host = gms::inet_address(host_str);
- hosts.push_back(host);
+ hosts.push_back(g.local().get_host_id(host));
} catch (std::exception& e) {
throw httpd::bad_param_exception(format("Failed to parse host address {}: {}", host_str, e.what()));
}
@@ -41,7 +42,7 @@ void set_hinted_handoff(http_context& ctx, routes& r, sharded<service::storage_p
return hosts;
};

- std::vector<gms::inet_address> target_hosts = parse_hosts_list(req->get_query_param("target_hosts"));
+ std::vector<locator::host_id> target_hosts = parse_hosts_list(req->get_query_param("target_hosts"));
return proxy.local().create_hint_sync_point(std::move(target_hosts)).then([] (db::hints::sync_point sync_point) {
return json::json_return_type(sync_point.encode());
});
diff --git a/db/hints/manager.cc b/db/hints/manager.cc
index 95d5ea0f2a0..17332a0f52d 100644
--- a/db/hints/manager.cc
+++ b/db/hints/manager.cc
@@ -266,21 +266,14 @@ void manager::forbid_hints_for_eps_with_pending_hints() {
}
}

-sync_point::shard_rps manager::calculate_current_sync_point(std::span<const gms::inet_address> target_eps) const {
+sync_point::shard_rps manager::calculate_current_sync_point(std::span<const locator::host_id> target_eps) const {
sync_point::shard_rps rps;
- const auto tmptr = _proxy.get_token_metadata_ptr();

for (auto addr : target_eps) {
- const auto hid = tmptr->get_host_id_if_known(addr);
- // Ignore the IPs that we cannot map.
- if (!hid) {
- continue;
- }
-
- auto it = _ep_managers.find(*hid);
+ auto it = _ep_managers.find(addr);
if (it != _ep_managers.end()) {
const hint_endpoint_manager& ep_man = it->second;
- rps[*hid] = ep_man.last_written_replay_position();
+ rps[addr] = ep_man.last_written_replay_position();
}
}

diff --git a/main.cc b/main.cc
index 6a7a3b66d06..8ccb6d846c2 100644
--- a/main.cc
+++ b/main.cc
@@ -2262,7 +2262,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
supervisor::notify("allow replaying hints");
proxy.invoke_on_all(&service::storage_proxy::allow_replaying_hints).get();

- api::set_hinted_handoff(ctx, proxy).get();
+ api::set_hinted_handoff(ctx, proxy, gossiper).get();
auto stop_hinted_handoff_api = defer_verbose_shutdown("hinted handoff API", [&ctx] {
api::unset_hinted_handoff(ctx).get();
});
diff --git a/repair/row_level.cc b/repair/row_level.cc
index ac05787d91e..ca70ae15f18 100644
--- a/repair/row_level.cc
+++ b/repair/row_level.cc
@@ -2317,7 +2317,7 @@ future<repair_flush_hints_batchlog_response> repair_service::repair_flush_hints_
auto flush_time = now;
if (cache_disabled || (now - _flush_hints_batchlog_time > cache_time)) {
// Empty targets meants all nodes
- db::hints::sync_point sync_point = co_await _sp.local().create_hint_sync_point(std::vector<gms::inet_address>{});
+ db::hints::sync_point sync_point = co_await _sp.local().create_hint_sync_point(std::vector<locator::host_id>{});
lowres_clock::time_point deadline = lowres_clock::now() + req.hints_timeout;
try {
bool bm_throw = utils::get_local_injector().enter("repair_flush_hints_batchlog_handler_bm_uninitialized");
diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index da28f1059dd..818de6f8e6c 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -6795,7 +6795,7 @@ const db::hints::host_filter& storage_proxy::get_hints_host_filter() const {
return _hints_manager.get_host_filter();
}

-future<db::hints::sync_point> storage_proxy::create_hint_sync_point(std::vector<gms::inet_address> target_hosts) const {
+future<db::hints::sync_point> storage_proxy::create_hint_sync_point(std::vector<locator::host_id> target_hosts) const {
db::hints::sync_point spoint;
spoint.regular_per_shard_rps.resize(smp::count);
spoint.mv_per_shard_rps.resize(smp::count);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:30 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
db/hints/manager.cc | 9 +++++----
1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/db/hints/manager.cc b/db/hints/manager.cc
index 17332a0f52d..7cdb515352b 100644
--- a/db/hints/manager.cc
+++ b/db/hints/manager.cc
@@ -335,10 +335,11 @@ future<> manager::wait_for_sync_point(abort_source& as, const sync_point::shard_

for (const auto& [addr, rp] : rps) {
if (std::holds_alternative<gms::inet_address>(addr)) {
- const auto maybe_hid = tmptr->get_host_id_if_known(std::get<gms::inet_address>(addr));
- // Ignore the IPs we cannot map.
- if (maybe_hid) [[likely]] {
- hid_rps.emplace(*maybe_hid, rp);
+ try {
+ const auto hid = _gossiper_anchor->get_host_id(std::get<gms::inet_address>(addr));
+ hid_rps.emplace(hid, rp);
+ } catch (...) {
+ // Ignore the IPs we cannot map.
}
} else {
hid_rps.emplace(std::get<locator::host_id>(addr), rp);
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:30 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ip knowledge from the token_metadata, so use gossiper to
retrieve the mapping instead.
---
api/api_init.hh | 2 +-
api/token_metadata.hh | 3 ++-
locator/token_metadata.hh | 2 +-
api/api.cc | 4 ++--
api/token_metadata.cc | 45 ++++++++++++++++++++++++++-------------
locator/token_metadata.cc | 30 +++++++++-----------------
main.cc | 2 +-
7 files changed, 47 insertions(+), 41 deletions(-)

diff --git a/api/api_init.hh b/api/api_init.hh
index 7c74f69ba93..9cd9193645e 100644
--- a/api/api_init.hh
+++ b/api/api_init.hh
@@ -112,7 +112,7 @@ future<> set_server_authorization_cache(http_context& ctx, sharded<auth::service
future<> unset_server_authorization_cache(http_context& ctx);
future<> set_server_snapshot(http_context& ctx, sharded<db::snapshot_ctl>& snap_ctl);
future<> unset_server_snapshot(http_context& ctx);
-future<> set_server_token_metadata(http_context& ctx, sharded<locator::shared_token_metadata>& tm);
+future<> set_server_token_metadata(http_context& ctx, sharded<locator::shared_token_metadata>& tm, sharded<gms::gossiper>& g);
future<> unset_server_token_metadata(http_context& ctx);
future<> set_server_gossip(http_context& ctx, sharded<gms::gossiper>& g);
future<> unset_server_gossip(http_context& ctx);
diff --git a/api/token_metadata.hh b/api/token_metadata.hh
index 0bab6d999fd..3e804050fc0 100644
--- a/api/token_metadata.hh
+++ b/api/token_metadata.hh
@@ -15,10 +15,11 @@ class routes;
}

namespace locator { class shared_token_metadata; }
+namespace gms { class gossiper; }

namespace api {
struct http_context;
-void set_token_metadata(http_context& ctx, seastar::httpd::routes& r, seastar::sharded<locator::shared_token_metadata>& tm);
+void set_token_metadata(http_context& ctx, seastar::httpd::routes& r, seastar::sharded<locator::shared_token_metadata>& tm, seastar::sharded<gms::gossiper>& g);
void unset_token_metadata(http_context& ctx, seastar::httpd::routes& r);

}
diff --git a/locator/token_metadata.hh b/locator/token_metadata.hh
index 0f604bdf858..541c91f086e 100644
--- a/locator/token_metadata.hh
+++ b/locator/token_metadata.hh
@@ -245,7 +245,7 @@ class token_metadata final {
inet_address get_endpoint_for_host_id(locator::host_id host_id) const;

/** @return a copy of the endpoint-to-id map for read-only operations */
- std::unordered_map<inet_address, host_id> get_endpoint_to_host_id_map() const;
+ std::unordered_set<host_id> get_host_ids() const;

/// Returns host_id of the local node.
host_id get_my_id() const;
diff --git a/api/api.cc b/api/api.cc
index 86be4257ef0..1753153f87d 100644
--- a/api/api.cc
+++ b/api/api.cc
@@ -188,8 +188,8 @@ future<> unset_server_snapshot(http_context& ctx) {
return ctx.http_server.set_routes([&ctx] (routes& r) { unset_snapshot(ctx, r); });
}

-future<> set_server_token_metadata(http_context& ctx, sharded<locator::shared_token_metadata>& tm) {
- return ctx.http_server.set_routes([&ctx, &tm] (routes& r) { set_token_metadata(ctx, r, tm); });
+future<> set_server_token_metadata(http_context& ctx, sharded<locator::shared_token_metadata>& tm, sharded<gms::gossiper>& g) {
+ return ctx.http_server.set_routes([&ctx, &tm, &g] (routes& r) { set_token_metadata(ctx, r, tm, g); });
}

future<> unset_server_token_metadata(http_context& ctx) {
diff --git a/api/token_metadata.cc b/api/token_metadata.cc
index a8c3234befe..081388d329d 100644
--- a/api/token_metadata.cc
+++ b/api/token_metadata.cc
@@ -10,6 +10,7 @@
#include "api/api-doc/storage_service.json.hh"
#include "api/api-doc/endpoint_snitch_info.json.hh"
#include "locator/token_metadata.hh"
+#include "gms/gossiper.hh"

using namespace seastar::httpd;

@@ -18,7 +19,7 @@ namespace api {
namespace ss = httpd::storage_service_json;
using namespace json;

-void set_token_metadata(http_context& ctx, routes& r, sharded<locator::shared_token_metadata>& tm) {
+void set_token_metadata(http_context& ctx, routes& r, sharded<locator::shared_token_metadata>& tm, sharded<gms::gossiper>& g) {
ss::local_hostid.set(r, [&tm](std::unique_ptr<http::request> req) {
auto id = tm.local().get()->get_my_id();
if (!bool(id)) {
@@ -33,22 +34,25 @@ void set_token_metadata(http_context& ctx, routes& r, sharded<locator::shared_to
}));
});

- ss::get_node_tokens.set(r, [&tm] (std::unique_ptr<http::request> req) {
+ ss::get_node_tokens.set(r, [&tm, &g] (std::unique_ptr<http::request> req) {
gms::inet_address addr(req->get_path_param("endpoint"));
auto& local_tm = *tm.local().get();
- const auto host_id = local_tm.get_host_id_if_known(addr);
+ std::optional<locator::host_id> host_id;
+ try {
+ host_id = g.local().get_host_id(addr);
+ } catch (...) {}
return make_ready_future<json::json_return_type>(stream_range_as_array(host_id ? local_tm.get_tokens(*host_id): std::vector<dht::token>{}, [](const dht::token& i) {
return fmt::to_string(i);
}));
});

- ss::get_leaving_nodes.set(r, [&tm](const_req req) {
+ ss::get_leaving_nodes.set(r, [&tm, &g](const_req req) {
const auto& local_tm = *tm.local().get();
const auto& leaving_host_ids = local_tm.get_leaving_endpoints();
std::unordered_set<gms::inet_address> eps;
eps.reserve(leaving_host_ids.size());
for (const auto host_id: leaving_host_ids) {
- eps.insert(local_tm.get_endpoint_for_host_id(host_id));
+ eps.insert(g.local().get_address_map().get(host_id));
}
return container_to_vec(eps);
});
@@ -58,20 +62,23 @@ void set_token_metadata(http_context& ctx, routes& r, sharded<locator::shared_to
return container_to_vec(addr);
});

- ss::get_joining_nodes.set(r, [&tm](const_req req) {
+ ss::get_joining_nodes.set(r, [&tm, &g](const_req req) {
const auto& local_tm = *tm.local().get();
const auto& points = local_tm.get_bootstrap_tokens();
std::unordered_set<gms::inet_address> eps;
eps.reserve(points.size());
for (const auto& [token, host_id]: points) {
- eps.insert(local_tm.get_endpoint_for_host_id(host_id));
+ eps.insert(g.local().get_address_map().get(host_id));
}
return container_to_vec(eps);
});

- ss::get_host_id_map.set(r, [&tm](const_req req) {
+ ss::get_host_id_map.set(r, [&tm, &g](const_req req) {
std::vector<ss::mapper> res;
- return map_to_key_value(tm.local().get()->get_endpoint_to_host_id_map(), res);
+ auto map = tm.local().get()->get_host_ids() |
+ std::views::transform([&g] (locator::host_id id) { return std::make_pair(g.local().get_address_map().get(id), id); }) |
+ std::ranges::to<std::unordered_map>();
+ return map_to_key_value(std::move(map), res);
});

static auto host_or_broadcast = [&tm](const_req req) {
@@ -79,26 +86,34 @@ void set_token_metadata(http_context& ctx, routes& r, sharded<locator::shared_to
return host.empty() ? tm.local().get()->get_topology().my_address() : gms::inet_address(host);
};

- httpd::endpoint_snitch_info_json::get_datacenter.set(r, [&tm](const_req req) {
+ httpd::endpoint_snitch_info_json::get_datacenter.set(r, [&tm, &g](const_req req) {
auto& topology = tm.local().get()->get_topology();
auto ep = host_or_broadcast(req);
- if (!topology.has_endpoint(ep)) {
+ std::optional<locator::host_id> host_id;
+ try {
+ host_id = g.local().get_host_id(ep);
+ } catch (...) {}
+ if (!host_id || !topology.has_node(*host_id)) {
// Cannot return error here, nodetool status can race, request
// info about just-left node and not handle it nicely
return locator::endpoint_dc_rack::default_location.dc;
}
- return topology.get_datacenter(ep);
+ return topology.get_datacenter(*host_id);
});

- httpd::endpoint_snitch_info_json::get_rack.set(r, [&tm](const_req req) {
+ httpd::endpoint_snitch_info_json::get_rack.set(r, [&tm, &g](const_req req) {
auto& topology = tm.local().get()->get_topology();
auto ep = host_or_broadcast(req);
- if (!topology.has_endpoint(ep)) {
+ std::optional<locator::host_id> host_id;
+ try {
+ host_id = g.local().get_host_id(ep);
+ } catch (...) {}
+ if (!host_id || !topology.has_node(*host_id)) {
// Cannot return error here, nodetool status can race, request
// info about just-left node and not handle it nicely
return locator::endpoint_dc_rack::default_location.rack;
}
- return topology.get_rack(ep);
+ return topology.get_rack(*host_id);
});
}

diff --git a/locator/token_metadata.cc b/locator/token_metadata.cc
index 35c7efd7924..3c369e44ce5 100644
--- a/locator/token_metadata.cc
+++ b/locator/token_metadata.cc
@@ -162,8 +162,8 @@ class token_metadata_impl final {
/** Return the end-point for a unique host ID.*/
inet_address get_endpoint_for_host_id(host_id) const;

- /** @return a copy of the endpoint-to-id map for read-only operations */
- std::unordered_map<inet_address, host_id> get_endpoint_to_host_id_map() const;
+ /** @return a copy of host id set for read-only operations */
+ std::unordered_set<host_id> get_host_ids() const;

void add_bootstrap_token(token t, host_id endpoint);

@@ -567,21 +567,11 @@ inet_address token_metadata_impl::get_endpoint_for_host_id(host_id host_id) cons
}
}

-std::unordered_map<inet_address, host_id> token_metadata_impl::get_endpoint_to_host_id_map() const {
- const auto& nodes = _topology.get_nodes_by_endpoint();
- std::unordered_map<inet_address, host_id> map;
- map.reserve(nodes.size());
- for (const auto& [endpoint, node] : nodes) {
- if (node.get().left() || node.get().is_none()) {
- continue;
- }
- if (const auto& host_id = node.get().host_id()) {
- map[endpoint] = host_id;
- } else {
- tlogger.info("get_endpoint_to_host_id_map: endpoint {} has null host_id: state={}", endpoint, node.get().get_state());
- }
- }
- return map;
+std::unordered_set<host_id> token_metadata_impl::get_host_ids() const {
+ return _topology.get_nodes() |
+ std::views::filter([&] (const node& n) { return !n.left() && !n.is_none(); }) |
+ std::views::transform([] (const node& n) { return n.host_id(); }) |
+ std::ranges::to<std::unordered_set>();
}

bool token_metadata_impl::is_normal_token_owner(host_id endpoint) const {
@@ -1067,9 +1057,9 @@ token_metadata::get_endpoint_for_host_id(host_id host_id) const {
return _impl->get_endpoint_for_host_id(host_id);
}

-std::unordered_map<inet_address, host_id>
-token_metadata::get_endpoint_to_host_id_map() const {
- return _impl->get_endpoint_to_host_id_map();
+std::unordered_set<host_id>
+token_metadata::get_host_ids() const {
+ return _impl->get_host_ids();
}

void
diff --git a/main.cc b/main.cc
index fbc7ac992ee..6a7a3b66d06 100644
--- a/main.cc
+++ b/main.cc
@@ -1093,7 +1093,7 @@ To start the scylla server proper, simply invoke as: scylla server (or just scyl
// token_metadata.stop().get();
//});

- api::set_server_token_metadata(ctx, token_metadata).get();
+ api::set_server_token_metadata(ctx, token_metadata, gossiper).get();
auto stop_tokens_api = defer_verbose_shutdown("token metadata API", [&ctx] {
api::unset_server_token_metadata(ctx).get();
});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:30 AMJan 16
to scylladb-dev@googlegroups.com
Now that all topology related code uses host ids there is not point to
maintain ip to id (and back) mappings in the token metadata. After the
patch the mapping will be maintained in the gossiper only. The rest of
the system will use host ids and in rare cases where translation is
needed (mostly for UX compatibility reasons) the translation will be
done using gossiper.

Fixes: scylladb/scylla#21777

CI: https://jenkins.scylladb.com/job/scylla-master/job/scylla-ci/14485/

Also in scylla-dev gleb/drop-ip-from-tm-v2

Changelog:
v1->v2:
- fix typos
- split unrelated changes to a separate patch
- added one more patch (the last one) to fix #22293
- made an error message more clear
- drop redundant loading of peers

Gleb Natapov (58):
service: address_map: add lookup function that expects address to
exist
hints: simplify can_send() function
api: do not use token_metadata to retrieve ip to id mapping in
token_metadata RESTful endpoints
hints: move create_hint_sync_point function to host ids
storage_service: drop ip based topology::get_datacenter() usage
messaging_service: drop the usage of ip based token_metadata APIs
consistency_level: drop templates since the same types of ranges are
used by all the callers
locator: topology: drop no longer used ip based overloads
locator: topology: remove get_location overload that works on ip and
its last users
storage_proxy: drop is_alive that works on ip since it is not used any
more
locator: topology: drop is_me ip overload along with remaning users
locator: topology: change get_datacenter_endpoints and
get_datacenter_racks to return host ids and amend users
storage_service: change get_dc_rack_for() to work on host ids
hints: move id to ip translation into store_hint() function
node_ops, cdc: drop remaining
token_metadata::get_endpoint_for_host_id() usage
topology coordinator: remove gossiper entry only if host id matches
provided one
storage_service: fix logging
storage_service: ip_address_updater: check peers table instead of
token_metadata whether ip was changed
host_id_or_endpoint: use gossiper to resolve ip to id and back
mappings
token_metadata: drop no longer used functions
topology coordinator: do not update topology on address change

api/api_init.hh | 6 +-
api/hinted_handoff.hh | 3 +-
api/storage_service.hh | 2 +-
api/token_metadata.hh | 3 +-
db/consistency_level.hh | 10 +-
db/hints/manager.hh | 4 +-
db/view/view_builder.hh | 2 +-
gms/gossiper.hh | 9 +-
locator/abstract_replication_strategy.hh | 5 -
locator/token_metadata.hh | 37 +-
locator/topology.hh | 58 +-
locator/util.hh | 5 +
message/messaging_service.hh | 10 +-
repair/repair.hh | 3 -
repair/task_manager_module.hh | 4 +-
service/address_map.hh | 9 +
service/storage_proxy.hh | 5 +-
service/storage_service.hh | 21 +-
alternator/server.cc | 5 +-
api/api.cc | 14 +-
api/hinted_handoff.cc | 13 +-
api/messaging_service.cc | 2 +-
api/storage_service.cc | 6 +-
api/token_metadata.cc | 45 +-
cdc/generation.cc | 2 +-
cql3/statements/select_statement.cc | 2 +-
db/consistency_level.cc | 19 +-
db/hints/internal/hint_sender.cc | 33 +-
db/hints/manager.cc | 36 +-
db/system_keyspace.cc | 2 +-
db/view/view.cc | 8 +-
db/virtual_tables.cc | 19 +-
gms/gossiper.cc | 27 +-
locator/abstract_replication_strategy.cc | 23 -
locator/network_topology_strategy.cc | 4 +-
locator/tablets.cc | 17 -
locator/token_metadata.cc | 173 +-----
locator/topology.cc | 130 +----
locator/util.cc | 20 +-
main.cc | 22 +-
message/messaging_service.cc | 31 +-
node_ops/node_ops_ctl.cc | 2 +-
repair/repair.cc | 32 +-
repair/row_level.cc | 2 +-
service/migration_manager.cc | 10 +-
service/qos/service_level_controller.cc | 2 +-
service/storage_proxy.cc | 48 +-
service/storage_service.cc | 577 ++++++++++---------
service/topology_coordinator.cc | 35 +-
test/boost/locator_topology_test.cc | 130 +----
test/boost/network_topology_strategy_test.cc | 68 +--
test/boost/tablets_test.cc | 68 +--
test/boost/token_metadata_test.cc | 38 --
test/lib/cql_test_env.cc | 5 +-
test/manual/message.cc | 2 +-
test/perf/perf_sort_by_proximity.cc | 1 -
test/perf/tablet_load_balancing.cc | 1 -
57 files changed, 668 insertions(+), 1202 deletions(-)

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:32 AMJan 16
to scylladb-dev@googlegroups.com
Check directly in the topology instead.
---
db/view/view.cc | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/db/view/view.cc b/db/view/view.cc
index 03e654d2dfb..7f73336234b 100644
--- a/db/view/view.cc
+++ b/db/view/view.cc
@@ -2671,7 +2671,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst
// In the v1 table we may have left over rows that belong to nodes that were removed
// and we didn't clean them, so do that now.
auto host_id = row.get_as<utils::UUID>("host_id");
- if (!tmptr->get_endpoint_for_host_id_if_known(locator::host_id(host_id))) {
+ if (!tmptr->get_topology().find_node(locator::host_id(host_id))) {
vlogger.warn("Dropping a row from view_build_status: host {} does not exist", host_id);
continue;
}
@@ -3151,7 +3151,7 @@ future<bool> view_builder::check_view_build_ongoing(const locator::token_metadat
return view_status(ks_name, cf_name).then([&tm] (view_statuses_type&& view_statuses) {
return std::ranges::any_of(view_statuses, [&tm] (const view_statuses_type::value_type& view_status) {
// Only consider status of known hosts.
- return view_status.second == "STARTED" && tm.get_endpoint_for_host_id_if_known(view_status.first);
+ return view_status.second == "STARTED" && tm.get_topology().find_node(view_status.first);
});
});
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:33 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index ac0095f3971..22cdaf339eb 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -7227,7 +7227,7 @@ future<> storage_service::force_remove_completion() {
auto leaving = tm.get_leaving_endpoints();
slogger.warn("Removal not confirmed, Leaving={}", leaving);
for (auto host_id : leaving) {
- const auto endpoint = tm.get_endpoint_for_host_id_if_known(host_id);
+ const auto endpoint = ss._address_map.find(host_id);
if (!endpoint) {
slogger.warn("No endpoint is found for host_id {}", host_id);
continue;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:33 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
repair/task_manager_module.hh | 4 +++-
repair/repair.cc | 26 +++++++++++++++-----------
2 files changed, 18 insertions(+), 12 deletions(-)

diff --git a/repair/task_manager_module.hh b/repair/task_manager_module.hh
index c801ca1c3e8..1fbad623a3c 100644
--- a/repair/task_manager_module.hh
+++ b/repair/task_manager_module.hh
@@ -49,8 +49,9 @@ class user_requested_repair_task_impl : public repair_task_impl {
std::unordered_set<locator::host_id> _ignore_nodes;
bool _small_table_optimization;
std::optional<int> _ranges_parallelism;
+ gms::gossiper& _gossiper;
public:
- user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism) noexcept
+ user_requested_repair_task_impl(tasks::task_manager::module_ptr module, repair_uniq_id id, std::string keyspace, std::string entity, lw_shared_ptr<locator::global_vnode_effective_replication_map> germs, std::vector<sstring> cfs, dht::token_range_vector ranges, std::vector<sstring> hosts, std::vector<sstring> data_centers, std::unordered_set<locator::host_id> ignore_nodes, bool small_table_optimization, std::optional<int> ranges_parallelism, gms::gossiper& gossiper) noexcept
: repair_task_impl(module, id.uuid(), id.id, "keyspace", std::move(keyspace), "", std::move(entity), tasks::task_id::create_null_id(), streaming::stream_reason::repair)
, _germs(germs)
, _cfs(std::move(cfs))
@@ -60,6 +61,7 @@ class user_requested_repair_task_impl : public repair_task_impl {
, _ignore_nodes(std::move(ignore_nodes))
, _small_table_optimization(small_table_optimization)
, _ranges_parallelism(ranges_parallelism)
+ , _gossiper(gossiper)
{}

virtual tasks::is_abortable is_abortable() const noexcept override {
diff --git a/repair/repair.cc b/repair/repair.cc
index 67b68eb8064..51c0aa8d7db 100644
--- a/repair/repair.cc
+++ b/repair/repair.cc
@@ -226,6 +226,7 @@ repair_neighbors::repair_neighbors(std::vector<locator::host_id> nodes, std::vec

// Return all of the neighbors with whom we share the provided range.
static std::vector<locator::host_id> get_neighbors(
+ const gms::gossiper& gossiper,
const locator::effective_replication_map& erm,
const sstring& ksname, query::range<dht::token> range,
const std::vector<sstring>& data_centers,
@@ -282,17 +283,19 @@ static std::vector<locator::host_id> get_neighbors(
} catch(...) {
throw std::runtime_error(format("Unknown host specified: {}", host));
}
- auto endpoint = erm.get_token_metadata().get_host_id_if_known(ip);
- if (endpoint) {
+
+ try {
+ auto endpoint = gossiper.get_host_id(ip);
+
if (endpoint == my_address) {
found_me = true;
- } else if (neighbor_set.contains(*endpoint)) {
- ret.push_back(*endpoint);
+ } else if (neighbor_set.contains(endpoint)) {
+ ret.push_back(endpoint);
// If same host is listed twice, don't add it again later
- neighbor_set.erase(*endpoint);
- } else {
- rlogger.warn("Provided host ip {} has no corresponding host id", ip);
+ neighbor_set.erase(endpoint);
}
+ } catch (...) {
+ rlogger.warn("Provided host ip {} has no corresponding host id", ip);
}
// Nodes which aren't neighbors for this range are ignored.
// This allows the user to give a list of "good" nodes, where
@@ -329,6 +332,7 @@ static std::vector<locator::host_id> get_neighbors(
}

static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
+ const gms::gossiper& gossiper,
const locator::effective_replication_map& erm,
const sstring& ksname,
const dht::token_range_vector& ranges,
@@ -344,7 +348,7 @@ static future<std::list<locator::host_id>> get_hosts_participating_in_repair(
participating_hosts.insert(my_address);

co_await do_for_each(ranges, [&] (const dht::token_range& range) {
- const auto nbs = get_neighbors(erm, ksname, range, data_centers, hosts, ignore_nodes);
+ const auto nbs = get_neighbors(gossiper, erm, ksname, range, data_centers, hosts, ignore_nodes);
for (const auto& nb : nbs) {
participating_hosts.insert(nb);
}
@@ -676,7 +680,7 @@ void repair::shard_repair_task_impl::check_in_abort_or_shutdown() {

repair_neighbors repair::shard_repair_task_impl::get_repair_neighbors(const dht::token_range& range) {
return neighbors.empty() ?
- repair_neighbors(get_neighbors(*erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
+ repair_neighbors(get_neighbors(gossiper, *erm, _status.keyspace, range, data_centers, hosts, ignore_nodes, _small_table_optimization)) :
neighbors[range];
}

@@ -1306,7 +1310,7 @@ future<int> repair_service::do_repair_start(gms::gossip_address_map& addr_map, s
}

auto ranges_parallelism = options.ranges_parallelism == -1 ? std::nullopt : std::optional<int>(options.ranges_parallelism);
- auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism);
+ auto task = co_await _repair_module->make_and_start_task<repair::user_requested_repair_task_impl>({}, id, std::move(keyspace), "", germs, std::move(cfs), std::move(ranges), std::move(options.hosts), std::move(options.data_centers), std::move(ignore_nodes), small_table_optimization, ranges_parallelism, _gossiper.local());
co_return id.id;
}

@@ -1331,7 +1335,7 @@ future<> repair::user_requested_repair_task_impl::run() {
auto normal_nodes = germs->get().get_token_metadata().get_normal_token_owners();
participants = std::list<locator::host_id>(normal_nodes.begin(), normal_nodes.end());
} else {
- participants = get_hosts_participating_in_repair(germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
+ participants = get_hosts_participating_in_repair(_gossiper, germs->get(), keyspace, ranges, data_centers, hosts, ignore_nodes).get();
}
auto [hints_batchlog_flushed, flush_time] = rs.flush_hints(id, keyspace, cfs, ignore_nodes).get();

--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:33 AMJan 16
to scylladb-dev@googlegroups.com
The state always has host id now, so there is no point to looks it up in
the token metadata.
---
db/virtual_tables.cc | 20 ++++++++------------
1 file changed, 8 insertions(+), 12 deletions(-)

diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc
index b42be729489..b9b3e0d5f0c 100644
--- a/db/virtual_tables.cc
+++ b/db/virtual_tables.cc
@@ -75,7 +75,7 @@ class cluster_status_table : public memtable_filling_virtual_table {
std::vector<frozen_mutation> muts;
muts.reserve(gossiper.num_endpoints());

- gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state&) {
+ gossiper.for_each_endpoint_state([&] (const gms::inet_address& endpoint, const gms::endpoint_state& eps) {
static thread_local auto s = build_schema();
mutation m(s, partition_key::from_single_value(*s, data_value(endpoint).serialize_nonnull()));
row& cr = m.partition().clustered_row(*schema(), clustering_key::make_empty()).cells();
@@ -86,24 +86,20 @@ class cluster_status_table : public memtable_filling_virtual_table {
}
set_cell(cr, "load", gossiper.get_application_state_value(endpoint, gms::application_state::LOAD));

- auto hostid = tm.get_host_id_if_known(endpoint);
- if (hostid) {
- if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) {
- set_cell(cr, "status", boost::to_upper_copy<std::string>(fmt::format("{}", ss.get_node_state(*hostid))));
- }
- set_cell(cr, "host_id", hostid->uuid());
+ auto hostid = eps.get_host_id();
+ if (ss.raft_topology_change_enabled() && !gossiper.is_shutdown(endpoint)) {
+ set_cell(cr, "status", boost::to_upper_copy<std::string>(fmt::format("{}", ss.get_node_state(hostid))));
}
+ set_cell(cr, "host_id", hostid.uuid());

- if (hostid) {
- sstring dc = tm.get_topology().get_location(endpoint).dc;
- set_cell(cr, "dc", dc);
- }
+ sstring dc = tm.get_topology().get_location(endpoint).dc;
+ set_cell(cr, "dc", dc);

if (ownership.contains(endpoint)) {
set_cell(cr, "owns", ownership[endpoint]);
}

- set_cell(cr, "tokens", int32_t(hostid ? tm.get_tokens(*hostid).size() : 0));
+ set_cell(cr, "tokens", int32_t(tm.get_tokens(hostid).size()));

muts.push_back(freeze(std::move(m)));
});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:34 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_proxy.cc | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/service/storage_proxy.cc b/service/storage_proxy.cc
index 818de6f8e6c..86a7958bccb 100644
--- a/service/storage_proxy.cc
+++ b/service/storage_proxy.cc
@@ -1058,7 +1058,9 @@ class storage_proxy::remote {
void connection_dropped(gms::inet_address addr, std::optional<locator::host_id> id) {
slogger.debug("Drop hit rate info for {} because of disconnect", addr);
if (!id) {
- id = _sp.get_token_metadata_ptr()->get_host_id_if_known(addr);
+ try {
+ id = _gossiper.get_host_id(addr);
+ } catch (...) {}
}
if (!id) {
return;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:34 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 15 +++++++++------
1 file changed, 9 insertions(+), 6 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 22cdaf339eb..e7374af7207 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -4469,11 +4469,11 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
// and waits for ring_delay. It's possible the node being decommissioned might
// die after it has sent this notification. If this happens, the node would
// have already been removed from this token_metadata, so we wouldn't find it here.
- const auto node_id = tmptr->get_host_id_if_known(node);
- slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
- if (node_id) {
- tmptr->del_leaving_endpoint(*node_id);
- }
+ try {
+ const auto node_id = _gossiper.get_host_id(node);
+ slogger.info("decommission[{}]: Removed node={} as leaving node, coordinator={}", req.ops_uuid, node, coordinator);
+ tmptr->del_leaving_endpoint(node_id);
+ } catch (...) {}
}
return update_topology_change_info(tmptr, ::format("decommission {}", req.leaving_nodes));
});
@@ -4489,7 +4489,10 @@ future<node_ops_cmd_response> storage_service::node_ops_cmd_handler(gms::inet_ad
check_again = false;
for (auto& node : req.leaving_nodes) {
auto tmptr = get_token_metadata_ptr();
- const auto host_id = tmptr->get_host_id_if_known(node);
+ std::optional<locator::host_id> host_id;
+ try {
+ host_id = _gossiper.get_host_id(node);
+ } catch(...) {};
if (host_id && tmptr->is_normal_token_owner(*host_id)) {
check_again = true;
if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:35 AMJan 16
to scylladb-dev@googlegroups.com
Now that we have gossiper::get_endpoint_state_ptr that works on host ids
there is no need to translate id to ip at all.
---
service/topology_coordinator.cc | 35 ++++++++++++++-------------------
1 file changed, 15 insertions(+), 20 deletions(-)

diff --git a/service/topology_coordinator.cc b/service/topology_coordinator.cc
index 4e8ca60ef2c..a50f8838d5c 100644
--- a/service/topology_coordinator.cc
+++ b/service/topology_coordinator.cc
@@ -3057,11 +3057,11 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
rtlogger.info("building initial raft topology state and CDC generation");
guard = co_await start_operation();

- auto get_application_state = [&] (locator::host_id host_id, gms::inet_address ep, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
+ auto get_application_state = [&] (locator::host_id host_id, const gms::application_state_map& epmap, gms::application_state app_state) -> sstring {
const auto it = epmap.find(app_state);
if (it == epmap.end()) {
- throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{}: application state {} is missing in gossip",
- host_id, ep, app_state));
+ throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}: application state {} is missing in gossip",
+ host_id, app_state));
}
// it's versioned_value::value(), not std::optional::value() - it does not throw
return it->second.value();
@@ -3069,17 +3069,13 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {

// Create a new CDC generation
auto get_sharding_info_for_host_id = [&] (locator::host_id host_id) -> std::pair<size_t, uint8_t> {
- const auto ep = tmptr->get_endpoint_for_host_id_if_known(host_id);
- if (!ep) {
- throw std::runtime_error(format("IP of node with ID {} is not known", host_id));
- }
- const auto eptr = _gossiper.get_endpoint_state_ptr(*ep);
+ const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
if (!eptr) {
- throw std::runtime_error(format("no gossiper endpoint state for node {}/{}", host_id, *ep));
+ throw std::runtime_error(format("no gossiper endpoint state for node {}", host_id));
}
const auto& epmap = eptr->get_application_state_map();
- const auto shard_count = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::SHARD_COUNT));
- const auto ignore_msb = std::stoi(get_application_state(host_id, *ep, epmap, gms::application_state::IGNORE_MSB_BITS));
+ const auto shard_count = std::stoi(get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT));
+ const auto ignore_msb = std::stoi(get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS));
return std::make_pair<size_t, uint8_t>(shard_count, ignore_msb);
};
auto [cdc_gen_uuid, guard_, mutation] = co_await prepare_and_broadcast_cdc_generation_data(tmptr, std::move(guard), std::nullopt, get_sharding_info_for_host_id);
@@ -3096,23 +3092,22 @@ future<> topology_coordinator::build_coordinator_state(group0_guard guard) {
}

const auto& host_id = node.get().host_id();
- const auto& ep = node.get().endpoint();
- const auto eptr = _gossiper.get_endpoint_state_ptr(ep);
+ const auto eptr = _gossiper.get_endpoint_state_ptr(host_id);
if (!eptr) {
- throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {}/{} as gossip contains no data for it", host_id, ep));
+ throw std::runtime_error(format("failed to build initial raft topology state from gossip for node {} as gossip contains no data for it", host_id));
}

const auto& epmap = eptr->get_application_state_map();

- const auto datacenter = get_application_state(host_id, ep, epmap, gms::application_state::DC);
- const auto rack = get_application_state(host_id, ep, epmap, gms::application_state::RACK);
+ const auto datacenter = get_application_state(host_id, epmap, gms::application_state::DC);
+ const auto rack = get_application_state(host_id, epmap, gms::application_state::RACK);
const auto tokens_v = tmptr->get_tokens(host_id);
const std::unordered_set<dht::token> tokens(tokens_v.begin(), tokens_v.end());
- const auto release_version = get_application_state(host_id, ep, epmap, gms::application_state::RELEASE_VERSION);
+ const auto release_version = get_application_state(host_id, epmap, gms::application_state::RELEASE_VERSION);
const auto num_tokens = tokens.size();
- const auto shard_count = get_application_state(host_id, ep, epmap, gms::application_state::SHARD_COUNT);
- const auto ignore_msb = get_application_state(host_id, ep, epmap, gms::application_state::IGNORE_MSB_BITS);
- const auto supported_features_s = get_application_state(host_id, ep, epmap, gms::application_state::SUPPORTED_FEATURES);
+ const auto shard_count = get_application_state(host_id, epmap, gms::application_state::SHARD_COUNT);
+ const auto ignore_msb = get_application_state(host_id, epmap, gms::application_state::IGNORE_MSB_BITS);
+ const auto supported_features_s = get_application_state(host_id, epmap, gms::application_state::SUPPORTED_FEATURES);
const auto supported_features = gms::feature_service::to_feature_set(supported_features_s);

if (enabled_features.empty()) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:36 AMJan 16
to scylladb-dev@googlegroups.com
The functions are called from RESful API so has to return ips for backwards
compatibility, but internally we can use host ids as long as possible
and convert to ips just before returning. This also drops usage of ip
based erm function which we want to get rid of.
---
locator/util.hh | 2 +-
locator/util.cc | 20 ++++++++++----------
service/storage_service.cc | 6 +++++-
3 files changed, 16 insertions(+), 12 deletions(-)

diff --git a/locator/util.hh b/locator/util.hh
index 3a3360d3d94..c66e400680a 100644
--- a/locator/util.hh
+++ b/locator/util.hh
@@ -23,6 +23,6 @@ namespace gms {

namespace locator {
future<std::vector<dht::token_range_endpoints>> describe_ring(const replica::database& db, const gms::gossiper& gossiper, const sstring& keyspace, bool include_only_local_dc = false);
- future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>> get_range_to_address_map(
+ future<std::unordered_map<dht::token_range, host_id_vector_replica_set>> get_range_to_address_map(
locator::effective_replication_map_ptr erm, const std::vector<token>& sorted_tokens);
}
\ No newline at end of file
diff --git a/locator/util.cc b/locator/util.cc
index d7f8768cdae..55c0e0d4911 100644
--- a/locator/util.cc
+++ b/locator/util.cc
@@ -12,14 +12,14 @@

namespace locator {

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+static future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
construct_range_to_endpoint_map(
locator::effective_replication_map_ptr erm,
const dht::token_range_vector& ranges) {
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> res;
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> res;
res.reserve(ranges.size());
for (auto r : ranges) {
- res[r] = erm->get_natural_endpoints(
+ res[r] = erm->get_natural_replicas(
r.end() ? r.end()->value() : dht::maximum_token());
co_await coroutine::maybe_yield();
}
@@ -47,7 +47,7 @@ get_all_ranges(const std::vector<token>& sorted_tokens) {
}

// Caller is responsible to hold token_metadata valid until the returned future is resolved
-future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map(locator::effective_replication_map_ptr erm,
const std::vector<token>& sorted_tokens) {
co_return co_await construct_range_to_endpoint_map(erm, co_await get_all_ranges(sorted_tokens));
@@ -67,12 +67,12 @@ get_tokens_in_local_dc(const locator::token_metadata& tm) {
co_return filtered_tokens;
}

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+static future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map_in_local_dc(
locator::effective_replication_map_ptr erm) {
auto tmptr = erm->get_token_metadata_ptr();
auto orig_map = co_await get_range_to_address_map(erm, co_await get_tokens_in_local_dc(*tmptr));
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> filtered_map;
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> filtered_map;
filtered_map.reserve(orig_map.size());
auto local_dc_filter = tmptr->get_topology().get_local_dc_filter();
for (auto entry : orig_map) {
@@ -90,7 +90,7 @@ get_range_to_address_map_in_local_dc(
// return get_range_to_address_map(db.find_keyspace(keyspace).get_vnode_effective_replication_map());
// }

-static future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
+future<std::unordered_map<dht::token_range, host_id_vector_replica_set>>
get_range_to_address_map(locator::effective_replication_map_ptr erm) {
return get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
}
@@ -100,7 +100,7 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const
std::vector<dht::token_range_endpoints> ranges;

auto erm = db.find_keyspace(keyspace).get_vnode_effective_replication_map();
- std::unordered_map<dht::token_range, inet_address_vector_replica_set> range_to_address_map = co_await (
+ std::unordered_map<dht::token_range, host_id_vector_replica_set> range_to_address_map = co_await (
include_only_local_dc
? get_range_to_address_map_in_local_dc(erm)
: get_range_to_address_map(erm)
@@ -119,10 +119,10 @@ describe_ring(const replica::database& db, const gms::gossiper& gossiper, const
}
for (auto endpoint : addresses) {
dht::endpoint_details details;
- details._host = endpoint;
+ details._host = gossiper.get_address_map().get(endpoint);
details._datacenter = topology.get_datacenter(endpoint);
details._rack = topology.get_rack(endpoint);
- tr._rpc_endpoints.push_back(gossiper.get_rpc_address(endpoint));
+ tr._rpc_endpoints.push_back(gossiper.get_rpc_address(details._host));
tr._endpoints.push_back(fmt::to_string(details._host));
tr._endpoint_details.push_back(details);
}
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 17cb7a8fa3a..73fae5726e2 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -13,6 +13,7 @@
#include "compaction/task_manager_module.hh"
#include "gc_clock.hh"
#include "raft/raft.hh"
+#include <ranges>
#include <seastar/core/sleep.hh>
#include "service/qos/raft_service_level_distributed_data_accessor.hh"
#include "service/qos/service_level_controller.hh"
@@ -2303,7 +2304,10 @@ future<> storage_service::bootstrap(std::unordered_set<token>& bootstrap_tokens,

future<std::unordered_map<dht::token_range, inet_address_vector_replica_set>>
storage_service::get_range_to_address_map(locator::effective_replication_map_ptr erm) const {
- return locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens());
+ co_return (co_await locator::get_range_to_address_map(erm, erm->get_token_metadata_ptr()->sorted_tokens())) |
+ std::views::transform([&] (auto tid) { return std::make_pair(tid.first,
+ tid.second | std::views::transform([&] (auto id) { return _address_map.get(id); }) | std::ranges::to<inet_address_vector_replica_set>()); }) |
+ std::ranges::to<std::unordered_map>();
}

future<> storage_service::handle_state_bootstrap(inet_address endpoint, gms::permit_id pid) {
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:36 AMJan 16
to scylladb-dev@googlegroups.com
Will be used later to simplify code.
---
gms/gossiper.hh | 1 +
gms/gossiper.cc | 8 ++++++++
2 files changed, 9 insertions(+)

diff --git a/gms/gossiper.hh b/gms/gossiper.hh
index 2ff817f7000..595f7103ca6 100644
--- a/gms/gossiper.hh
+++ b/gms/gossiper.hh
@@ -435,6 +435,7 @@ class gossiper : public seastar::async_sharded_service<gossiper>, public seastar
// The endpoint_state is immutable (except for its update_timestamp), guaranteed not to change while
// the endpoint_state_ptr is held.
endpoint_state_ptr get_endpoint_state_ptr(inet_address ep) const noexcept;
+ endpoint_state_ptr get_endpoint_state_ptr(locator::host_id ep) const noexcept;

// Return this node's endpoint_state_ptr
endpoint_state_ptr get_this_endpoint_state_ptr() const noexcept {
diff --git a/gms/gossiper.cc b/gms/gossiper.cc
index 19747be504f..c303230316e 100644
--- a/gms/gossiper.cc
+++ b/gms/gossiper.cc
@@ -1505,6 +1505,14 @@ endpoint_state_ptr gossiper::get_endpoint_state_ptr(inet_address ep) const noexc
}
}

+endpoint_state_ptr gossiper::get_endpoint_state_ptr(locator::host_id id) const noexcept {
+ auto ip = _address_map.find(id);
+ if (!ip) {
+ return nullptr;
+ }
+ return get_endpoint_state_ptr(*ip);
+}
+
void gossiper::update_timestamp(const endpoint_state_ptr& eps) noexcept {
const_cast<endpoint_state&>(*eps).update_timestamp();
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:36 AMJan 16
to scylladb-dev@googlegroups.com
Currently the entry is removed only if ip is not used by any normal or
transitioning node. This is done to not remove a wrong entry that just
happen to use the same ip, but the same can be achieved by checking host
id in the entry.
---
service/storage_service.cc | 16 +---------------
1 file changed, 1 insertion(+), 15 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index a996b7c30f8..86952acfd20 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -428,20 +428,6 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
}
};

- auto get_used_ips = [&, used_ips = std::optional<std::unordered_set<inet_address>>{}]() mutable
- -> const std::unordered_set<inet_address>&
- {
- if (!used_ips) {
- used_ips.emplace();
- for (const auto& [sid, rs]: boost::range::join(t.normal_nodes, t.transition_nodes)) {
- if (const auto used_ip = am.find(locator::host_id{sid.uuid()})) {
- used_ips->insert(*used_ip);
- }
- }
- }
- return *used_ips;
- };
-
using host_id_to_ip_map_t = std::unordered_map<locator::host_id, gms::inet_address>;
auto get_host_id_to_ip_map = [&, map = std::optional<host_id_to_ip_map_t>{}]() mutable -> future<const host_id_to_ip_map_t*> {
if (!map.has_value()) {
@@ -464,7 +450,7 @@ future<storage_service::nodes_to_notify_after_sync> storage_service::sync_raft_t
auto remove_ip = [&](inet_address ip, locator::host_id host_id, bool notify) -> future<> {
sys_ks_futures.push_back(_sys_ks.local().remove_endpoint(ip));

- if (_gossiper.get_endpoint_state_ptr(ip) && !get_used_ips().contains(ip)) {
+ if (const auto ep = _gossiper.get_endpoint_state_ptr(ip); ep && ep->get_host_id() == host_id) {
co_await _gossiper.force_remove_endpoint(ip, gms::null_permit_id);
if (notify) {
nodes_to_notify.left.push_back({ip, host_id});
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:36 AMJan 16
to scylladb-dev@googlegroups.com
Use address map to translate id to ip instead. We want to drop ips from token_metadata.
---
cdc/generation.cc | 2 +-
node_ops/node_ops_ctl.cc | 2 +-
service/storage_service.cc | 29 +++++++++++------------------
3 files changed, 13 insertions(+), 20 deletions(-)

diff --git a/cdc/generation.cc b/cdc/generation.cc
index 8ace3cea7e2..22645a7c27c 100644
--- a/cdc/generation.cc
+++ b/cdc/generation.cc
@@ -402,7 +402,7 @@ future<cdc::generation_id> generation_service::legacy_make_new_generation(const
throw std::runtime_error(
format("Can't find endpoint for token {}", end));
}
- const auto ep = tmptr->get_endpoint_for_host_id(*endpoint);
+ const auto ep = _gossiper.get_address_map().get(*endpoint);
auto sc = get_shard_count(ep, _gossiper);
return {sc > 0 ? sc : 1, get_sharding_ignore_msb(ep, _gossiper)};
}
diff --git a/node_ops/node_ops_ctl.cc b/node_ops/node_ops_ctl.cc
index 1eaaf2e333e..64a1ab36186 100644
--- a/node_ops/node_ops_ctl.cc
+++ b/node_ops/node_ops_ctl.cc
@@ -143,7 +143,7 @@ future<> node_ops_ctl::abort_on_error(node_ops_cmd cmd, std::exception_ptr ex) n
future<> node_ops_ctl::send_to_all(node_ops_cmd cmd) {
req.cmd = cmd;
req.ignore_nodes = ignore_nodes |
- std::views::transform([&] (locator::host_id id) { return tmptr->get_endpoint_for_host_id(id); }) |
+ std::views::transform([&] (locator::host_id id) { return ss.gossiper().get_address_map().get(id); }) |
std::ranges::to<std::list>();
sstring op_desc = ::format("{}", cmd);
nlogger.info("{}[{}]: Started {}", desc, uuid(), req);
diff --git a/service/storage_service.cc b/service/storage_service.cc
index 5e1452a6883..a996b7c30f8 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -356,16 +356,6 @@ static future<> set_gossip_tokens(gms::gossiper& g,
);
}

-static std::unordered_map<token, gms::inet_address> get_token_to_endpoint(const locator::token_metadata& tm) {
- const auto& map = tm.get_token_to_endpoint();
- std::unordered_map<token, gms::inet_address> result;
- result.reserve(map.size());
- for (const auto [t, id]: map) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
- }
- return result;
-}
-
/*
* The helper waits for two things
* 1) for schema agreement
@@ -1964,7 +1954,7 @@ future<> storage_service::join_topology(sharded<db::system_distributed_keyspace>
for (auto token : bootstrap_tokens) {
auto existing = tmptr->get_endpoint(token);
if (existing) {
- auto eps = _gossiper.get_endpoint_state_ptr(tmptr->get_endpoint_for_host_id(*existing));
+ auto eps = _gossiper.get_endpoint_state_ptr(*existing);
if (eps && eps->get_update_timestamp() > gms::gossiper::clk::now() - delay) {
throw std::runtime_error("Cannot replace a live node...");
}
@@ -2468,7 +2458,10 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
// token_to_endpoint_map is used to track the current token owners for the purpose of removing replaced endpoints.
// when any token is replaced by a new owner, we track the existing owner in `candidates_for_removal`
// and eventually, if any candidate for removal ends up owning no tokens, it is removed from token_metadata.
- std::unordered_map<token, inet_address> token_to_endpoint_map = get_token_to_endpoint(get_token_metadata());
+ std::unordered_map<token, inet_address> token_to_endpoint_map = get_token_metadata().get_token_to_endpoint() |
+ std::views::transform([this] (auto& e) {
+ return std::make_pair(e.first, _address_map.get(e.second));
+ }) | std::ranges::to<std::unordered_map>();
std::unordered_set<inet_address> candidates_for_removal;

// Here we convert endpoint tokens from gossiper to owned_tokens, which will be assigned as a new
@@ -2587,7 +2580,7 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
const auto& tm = get_token_metadata();
auto ver = tm.get_ring_version();
for (auto& x : tm.get_token_to_endpoint()) {
- slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, tm.get_endpoint_for_host_id(x.second), x.second);
+ slogger.debug("handle_state_normal: token_metadata.ring_version={}, token={} -> endpoint={}/{}", ver, x.first, _address_map.get(x.second), x.second);
}
}
_normal_state_handled_on_boot.insert(endpoint);
@@ -3472,7 +3465,7 @@ storage_service::prepare_replacement_info(std::unordered_set<gms::inet_address>
}

future<std::map<gms::inet_address, float>> storage_service::get_ownership() {
- return run_with_no_api_lock([] (storage_service& ss) {
+ return run_with_no_api_lock([this] (storage_service& ss) {
const auto& tm = ss.get_token_metadata();
auto token_map = dht::token::describe_ownership(tm.sorted_tokens());
// describeOwnership returns tokens in an unspecified order, let's re-order them
@@ -3480,7 +3473,7 @@ future<std::map<gms::inet_address, float>> storage_service::get_ownership() {
for (auto entry : token_map) {
locator::host_id id = tm.get_endpoint(entry.first).value();
auto token_ownership = entry.second;
- ownership[tm.get_endpoint_for_host_id(id)] += token_ownership;
+ ownership[_address_map.get(id)] += token_ownership;
}
return ownership;
});
@@ -5299,10 +5292,10 @@ std::map<token, inet_address> storage_service::get_token_to_endpoint_map() {
const auto& tm = get_token_metadata();
std::map<token, inet_address> result;
for (const auto [t, id]: tm.get_token_to_endpoint()) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
+ result.insert({t, _address_map.get(id)});
}
for (const auto [t, id]: tm.get_bootstrap_tokens()) {
- result.insert({t, tm.get_endpoint_for_host_id(id)});
+ result.insert({t, _address_map.get(id)});
}
return result;
}
@@ -5312,7 +5305,7 @@ future<std::map<token, inet_address>> storage_service::get_tablet_to_endpoint_ma
const auto& tmap = tm.tablets().get_tablet_map(table);
std::map<token, inet_address> result;
for (std::optional<locator::tablet_id> tid = tmap.first_tablet(); tid; tid = tmap.next_tablet(*tid)) {
- result.emplace(tmap.get_last_token(*tid), tm.get_endpoint_for_host_id(tmap.get_primary_replica(*tid).host));
+ result.emplace(tmap.get_last_token(*tid), _address_map.get(tmap.get_primary_replica(*tid).host));
co_await coroutine::maybe_yield();
}
co_return result;
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:36 AMJan 16
to scylladb-dev@googlegroups.com
We want to drop ips from token_metadata so move to different API to map
ip to id.
---
service/storage_service.cc | 32 +++++++++++++++++++++-----------
1 file changed, 21 insertions(+), 11 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index e7374af7207..2db8395b3cd 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2637,19 +2637,18 @@ future<> storage_service::on_join(gms::inet_address endpoint, gms::endpoint_stat

future<> storage_service::on_alive(gms::inet_address endpoint, gms::endpoint_state_ptr state, gms::permit_id pid) {
const auto& tm = get_token_metadata();
- const auto tm_host_id_opt = tm.get_host_id_if_known(endpoint);
- slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, tm_host_id_opt, pid);
+ const auto host_id = state->get_host_id();
+ slogger.debug("endpoint={}/{} on_alive: permit_id={}", endpoint, host_id, pid);
const auto* node = tm.get_topology().find_node(endpoint);
if (node && node->is_member()) {
co_await notify_up(endpoint);
} else if (raft_topology_change_enabled()) {
slogger.debug("ignore on_alive since topology changes are using raft and "
- "endpoint {}/{} is not a topology member", endpoint, tm_host_id_opt);
+ "endpoint {}/{} is not a topology member", endpoint, host_id);
} else {
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
const auto dc_rack = get_dc_rack_for(endpoint);
- const auto host_id = _gossiper.get_host_id(endpoint);
tmptr->update_host_id(host_id, endpoint);
tmptr->update_topology(host_id, dc_rack);
co_await replicate_to_all_cores(std::move(tmptr));
@@ -2740,15 +2739,26 @@ future<> storage_service::on_remove(gms::inet_address endpoint, gms::permit_id p
co_return;
}

+ locator::host_id host_id;
+
+ try {
+ // It seams gossiper does not check for endpoint existance before calling the callback
+ // so the lookup may fail, but there is nothing to do in this case.
+ host_id = _gossiper.get_host_id(endpoint);
+ } catch (...) {
+ co_return;
+ }
+
+ // We should handle the case when the host id is mapped to a different address.
+ // This could happen when an address for the host id changes and the callback here is called
+ // with the old ip. We should just skip the remove in that case.
+ if (_address_map.get(host_id) != endpoint) {
+ co_return;
+ }
+
auto tmlock = co_await get_token_metadata_lock();
auto tmptr = co_await get_mutable_token_metadata_ptr();
- // We should handle the case when we aren't able to find endpoint -> ip mapping in token_metadata.
- // This could happen e.g. when the new endpoint has bigger generation in handle_state_normal - the code
- // in handle_state_normal will remap host_id to the new IP and we won't find
- // old IP here. We should just skip the remove in that case.
- if (const auto host_id = tmptr->get_host_id_if_known(endpoint); host_id) {
- tmptr->remove_endpoint(*host_id);
- }
+ tmptr->remove_endpoint(host_id);
co_await update_topology_change_info(tmptr, ::format("on_remove {}", endpoint));
co_await replicate_to_all_cores(std::move(tmptr));
}
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:37 AMJan 16
to scylladb-dev@googlegroups.com
Move the test to work on host ids. IPs will be dropped eventually.
---
test/boost/network_topology_strategy_test.cc | 32 ++++++++++----------
1 file changed, 16 insertions(+), 16 deletions(-)

diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc
index 58cbb556d73..a4369457c64 100644
--- a/test/boost/network_topology_strategy_test.cc
+++ b/test/boost/network_topology_strategy_test.cc
@@ -50,7 +50,7 @@ struct ring_point {
host_id id = host_id::create_random_id();
};

-void print_natural_endpoints(double point, const inet_address_vector_replica_set v) {
+void print_natural_endpoints(double point, const host_id_vector_replica_set v) {
testlog.debug("Natural endpoints for a token {}:", point);
std::string str;
std::ostringstream strm(str);
@@ -104,11 +104,11 @@ void strategy_sanity_check(
void endpoints_check(
replication_strategy_ptr ars_ptr,
const token_metadata_ptr& tm,
- const inet_address_vector_replica_set& endpoints,
+ const host_id_vector_replica_set& endpoints,
const locator::topology& topo,
bool strict_dc_rf = false) {

- auto&& nodes_per_dc = tm->get_datacenter_token_owners_ips();
+ auto&& nodes_per_dc = tm->get_datacenter_token_owners();
const network_topology_strategy* nts_ptr =
dynamic_cast<const network_topology_strategy*>(ars_ptr.get());

@@ -123,7 +123,7 @@ void endpoints_check(
BOOST_CHECK_LE(total_rf, ars_ptr->get_replication_factor(*tm));

// Check the uniqueness
- std::unordered_set<inet_address> ep_set(endpoints.begin(), endpoints.end());
+ std::unordered_set<locator::host_id> ep_set(endpoints.begin(), endpoints.end());
BOOST_CHECK_EQUAL(endpoints.size(), ep_set.size());

// Check the per-DC RF
@@ -166,7 +166,7 @@ void full_ring_check(const std::vector<ring_point>& ring_points,
for (auto& rp : ring_points) {
double cur_point1 = rp.point - 0.5;
token t1(tests::d2t(cur_point1 / ring_points.size()));
- auto endpoints1 = erm->get_natural_endpoints(t1);
+ auto endpoints1 = erm->get_natural_replicas(t1);

endpoints_check(ars_ptr, tmptr, endpoints1, topo);

@@ -179,7 +179,7 @@ void full_ring_check(const std::vector<ring_point>& ring_points,
//
double cur_point2 = rp.point - 0.2;
token t2(tests::d2t(cur_point2 / ring_points.size()));
- auto endpoints2 = erm->get_natural_endpoints(t2);
+ auto endpoints2 = erm->get_natural_replicas(t2);

endpoints_check(ars_ptr, tmptr, endpoints2, topo);
check_ranges_are_sorted(erm, rp.id).get();
@@ -193,17 +193,17 @@ void full_ring_check(const tablet_map& tmap,
auto& tm = *tmptr;
const auto& topo = tm.get_topology();

- auto to_endpoint_set = [&] (const tablet_replica_set& replicas) {
- inet_address_vector_replica_set result;
+ auto to_replica_set = [&] (const tablet_replica_set& replicas) {
+ host_id_vector_replica_set result;
result.reserve(replicas.size());
for (auto&& replica : replicas) {
- result.emplace_back(tm.get_endpoint_for_host_id(replica.host));
+ result.emplace_back(replica.host);
}
return result;
};

for (tablet_id tb : tmap.tablet_ids()) {
- endpoints_check(rs_ptr, tmptr, to_endpoint_set(tmap.get_tablet_info(tb).replicas), topo, true);
+ endpoints_check(rs_ptr, tmptr, to_replica_set(tmap.get_tablet_info(tb).replicas), topo, true);
}
}

@@ -672,7 +672,7 @@ static size_t get_replication_factor(const sstring& dc,

static bool has_sufficient_replicas(const sstring& dc,
const std::unordered_map<sstring, std::unordered_set<host_id>>& dc_replicas,
- const std::unordered_map<sstring, std::unordered_set<inet_address>>& all_endpoints,
+ const std::unordered_map<sstring, std::unordered_set<host_id>>& all_endpoints,
const std::unordered_map<sstring, size_t>& datacenters) noexcept {
auto dc_replicas_it = dc_replicas.find(dc);
if (dc_replicas_it == dc_replicas.end()) {
@@ -690,7 +690,7 @@ static bool has_sufficient_replicas(const sstring& dc,

static bool has_sufficient_replicas(
const std::unordered_map<sstring, std::unordered_set<host_id>>& dc_replicas,
- const std::unordered_map<sstring, std::unordered_set<inet_address>>& all_endpoints,
+ const std::unordered_map<sstring, std::unordered_set<host_id>>& all_endpoints,
const std::unordered_map<sstring, size_t>& datacenters) noexcept {

for (auto& dc : datacenters | std::views::keys) {
@@ -741,16 +741,16 @@ static locator::host_id_set calculate_natural_endpoints(
// the token-owning members of a DC
//
const std::unordered_map<sstring,
- std::unordered_set<inet_address>>
- all_endpoints = tm.get_datacenter_token_owners_ips();
+ std::unordered_set<locator::host_id>>
+ all_endpoints = tm.get_datacenter_token_owners();
//
// all racks (with non-token owners filtered out) in a DC so we can check
// when we have exhausted all racks in a DC
//
const std::unordered_map<sstring,
std::unordered_map<sstring,
- std::unordered_set<inet_address>>>
- racks = tm.get_datacenter_racks_token_owners_ips();
+ std::unordered_set<host_id>>>
+ racks = tm.get_datacenter_racks_token_owners();

// not aware of any cluster members
SCYLLA_ASSERT(!all_endpoints.empty() && !racks.empty());
--
2.47.1

Gleb Natapov

<gleb@scylladb.com>
unread,
Jan 16, 2025, 5:02:37 AMJan 16
to scylladb-dev@googlegroups.com
Instead use gossiper and peers table to retrieve same information.
Token_metadata is created from the mix of those two anyway. The goal is
to drop ips from token_metadata entirely.
---
service/storage_service.cc | 28 ++++++++++++++++++++++------
1 file changed, 22 insertions(+), 6 deletions(-)

diff --git a/service/storage_service.cc b/service/storage_service.cc
index 73fae5726e2..16739957880 100644
--- a/service/storage_service.cc
+++ b/service/storage_service.cc
@@ -2370,11 +2370,26 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
if (tmptr->is_normal_token_owner(host_id)) {
slogger.info("handle_state_normal: node {}/{} was already a normal token owner", endpoint, host_id);
}
- auto existing = tmptr->get_endpoint_for_host_id_if_known(host_id);

// Old node in replace-with-same-IP scenario.
std::optional<locator::host_id> replaced_id;

+ auto ips = _gossiper.get_nodes_with_host_id(host_id);
+
+ std::optional<inet_address> existing;
+
+ if (tmptr->get_topology().find_node(host_id)) {
+ // If node is not in the topology there is no existsing address
+ // If there are two addresses for the same id the "other" one is existing
+ // If there is only one it is existing
+ if (ips.size() == 2) {
+ if (ips.erase(endpoint) == 0) {
+ on_internal_error(slogger, fmt::format("Gossiper has to ips {} for host id {} but non of them is {}", ips, endpoint, host_id));
+ }
+ }
+ existing = *ips.begin();
+ }
+
if (existing && *existing != endpoint) {
// This branch in taken when a node changes its IP address.

@@ -2420,11 +2435,12 @@ future<> storage_service::handle_state_normal(inet_address endpoint, gms::permit
// We do this after update_normal_tokens, allowing for tokens to be properly
// migrated to the new host_id.

- slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
- if (const auto old_host_id = tmptr->get_host_id_if_known(endpoint); old_host_id && *old_host_id != host_id) {
- // Replace with same IP scenario
- slogger.info("The IP {} previously owned host ID {}", endpoint, *old_host_id);
- replaced_id = *old_host_id;
+ auto peers = co_await _sys_ks.local().load_host_ids();
+ if (peers.contains(endpoint) && peers[endpoint] != host_id) {
+ replaced_id = peers[endpoint];
+ slogger.info("The IP {} previously owned host ID {}", endpoint, *replaced_id);
+ } else {
+ slogger.info("Host ID {} continues to be owned by {}", host_id, endpoint);
}
} else {
// This branch is taken if this node wasn't involved in node_ops
--
2.47.1

It is loading more messages.
0 new messages