ClickHouse/src/Coordination/KeeperStateManager.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

539 lines
18 KiB
C++
Raw Normal View History

2021-03-29 08:24:56 +00:00
#include <Coordination/KeeperStateManager.h>
2021-11-19 09:30:58 +00:00
2022-04-21 13:32:06 +00:00
#include <filesystem>
#include <Coordination/Defines.h>
2022-04-21 13:32:06 +00:00
#include <Common/DNSResolver.h>
2021-02-11 09:17:57 +00:00
#include <Common/Exception.h>
2022-03-02 19:02:02 +00:00
#include <Common/isLocalAddress.h>
2022-07-11 12:56:09 +00:00
#include <IO/ReadHelpers.h>
2023-05-22 17:51:58 +00:00
#include <IO/ReadBufferFromFile.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Disks/DiskLocal.h>
#include <Common/logger_useful.h>
#include "Coordination/CoordinationSettings.h"
2021-01-13 10:32:20 +00:00
namespace DB
{
2021-02-11 09:17:57 +00:00
namespace ErrorCodes
{
extern const int RAFT_ERROR;
extern const int CORRUPTED_DATA;
2021-02-11 09:17:57 +00:00
}
2022-03-02 19:02:02 +00:00
namespace
{
2023-06-01 14:39:01 +00:00
const std::string copy_lock_file = "STATE_COPY_LOCK";
2022-03-02 19:02:02 +00:00
bool isLocalhost(const std::string & hostname)
{
2022-03-03 10:29:43 +00:00
try
{
2022-03-04 12:14:38 +00:00
return isLocalAddress(DNSResolver::instance().resolveHost(hostname));
2022-03-03 10:29:43 +00:00
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
2022-03-04 12:14:38 +00:00
2022-03-03 10:29:43 +00:00
return false;
2022-03-02 19:02:02 +00:00
}
2022-03-07 11:13:37 +00:00
std::unordered_map<UInt64, std::string> getClientPorts(const Poco::Util::AbstractConfiguration & config)
{
2022-04-21 12:30:35 +00:00
using namespace std::string_literals;
static const std::array config_port_names = {
"keeper_server.tcp_port"s,
"keeper_server.tcp_port_secure"s,
"interserver_http_port"s,
"interserver_https_port"s,
"tcp_port"s,
"tcp_with_proxy_port"s,
"tcp_port_secure"s,
"mysql_port"s,
"postgresql_port"s,
"grpc_port"s,
"prometheus.port"s,
2022-03-07 11:13:37 +00:00
};
std::unordered_map<UInt64, std::string> ports;
for (const auto & config_port_name : config_port_names)
{
if (config.has(config_port_name))
ports[config.getUInt64(config_port_name)] = config_port_name;
}
return ports;
}
2022-03-02 19:02:02 +00:00
}
2022-03-02 19:37:59 +00:00
/// this function quite long because contains a lot of sanity checks in config:
/// 1. No duplicate endpoints
/// 2. No "localhost" or "127.0.0.1" or another local addresses mixed with normal addresses
2022-03-07 11:13:37 +00:00
/// 3. Raft internal port is not equal to any other port for client
2022-03-02 19:37:59 +00:00
/// 4. No duplicate IDs
/// 5. Our ID present in hostnames list
2022-04-21 13:32:06 +00:00
KeeperStateManager::KeeperConfigurationWrapper
KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us, bool enable_async_replication) const
2021-01-13 10:32:20 +00:00
{
2022-04-22 08:13:14 +00:00
const bool hostname_checks_enabled = config.getBool(config_prefix + ".hostname_checks_enabled", true);
2022-04-20 10:31:53 +00:00
2021-10-19 13:11:29 +00:00
KeeperConfigurationWrapper result;
2021-10-18 15:27:51 +00:00
result.cluster_config = std::make_shared<nuraft::cluster_config>();
2021-02-11 09:17:57 +00:00
Poco::Util::AbstractConfiguration::Keys keys;
2021-02-16 19:02:18 +00:00
config.keys(config_prefix + ".raft_configuration", keys);
2021-02-11 09:17:57 +00:00
2022-03-07 11:13:37 +00:00
auto client_ports = getClientPorts(config);
2022-03-02 19:37:59 +00:00
/// Sometimes (especially in cloud envs) users can provide incorrect
/// configuration with duplicated raft ids or endpoints. We check them
/// on config parsing stage and never commit to quorum.
std::unordered_map<std::string, int> check_duplicated_hostnames;
2021-10-18 15:27:51 +00:00
size_t total_servers = 0;
2022-04-21 13:32:06 +00:00
bool localhost_present = false;
2022-03-02 19:02:02 +00:00
std::string non_local_hostname;
2022-03-04 12:14:38 +00:00
size_t local_address_counter = 0;
2021-02-11 09:17:57 +00:00
for (const auto & server_key : keys)
{
2021-04-12 12:40:01 +00:00
if (!startsWith(server_key, "server"))
continue;
2021-02-16 19:02:18 +00:00
std::string full_prefix = config_prefix + ".raft_configuration." + server_key;
if (getMultipleValuesFromConfig(config, full_prefix, "id").size() > 1
|| getMultipleValuesFromConfig(config, full_prefix, "hostname").size() > 1
|| getMultipleValuesFromConfig(config, full_prefix, "port").size() > 1)
{
throw Exception(ErrorCodes::RAFT_ERROR, "Multiple <id> or <hostname> or <port> specified for a single <server>");
}
2021-10-18 15:27:51 +00:00
int new_server_id = config.getInt(full_prefix + ".id");
2021-02-11 09:17:57 +00:00
std::string hostname = config.getString(full_prefix + ".hostname");
int port = config.getInt(full_prefix + ".port");
bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true);
int32_t priority = config.getInt(full_prefix + ".priority", 1);
2021-02-11 10:25:10 +00:00
bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false);
if (client_ports.contains(port))
2022-03-02 19:37:59 +00:00
{
2022-04-21 13:32:06 +00:00
throw Exception(
ErrorCodes::RAFT_ERROR,
"Raft configuration contains hostname '{}' with port '{}' which is equal to '{}' in server configuration",
hostname,
port,
client_ports[port]);
2022-03-02 19:37:59 +00:00
}
2022-04-22 08:13:14 +00:00
if (hostname_checks_enabled)
2022-03-04 12:14:38 +00:00
{
2022-04-21 13:32:06 +00:00
if (hostname == "localhost")
2022-04-20 10:31:53 +00:00
{
2022-04-21 13:32:06 +00:00
localhost_present = true;
2022-04-20 10:31:53 +00:00
local_address_counter++;
}
else if (isLocalhost(hostname))
{
local_address_counter++;
}
else
{
non_local_hostname = hostname;
}
2022-03-04 12:14:38 +00:00
}
2022-03-02 19:02:02 +00:00
2021-02-11 10:25:10 +00:00
if (start_as_follower)
2021-10-18 15:27:51 +00:00
result.servers_start_as_followers.insert(new_server_id);
2021-02-11 09:17:57 +00:00
auto endpoint = hostname + ":" + std::to_string(port);
if (check_duplicated_hostnames.contains(endpoint))
{
2022-04-21 13:32:06 +00:00
throw Exception(
ErrorCodes::RAFT_ERROR,
"Raft config contains duplicate endpoints: "
"endpoint {} has been already added with id {}, but going to add it one more time with id {}",
endpoint,
check_duplicated_hostnames[endpoint],
new_server_id);
}
else
{
/// Fullscan to check duplicated ids
for (const auto & [id_endpoint, id] : check_duplicated_hostnames)
{
if (new_server_id == id)
2022-04-21 13:32:06 +00:00
throw Exception(
ErrorCodes::RAFT_ERROR,
"Raft config contains duplicate ids: id {} has been already added with endpoint {}, "
"but going to add it one more time with endpoint {}",
id,
id_endpoint,
endpoint);
}
check_duplicated_hostnames.emplace(endpoint, new_server_id);
}
2021-10-18 15:27:51 +00:00
auto peer_config = nuraft::cs_new<nuraft::srv_config>(new_server_id, 0, endpoint, "", !can_become_leader, priority);
if (my_server_id == new_server_id)
2021-02-11 09:17:57 +00:00
{
2021-10-18 15:27:51 +00:00
result.config = peer_config;
result.port = port;
2021-02-11 09:17:57 +00:00
}
2021-10-18 15:27:51 +00:00
result.cluster_config->get_servers().push_back(peer_config);
total_servers++;
2021-02-11 09:17:57 +00:00
}
result.cluster_config->set_async_replication(enable_async_replication);
2021-10-19 13:37:28 +00:00
if (!result.config && !allow_without_us)
2021-02-16 19:02:18 +00:00
throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id);
2021-02-11 10:25:10 +00:00
2021-10-18 15:27:51 +00:00
if (result.servers_start_as_followers.size() == total_servers)
2021-02-11 10:25:10 +00:00
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
2021-10-18 15:27:51 +00:00
2022-04-22 08:13:14 +00:00
if (hostname_checks_enabled)
2022-03-04 12:14:38 +00:00
{
2022-04-21 13:32:06 +00:00
if (localhost_present && !non_local_hostname.empty())
2022-04-20 10:31:53 +00:00
{
throw Exception(
ErrorCodes::RAFT_ERROR,
2022-04-21 13:32:06 +00:00
"Mixing 'localhost' and non-local hostnames ('{}') in raft_configuration is not allowed. "
"Different hosts can resolve 'localhost' to themselves so it's not allowed.",
non_local_hostname);
2022-04-20 10:31:53 +00:00
}
2022-03-04 12:14:38 +00:00
2022-04-20 10:31:53 +00:00
if (!non_local_hostname.empty() && local_address_counter > 1)
{
throw Exception(
ErrorCodes::RAFT_ERROR,
"Local address specified more than once ({} times) and non-local hostnames also exists ('{}') in raft_configuration. "
"Such configuration is not allowed because single host can vote multiple times.",
2022-04-21 13:32:06 +00:00
local_address_counter,
non_local_hostname);
2022-04-20 10:31:53 +00:00
}
2022-03-02 19:02:02 +00:00
}
2021-10-18 15:27:51 +00:00
return result;
}
2023-05-22 12:24:16 +00:00
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, KeeperContextPtr keeper_context_)
2022-07-10 19:10:27 +00:00
: my_server_id(server_id_)
, secure(false)
2023-05-22 12:24:16 +00:00
, log_store(nuraft::cs_new<KeeperLogStore>(
LogFileSettings{.force_sync = false, .compress_logs = false, .rotate_interval = 5000},
FlushSettings{},
2023-05-22 12:24:16 +00:00
keeper_context_))
, server_state_file_name("state")
, keeper_context(keeper_context_)
2022-07-10 19:10:27 +00:00
, logger(&Poco::Logger::get("KeeperStateManager"))
2021-10-18 15:27:51 +00:00
{
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
2021-10-19 12:00:26 +00:00
configuration_wrapper.cluster_config = nuraft::cs_new<nuraft::cluster_config>();
configuration_wrapper.port = port;
configuration_wrapper.config = peer_config;
configuration_wrapper.cluster_config->get_servers().push_back(peer_config);
2021-10-18 15:27:51 +00:00
}
KeeperStateManager::KeeperStateManager(
2021-10-27 12:26:42 +00:00
int my_server_id_,
2021-10-18 15:27:51 +00:00
const std::string & config_prefix_,
const std::string & server_state_file_name_,
2021-10-18 15:27:51 +00:00
const Poco::Util::AbstractConfiguration & config,
2023-05-22 12:24:16 +00:00
const CoordinationSettingsPtr & coordination_settings,
KeeperContextPtr keeper_context_)
2021-10-27 12:26:42 +00:00
: my_server_id(my_server_id_)
2021-10-27 15:21:26 +00:00
, secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false))
2021-10-18 15:27:51 +00:00
, config_prefix(config_prefix_)
, configuration_wrapper(parseServersConfiguration(config, false, coordination_settings->async_replication))
2021-10-18 15:27:51 +00:00
, log_store(nuraft::cs_new<KeeperLogStore>(
2023-01-26 09:44:37 +00:00
LogFileSettings
{
2023-09-08 12:45:01 +00:00
.force_sync = coordination_settings->force_sync,
.compress_logs = coordination_settings->compress_logs,
.rotate_interval = coordination_settings->rotate_log_storage_interval,
.max_size = coordination_settings->max_log_file_size,
.overallocate_size = coordination_settings->log_file_overallocate_size},
FlushSettings
{
2023-09-08 12:45:01 +00:00
.max_flush_batch_size = coordination_settings->max_flush_batch_size,
},
2023-05-22 12:24:16 +00:00
keeper_context_))
, server_state_file_name(server_state_file_name_)
2023-05-22 12:24:16 +00:00
, keeper_context(keeper_context_)
2022-07-10 15:01:38 +00:00
, logger(&Poco::Logger::get("KeeperStateManager"))
2021-10-18 15:27:51 +00:00
{
2021-01-13 10:32:20 +00:00
}
2021-04-08 14:17:57 +00:00
void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep)
2021-02-16 19:02:18 +00:00
{
2021-03-04 11:22:59 +00:00
log_store->init(last_commited_index, logs_to_keep);
2023-11-21 12:08:04 +00:00
log_store_initialized = true;
2021-02-16 19:02:18 +00:00
}
Replace exit() with abort() in case of NuRaft errors CI founds [1]: WARNING: ThreadSanitizer: data race (pid=99) Write of size 8 at 0x7b14002192b0 by thread T27: 12 std::__1::map<> 13 DB::OpenedFileCache::~OpenedFileCache() obj-x86_64-linux-gnu/../src/IO/OpenedFileCache.h:27:7 (clickhouse+0xac66de6) 14 cxa_at_exit_wrapper(void*) crtstuff.c (clickhouse+0xaa3646f) 15 decltype(*(std::__1::forward<nuraft::raft_server*&>(fp0)).*fp()) std::__1::__invoke<void ()(), nuraft::raft_server*&, void>() Previous read of size 8 at 0x7b14002192b0 by thread T37 (mutexes: write M732116415018761216): 4 DB::OpenedFileCache::get() obj-x86_64-linux-gnu/../src/IO/OpenedFileCache.h:47:37 (clickhouse+0xac66784) Thread T27 'nuraft_commit' (tid=193, running) created by main thread at: ... Thread T37 'MergeMutate' (tid=204, running) created by main thread at: ... But it also reports that the mutex was already destroyed: Mutex M732116415018761216 is already destroyed. The problem is that [nuraft can call `exit()`](https://github.com/ClickHouse-Extras/NuRaft/blob/1707a7572aa66ec5d0a2dbe2bf5effa3352e6b2d/src/handle_commit.cxx#L157-L158) which will call atexit handlers: 2022.02.17 22:54:03.495450 [ 193 ] {} <Error> RaftInstance: background committing thread encounter err Memory limit (total) exceeded: would use 56.56 GiB (attempt to allocate chunk of 8192 bytes), maximum: 55.82 GiB, exiting to protect the system [1]: https://s3.amazonaws.com/clickhouse-test-reports/33057/5a8cf3ac98808dadf125068a33ed9c622998a484/fuzzer_astfuzzertsan,actions//report.html Let's replace exit() with abort() to avoid this. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-19 10:00:35 +00:00
void KeeperStateManager::system_exit(const int /* exit_code */)
{
/// NuRaft itself calls exit() which will call atexit handlers
/// and this may lead to an issues in multi-threaded program.
///
/// Override this with abort().
abort();
}
2021-10-18 15:27:51 +00:00
ClusterConfigPtr KeeperStateManager::getLatestConfigFromLogStore() const
{
auto entry_with_change = log_store->getLatestConfigChange();
if (entry_with_change)
return ClusterConfig::deserialize(entry_with_change->get_buf());
return nullptr;
}
void KeeperStateManager::flushAndShutDownLogStore()
2021-02-17 20:36:25 +00:00
{
log_store->flushChangelogAndShutdown();
2021-02-17 20:36:25 +00:00
}
2021-03-29 08:24:56 +00:00
void KeeperStateManager::save_config(const nuraft::cluster_config & config)
2021-01-13 10:32:20 +00:00
{
2021-10-19 13:11:29 +00:00
std::lock_guard lock(configuration_wrapper_mutex);
2021-01-13 10:32:20 +00:00
nuraft::ptr<nuraft::buffer> buf = config.serialize();
2021-10-19 12:00:26 +00:00
configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf);
2021-01-13 10:32:20 +00:00
}
const String & KeeperStateManager::getOldServerStatePath()
2022-07-10 09:33:49 +00:00
{
2022-07-10 14:40:08 +00:00
static auto old_path = [this]
{
return server_state_file_name + "-OLD";
2022-07-10 09:33:49 +00:00
}();
return old_path;
}
2023-05-22 12:24:16 +00:00
DiskPtr KeeperStateManager::getStateFileDisk() const
{
return keeper_context->getStateFileDisk();
}
2022-07-11 12:56:09 +00:00
namespace
{
enum ServerStateVersion : uint8_t
{
V1 = 0
};
constexpr auto current_server_state_version = ServerStateVersion::V1;
}
2021-03-29 08:24:56 +00:00
void KeeperStateManager::save_state(const nuraft::srv_state & state)
2021-01-13 10:32:20 +00:00
{
2022-07-10 09:33:49 +00:00
const auto & old_path = getOldServerStatePath();
2023-05-22 12:24:16 +00:00
auto disk = getStateFileDisk();
if (disk->exists(server_state_file_name))
2023-06-01 14:39:01 +00:00
{
auto buf = disk->writeFile(copy_lock_file);
buf->finalize();
disk->copyFile(server_state_file_name, *disk, old_path);
disk->removeFile(copy_lock_file);
disk->removeFile(old_path);
}
2022-07-10 09:33:49 +00:00
auto server_state_file = disk->writeFile(server_state_file_name);
2022-07-10 09:33:49 +00:00
auto buf = state.serialize();
2022-07-10 15:01:38 +00:00
// calculate checksum
SipHash hash;
2022-07-11 12:56:09 +00:00
hash.update(current_server_state_version);
2022-07-10 15:01:38 +00:00
hash.update(reinterpret_cast<const char *>(buf->data_begin()), buf->size());
writeIntBinary(hash.get64(), *server_state_file);
2022-07-10 15:01:38 +00:00
writeIntBinary(static_cast<uint8_t>(current_server_state_version), *server_state_file);
2022-07-11 12:56:09 +00:00
server_state_file->write(reinterpret_cast<const char *>(buf->data_begin()), buf->size());
server_state_file->sync();
2023-06-01 14:39:01 +00:00
server_state_file->finalize();
2022-07-10 09:33:49 +00:00
disk->removeFileIfExists(old_path);
2022-07-10 09:33:49 +00:00
}
nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
{
2023-11-21 12:08:04 +00:00
chassert(log_store_initialized);
2022-07-10 09:33:49 +00:00
const auto & old_path = getOldServerStatePath();
2023-05-22 12:24:16 +00:00
auto disk = getStateFileDisk();
const auto try_read_file = [&](const auto & path) -> nuraft::ptr<nuraft::srv_state>
2022-07-10 09:33:49 +00:00
{
try
{
auto read_buf = disk->readFile(path);
auto content_size = read_buf->getFileSize();
2022-07-10 19:10:27 +00:00
if (content_size == 0)
return nullptr;
uint64_t read_checksum{0};
readIntBinary(read_checksum, *read_buf);
2022-07-10 15:01:38 +00:00
2022-07-11 12:56:09 +00:00
uint8_t version;
readIntBinary(version, *read_buf);
2022-07-11 12:56:09 +00:00
auto buffer_size = content_size - sizeof read_checksum - sizeof version;
2022-07-10 15:01:38 +00:00
auto state_buf = nuraft::buffer::alloc(buffer_size);
read_buf->readStrict(reinterpret_cast<char *>(state_buf->data_begin()), buffer_size);
2022-07-10 15:01:38 +00:00
SipHash hash;
2022-07-11 12:56:09 +00:00
hash.update(version);
2022-07-10 15:01:38 +00:00
hash.update(reinterpret_cast<const char *>(state_buf->data_begin()), state_buf->size());
if (read_checksum != hash.get64())
{
2023-01-16 23:11:59 +00:00
constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}";
#ifdef NDEBUG
LOG_ERROR(logger, error_format, path, hash.get64(), read_checksum);
2022-07-10 15:01:38 +00:00
return nullptr;
#else
throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, disk->getPath() + path, hash.get64(), read_checksum);
#endif
2022-07-10 15:01:38 +00:00
}
2022-07-10 09:33:49 +00:00
auto state = nuraft::srv_state::deserialize(*state_buf);
2023-05-24 07:36:39 +00:00
LOG_INFO(logger, "Read state from {}", fs::path(disk->getPath()) / path);
2022-07-10 09:33:49 +00:00
return state;
}
catch (const std::exception & e)
2022-07-10 09:33:49 +00:00
{
if (const auto * exception = dynamic_cast<const Exception *>(&e);
exception != nullptr && exception->code() == ErrorCodes::CORRUPTED_DATA)
{
throw;
}
LOG_ERROR(logger, "Failed to deserialize state from {}", disk->getPath() + path);
2022-07-10 09:33:49 +00:00
return nullptr;
}
};
if (disk->exists(server_state_file_name))
2022-07-10 09:33:49 +00:00
{
auto state = try_read_file(server_state_file_name);
2022-07-10 09:33:49 +00:00
if (state)
2022-07-11 12:56:09 +00:00
{
disk->removeFileIfExists(old_path);
2022-07-10 09:33:49 +00:00
return state;
2022-07-11 12:56:09 +00:00
}
disk->removeFile(server_state_file_name);
2022-07-10 09:33:49 +00:00
}
if (disk->exists(old_path))
2022-07-10 09:33:49 +00:00
{
2023-06-01 14:39:01 +00:00
if (disk->exists(copy_lock_file))
2022-07-11 12:56:09 +00:00
{
2023-06-01 14:39:01 +00:00
disk->removeFile(old_path);
disk->removeFile(copy_lock_file);
2022-07-11 12:56:09 +00:00
}
2023-06-01 14:39:01 +00:00
else
{
auto state = try_read_file(old_path);
if (state)
{
disk->moveFile(old_path, server_state_file_name);
return state;
}
disk->removeFile(old_path);
}
}
else if (disk->exists(copy_lock_file))
{
disk->removeFile(copy_lock_file);
2022-07-10 09:33:49 +00:00
}
2023-11-21 12:08:04 +00:00
if (log_store->next_slot() != 1)
LOG_ERROR(
logger,
"No state was read but Keeper contains data which indicates that the state file was lost. This is dangerous and can lead to "
"data loss.");
2022-07-10 09:33:49 +00:00
return nullptr;
2021-10-18 15:27:51 +00:00
}
ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(
const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) const
2021-10-18 15:27:51 +00:00
{
auto new_configuration_wrapper = parseServersConfiguration(config, true, coordination_settings->async_replication);
2021-10-18 15:27:51 +00:00
std::unordered_map<int, KeeperServerConfigPtr> new_ids, old_ids;
2021-10-19 19:42:40 +00:00
for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers())
2021-10-18 15:27:51 +00:00
new_ids[new_server->get_id()] = new_server;
2021-10-19 13:11:29 +00:00
{
std::lock_guard lock(configuration_wrapper_mutex);
2021-10-19 19:42:40 +00:00
for (const auto & old_server : configuration_wrapper.cluster_config->get_servers())
2021-10-19 13:11:29 +00:00
old_ids[old_server->get_id()] = old_server;
}
2021-10-18 15:27:51 +00:00
2023-04-20 13:26:02 +00:00
ClusterUpdateActions result;
2021-10-18 15:27:51 +00:00
2021-10-19 13:11:29 +00:00
/// First of all add new servers
2022-04-28 08:54:05 +00:00
for (const auto & [new_id, server_config] : new_ids)
2021-10-19 07:14:53 +00:00
{
2022-04-28 08:54:05 +00:00
auto old_server_it = old_ids.find(new_id);
if (old_server_it == old_ids.end())
2023-04-20 13:26:02 +00:00
result.emplace_back(AddRaftServer{RaftServerConfig{*server_config}});
2022-04-28 08:54:05 +00:00
else
{
const auto & old_endpoint = old_server_it->second->get_endpoint();
if (old_endpoint != server_config->get_endpoint())
{
LOG_WARNING(
&Poco::Logger::get("RaftConfiguration"),
"Config will be ignored because a server with ID {} is already present in the cluster on a different endpoint ({}). "
"The endpoint of the current servers should not be changed. For servers on a new endpoint, please use a new ID.",
new_id,
old_endpoint);
return {};
}
}
2021-10-19 07:14:53 +00:00
}
2021-10-18 15:27:51 +00:00
2021-10-19 13:11:29 +00:00
/// After that remove old ones
2021-10-19 12:00:26 +00:00
for (auto [old_id, server_config] : old_ids)
if (!new_ids.contains(old_id))
2023-04-20 13:26:02 +00:00
result.emplace_back(RemoveRaftServer{old_id});
2021-10-19 12:00:26 +00:00
2021-10-18 15:27:51 +00:00
{
2021-10-19 13:11:29 +00:00
std::lock_guard lock(configuration_wrapper_mutex);
/// And update priority if required
for (const auto & old_server : configuration_wrapper.cluster_config->get_servers())
2021-10-18 15:27:51 +00:00
{
2021-10-19 13:11:29 +00:00
for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers())
2021-10-18 15:27:51 +00:00
{
2021-10-19 13:11:29 +00:00
if (old_server->get_id() == new_server->get_id())
2021-10-19 12:00:26 +00:00
{
2021-10-19 13:11:29 +00:00
if (old_server->get_priority() != new_server->get_priority())
{
2023-04-20 13:26:02 +00:00
result.emplace_back(UpdateRaftServerPriority{
.id = new_server->get_id(),
.priority = new_server->get_priority()
});
2021-10-19 13:11:29 +00:00
}
break;
2021-10-19 12:00:26 +00:00
}
2021-10-18 15:27:51 +00:00
}
}
}
return result;
}
2021-01-13 10:32:20 +00:00
}