ClickHouse/src/Coordination/KeeperStateManager.cpp

190 lines
6.9 KiB
C++
Raw Normal View History

2021-03-29 08:24:56 +00:00
#include <Coordination/KeeperStateManager.h>
#include <Coordination/Defines.h>
2021-02-11 09:17:57 +00:00
#include <Common/Exception.h>
2021-05-12 14:05:44 +00:00
#include <filesystem>
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;
}
2021-05-12 14:05:44 +00:00
namespace
{
2021-05-18 14:08:56 +00:00
std::string getLogsPathFromConfig(
const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, bool standalone_keeper)
2021-05-12 14:05:44 +00:00
{
/// the most specialized path
if (config.has(config_prefix + ".log_storage_path"))
return config.getString(config_prefix + ".log_storage_path");
if (config.has(config_prefix + ".storage_path"))
return std::filesystem::path{config.getString(config_prefix + ".storage_path")} / "logs";
2021-05-18 14:08:56 +00:00
if (standalone_keeper)
return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs";
else
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs";
2021-05-12 14:05:44 +00:00
}
}
2021-02-11 12:12:01 +00:00
2021-10-18 15:27:51 +00:00
KeeperServersConfiguration KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const
2021-01-13 10:32:20 +00:00
{
2021-10-18 15:27:51 +00:00
KeeperServersConfiguration result;
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
2021-10-18 15:27:51 +00:00
size_t total_servers = 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;
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);
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);
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
}
2021-10-18 15:27:51 +00:00
if (!result.config)
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
return result;
}
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
: my_server_id(server_id_)
, secure(false)
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, 5000, false, false))
{
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(
int server_id_,
const std::string & config_prefix_,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings,
bool standalone_keeper)
: my_server_id(server_id_)
, secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false))
, config_prefix(config_prefix_)
2021-10-19 12:00:26 +00:00
, configuration_wrapper(parseServersConfiguration(config))
2021-10-18 15:27:51 +00:00
, log_store(nuraft::cs_new<KeeperLogStore>(
getLogsPathFromConfig(config_prefix_, config, standalone_keeper),
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs))
{
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);
2021-02-16 19:02:18 +00:00
}
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::setClusterConfig(const ClusterConfigPtr & cluster_config)
{
2021-10-19 12:00:26 +00:00
configuration_wrapper.cluster_config = cluster_config;
2021-10-18 15:27:51 +00:00
}
2021-03-29 08:24:56 +00:00
void KeeperStateManager::flushLogStore()
2021-02-17 20:36:25 +00:00
{
log_store->flush();
}
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
{
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
}
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
{
nuraft::ptr<nuraft::buffer> buf = state.serialize();
server_state = nuraft::srv_state::deserialize(*buf);
2021-10-18 15:27:51 +00:00
}
ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const
{
2021-10-19 12:00:26 +00:00
auto new_configuration_wrapper = parseServersConfiguration(config);
if (new_configuration_wrapper.port != configuration_wrapper.port)
2021-10-18 15:27:51 +00:00
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot change port of already running RAFT server");
std::unordered_map<int, KeeperServerConfigPtr> new_ids, old_ids;
2021-10-19 12:00:26 +00:00
for (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 12:00:26 +00:00
for (auto old_server : configuration_wrapper.cluster_config->get_servers())
2021-10-18 15:27:51 +00:00
old_ids[old_server->get_id()] = old_server;
ConfigUpdateActions result;
2021-10-19 07:14:53 +00:00
for (auto [new_id, server_config] : new_ids)
{
if (!old_ids.count(new_id))
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config});
}
2021-10-18 15:27:51 +00:00
2021-10-19 12:00:26 +00:00
for (auto [old_id, server_config] : old_ids)
{
if (!new_ids.count(old_id))
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config});
}
/// 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 12:00:26 +00:00
for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers())
2021-10-18 15:27:51 +00:00
{
if (old_server->get_id() == new_server->get_id())
{
if (old_server->get_priority() != new_server->get_priority())
2021-10-19 12:00:26 +00:00
{
2021-10-18 15:27:51 +00:00
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server});
2021-10-19 12:00:26 +00:00
}
2021-10-18 15:27:51 +00:00
break;
}
}
}
return result;
}
2021-01-13 10:32:20 +00:00
}