2021-03-29 08:24:56 +00:00
|
|
|
#include <Coordination/KeeperStateManager.h>
|
2021-10-03 09:54:23 +00:00
|
|
|
#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-03-29 08:24:56 +00:00
|
|
|
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
|
2021-02-11 12:12:01 +00:00
|
|
|
: my_server_id(server_id_)
|
|
|
|
, my_port(port)
|
2021-04-12 12:25:52 +00:00
|
|
|
, secure(false)
|
2021-09-22 10:38:06 +00:00
|
|
|
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, 5000, false, false))
|
2021-02-11 12:12:01 +00:00
|
|
|
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
|
|
|
|
{
|
|
|
|
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
|
|
|
|
cluster_config->get_servers().push_back(peer_config);
|
|
|
|
}
|
|
|
|
|
2021-03-29 08:24:56 +00:00
|
|
|
KeeperStateManager::KeeperStateManager(
|
2021-02-11 09:17:57 +00:00
|
|
|
int my_server_id_,
|
|
|
|
const std::string & config_prefix,
|
2021-02-16 19:02:18 +00:00
|
|
|
const Poco::Util::AbstractConfiguration & config,
|
2021-05-18 14:08:56 +00:00
|
|
|
const CoordinationSettingsPtr & coordination_settings,
|
|
|
|
bool standalone_keeper)
|
2021-01-13 10:32:20 +00:00
|
|
|
: my_server_id(my_server_id_)
|
2021-04-12 12:40:01 +00:00
|
|
|
, secure(config.getBool(config_prefix + ".raft_configuration.secure", false))
|
2021-03-29 08:24:56 +00:00
|
|
|
, log_store(nuraft::cs_new<KeeperLogStore>(
|
2021-05-18 14:08:56 +00:00
|
|
|
getLogsPathFromConfig(config_prefix, config, standalone_keeper),
|
2021-09-21 14:29:05 +00:00
|
|
|
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs))
|
2021-01-13 10:32:20 +00:00
|
|
|
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
|
|
|
|
{
|
2021-02-16 19:02:18 +00:00
|
|
|
|
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-25 20:30:55 +00:00
|
|
|
total_servers = keys.size();
|
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-02-11 09:17:57 +00:00
|
|
|
int server_id = config.getInt(full_prefix + ".id");
|
|
|
|
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-04-12 12:25:52 +00:00
|
|
|
|
2021-02-11 10:25:10 +00:00
|
|
|
if (start_as_follower)
|
|
|
|
start_as_follower_servers.insert(server_id);
|
2021-02-11 09:17:57 +00:00
|
|
|
|
|
|
|
auto endpoint = hostname + ":" + std::to_string(port);
|
|
|
|
auto peer_config = nuraft::cs_new<nuraft::srv_config>(server_id, 0, endpoint, "", !can_become_leader, priority);
|
|
|
|
if (server_id == my_server_id)
|
|
|
|
{
|
|
|
|
my_server_config = peer_config;
|
|
|
|
my_port = port;
|
|
|
|
}
|
|
|
|
|
|
|
|
cluster_config->get_servers().push_back(peer_config);
|
|
|
|
}
|
2021-04-12 12:25:52 +00:00
|
|
|
|
2021-02-11 09:17:57 +00:00
|
|
|
if (!my_server_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
|
|
|
|
|
|
|
if (start_as_follower_servers.size() == cluster_config->get_servers().size())
|
|
|
|
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
|
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-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
|
|
|
{
|
|
|
|
// Just keep in memory in this example.
|
|
|
|
// Need to write to disk here, if want to make it durable.
|
|
|
|
nuraft::ptr<nuraft::buffer> buf = config.serialize();
|
|
|
|
cluster_config = nuraft::cluster_config::deserialize(*buf);
|
|
|
|
}
|
|
|
|
|
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
|
|
|
{
|
|
|
|
// Just keep in memory in this example.
|
|
|
|
// Need to write to disk here, if want to make it durable.
|
|
|
|
nuraft::ptr<nuraft::buffer> buf = state.serialize();
|
|
|
|
server_state = nuraft::srv_state::deserialize(*buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|