Merge branch 'master' into kvm-clock

This commit is contained in:
Sergei Trifonov 2022-09-13 00:17:09 +02:00 committed by GitHub
commit 85e43566b6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 229 additions and 97 deletions

View File

@ -40,6 +40,7 @@
#include <Common/getMappedArea.h> #include <Common/getMappedArea.h>
#include <Common/remapExecutable.h> #include <Common/remapExecutable.h>
#include <Common/TLDListsHolder.h> #include <Common/TLDListsHolder.h>
#include <Common/Config/AbstractConfigurationComparison.h>
#include <Core/ServerUUID.h> #include <Core/ServerUUID.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
@ -1270,6 +1271,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
CertificateReloader::instance().tryLoad(*config); CertificateReloader::instance().tryLoad(*config);
#endif #endif
ProfileEvents::increment(ProfileEvents::MainConfigLoads); ProfileEvents::increment(ProfileEvents::MainConfigLoads);
/// Must be the last.
latest_config = config;
}, },
/* already_loaded = */ false); /// Reload it right now (initial loading) /* already_loaded = */ false); /// Reload it right now (initial loading)
@ -1887,7 +1891,7 @@ void Server::createServers(
port_name, port_name,
"http://" + address.toString(), "http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); context(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
}); });
/// HTTPS /// HTTPS
@ -1904,7 +1908,7 @@ void Server::createServers(
port_name, port_name,
"https://" + address.toString(), "https://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); context(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
#else #else
UNUSED(port); UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
@ -2029,7 +2033,7 @@ void Server::createServers(
port_name, port_name,
"Prometheus: http://" + address.toString(), "Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); context(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
}); });
} }
@ -2050,7 +2054,7 @@ void Server::createServers(
"replica communication (interserver): http://" + address.toString(), "replica communication (interserver): http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), context(),
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
server_pool, server_pool,
socket, socket,
http_params)); http_params));
@ -2070,7 +2074,7 @@ void Server::createServers(
"secure replica communication (interserver): https://" + address.toString(), "secure replica communication (interserver): https://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), context(),
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
server_pool, server_pool,
socket, socket,
http_params)); http_params));
@ -2112,13 +2116,24 @@ void Server::updateServers(
std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)"));
Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config();
for (auto & server : servers) for (auto & server : servers)
{ {
if (!server.isStopping()) if (!server.isStopping())
{ {
bool has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); bool has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end();
bool has_port = !config.getString(server.getPortName(), "").empty(); bool has_port = !config.getString(server.getPortName(), "").empty();
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber())
/// NOTE: better to compare using getPortName() over using
/// dynamic_cast<> since HTTPServer is also used for prometheus and
/// internal replication communications.
bool is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port";
bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers");
if (force_restart)
LOG_TRACE(log, "<http_handlers> had been changed, will reload {}", server.getDescription());
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart)
{ {
server.stop(); server.stop();
LOG_INFO(log, "Stopped listening for {}", server.getDescription()); LOG_INFO(log, "Stopped listening for {}", server.getDescription());

View File

@ -67,6 +67,9 @@ protected:
private: private:
ContextMutablePtr global_context; ContextMutablePtr global_context;
/// Updated/recent config, to compare http_handlers
ConfigurationPtr latest_config;
Poco::Net::SocketAddress socketBindListen( Poco::Net::SocketAddress socketBindListen(
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
Poco::Net::ServerSocket & socket, Poco::Net::ServerSocket & socket,

View File

@ -248,6 +248,7 @@ BackupCoordinationReplicatedTables::getMutations(const String & table_shared_id,
return {}; return {};
std::vector<MutationInfo> res; std::vector<MutationInfo> res;
res.reserve(table_info.mutations.size());
for (const auto & [mutation_id, mutation_entry] : table_info.mutations) for (const auto & [mutation_id, mutation_entry] : table_info.mutations)
res.emplace_back(MutationInfo{mutation_id, mutation_entry}); res.emplace_back(MutationInfo{mutation_id, mutation_entry});
return res; return res;

View File

@ -2974,7 +2974,7 @@ const IHostContextPtr & Context::getHostContext() const
} }
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager() std::shared_ptr<ActionLocksManager> Context::getActionLocksManager() const
{ {
auto lock = getLock(); auto lock = getLock();

View File

@ -936,7 +936,7 @@ public:
bool applyDeletedMask() const { return apply_deleted_mask; } bool applyDeletedMask() const { return apply_deleted_mask; }
void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; } void setApplyDeletedMask(bool apply) { apply_deleted_mask = apply; }
ActionLocksManagerPtr getActionLocksManager(); ActionLocksManagerPtr getActionLocksManager() const;
enum class ApplicationType enum class ApplicationType
{ {

View File

@ -180,30 +180,41 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
{ {
for (auto & elem : DatabaseCatalog::instance().getDatabases()) for (auto & elem : DatabaseCatalog::instance().getDatabases())
{ {
for (auto iterator = elem.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) startStopActionInDatabase(action_type, start, elem.first, elem.second, getContext(), log);
{
StoragePtr table = iterator->table();
if (!table)
continue;
if (!access->isGranted(required_access_type, elem.first, iterator->name()))
{
LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), elem.first, iterator->name());
continue;
}
if (start)
{
manager->remove(table, action_type);
table->onActionLockRemove(action_type);
}
else
manager->add(table, action_type);
}
} }
} }
} }
void InterpreterSystemQuery::startStopActionInDatabase(StorageActionBlockType action_type, bool start,
const String & database_name, const DatabasePtr & database,
const ContextPtr & local_context, Poco::Logger * log)
{
auto manager = local_context->getActionLocksManager();
auto access = local_context->getAccess();
auto required_access_type = getRequiredAccessType(action_type);
for (auto iterator = database->getTablesIterator(local_context); iterator->isValid(); iterator->next())
{
StoragePtr table = iterator->table();
if (!table)
continue;
if (!access->isGranted(required_access_type, database_name, iterator->name()))
{
LOG_INFO(log, "Access {} denied, skipping {}.{}", toString(required_access_type), database_name, iterator->name());
continue;
}
if (start)
{
manager->remove(table, action_type);
table->onActionLockRemove(action_type);
}
else
manager->add(table, action_type);
}
}
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery")) : WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery"))

View File

@ -16,6 +16,9 @@ namespace DB
class Context; class Context;
class AccessRightsElements; class AccessRightsElements;
class ASTSystemQuery; class ASTSystemQuery;
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
/** Implement various SYSTEM queries. /** Implement various SYSTEM queries.
@ -37,6 +40,10 @@ public:
BlockIO execute() override; BlockIO execute() override;
static void startStopActionInDatabase(StorageActionBlockType action_type, bool start,
const String & database_name, const DatabasePtr & database,
const ContextPtr & local_context, Poco::Logger * log);
private: private:
ASTPtr query_ptr; ASTPtr query_ptr;
Poco::Logger * log = nullptr; Poco::Logger * log = nullptr;

View File

@ -5,6 +5,7 @@
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateQuery.h> #include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterSystemQuery.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/loadMetadata.h> #include <Interpreters/loadMetadata.h>
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
@ -32,6 +33,14 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
namespace ActionLocks
{
extern StorageActionBlockType PartsMerge;
extern StorageActionBlockType PartsFetch;
extern StorageActionBlockType PartsSend;
extern StorageActionBlockType DistributedSend;
}
static void executeCreateQuery( static void executeCreateQuery(
const String & query, const String & query,
ContextMutablePtr context, ContextMutablePtr context,
@ -327,9 +336,24 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons
} }
auto local_context = Context::createCopy(context); auto local_context = Context::createCopy(context);
/// We have to stop background operations that may lock table for share to avoid DEADLOCK_AVOIDED error
/// on moving tables from Ordinary database. Server has not started to accept connections yet,
/// so there are no user queries, only background operations
LOG_INFO(log, "Will stop background operations to be able to rename tables in Ordinary database {}", database_name);
static const auto actions_to_stop = {
ActionLocks::PartsMerge, ActionLocks::PartsFetch, ActionLocks::PartsSend, ActionLocks::DistributedSend
};
for (const auto & action : actions_to_stop)
InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ false, database_name, database, context, log);
local_context->setSetting("check_table_dependencies", false); local_context->setSetting("check_table_dependencies", false);
convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name); convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name);
LOG_INFO(log, "Will start background operations after renaming tables in database {}", database_name);
for (const auto & action : actions_to_stop)
InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ true, database_name, database, context, log);
auto new_database = DatabaseCatalog::instance().getDatabase(database_name); auto new_database = DatabaseCatalog::instance().getDatabase(database_name);
UUID db_uuid = new_database->getUUID(); UUID db_uuid = new_database->getUUID();
std::vector<UUID> tables_uuids; std::vector<UUID> tables_uuids;

View File

@ -1158,18 +1158,20 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo
return predefined_query; return predefined_query;
} }
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{ {
auto query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
std::optional<String> content_type_override; std::optional<String> content_type_override;
if (server.config().has(config_prefix + ".handler.content_type")) if (config.has(config_prefix + ".handler.content_type"))
content_type_override = server.config().getString(config_prefix + ".handler.content_type"); content_type_override = config.getString(config_prefix + ".handler.content_type");
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>( auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(
server, std::move(query_param_name), std::move(content_type_override)); server, std::move(query_param_name), std::move(content_type_override));
factory->addFiltersFromConfig(server.config(), config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }
@ -1197,23 +1199,23 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression)
return compiled_regex; return compiled_regex;
} }
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix) HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{ {
Poco::Util::AbstractConfiguration & configuration = server.config(); if (!config.has(config_prefix + ".handler.query"))
if (!configuration.has(config_prefix + ".handler.query"))
throw Exception("There is no path '" + config_prefix + ".handler.query' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); throw Exception("There is no path '" + config_prefix + ".handler.query' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
std::string predefined_query = configuration.getString(config_prefix + ".handler.query"); std::string predefined_query = config.getString(config_prefix + ".handler.query");
NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query); NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query);
std::unordered_map<String, CompiledRegexPtr> headers_name_with_regex; std::unordered_map<String, CompiledRegexPtr> headers_name_with_regex;
Poco::Util::AbstractConfiguration::Keys headers_name; Poco::Util::AbstractConfiguration::Keys headers_name;
configuration.keys(config_prefix + ".headers", headers_name); config.keys(config_prefix + ".headers", headers_name);
for (const auto & header_name : headers_name) for (const auto & header_name : headers_name)
{ {
auto expression = configuration.getString(config_prefix + ".headers." + header_name); auto expression = config.getString(config_prefix + ".headers." + header_name);
if (!startsWith(expression, "regex:")) if (!startsWith(expression, "regex:"))
continue; continue;
@ -1225,14 +1227,14 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
} }
std::optional<String> content_type_override; std::optional<String> content_type_override;
if (configuration.has(config_prefix + ".handler.content_type")) if (config.has(config_prefix + ".handler.content_type"))
content_type_override = configuration.getString(config_prefix + ".handler.content_type"); content_type_override = config.getString(config_prefix + ".handler.content_type");
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory; std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;
if (configuration.has(config_prefix + ".url")) if (config.has(config_prefix + ".url"))
{ {
auto url_expression = configuration.getString(config_prefix + ".url"); auto url_expression = config.getString(config_prefix + ".url");
if (startsWith(url_expression, "regex:")) if (startsWith(url_expression, "regex:"))
url_expression = url_expression.substr(6); url_expression = url_expression.substr(6);
@ -1247,7 +1249,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
std::move(regex), std::move(regex),
std::move(headers_name_with_regex), std::move(headers_name_with_regex),
std::move(content_type_override)); std::move(content_type_override));
factory->addFiltersFromConfig(configuration, config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }
} }
@ -1259,7 +1261,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
CompiledRegexPtr{}, CompiledRegexPtr{},
std::move(headers_name_with_regex), std::move(headers_name_with_regex),
std::move(content_type_override)); std::move(content_type_override));
factory->addFiltersFromConfig(configuration, config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }

View File

@ -5,7 +5,7 @@
#include <Access/Credentials.h> #include <Access/Credentials.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Poco/Util/LayeredConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include "HTTPHandler.h" #include "HTTPHandler.h"
#include "NotFoundHandler.h" #include "NotFoundHandler.h"
@ -27,7 +27,11 @@ namespace ErrorCodes
} }
static void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server); static void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server);
static void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server, AsynchronousMetrics & async_metrics); static void addDefaultHandlersFactory(
HTTPRequestHandlerFactoryMain & factory,
IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics);
HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_)
: log(&Poco::Logger::get(name_)), name(name_) : log(&Poco::Logger::get(name_)), name(name_)
@ -59,37 +63,41 @@ std::unique_ptr<HTTPRequestHandler> HTTPRequestHandlerFactoryMain::createRequest
} }
static inline auto createHandlersFactoryFromConfig( static inline auto createHandlersFactoryFromConfig(
IServer & server, const std::string & name, const String & prefix, AsynchronousMetrics & async_metrics) IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & name,
const String & prefix,
AsynchronousMetrics & async_metrics)
{ {
auto main_handler_factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name); auto main_handler_factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys keys;
server.config().keys(prefix, keys); config.keys(prefix, keys);
for (const auto & key : keys) for (const auto & key : keys)
{ {
if (key == "defaults") if (key == "defaults")
{ {
addDefaultHandlersFactory(*main_handler_factory, server, async_metrics); addDefaultHandlersFactory(*main_handler_factory, server, config, async_metrics);
} }
else if (startsWith(key, "rule")) else if (startsWith(key, "rule"))
{ {
const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); const auto & handler_type = config.getString(prefix + "." + key + ".handler.type", "");
if (handler_type.empty()) if (handler_type.empty())
throw Exception("Handler type in config is not specified here: " + prefix + "." + key + ".handler.type", throw Exception("Handler type in config is not specified here: " + prefix + "." + key + ".handler.type",
ErrorCodes::INVALID_CONFIG_PARAMETER); ErrorCodes::INVALID_CONFIG_PARAMETER);
if (handler_type == "static") if (handler_type == "static")
main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix + "." + key)); main_handler_factory->addHandler(createStaticHandlerFactory(server, config, prefix + "." + key));
else if (handler_type == "dynamic_query_handler") else if (handler_type == "dynamic_query_handler")
main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix + "." + key)); main_handler_factory->addHandler(createDynamicHandlerFactory(server, config, prefix + "." + key));
else if (handler_type == "predefined_query_handler") else if (handler_type == "predefined_query_handler")
main_handler_factory->addHandler(createPredefinedHandlerFactory(server, prefix + "." + key)); main_handler_factory->addHandler(createPredefinedHandlerFactory(server, config, prefix + "." + key));
else if (handler_type == "prometheus") else if (handler_type == "prometheus")
main_handler_factory->addHandler(createPrometheusHandlerFactory(server, async_metrics, prefix + "." + key)); main_handler_factory->addHandler(createPrometheusHandlerFactory(server, config, async_metrics, prefix + "." + key));
else if (handler_type == "replicas_status") else if (handler_type == "replicas_status")
main_handler_factory->addHandler(createReplicasStatusHandlerFactory(server, prefix + "." + key)); main_handler_factory->addHandler(createReplicasStatusHandlerFactory(server, config, prefix + "." + key));
else else
throw Exception("Unknown handler type '" + handler_type + "' in config here: " + prefix + "." + key + ".handler.type", throw Exception("Unknown handler type '" + handler_type + "' in config here: " + prefix + "." + key + ".handler.type",
ErrorCodes::INVALID_CONFIG_PARAMETER); ErrorCodes::INVALID_CONFIG_PARAMETER);
@ -103,16 +111,16 @@ static inline auto createHandlersFactoryFromConfig(
} }
static inline HTTPRequestHandlerFactoryPtr static inline HTTPRequestHandlerFactoryPtr
createHTTPHandlerFactory(IServer & server, const std::string & name, AsynchronousMetrics & async_metrics) createHTTPHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics)
{ {
if (server.config().has("http_handlers")) if (config.has("http_handlers"))
{ {
return createHandlersFactoryFromConfig(server, name, "http_handlers", async_metrics); return createHandlersFactoryFromConfig(server, config, name, "http_handlers", async_metrics);
} }
else else
{ {
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name); auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
addDefaultHandlersFactory(*factory, server, async_metrics); addDefaultHandlersFactory(*factory, server, config, async_metrics);
return factory; return factory;
} }
} }
@ -129,18 +137,18 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I
return factory; return factory;
} }
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name) HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, AsynchronousMetrics & async_metrics, const std::string & name)
{ {
if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory")
return createHTTPHandlerFactory(server, name, async_metrics); return createHTTPHandlerFactory(server, config, name, async_metrics);
else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory")
return createInterserverHTTPHandlerFactory(server, name); return createInterserverHTTPHandlerFactory(server, name);
else if (name == "PrometheusHandler-factory") else if (name == "PrometheusHandler-factory")
{ {
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name); auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>( auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics)); server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
handler->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics")); handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
handler->allowGetAndHeadRequest(); handler->allowGetAndHeadRequest();
factory->addHandler(handler); factory->addHandler(handler);
return factory; return factory;
@ -185,7 +193,11 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS
factory.addHandler(js_handler); factory.addHandler(js_handler);
} }
void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer & server, AsynchronousMetrics & async_metrics) void addDefaultHandlersFactory(
HTTPRequestHandlerFactoryMain & factory,
IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics)
{ {
addCommonDefaultHandlersFactory(factory, server); addCommonDefaultHandlersFactory(factory, server);
@ -195,11 +207,11 @@ void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer
/// We check that prometheus handler will be served on current (default) port. /// We check that prometheus handler will be served on current (default) port.
/// Otherwise it will be created separately, see createHandlerFactory(...). /// Otherwise it will be created separately, see createHandlerFactory(...).
if (server.config().has("prometheus") && server.config().getInt("prometheus.port", 0) == 0) if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0)
{ {
auto prometheus_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>( auto prometheus_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics)); server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
prometheus_handler->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics")); prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
prometheus_handler->allowGetAndHeadRequest(); prometheus_handler->allowGetAndHeadRequest();
factory.addHandler(prometheus_handler); factory.addHandler(prometheus_handler);
} }

View File

@ -7,7 +7,7 @@
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Poco/Util/LayeredConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
namespace DB namespace DB
{ {
@ -63,7 +63,7 @@ public:
}; };
} }
void addFiltersFromConfig(Poco::Util::AbstractConfiguration & config, const std::string & prefix) void addFiltersFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & prefix)
{ {
Poco::Util::AbstractConfiguration::Keys filters_type; Poco::Util::AbstractConfiguration::Keys filters_type;
config.keys(prefix, filters_type); config.keys(prefix, filters_type);
@ -126,16 +126,34 @@ private:
std::function<std::unique_ptr<HTTPRequestHandler> ()> creator; std::function<std::unique_ptr<HTTPRequestHandler> ()> creator;
}; };
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server, const std::string & config_prefix); HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr HTTPRequestHandlerFactoryPtr
createPrometheusHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & config_prefix); createPrometheusHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & config_prefix);
/// @param server - used in handlers to check IServer::isCancelled()
/// @param config - not the same as server.config(), since it can be newer
/// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true)
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & name);
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name);
} }

View File

@ -39,7 +39,7 @@ static inline bool checkExpression(std::string_view match_str, const std::pair<S
return match_str == expression.first; return match_str == expression.first;
} }
static inline auto methodsFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT static inline auto methodsFilter(const Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
{ {
std::vector<String> methods; std::vector<String> methods;
Poco::StringTokenizer tokenizer(config.getString(config_path), ","); Poco::StringTokenizer tokenizer(config.getString(config_path), ",");
@ -63,7 +63,7 @@ static inline auto getExpression(const std::string & expression)
return std::make_pair(expression, compiled_regex); return std::make_pair(expression, compiled_regex);
} }
static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT static inline auto urlFilter(const Poco::Util::AbstractConfiguration & config, const std::string & config_path) /// NOLINT
{ {
return [expression = getExpression(config.getString(config_path))](const HTTPServerRequest & request) return [expression = getExpression(config.getString(config_path))](const HTTPServerRequest & request)
{ {
@ -74,7 +74,7 @@ static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const s
}; };
} }
static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, const std::string & prefix) /// NOLINT static inline auto headersFilter(const Poco::Util::AbstractConfiguration & config, const std::string & prefix) /// NOLINT
{ {
std::unordered_map<String, std::pair<String, CompiledRegexPtr>> headers_expression; std::unordered_map<String, std::pair<String, CompiledRegexPtr>> headers_expression;
Poco::Util::AbstractConfiguration::Keys headers_name; Poco::Util::AbstractConfiguration::Keys headers_name;

View File

@ -42,11 +42,14 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
} }
HTTPRequestHandlerFactoryPtr HTTPRequestHandlerFactoryPtr
createPrometheusHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & config_prefix) createPrometheusHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & config_prefix)
{ {
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>( auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(server.config(), config_prefix + ".handler", async_metrics)); server, PrometheusMetricsWriter(config, config_prefix + ".handler", async_metrics));
factory->addFiltersFromConfig(server.config(), config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }

View File

@ -108,10 +108,12 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
} }
} }
HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server, const std::string & config_prefix) HTTPRequestHandlerFactoryPtr createReplicasStatusHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{ {
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<ReplicasStatusHandler>>(server); auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<ReplicasStatusHandler>>(server);
factory->addFiltersFromConfig(server.config(), config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }

View File

@ -169,15 +169,17 @@ StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & exp
{ {
} }
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server, const std::string & config_prefix) HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{ {
int status = server.config().getInt(config_prefix + ".handler.status", 200); int status = config.getInt(config_prefix + ".handler.status", 200);
std::string response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n");
std::string response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>( auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(
server, std::move(response_content), std::move(status), std::move(response_content_type)); server, std::move(response_content), std::move(status), std::move(response_content_type));
factory->addFiltersFromConfig(server.config(), config_prefix); factory->addFiltersFromConfig(config, config_prefix);
return factory; return factory;
} }

View File

@ -6,4 +6,5 @@
<mysql_port from_zk="/clickhouse/ports/mysql" replace="replace" /> <mysql_port from_zk="/clickhouse/ports/mysql" replace="replace" />
<postgresql_port from_zk="/clickhouse/ports/postgresql" replace="replace" /> <postgresql_port from_zk="/clickhouse/ports/postgresql" replace="replace" />
<grpc_port from_zk="/clickhouse/ports/grpc" replace="replace" /> <grpc_port from_zk="/clickhouse/ports/grpc" replace="replace" />
</yandex> <http_handlers from_zk="/clickhouse/http_handlers" replace="replace" />
</yandex>

View File

@ -25,7 +25,7 @@ cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance( instance = cluster.add_instance(
"instance", "instance",
main_configs=[ main_configs=[
"configs/ports_from_zk.xml", "configs/overrides_from_zk.xml",
"configs/ssl_conf.xml", "configs/ssl_conf.xml",
"configs/dhparam.pem", "configs/dhparam.pem",
"configs/server.crt", "configs/server.crt",
@ -58,7 +58,7 @@ import clickhouse_grpc_pb2_grpc
@pytest.fixture(name="cluster", scope="module") @pytest.fixture(name="cluster", scope="module")
def fixture_cluster(): def fixture_cluster():
try: try:
cluster.add_zookeeper_startup_command(configure_ports_from_zk) cluster.add_zookeeper_startup_command(configure_from_zk)
cluster.start() cluster.start()
yield cluster yield cluster
finally: finally:
@ -128,7 +128,7 @@ def grpc_query(channel, query_text):
return result.output.decode() return result.output.decode()
def configure_ports_from_zk(zk, querier=None): def configure_from_zk(zk, querier=None):
default_config = [ default_config = [
("/clickhouse/listen_hosts", b"<listen_host>0.0.0.0</listen_host>"), ("/clickhouse/listen_hosts", b"<listen_host>0.0.0.0</listen_host>"),
("/clickhouse/ports/tcp", b"9000"), ("/clickhouse/ports/tcp", b"9000"),
@ -136,6 +136,7 @@ def configure_ports_from_zk(zk, querier=None):
("/clickhouse/ports/mysql", b"9004"), ("/clickhouse/ports/mysql", b"9004"),
("/clickhouse/ports/postgresql", b"9005"), ("/clickhouse/ports/postgresql", b"9005"),
("/clickhouse/ports/grpc", b"9100"), ("/clickhouse/ports/grpc", b"9100"),
("/clickhouse/http_handlers", b"<defaults/>"),
] ]
for path, value in default_config: for path, value in default_config:
if querier is not None: if querier is not None:
@ -182,7 +183,7 @@ def default_client(cluster, zk, restore_via_http=False):
yield client yield client
finally: finally:
querier = instance.http_query if restore_via_http else client.query querier = instance.http_query if restore_via_http else client.query
configure_ports_from_zk(zk, querier) configure_from_zk(zk, querier)
def test_change_tcp_port(cluster, zk): def test_change_tcp_port(cluster, zk):
@ -320,7 +321,7 @@ def test_change_listen_host(cluster, zk):
assert localhost_client.query("SELECT 1") == "1\n" assert localhost_client.query("SELECT 1") == "1\n"
finally: finally:
with sync_loaded_config(localhost_client.query): with sync_loaded_config(localhost_client.query):
configure_ports_from_zk(zk) configure_from_zk(zk)
# This is a regression test for the case when the clickhouse-server was waiting # This is a regression test for the case when the clickhouse-server was waiting
@ -371,7 +372,7 @@ def test_reload_via_client(cluster, zk):
while True: while True:
try: try:
with sync_loaded_config(localhost_client.query): with sync_loaded_config(localhost_client.query):
configure_ports_from_zk(zk) configure_from_zk(zk)
break break
except QueryRuntimeException: except QueryRuntimeException:
logging.exception("The new socket is not binded yet") logging.exception("The new socket is not binded yet")
@ -379,3 +380,33 @@ def test_reload_via_client(cluster, zk):
if exception: if exception:
raise exception raise exception
def test_change_http_handlers(cluster, zk):
with default_client(cluster, zk) as client:
curl_result = instance.exec_in_container(
["bash", "-c", "curl -s '127.0.0.1:8123/it_works'"]
)
assert "There is no handle /it_works" in curl_result
with sync_loaded_config(client.query):
zk.set(
"/clickhouse/http_handlers",
b"""
<defaults/>
<rule>
<url>/it_works</url>
<methods>GET</methods>
<handler>
<type>predefined_query_handler</type>
<query>SELECT 'It works.'</query>
</handler>
</rule>
""",
)
curl_result = instance.exec_in_container(
["bash", "-c", "curl -s '127.0.0.1:8123/it_works'"]
)
assert curl_result == "It works.\n"