ClickHouse/src/Coordination/KeeperStateManager.h

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

138 lines
4.2 KiB
C++
Raw Normal View History

2021-01-13 10:32:20 +00:00
#pragma once
#include <Core/Types.h>
#include <string>
2021-03-29 08:24:56 +00:00
#include <Coordination/KeeperLogStore.h>
2021-02-16 19:02:18 +00:00
#include <Coordination/CoordinationSettings.h>
2021-10-27 23:10:39 +00:00
#include <libnuraft/nuraft.hxx>
2021-02-11 09:17:57 +00:00
#include <Poco/Util/AbstractConfiguration.h>
2022-04-07 09:25:01 +00:00
#include "Coordination/KeeperStateMachine.h"
2023-04-20 13:26:02 +00:00
#include "Coordination/RaftServerConfig.h"
2021-10-18 15:27:51 +00:00
#include <Coordination/KeeperSnapshotManager.h>
2021-01-13 10:32:20 +00:00
namespace DB
{
2021-10-18 15:27:51 +00:00
using KeeperServerConfigPtr = nuraft::ptr<nuraft::srv_config>;
2021-10-19 13:11:29 +00:00
/// Responsible for managing our and cluster configuration
2021-03-29 08:24:56 +00:00
class KeeperStateManager : public nuraft::state_mgr
2021-01-13 10:32:20 +00:00
{
public:
2021-03-29 08:24:56 +00:00
KeeperStateManager(
2021-02-11 09:17:57 +00:00
int server_id_,
2021-10-18 15:27:51 +00:00
const std::string & config_prefix_,
2023-05-22 12:24:16 +00:00
const std::string & server_state_file_name_,
2021-02-16 19:02:18 +00:00
const Poco::Util::AbstractConfiguration & config,
2023-05-22 12:24:16 +00:00
KeeperContextPtr keeper_context_);
2021-01-13 10:32:20 +00:00
2021-10-19 13:11:29 +00:00
/// Constructor for tests
2021-03-29 08:24:56 +00:00
KeeperStateManager(
2021-02-11 12:12:01 +00:00
int server_id_,
const std::string & host,
2021-02-16 19:02:18 +00:00
int port,
2023-05-22 12:24:16 +00:00
KeeperContextPtr keeper_context_);
2021-02-16 19:02:18 +00:00
2021-04-08 14:17:57 +00:00
void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep);
2021-02-11 12:12:01 +00:00
/// Flush logstore and call shutdown of background thread
void flushAndShutDownLogStore();
2021-02-17 20:36:25 +00:00
2021-10-19 13:11:29 +00:00
/// Called on server start, in our case we don't use any separate logic for load
nuraft::ptr<nuraft::cluster_config> load_config() override
{
std::lock_guard lock(configuration_wrapper_mutex);
return configuration_wrapper.cluster_config;
}
2021-01-13 10:32:20 +00:00
2021-10-19 13:11:29 +00:00
/// Save cluster config (i.e. nodes, their priorities and so on)
2021-01-13 10:32:20 +00:00
void save_config(const nuraft::cluster_config & config) override;
void save_state(const nuraft::srv_state & state) override;
2022-07-10 09:33:49 +00:00
nuraft::ptr<nuraft::srv_state> read_state() override;
2021-01-13 10:32:20 +00:00
nuraft::ptr<nuraft::log_store> load_log_store() override { return log_store; }
2021-10-18 15:27:51 +00:00
int32_t server_id() override { return my_server_id; }
2021-01-13 10:32:20 +00:00
2023-04-20 13:26:02 +00:00
nuraft::ptr<nuraft::srv_config> get_srv_config() const
{
std::lock_guard lk(configuration_wrapper_mutex);
return configuration_wrapper.config;
}
2021-01-13 10:32:20 +00:00
void system_exit(const int exit_code) override; /// NOLINT
2021-01-13 10:32:20 +00:00
2021-10-19 13:11:29 +00:00
int getPort() const
{
std::lock_guard lock(configuration_wrapper_mutex);
return configuration_wrapper.port;
}
2021-02-11 09:17:57 +00:00
2021-02-11 10:25:10 +00:00
bool shouldStartAsFollower() const
{
2021-10-19 13:11:29 +00:00
std::lock_guard lock(configuration_wrapper_mutex);
2022-04-05 06:27:03 +00:00
return configuration_wrapper.servers_start_as_followers.contains(my_server_id);
2021-02-11 10:25:10 +00:00
}
bool isSecure() const
{
return secure;
}
2021-03-29 08:24:56 +00:00
nuraft::ptr<KeeperLogStore> getLogStore() const { return log_store; }
2021-10-19 13:11:29 +00:00
uint64_t getTotalServers() const
{
std::lock_guard lock(configuration_wrapper_mutex);
return configuration_wrapper.cluster_config->get_servers().size();
}
2021-10-18 15:27:51 +00:00
2021-10-19 13:11:29 +00:00
/// Read all log entries in log store from the begging and return latest config (with largest log_index)
2021-10-18 15:27:51 +00:00
ClusterConfigPtr getLatestConfigFromLogStore() const;
2023-04-20 13:26:02 +00:00
// TODO (myrrc) This should be removed once "reconfig" is stabilized
ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) const;
2021-01-13 10:32:20 +00:00
private:
const String & getOldServerStatePath();
2022-07-10 09:33:49 +00:00
2023-05-22 12:24:16 +00:00
DiskPtr getStateFileDisk() const;
2021-11-19 09:30:58 +00:00
/// Wrapper struct for Keeper cluster config. We parse this
/// info from XML files.
struct KeeperConfigurationWrapper
{
/// Our port
int port;
/// Our config
KeeperServerConfigPtr config;
/// Servers id's to start as followers
std::unordered_set<int> servers_start_as_followers;
/// Cluster config
ClusterConfigPtr cluster_config;
};
2021-01-13 10:32:20 +00:00
int my_server_id;
bool secure;
2021-10-18 15:27:51 +00:00
std::string config_prefix;
2021-10-19 12:00:26 +00:00
2021-10-19 13:11:29 +00:00
mutable std::mutex configuration_wrapper_mutex;
2023-04-20 13:26:02 +00:00
KeeperConfigurationWrapper configuration_wrapper TSA_GUARDED_BY(configuration_wrapper_mutex);
2021-10-19 13:11:29 +00:00
2023-11-21 12:08:04 +00:00
bool log_store_initialized = false;
2021-03-29 08:24:56 +00:00
nuraft::ptr<KeeperLogStore> log_store;
2022-07-10 09:33:49 +00:00
const String server_state_file_name;
2021-10-18 15:27:51 +00:00
2023-05-22 12:24:16 +00:00
KeeperContextPtr keeper_context;
2024-01-23 17:04:50 +00:00
LoggerPtr logger;
2022-07-10 15:01:38 +00:00
2022-04-11 06:41:46 +00:00
public:
2021-10-19 13:11:29 +00:00
/// Parse configuration from xml config.
KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us, bool enable_async_replication) const;
2021-01-13 10:32:20 +00:00
};
}