2021-05-12 10:39:07 +00:00
|
|
|
#include "Keeper.h"
|
|
|
|
|
2021-10-03 09:54:23 +00:00
|
|
|
#include <filesystem>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <pwd.h>
|
|
|
|
#include <Coordination/Defines.h>
|
2021-10-03 09:54:23 +00:00
|
|
|
#include <Core/ServerUUID.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <IO/UseSSL.h>
|
|
|
|
#include <Interpreters/DNSCacheUpdater.h>
|
2023-05-19 14:23:56 +00:00
|
|
|
#include <Server/waitServersToFinish.h>
|
2022-03-02 09:59:10 +00:00
|
|
|
#include <base/safeExit.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <base/scope_guard.h>
|
|
|
|
#include <sys/stat.h>
|
|
|
|
#include <Poco/Environment.h>
|
2021-10-03 09:54:23 +00:00
|
|
|
#include <Poco/Net/NetException.h>
|
|
|
|
#include <Poco/Net/TCPServer.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <Poco/Net/TCPServerParams.h>
|
2021-05-12 10:39:07 +00:00
|
|
|
#include <Poco/Util/HelpFormatter.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <Common/ClickHouseRevision.h>
|
|
|
|
#include <Common/Config/ConfigReloader.h>
|
|
|
|
#include <Common/DNSResolver.h>
|
|
|
|
#include <Common/ErrorHandlers.h>
|
|
|
|
#include <Common/assertProcessUserMatchesDataOwner.h>
|
|
|
|
#include <Common/getMultipleKeysFromConfig.h>
|
|
|
|
#include <Common/logger_useful.h>
|
|
|
|
#include <Common/makeSocketAddress.h>
|
2022-11-09 12:37:42 +00:00
|
|
|
|
2023-05-17 13:32:51 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
|
2021-10-27 12:26:42 +00:00
|
|
|
#include <Coordination/FourLetterCommand.h>
|
2022-11-09 12:37:42 +00:00
|
|
|
#include <Coordination/KeeperAsynchronousMetrics.h>
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2022-11-09 08:02:04 +00:00
|
|
|
#include <Server/HTTP/HTTPServer.h>
|
2022-11-09 12:37:42 +00:00
|
|
|
#include <Server/HTTPHandlerFactory.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <Server/TCPServer.h>
|
2022-11-09 08:02:04 +00:00
|
|
|
|
|
|
|
#include "Core/Defines.h"
|
2022-09-28 13:29:29 +00:00
|
|
|
#include "config.h"
|
2023-06-14 00:23:39 +00:00
|
|
|
#include "config_tools.h"
|
2023-07-11 10:04:43 +00:00
|
|
|
#include "config_version.h"
|
2023-06-14 00:23:39 +00:00
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
#if USE_SSL
|
|
|
|
# include <Poco/Net/Context.h>
|
|
|
|
# include <Poco/Net/SecureServerSocket.h>
|
|
|
|
#endif
|
|
|
|
|
2021-05-22 23:26:40 +00:00
|
|
|
#include <Server/KeeperTCPHandlerFactory.h>
|
2023-07-11 10:04:43 +00:00
|
|
|
#include <Server/ProtocolServerAdapter.h>
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2023-05-17 13:32:51 +00:00
|
|
|
#include <Disks/registerDisks.h>
|
|
|
|
|
2021-05-17 09:38:44 +00:00
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
int mainEntryClickHouseKeeper(int argc, char ** argv)
|
|
|
|
{
|
|
|
|
DB::Keeper app;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
return app.run(argc, argv);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
|
|
|
|
auto code = DB::getCurrentExceptionCode();
|
|
|
|
return code ? code : 1;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-05-02 17:41:42 +00:00
|
|
|
#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
|
2022-11-10 09:49:00 +00:00
|
|
|
|
2022-11-30 12:24:08 +00:00
|
|
|
// Weak symbols don't work correctly on Darwin
|
|
|
|
// so we have a stub implementation to avoid linker errors
|
2023-07-11 10:04:43 +00:00
|
|
|
void collectCrashLog(Int32, UInt64, const String &, const StackTrace &)
|
|
|
|
{
|
|
|
|
}
|
2022-11-10 09:49:00 +00:00
|
|
|
|
|
|
|
#endif
|
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NO_ELEMENTS_IN_CONFIG;
|
|
|
|
extern const int SUPPORT_IS_DISABLED;
|
|
|
|
extern const int NETWORK_ERROR;
|
2022-04-15 08:52:34 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2021-05-12 10:39:07 +00:00
|
|
|
}
|
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Net::SocketAddress
|
|
|
|
Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const
|
2021-05-12 10:39:07 +00:00
|
|
|
{
|
|
|
|
auto address = makeSocketAddress(host, port, &logger());
|
|
|
|
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false));
|
|
|
|
socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64));
|
|
|
|
|
|
|
|
return address;
|
|
|
|
}
|
|
|
|
|
|
|
|
void Keeper::createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const
|
|
|
|
{
|
|
|
|
/// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file.
|
|
|
|
if (!config().has(port_name))
|
|
|
|
return;
|
|
|
|
|
|
|
|
auto port = config().getInt(port_name);
|
|
|
|
try
|
|
|
|
{
|
|
|
|
func(port);
|
|
|
|
}
|
|
|
|
catch (const Poco::Exception &)
|
|
|
|
{
|
|
|
|
std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false);
|
|
|
|
|
|
|
|
if (listen_try)
|
|
|
|
{
|
2023-07-11 10:04:43 +00:00
|
|
|
LOG_WARNING(
|
|
|
|
&logger(),
|
|
|
|
"{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
|
2021-05-12 10:39:07 +00:00
|
|
|
"specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
|
|
|
|
"file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> ."
|
|
|
|
" Example for disabled IPv4: <listen_host>::</listen_host>",
|
|
|
|
message);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-01-23 13:16:14 +00:00
|
|
|
throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR);
|
2021-05-12 10:39:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void Keeper::uninitialize()
|
|
|
|
{
|
|
|
|
logger().information("shutting down");
|
|
|
|
BaseDaemon::uninitialize();
|
|
|
|
}
|
|
|
|
|
|
|
|
int Keeper::run()
|
|
|
|
{
|
|
|
|
if (config().hasOption("help"))
|
|
|
|
{
|
|
|
|
Poco::Util::HelpFormatter help_formatter(Keeper::options());
|
2023-07-11 10:04:43 +00:00
|
|
|
auto header_str = fmt::format(
|
|
|
|
"{0} [OPTION] [-- [ARG]...]\n"
|
2023-06-14 00:23:39 +00:00
|
|
|
#if ENABLE_CLICKHOUSE_KEEPER_CLIENT
|
2023-07-11 10:04:43 +00:00
|
|
|
"{0} client [OPTION]\n"
|
2023-06-14 00:23:39 +00:00
|
|
|
#endif
|
2023-07-11 10:04:43 +00:00
|
|
|
"positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010",
|
|
|
|
commandName());
|
2021-05-12 10:39:07 +00:00
|
|
|
help_formatter.setHeader(header_str);
|
|
|
|
help_formatter.format(std::cout);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
if (config().hasOption("version"))
|
|
|
|
{
|
2021-05-12 13:04:34 +00:00
|
|
|
std::cout << DBMS_NAME << " keeper version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
|
2021-05-12 10:39:07 +00:00
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
return Application::run(); // NOLINT
|
|
|
|
}
|
|
|
|
|
|
|
|
void Keeper::initialize(Poco::Util::Application & self)
|
|
|
|
{
|
|
|
|
BaseDaemon::initialize(self);
|
|
|
|
logger().information("starting up");
|
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
LOG_INFO(
|
|
|
|
&logger(),
|
|
|
|
"OS Name = {}, OS Version = {}, OS Architecture = {}",
|
2021-05-12 10:39:07 +00:00
|
|
|
Poco::Environment::osName(),
|
|
|
|
Poco::Environment::osVersion(),
|
|
|
|
Poco::Environment::osArchitecture());
|
|
|
|
}
|
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
std::string Keeper::getDefaultConfigFileName() const
|
2021-05-12 10:39:07 +00:00
|
|
|
{
|
2021-05-12 13:04:34 +00:00
|
|
|
return "keeper_config.xml";
|
2021-05-12 10:39:07 +00:00
|
|
|
}
|
|
|
|
|
2022-04-15 08:52:34 +00:00
|
|
|
void Keeper::handleCustomArguments(const std::string & arg, [[maybe_unused]] const std::string & value) // NOLINT
|
2022-04-08 07:18:18 +00:00
|
|
|
{
|
|
|
|
if (arg == "force-recovery")
|
|
|
|
{
|
|
|
|
assert(value.empty());
|
2022-04-14 12:07:33 +00:00
|
|
|
config().setBool("keeper_server.force_recovery", true);
|
2022-04-08 07:18:18 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid argument {} provided", arg);
|
|
|
|
}
|
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
void Keeper::defineOptions(Poco::Util::OptionSet & options)
|
|
|
|
{
|
2023-07-11 10:04:43 +00:00
|
|
|
options.addOption(Poco::Util::Option("help", "h", "show help and exit").required(false).repeatable(false).binding("help"));
|
|
|
|
options.addOption(Poco::Util::Option("version", "V", "show version and exit").required(false).repeatable(false).binding("version"));
|
2021-05-12 10:39:07 +00:00
|
|
|
options.addOption(
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Util::Option(
|
|
|
|
"force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum")
|
2021-05-12 10:39:07 +00:00
|
|
|
.required(false)
|
|
|
|
.repeatable(false)
|
2023-07-11 10:04:43 +00:00
|
|
|
.noArgument()
|
|
|
|
.callback(Poco::Util::OptionCallback<Keeper>(this, &Keeper::handleCustomArguments)));
|
2021-05-12 10:39:07 +00:00
|
|
|
BaseDaemon::defineOptions(options);
|
|
|
|
}
|
|
|
|
|
2023-05-22 12:24:16 +00:00
|
|
|
namespace
|
2022-11-09 08:02:04 +00:00
|
|
|
{
|
2023-05-22 12:24:16 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
struct KeeperHTTPContext : public IHTTPContext
|
2022-11-09 08:02:04 +00:00
|
|
|
{
|
2023-07-11 10:04:43 +00:00
|
|
|
explicit KeeperHTTPContext(ContextPtr context_) : context(std::move(context_)) { }
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxHstsAge() const override { return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); }
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxUriSize() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); }
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxFields() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); }
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxFieldNameSize() const override
|
|
|
|
{
|
|
|
|
return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 128 * 1024);
|
|
|
|
}
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxFieldValueSize() const override
|
|
|
|
{
|
|
|
|
return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 128 * 1024);
|
|
|
|
}
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
uint64_t getMaxChunkSize() const override
|
|
|
|
{
|
|
|
|
return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB);
|
|
|
|
}
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Timespan getReceiveTimeout() const override
|
|
|
|
{
|
|
|
|
return {context->getConfigRef().getInt64("keeper_server.http_receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0};
|
|
|
|
}
|
2022-11-09 14:51:41 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Timespan getSendTimeout() const override
|
|
|
|
{
|
|
|
|
return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0};
|
|
|
|
}
|
2022-11-09 08:02:04 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
ContextPtr context;
|
|
|
|
};
|
|
|
|
|
|
|
|
HTTPContextPtr httpContext()
|
|
|
|
{
|
|
|
|
return std::make_shared<KeeperHTTPContext>(Context::getGlobalContextInstance());
|
|
|
|
}
|
2023-05-22 12:24:16 +00:00
|
|
|
|
2022-11-09 08:02:04 +00:00
|
|
|
}
|
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
int Keeper::main(const std::vector<std::string> & /*args*/)
|
2022-11-18 12:22:55 +00:00
|
|
|
try
|
2021-05-12 10:39:07 +00:00
|
|
|
{
|
|
|
|
Poco::Logger * log = &logger();
|
|
|
|
|
|
|
|
UseSSL use_ssl;
|
|
|
|
|
|
|
|
MainThreadStatus::getInstance();
|
|
|
|
|
|
|
|
#if !defined(NDEBUG) || !defined(__OPTIMIZE__)
|
2021-05-12 13:04:34 +00:00
|
|
|
LOG_WARNING(log, "Keeper was built in debug mode. It will work slowly.");
|
2021-05-12 10:39:07 +00:00
|
|
|
#endif
|
|
|
|
|
|
|
|
#if defined(SANITIZER)
|
2021-05-12 13:04:34 +00:00
|
|
|
LOG_WARNING(log, "Keeper was built with sanitizer. It will work slowly.");
|
2021-05-12 10:39:07 +00:00
|
|
|
#endif
|
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
if (!config().has("keeper_server"))
|
|
|
|
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Keeper configuration (<keeper_server> section) not found in config");
|
|
|
|
|
2021-05-12 14:05:44 +00:00
|
|
|
std::string path;
|
|
|
|
|
|
|
|
if (config().has("keeper_server.storage_path"))
|
|
|
|
path = config().getString("keeper_server.storage_path");
|
|
|
|
else if (config().has("keeper_server.log_storage_path"))
|
2021-08-25 16:15:56 +00:00
|
|
|
path = std::filesystem::path(config().getString("keeper_server.log_storage_path")).parent_path();
|
2021-05-12 14:05:44 +00:00
|
|
|
else if (config().has("keeper_server.snapshot_storage_path"))
|
2021-08-25 16:15:56 +00:00
|
|
|
path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path();
|
2021-05-12 14:05:44 +00:00
|
|
|
else
|
2021-05-18 14:08:56 +00:00
|
|
|
path = std::filesystem::path{KEEPER_DEFAULT_PATH};
|
2021-05-12 14:05:44 +00:00
|
|
|
|
2023-01-30 14:26:57 +00:00
|
|
|
std::filesystem::create_directories(path);
|
2021-05-12 14:05:44 +00:00
|
|
|
|
|
|
|
/// Check that the process user id matches the owner of the data.
|
2023-07-11 10:04:43 +00:00
|
|
|
assertProcessUserMatchesDataOwner(path, [&](const std::string & message) { LOG_WARNING(log, fmt::runtime(message)); });
|
2021-05-12 14:05:44 +00:00
|
|
|
|
2021-08-17 13:24:14 +00:00
|
|
|
DB::ServerUUID::load(path + "/uuid", log);
|
2021-08-16 18:30:53 +00:00
|
|
|
|
2021-10-19 14:29:49 +00:00
|
|
|
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
|
|
|
|
|
2021-11-12 13:24:47 +00:00
|
|
|
GlobalThreadPool::initialize(
|
|
|
|
config().getUInt("max_thread_pool_size", 100),
|
|
|
|
config().getUInt("max_thread_pool_free_size", 1000),
|
2023-07-11 10:04:43 +00:00
|
|
|
config().getUInt("thread_pool_queue_size", 10000));
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
static ServerErrorHandler error_handler;
|
|
|
|
Poco::ErrorHandler::set(&error_handler);
|
|
|
|
|
|
|
|
/// Initialize DateLUT early, to not interfere with running time of first query.
|
|
|
|
LOG_DEBUG(log, "Initializing DateLUT.");
|
2023-04-12 10:47:05 +00:00
|
|
|
DateLUT::serverTimezoneInstance();
|
|
|
|
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
/// Don't want to use DNS cache
|
|
|
|
DNSResolver::instance().setDisableCacheFlag();
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
|
2022-11-09 12:37:42 +00:00
|
|
|
std::mutex servers_lock;
|
|
|
|
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
|
|
|
|
|
2023-05-17 13:32:51 +00:00
|
|
|
auto shared_context = Context::createShared();
|
|
|
|
auto global_context = Context::createGlobal(shared_context.get());
|
|
|
|
|
|
|
|
global_context->makeGlobalContext();
|
|
|
|
global_context->setPath(path);
|
|
|
|
global_context->setRemoteHostFilter(config());
|
|
|
|
|
2023-05-22 12:24:16 +00:00
|
|
|
if (config().has("macros"))
|
|
|
|
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
|
|
|
|
|
2023-05-17 13:32:51 +00:00
|
|
|
registerDisks(/*global_skip_access_check=*/false);
|
|
|
|
|
2022-11-09 12:37:42 +00:00
|
|
|
/// This object will periodically calculate some metrics.
|
|
|
|
KeeperAsynchronousMetrics async_metrics(
|
2023-05-22 12:24:16 +00:00
|
|
|
global_context,
|
2022-11-09 12:37:42 +00:00
|
|
|
config().getUInt("asynchronous_metrics_update_period_s", 1),
|
|
|
|
[&]() -> std::vector<ProtocolServerMetrics>
|
|
|
|
{
|
|
|
|
std::vector<ProtocolServerMetrics> metrics;
|
|
|
|
|
|
|
|
std::lock_guard lock(servers_lock);
|
|
|
|
metrics.reserve(servers->size());
|
|
|
|
for (const auto & server : *servers)
|
|
|
|
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
|
|
|
|
return metrics;
|
2023-07-11 10:04:43 +00:00
|
|
|
});
|
2022-11-09 12:37:42 +00:00
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
|
|
|
|
|
|
|
|
bool listen_try = config().getBool("listen_try", false);
|
|
|
|
if (listen_hosts.empty())
|
|
|
|
{
|
|
|
|
listen_hosts.emplace_back("::1");
|
|
|
|
listen_hosts.emplace_back("127.0.0.1");
|
|
|
|
listen_try = true;
|
|
|
|
}
|
|
|
|
|
2021-10-18 09:13:24 +00:00
|
|
|
/// Initialize keeper RAFT. Do nothing if no keeper_server in config.
|
2023-05-22 12:24:16 +00:00
|
|
|
global_context->initializeKeeperDispatcher(/* start_async = */ true);
|
|
|
|
FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher());
|
2022-03-03 20:27:46 +00:00
|
|
|
|
2023-07-11 10:04:43 +00:00
|
|
|
auto config_getter = [&]() -> const Poco::Util::AbstractConfiguration & { return global_context->getConfigRef(); };
|
2021-10-27 12:26:42 +00:00
|
|
|
|
2023-03-28 12:15:28 +00:00
|
|
|
auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
|
|
|
|
auto tcp_send_timeout = config().getInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC);
|
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
for (const auto & listen_host : listen_hosts)
|
|
|
|
{
|
|
|
|
/// TCP Keeper
|
|
|
|
const char * port_name = "keeper_server.tcp_port";
|
2023-07-11 10:04:43 +00:00
|
|
|
createServer(
|
|
|
|
listen_host,
|
|
|
|
port_name,
|
|
|
|
listen_try,
|
|
|
|
[&](UInt16 port)
|
|
|
|
{
|
|
|
|
Poco::Net::ServerSocket socket;
|
|
|
|
auto address = socketBindListen(socket, listen_host, port);
|
|
|
|
socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0});
|
|
|
|
socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0});
|
|
|
|
servers->emplace_back(
|
|
|
|
listen_host,
|
|
|
|
port_name,
|
|
|
|
"Keeper (tcp): " + address.toString(),
|
|
|
|
std::make_unique<TCPServer>(
|
|
|
|
new KeeperTCPHandlerFactory(
|
|
|
|
config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, false),
|
|
|
|
server_pool,
|
|
|
|
socket));
|
|
|
|
});
|
2021-05-12 13:04:34 +00:00
|
|
|
|
|
|
|
const char * secure_port_name = "keeper_server.tcp_port_secure";
|
2023-07-11 10:04:43 +00:00
|
|
|
createServer(
|
|
|
|
listen_host,
|
|
|
|
secure_port_name,
|
|
|
|
listen_try,
|
|
|
|
[&](UInt16 port)
|
|
|
|
{
|
2021-05-12 10:39:07 +00:00
|
|
|
#if USE_SSL
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Net::SecureServerSocket socket;
|
|
|
|
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
|
|
|
socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0});
|
|
|
|
socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0});
|
|
|
|
servers->emplace_back(
|
|
|
|
listen_host,
|
|
|
|
secure_port_name,
|
|
|
|
"Keeper with secure protocol (tcp_secure): " + address.toString(),
|
|
|
|
std::make_unique<TCPServer>(
|
|
|
|
new KeeperTCPHandlerFactory(
|
|
|
|
config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, true),
|
|
|
|
server_pool,
|
|
|
|
socket));
|
2021-05-12 10:39:07 +00:00
|
|
|
#else
|
2023-07-11 10:04:43 +00:00
|
|
|
UNUSED(port);
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::SUPPORT_IS_DISABLED,
|
|
|
|
"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.");
|
2021-05-12 10:39:07 +00:00
|
|
|
#endif
|
2023-07-11 10:04:43 +00:00
|
|
|
});
|
2022-11-09 08:02:04 +00:00
|
|
|
|
|
|
|
const auto & config = config_getter();
|
2023-03-28 12:15:28 +00:00
|
|
|
auto http_context = httpContext();
|
2022-11-09 08:02:04 +00:00
|
|
|
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
|
|
|
|
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
2023-03-28 12:15:28 +00:00
|
|
|
http_params->setTimeout(http_context->getReceiveTimeout());
|
2022-11-09 08:02:04 +00:00
|
|
|
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
|
|
|
|
|
|
|
/// Prometheus (if defined and not setup yet with http_port)
|
|
|
|
port_name = "prometheus.port";
|
2023-07-11 10:04:43 +00:00
|
|
|
createServer(
|
|
|
|
listen_host,
|
|
|
|
port_name,
|
|
|
|
listen_try,
|
|
|
|
[&, my_http_context = std::move(http_context)](UInt16 port) mutable
|
|
|
|
{
|
|
|
|
Poco::Net::ServerSocket socket;
|
|
|
|
auto address = socketBindListen(socket, listen_host, port);
|
|
|
|
socket.setReceiveTimeout(my_http_context->getReceiveTimeout());
|
|
|
|
socket.setSendTimeout(my_http_context->getSendTimeout());
|
|
|
|
servers->emplace_back(
|
|
|
|
listen_host,
|
|
|
|
port_name,
|
|
|
|
"Prometheus: http://" + address.toString(),
|
|
|
|
std::make_unique<HTTPServer>(
|
|
|
|
std::move(my_http_context),
|
|
|
|
createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"),
|
|
|
|
server_pool,
|
|
|
|
socket,
|
|
|
|
http_params));
|
|
|
|
});
|
2021-05-12 13:04:34 +00:00
|
|
|
}
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
for (auto & server : *servers)
|
2021-10-22 07:15:34 +00:00
|
|
|
{
|
2021-05-12 10:39:07 +00:00
|
|
|
server.start();
|
2021-10-22 07:15:34 +00:00
|
|
|
LOG_INFO(log, "Listening for {}", server.getDescription());
|
|
|
|
}
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2022-11-09 12:37:42 +00:00
|
|
|
async_metrics.start();
|
|
|
|
|
2021-10-19 14:29:49 +00:00
|
|
|
zkutil::EventPtr unused_event = std::make_shared<Poco::Event>();
|
|
|
|
zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; });
|
|
|
|
/// ConfigReloader have to strict parameters which are redundant in our case
|
|
|
|
auto main_config_reloader = std::make_unique<ConfigReloader>(
|
|
|
|
config_path,
|
2023-07-11 10:04:43 +00:00
|
|
|
std::vector{{include_from_path}},
|
2021-10-19 14:29:49 +00:00
|
|
|
config().getString("path", ""),
|
|
|
|
std::move(unused_cache),
|
|
|
|
unused_event,
|
|
|
|
[&](ConfigurationPtr config, bool /* initial_loading */)
|
|
|
|
{
|
|
|
|
if (config->has("keeper_server"))
|
2023-05-22 12:24:16 +00:00
|
|
|
global_context->updateKeeperConfiguration(*config);
|
2021-10-19 14:29:49 +00:00
|
|
|
},
|
2023-07-11 10:04:43 +00:00
|
|
|
/* already_loaded = */ false); /// Reload it right now (initial loading)
|
2021-10-19 14:29:49 +00:00
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
SCOPE_EXIT({
|
|
|
|
LOG_INFO(log, "Shutting down.");
|
2021-10-19 14:29:49 +00:00
|
|
|
main_config_reloader.reset();
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2022-11-09 12:37:42 +00:00
|
|
|
async_metrics.stop();
|
|
|
|
|
2021-05-18 14:08:56 +00:00
|
|
|
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t current_connections = 0;
|
2021-05-12 10:39:07 +00:00
|
|
|
for (auto & server : *servers)
|
|
|
|
{
|
|
|
|
server.stop();
|
|
|
|
current_connections += server.currentConnections();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (current_connections)
|
|
|
|
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
|
|
|
|
else
|
|
|
|
LOG_INFO(log, "Closed all listening sockets.");
|
|
|
|
|
|
|
|
if (current_connections > 0)
|
2023-06-27 18:50:40 +00:00
|
|
|
current_connections = waitServersToFinish(*servers, servers_lock, config().getInt("shutdown_wait_unfinished", 5));
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
if (current_connections)
|
2023-07-11 10:04:43 +00:00
|
|
|
LOG_INFO(
|
|
|
|
log,
|
|
|
|
"Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.",
|
|
|
|
current_connections);
|
2021-05-12 10:39:07 +00:00
|
|
|
else
|
2021-05-18 14:08:56 +00:00
|
|
|
LOG_INFO(log, "Closed connections to Keeper.");
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2023-05-22 12:24:16 +00:00
|
|
|
global_context->shutdownKeeperDispatcher();
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
|
|
|
|
server_pool.joinAll();
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Destroyed global context.");
|
|
|
|
|
|
|
|
if (current_connections)
|
|
|
|
{
|
|
|
|
LOG_INFO(log, "Will shutdown forcefully.");
|
2022-03-02 09:59:10 +00:00
|
|
|
safeExit(0);
|
2021-05-12 10:39:07 +00:00
|
|
|
}
|
|
|
|
});
|
|
|
|
|
|
|
|
|
|
|
|
buildLoggers(config(), logger());
|
2021-10-19 14:29:49 +00:00
|
|
|
main_config_reloader->start();
|
2021-05-12 10:39:07 +00:00
|
|
|
|
|
|
|
LOG_INFO(log, "Ready for connections.");
|
|
|
|
|
|
|
|
waitForTerminationRequest();
|
|
|
|
|
|
|
|
return Application::EXIT_OK;
|
|
|
|
}
|
2022-11-18 12:22:55 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// Poco does not provide stacktrace.
|
|
|
|
tryLogCurrentException("Application");
|
|
|
|
throw;
|
|
|
|
}
|
2021-05-12 10:39:07 +00:00
|
|
|
|
2021-05-12 13:04:34 +00:00
|
|
|
|
|
|
|
void Keeper::logRevision() const
|
|
|
|
{
|
2023-07-11 10:04:43 +00:00
|
|
|
Poco::Logger::root().information(
|
|
|
|
"Starting ClickHouse Keeper " + std::string{VERSION_STRING} + "(revision : "
|
|
|
|
+ std::to_string(ClickHouseRevision::getVersionRevision()) + ", git hash: " + (git_hash.empty() ? "<unknown>" : git_hash)
|
|
|
|
+ ", build id: " + (build_id.empty() ? "<unknown>" : build_id) + ")" + ", PID " + std::to_string(getpid()));
|
2021-05-12 13:04:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-05-12 10:39:07 +00:00
|
|
|
}
|