mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into kvm-clock
This commit is contained in:
commit
85e43566b6
@ -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());
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
|
@ -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();
|
||||||
|
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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"))
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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);
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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>
|
@ -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"
|
||||||
|
Loading…
Reference in New Issue
Block a user