mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Review fixes
This commit is contained in:
parent
b390ebb083
commit
31181095e7
@ -16,7 +16,7 @@ services:
|
||||
target: /var/log/clickhouse-keeper
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir1:-}
|
||||
target: /var/lib/clickhouse/coordination
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "clickhouse keeper --config=/etc/clickhouse-keeper/keeper_config1.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
@ -46,7 +46,7 @@ services:
|
||||
target: /var/log/clickhouse-keeper
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir2:-}
|
||||
target: /var/lib/clickhouse/coordination
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "clickhouse keeper --config=/etc/clickhouse-keeper/keeper_config2.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
@ -76,7 +76,7 @@ services:
|
||||
target: /var/log/clickhouse-keeper
|
||||
- type: ${keeper_fs:-tmpfs}
|
||||
source: ${keeper_db_dir3:-}
|
||||
target: /var/lib/clickhouse/coordination
|
||||
target: /var/lib/clickhouse-keeper
|
||||
entrypoint: "clickhouse keeper --config=/etc/clickhouse-keeper/keeper_config3.xml --log-file=/var/log/clickhouse-keeper/clickhouse-keeper.log --errorlog-file=/var/log/clickhouse-keeper/clickhouse-keeper.err.log"
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
|
@ -290,7 +290,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
global_context = Context::createGlobal(shared_context.get());
|
||||
|
||||
global_context->makeGlobalContext();
|
||||
global_context->setApplicationType(Context::ApplicationType::SERVER);
|
||||
global_context->setApplicationType(Context::ApplicationType::KEEPER);
|
||||
|
||||
if (!config().has("keeper_server"))
|
||||
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Keeper configuration (<keeper_server> section) not found in config");
|
||||
@ -305,7 +305,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
else if (config().has("keeper_server.snapshot_storage_path"))
|
||||
path = config().getString("keeper_server.snapshot_storage_path");
|
||||
else
|
||||
path = std::filesystem::path{DBMS_DEFAULT_PATH} / "coordination/logs";
|
||||
path = std::filesystem::path{KEEPER_DEFAULT_PATH};
|
||||
|
||||
|
||||
/// Check that the process user id matches the owner of the data.
|
||||
@ -410,7 +410,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
global_context->shutdown();
|
||||
|
||||
LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish.");
|
||||
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
|
||||
int current_connections = 0;
|
||||
for (auto & server : *servers)
|
||||
{
|
||||
@ -427,9 +427,9 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
current_connections = waitServersToFinish(*servers, config().getInt("shutdown_wait_unfinished", 5));
|
||||
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections);
|
||||
LOG_INFO(log, "Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.", current_connections);
|
||||
else
|
||||
LOG_INFO(log, "Closed connections to servers for tables.");
|
||||
LOG_INFO(log, "Closed connections to Keeper.");
|
||||
|
||||
global_context->shutdownKeeperStorageDispatcher();
|
||||
|
||||
|
@ -60,7 +60,7 @@ void setSSLParams(nuraft::asio_service::options & asio_opts)
|
||||
}
|
||||
#endif
|
||||
|
||||
std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper)
|
||||
{
|
||||
/// the most specialized path
|
||||
if (config.has("keeper_server.snapshot_storage_path"))
|
||||
@ -69,7 +69,10 @@ std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration &
|
||||
if (config.has("keeper_server.storage_path"))
|
||||
return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots";
|
||||
|
||||
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots";
|
||||
if (standalone_keeper)
|
||||
return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots";
|
||||
else
|
||||
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots";
|
||||
}
|
||||
|
||||
}
|
||||
@ -79,14 +82,15 @@ KeeperServer::KeeperServer(
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_)
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
bool standalone_keeper)
|
||||
: server_id(server_id_)
|
||||
, coordination_settings(coordination_settings_)
|
||||
, state_machine(nuraft::cs_new<KeeperStateMachine>(
|
||||
responses_queue_, snapshots_queue_,
|
||||
getSnapshotsPathFromConfig(config),
|
||||
getSnapshotsPathFromConfig(config, standalone_keeper),
|
||||
coordination_settings))
|
||||
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings))
|
||||
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings, standalone_keeper))
|
||||
, log(&Poco::Logger::get("KeeperServer"))
|
||||
{
|
||||
if (coordination_settings->quorum_reads)
|
||||
|
@ -55,7 +55,8 @@ public:
|
||||
const CoordinationSettingsPtr & coordination_settings_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
ResponsesQueue & responses_queue_,
|
||||
SnapshotsQueue & snapshots_queue_);
|
||||
SnapshotsQueue & snapshots_queue_,
|
||||
bool standalone_keeper);
|
||||
|
||||
void startup();
|
||||
|
||||
|
@ -12,8 +12,8 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
std::string getLogsPathFromConfig(const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config)
|
||||
std::string getLogsPathFromConfig(
|
||||
const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, bool standalone_keeper)
|
||||
{
|
||||
/// the most specialized path
|
||||
if (config.has(config_prefix + ".log_storage_path"))
|
||||
@ -22,7 +22,10 @@ std::string getLogsPathFromConfig(const std::string & config_prefix, const Poco:
|
||||
if (config.has(config_prefix + ".storage_path"))
|
||||
return std::filesystem::path{config.getString(config_prefix + ".storage_path")} / "logs";
|
||||
|
||||
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs";
|
||||
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";
|
||||
}
|
||||
|
||||
}
|
||||
@ -42,11 +45,12 @@ KeeperStateManager::KeeperStateManager(
|
||||
int my_server_id_,
|
||||
const std::string & config_prefix,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const CoordinationSettingsPtr & coordination_settings)
|
||||
const CoordinationSettingsPtr & coordination_settings,
|
||||
bool standalone_keeper)
|
||||
: my_server_id(my_server_id_)
|
||||
, secure(config.getBool(config_prefix + ".raft_configuration.secure", false))
|
||||
, log_store(nuraft::cs_new<KeeperLogStore>(
|
||||
getLogsPathFromConfig(config_prefix, config),
|
||||
getLogsPathFromConfig(config_prefix, config, standalone_keeper),
|
||||
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync))
|
||||
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
|
||||
{
|
||||
|
@ -17,7 +17,8 @@ public:
|
||||
int server_id_,
|
||||
const std::string & config_prefix,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const CoordinationSettingsPtr & coordination_settings);
|
||||
const CoordinationSettingsPtr & coordination_settings,
|
||||
bool standalone_keeper);
|
||||
|
||||
KeeperStateManager(
|
||||
int server_id_,
|
||||
|
@ -234,7 +234,7 @@ bool KeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr
|
||||
return true;
|
||||
}
|
||||
|
||||
void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config)
|
||||
void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper)
|
||||
{
|
||||
LOG_DEBUG(log, "Initializing storage dispatcher");
|
||||
int myid = config.getInt("keeper_server.server_id");
|
||||
@ -246,7 +246,8 @@ void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration
|
||||
responses_thread = ThreadFromGlobalPool([this] { responseThread(); });
|
||||
snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); });
|
||||
|
||||
server = std::make_unique<KeeperServer>(myid, coordination_settings, config, responses_queue, snapshots_queue);
|
||||
server = std::make_unique<KeeperServer>(
|
||||
myid, coordination_settings, config, responses_queue, snapshots_queue, standalone_keeper);
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Waiting server to initialize");
|
||||
|
@ -86,7 +86,7 @@ private:
|
||||
public:
|
||||
KeeperStorageDispatcher();
|
||||
|
||||
void initialize(const Poco::Util::AbstractConfiguration & config);
|
||||
void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper);
|
||||
|
||||
void shutdown();
|
||||
|
||||
|
@ -98,6 +98,8 @@
|
||||
|
||||
#define DBMS_DEFAULT_PATH "/var/lib/clickhouse/"
|
||||
|
||||
#define KEEPER_DEFAULT_PATH "/var/lib/clickhouse-keeper/"
|
||||
|
||||
// more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html
|
||||
|
||||
/// Marks that extra information is sent to a shard. It could be any magic numbers.
|
||||
|
@ -314,8 +314,8 @@ struct ContextSharedPart
|
||||
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
|
||||
|
||||
#if USE_NURAFT
|
||||
mutable std::mutex nu_keeper_storage_dispatcher_mutex;
|
||||
mutable std::shared_ptr<KeeperStorageDispatcher> nu_keeper_storage_dispatcher;
|
||||
mutable std::mutex keeper_storage_dispatcher_mutex;
|
||||
mutable std::shared_ptr<KeeperStorageDispatcher> keeper_storage_dispatcher;
|
||||
#endif
|
||||
mutable std::mutex auxiliary_zookeepers_mutex;
|
||||
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
|
||||
@ -1678,16 +1678,16 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
|
||||
void Context::initializeKeeperStorageDispatcher() const
|
||||
{
|
||||
#if USE_NURAFT
|
||||
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
|
||||
std::lock_guard lock(shared->keeper_storage_dispatcher_mutex);
|
||||
|
||||
if (shared->nu_keeper_storage_dispatcher)
|
||||
if (shared->keeper_storage_dispatcher)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times");
|
||||
|
||||
const auto & config = getConfigRef();
|
||||
if (config.has("keeper_server"))
|
||||
{
|
||||
shared->nu_keeper_storage_dispatcher = std::make_shared<KeeperStorageDispatcher>();
|
||||
shared->nu_keeper_storage_dispatcher->initialize(config);
|
||||
shared->keeper_storage_dispatcher = std::make_shared<KeeperStorageDispatcher>();
|
||||
shared->keeper_storage_dispatcher->initialize(config, getApplicationType() == ApplicationType::KEEPER);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
@ -1695,22 +1695,22 @@ void Context::initializeKeeperStorageDispatcher() const
|
||||
#if USE_NURAFT
|
||||
std::shared_ptr<KeeperStorageDispatcher> & Context::getKeeperStorageDispatcher() const
|
||||
{
|
||||
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
|
||||
if (!shared->nu_keeper_storage_dispatcher)
|
||||
std::lock_guard lock(shared->keeper_storage_dispatcher_mutex);
|
||||
if (!shared->keeper_storage_dispatcher)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests");
|
||||
|
||||
return shared->nu_keeper_storage_dispatcher;
|
||||
return shared->keeper_storage_dispatcher;
|
||||
}
|
||||
#endif
|
||||
|
||||
void Context::shutdownKeeperStorageDispatcher() const
|
||||
{
|
||||
#if USE_NURAFT
|
||||
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
|
||||
if (shared->nu_keeper_storage_dispatcher)
|
||||
std::lock_guard lock(shared->keeper_storage_dispatcher_mutex);
|
||||
if (shared->keeper_storage_dispatcher)
|
||||
{
|
||||
shared->nu_keeper_storage_dispatcher->shutdown();
|
||||
shared->nu_keeper_storage_dispatcher.reset();
|
||||
shared->keeper_storage_dispatcher->shutdown();
|
||||
shared->keeper_storage_dispatcher.reset();
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
@ -734,7 +734,8 @@ public:
|
||||
{
|
||||
SERVER, /// The program is run as clickhouse-server daemon (default behavior)
|
||||
CLIENT, /// clickhouse-client
|
||||
LOCAL /// clickhouse-local
|
||||
LOCAL, /// clickhouse-local
|
||||
KEEPER, /// clickhouse-keeper (also daemon)
|
||||
};
|
||||
|
||||
ApplicationType getApplicationType() const;
|
||||
|
Loading…
Reference in New Issue
Block a user