Extract HTTPContext for HTTPServer

This commit is contained in:
Antonio Andelic 2022-11-09 09:02:04 +01:00
parent bb507356ef
commit e82a5d43b5
17 changed files with 216 additions and 39 deletions

View File

@ -24,6 +24,9 @@
#include <pwd.h> #include <pwd.h>
#include <Coordination/FourLetterCommand.h> #include <Coordination/FourLetterCommand.h>
#include <Server/HTTP/HTTPServer.h>
#include "Core/Defines.h"
#include "config.h" #include "config.h"
#include "config_version.h" #include "config_version.h"
@ -273,6 +276,54 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
BaseDaemon::defineOptions(options); BaseDaemon::defineOptions(options);
} }
struct Keeper::KeeperHTTPContext : public IHTTPContext
{
uint64_t getMaxHstsAge() const override
{
return 0;
}
uint64_t getMaxUriSize() const override
{
return 1048576;
}
uint64_t getMaxFields() const override
{
return 1000000;
}
uint64_t getMaxFieldNameSize() const override
{
return 1048576;
}
uint64_t getMaxFieldValueSize() const override
{
return 1048576;
}
uint64_t getMaxChunkSize() const override
{
return 100_GiB;
}
Poco::Timespan getReceiveTimeout() const override
{
return DEFAULT_HTTP_READ_BUFFER_TIMEOUT;
}
Poco::Timespan getSendTimeout() const override
{
return DEFAULT_HTTP_READ_BUFFER_TIMEOUT;
}
};
HTTPContextPtr Keeper::httpContext()
{
return std::make_shared<KeeperHTTPContext>();
}
int Keeper::main(const std::vector<std::string> & /*args*/) int Keeper::main(const std::vector<std::string> & /*args*/)
{ {
Poco::Logger * log = &logger(); Poco::Logger * log = &logger();
@ -412,6 +463,29 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #endif
}); });
const auto & config = config_getter();
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
http_params->setKeepAliveTimeout(keep_alive_timeout);
/// Prometheus (if defined and not setup yet with http_port)
port_name = "prometheus.port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port) -> ProtocolServerAdapter
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
// TODO(antonio2368): use config
socket.setReceiveTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
socket.setSendTimeout(DBMS_DEFAULT_SEND_TIMEOUT_SEC);
return ProtocolServerAdapter(
listen_host,
port_name,
"Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>(
httpContext(), createHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
});
} }
for (auto & server : *servers) for (auto & server : *servers)

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Server/IServer.h> #include <Server/IServer.h>
#include <Server/HTTP/HTTPContext.h>
#include <Daemon/BaseDaemon.h> #include <Daemon/BaseDaemon.h>
#include "TinyContext.h" #include "TinyContext.h"
@ -58,6 +59,9 @@ protected:
private: private:
TinyContext tiny_context; TinyContext tiny_context;
struct KeeperHTTPContext;
static HTTPContextPtr httpContext();
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
using CreateServerFunc = std::function<void(UInt16)>; using CreateServerFunc = std::function<void(UInt16)>;

View File

@ -1878,15 +1878,15 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this)); return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this));
if (type == "http") if (type == "http")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"))
); );
if (type == "prometheus") if (type == "prometheus")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"))
); );
if (type == "interserver") if (type == "interserver")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"))
); );
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type);
@ -1927,6 +1927,60 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return stack; return stack;
} }
struct Server::ServerHTTPContext : public IHTTPContext
{
explicit ServerHTTPContext(ContextPtr context_)
: context(Context::createCopy(context_))
{}
uint64_t getMaxHstsAge() const override
{
return context->getSettingsRef().hsts_max_age;
}
uint64_t getMaxUriSize() const override
{
return context->getSettingsRef().http_max_uri_size;
}
uint64_t getMaxFields() const override
{
return context->getSettingsRef().http_max_fields;
}
uint64_t getMaxFieldNameSize() const override
{
return context->getSettingsRef().http_max_field_name_size;
}
uint64_t getMaxFieldValueSize() const override
{
return context->getSettingsRef().http_max_field_value_size;
}
uint64_t getMaxChunkSize() const override
{
return context->getSettingsRef().http_max_chunk_size;
}
Poco::Timespan getReceiveTimeout() const override
{
return context->getSettingsRef().http_receive_timeout;
}
Poco::Timespan getSendTimeout() const override
{
return context->getSettingsRef().http_send_timeout;
}
ContextPtr context;
};
HTTPContextPtr Server::httpContext() const
{
return std::make_shared<ServerHTTPContext>(context());
}
void Server::createServers( void Server::createServers(
Poco::Util::AbstractConfiguration & config, Poco::Util::AbstractConfiguration & config,
const Strings & listen_hosts, const Strings & listen_hosts,
@ -2009,7 +2063,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, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
}); });
/// HTTPS /// HTTPS
@ -2026,7 +2080,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, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); httpContext(), 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.",
@ -2151,7 +2205,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, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
}); });
} }
@ -2171,7 +2225,7 @@ void Server::createServers(
port_name, port_name,
"replica communication (interserver): http://" + address.toString(), "replica communication (interserver): http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
server_pool, server_pool,
socket, socket,
@ -2191,7 +2245,7 @@ void Server::createServers(
port_name, port_name,
"secure replica communication (interserver): https://" + address.toString(), "secure replica communication (interserver): https://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
server_pool, server_pool,
socket, socket,

View File

@ -3,6 +3,7 @@
#include <Server/IServer.h> #include <Server/IServer.h>
#include <Daemon/BaseDaemon.h> #include <Daemon/BaseDaemon.h>
#include "Server/HTTP/HTTPContext.h"
#include <Server/TCPProtocolStackFactory.h> #include <Server/TCPProtocolStackFactory.h>
#include <Poco/Net/HTTPServerParams.h> #include <Poco/Net/HTTPServerParams.h>
@ -72,6 +73,9 @@ private:
/// Updated/recent config, to compare http_handlers /// Updated/recent config, to compare http_handlers
ConfigurationPtr latest_config; ConfigurationPtr latest_config;
struct ServerHTTPContext;
HTTPContextPtr httpContext() const;
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

@ -0,0 +1,24 @@
#pragma once
#include <Poco/Timespan.h>
namespace DB
{
struct IHTTPContext
{
virtual uint64_t getMaxHstsAge() const = 0;
virtual uint64_t getMaxUriSize() const = 0;
virtual uint64_t getMaxFields() const = 0;
virtual uint64_t getMaxFieldNameSize() const = 0;
virtual uint64_t getMaxFieldValueSize() const = 0;
virtual uint64_t getMaxChunkSize() const = 0;
virtual Poco::Timespan getReceiveTimeout() const = 0;
virtual Poco::Timespan getSendTimeout() const = 0;
virtual ~IHTTPContext() = default;
};
using HTTPContextPtr = std::shared_ptr<IHTTPContext>;
}

View File

@ -6,7 +6,7 @@
namespace DB namespace DB
{ {
HTTPServer::HTTPServer( HTTPServer::HTTPServer(
ContextPtr context, HTTPContextPtr context,
HTTPRequestHandlerFactoryPtr factory_, HTTPRequestHandlerFactoryPtr factory_,
Poco::ThreadPool & thread_pool, Poco::ThreadPool & thread_pool,
Poco::Net::ServerSocket & socket_, Poco::Net::ServerSocket & socket_,

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Server/HTTP/HTTPRequestHandlerFactory.h> #include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Server/TCPServer.h> #include <Server/TCPServer.h>
#include <Poco/Net/HTTPServerParams.h> #include <Poco/Net/HTTPServerParams.h>
@ -11,13 +12,11 @@
namespace DB namespace DB
{ {
class Context;
class HTTPServer : public TCPServer class HTTPServer : public TCPServer
{ {
public: public:
explicit HTTPServer( explicit HTTPServer(
ContextPtr context, HTTPContextPtr context,
HTTPRequestHandlerFactoryPtr factory, HTTPRequestHandlerFactoryPtr factory,
Poco::ThreadPool & thread_pool, Poco::ThreadPool & thread_pool,
Poco::Net::ServerSocket & socket, Poco::Net::ServerSocket & socket,

View File

@ -7,12 +7,12 @@ namespace DB
{ {
HTTPServerConnection::HTTPServerConnection( HTTPServerConnection::HTTPServerConnection(
ContextPtr context_, HTTPContextPtr context_,
TCPServer & tcp_server_, TCPServer & tcp_server_,
const Poco::Net::StreamSocket & socket, const Poco::Net::StreamSocket & socket,
Poco::Net::HTTPServerParams::Ptr params_, Poco::Net::HTTPServerParams::Ptr params_,
HTTPRequestHandlerFactoryPtr factory_) HTTPRequestHandlerFactoryPtr factory_)
: TCPServerConnection(socket), context(Context::createCopy(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false) : TCPServerConnection(socket), context(std::move(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false)
{ {
poco_check_ptr(factory); poco_check_ptr(factory);
} }
@ -36,7 +36,7 @@ void HTTPServerConnection::run()
if (request.isSecure()) if (request.isSecure())
{ {
size_t hsts_max_age = context->getSettingsRef().hsts_max_age.value; size_t hsts_max_age = context->getMaxHstsAge();
if (hsts_max_age > 0) if (hsts_max_age > 0)
response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age)); response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age));

View File

@ -2,6 +2,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h> #include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Poco/Net/HTTPServerParams.h> #include <Poco/Net/HTTPServerParams.h>
#include <Poco/Net/HTTPServerSession.h> #include <Poco/Net/HTTPServerSession.h>
@ -15,7 +16,7 @@ class HTTPServerConnection : public Poco::Net::TCPServerConnection
{ {
public: public:
HTTPServerConnection( HTTPServerConnection(
ContextPtr context, HTTPContextPtr context,
TCPServer & tcp_server, TCPServer & tcp_server,
const Poco::Net::StreamSocket & socket, const Poco::Net::StreamSocket & socket,
Poco::Net::HTTPServerParams::Ptr params, Poco::Net::HTTPServerParams::Ptr params,
@ -27,7 +28,7 @@ protected:
static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status); static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status);
private: private:
ContextPtr context; HTTPContextPtr context;
TCPServer & tcp_server; TCPServer & tcp_server;
Poco::Net::HTTPServerParams::Ptr params; Poco::Net::HTTPServerParams::Ptr params;
HTTPRequestHandlerFactoryPtr factory; HTTPRequestHandlerFactoryPtr factory;

View File

@ -5,8 +5,8 @@
namespace DB namespace DB
{ {
HTTPServerConnectionFactory::HTTPServerConnectionFactory( HTTPServerConnectionFactory::HTTPServerConnectionFactory(
ContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_) HTTPContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_)
: context(Context::createCopy(context_)), params(params_), factory(factory_) : context(std::move(context_)), params(params_), factory(factory_)
{ {
poco_check_ptr(factory); poco_check_ptr(factory);
} }

View File

@ -2,6 +2,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Server/HTTP/HTTPRequestHandlerFactory.h> #include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTP/HTTPContext.h>
#include <Server/TCPServerConnectionFactory.h> #include <Server/TCPServerConnectionFactory.h>
#include <Poco/Net/HTTPServerParams.h> #include <Poco/Net/HTTPServerParams.h>
@ -12,12 +13,12 @@ namespace DB
class HTTPServerConnectionFactory : public TCPServerConnectionFactory class HTTPServerConnectionFactory : public TCPServerConnectionFactory
{ {
public: public:
HTTPServerConnectionFactory(ContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory); HTTPServerConnectionFactory(HTTPContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory);
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override; Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override;
private: private:
ContextPtr context; HTTPContextPtr context;
Poco::Net::HTTPServerParams::Ptr params; Poco::Net::HTTPServerParams::Ptr params;
HTTPRequestHandlerFactoryPtr factory; HTTPRequestHandlerFactoryPtr factory;
}; };

View File

@ -21,11 +21,11 @@
namespace DB namespace DB
{ {
HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session) HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
: max_uri_size(context->getSettingsRef().http_max_uri_size) : max_uri_size(context->getMaxUriSize())
, max_fields_number(context->getSettingsRef().http_max_fields) , max_fields_number(context->getMaxFields())
, max_field_name_size(context->getSettingsRef().http_max_field_name_size) , max_field_name_size(context->getMaxFieldNameSize())
, max_field_value_size(context->getSettingsRef().http_max_field_value_size) , max_field_value_size(context->getMaxFieldValueSize())
{ {
response.attachRequest(this); response.attachRequest(this);
@ -34,8 +34,8 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
server_address = session.serverAddress(); server_address = session.serverAddress();
secure = session.socket().secure(); secure = session.socket().secure();
auto receive_timeout = context->getSettingsRef().http_receive_timeout; auto receive_timeout = context->getReceiveTimeout();
auto send_timeout = context->getSettingsRef().http_send_timeout; auto send_timeout = context->getSendTimeout();
session.socket().setReceiveTimeout(receive_timeout); session.socket().setReceiveTimeout(receive_timeout);
session.socket().setSendTimeout(send_timeout); session.socket().setSendTimeout(send_timeout);
@ -46,7 +46,7 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
readRequest(*in); /// Try parse according to RFC7230 readRequest(*in); /// Try parse according to RFC7230
if (getChunkedTransferEncoding()) if (getChunkedTransferEncoding())
stream = std::make_unique<HTTPChunkedReadBuffer>(std::move(in), context->getSettingsRef().http_max_chunk_size); stream = std::make_unique<HTTPChunkedReadBuffer>(std::move(in), context->getMaxChunkSize());
else if (hasContentLength()) else if (hasContentLength())
stream = std::make_unique<LimitReadBuffer>(std::move(in), getContentLength(), false); stream = std::make_unique<LimitReadBuffer>(std::move(in), getContentLength(), false);
else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE)

View File

@ -3,6 +3,7 @@
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <Server/HTTP/HTTPRequest.h> #include <Server/HTTP/HTTPRequest.h>
#include <Server/HTTP/HTTPContext.h>
#include "config.h" #include "config.h"
#include <Poco/Net/HTTPServerSession.h> #include <Poco/Net/HTTPServerSession.h>
@ -18,7 +19,7 @@ class ReadBufferFromPocoSocket;
class HTTPServerRequest : public HTTPRequest class HTTPServerRequest : public HTTPRequest
{ {
public: public:
HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session); HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session);
/// FIXME: it's a little bit inconvenient interface. The rationale is that all other ReadBuffer's wrap each other /// FIXME: it's a little bit inconvenient interface. The rationale is that all other ReadBuffer's wrap each other
/// via unique_ptr - but we can't inherit HTTPServerRequest from ReadBuffer and pass it around, /// via unique_ptr - but we can't inherit HTTPServerRequest from ReadBuffer and pass it around,

View File

@ -144,15 +144,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::
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")
{ return createPrometheusMainHandlerFactory(server, config, async_metrics, name);
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
handler->allowGetAndHeadRequest();
factory->addHandler(handler);
return factory;
}
throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR);
} }

View File

@ -5,6 +5,7 @@
#include <Server/HTTP/HTTPRequestHandlerFactory.h> #include <Server/HTTP/HTTPRequestHandlerFactory.h>
#include <Server/HTTPHandlerRequestFilter.h> #include <Server/HTTPHandlerRequestFilter.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Daemon/BaseDaemon.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
@ -148,6 +149,12 @@ createPrometheusHandlerFactory(IServer & server,
AsynchronousMetrics & async_metrics, AsynchronousMetrics & async_metrics,
const std::string & config_prefix); const std::string & config_prefix);
HTTPRequestHandlerFactoryPtr
createPrometheusMainHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & name);
/// @param server - used in handlers to check IServer::isCancelled() /// @param server - used in handlers to check IServer::isCancelled()
/// @param config - not the same as server.config(), since it can be newer /// @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) /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true)

View File

@ -53,4 +53,19 @@ createPrometheusHandlerFactory(IServer & server,
return factory; return factory;
} }
HTTPRequestHandlerFactoryPtr
createPrometheusMainHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & name)
{
auto factory = std::make_shared<HTTPRequestHandlerFactoryMain>(name);
auto handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(
server, PrometheusMetricsWriter(config, "prometheus", async_metrics));
handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
handler->allowGetAndHeadRequest();
factory->addHandler(handler);
return factory;
}
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Server/HTTP/HTTPRequestHandler.h> #include <Server/HTTP/HTTPRequestHandler.h>
#include <Daemon/BaseDaemon.h>
#include "PrometheusMetricsWriter.h" #include "PrometheusMetricsWriter.h"