mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Extract HTTPContext for HTTPServer
This commit is contained in:
parent
bb507356ef
commit
e82a5d43b5
@ -24,6 +24,9 @@
|
||||
#include <pwd.h>
|
||||
#include <Coordination/FourLetterCommand.h>
|
||||
|
||||
#include <Server/HTTP/HTTPServer.h>
|
||||
|
||||
#include "Core/Defines.h"
|
||||
#include "config.h"
|
||||
#include "config_version.h"
|
||||
|
||||
@ -273,6 +276,54 @@ void Keeper::defineOptions(Poco::Util::OptionSet & 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*/)
|
||||
{
|
||||
Poco::Logger * log = &logger();
|
||||
@ -412,6 +463,29 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#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)
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Server/IServer.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include "TinyContext.h"
|
||||
|
||||
@ -58,6 +59,9 @@ protected:
|
||||
private:
|
||||
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;
|
||||
|
||||
using CreateServerFunc = std::function<void(UInt16)>;
|
||||
|
@ -1878,15 +1878,15 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
|
||||
return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this));
|
||||
if (type == "http")
|
||||
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")
|
||||
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")
|
||||
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);
|
||||
@ -1927,6 +1927,60 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
|
||||
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(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const Strings & listen_hosts,
|
||||
@ -2009,7 +2063,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"http://" + address.toString(),
|
||||
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
|
||||
@ -2026,7 +2080,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"https://" + address.toString(),
|
||||
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
|
||||
UNUSED(port);
|
||||
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
|
||||
@ -2151,7 +2205,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"Prometheus: http://" + address.toString(),
|
||||
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,
|
||||
"replica communication (interserver): http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
httpContext(),
|
||||
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
@ -2191,7 +2245,7 @@ void Server::createServers(
|
||||
port_name,
|
||||
"secure replica communication (interserver): https://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
httpContext(),
|
||||
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Server/IServer.h>
|
||||
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include "Server/HTTP/HTTPContext.h"
|
||||
#include <Server/TCPProtocolStackFactory.h>
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
|
||||
@ -72,6 +73,9 @@ private:
|
||||
/// Updated/recent config, to compare http_handlers
|
||||
ConfigurationPtr latest_config;
|
||||
|
||||
struct ServerHTTPContext;
|
||||
HTTPContextPtr httpContext() const;
|
||||
|
||||
Poco::Net::SocketAddress socketBindListen(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
Poco::Net::ServerSocket & socket,
|
||||
|
24
src/Server/HTTP/HTTPContext.h
Normal file
24
src/Server/HTTP/HTTPContext.h
Normal 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>;
|
||||
|
||||
}
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
HTTPServer::HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory_,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
Poco::Net::ServerSocket & socket_,
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Server/TCPServer.h>
|
||||
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
@ -11,13 +12,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
class HTTPServer : public TCPServer
|
||||
{
|
||||
public:
|
||||
explicit HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
Poco::Net::ServerSocket & socket,
|
||||
|
@ -7,12 +7,12 @@ namespace DB
|
||||
{
|
||||
|
||||
HTTPServerConnection::HTTPServerConnection(
|
||||
ContextPtr context_,
|
||||
HTTPContextPtr context_,
|
||||
TCPServer & tcp_server_,
|
||||
const Poco::Net::StreamSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params_,
|
||||
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);
|
||||
}
|
||||
@ -36,7 +36,7 @@ void HTTPServerConnection::run()
|
||||
|
||||
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)
|
||||
response.add("Strict-Transport-Security", "max-age=" + std::to_string(hsts_max_age));
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
#include <Poco/Net/HTTPServerSession.h>
|
||||
@ -15,7 +16,7 @@ class HTTPServerConnection : public Poco::Net::TCPServerConnection
|
||||
{
|
||||
public:
|
||||
HTTPServerConnection(
|
||||
ContextPtr context,
|
||||
HTTPContextPtr context,
|
||||
TCPServer & tcp_server,
|
||||
const Poco::Net::StreamSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params,
|
||||
@ -27,7 +28,7 @@ protected:
|
||||
static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status);
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
HTTPContextPtr context;
|
||||
TCPServer & tcp_server;
|
||||
Poco::Net::HTTPServerParams::Ptr params;
|
||||
HTTPRequestHandlerFactoryPtr factory;
|
||||
|
@ -5,8 +5,8 @@
|
||||
namespace DB
|
||||
{
|
||||
HTTPServerConnectionFactory::HTTPServerConnectionFactory(
|
||||
ContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_)
|
||||
: context(Context::createCopy(context_)), params(params_), factory(factory_)
|
||||
HTTPContextPtr context_, Poco::Net::HTTPServerParams::Ptr params_, HTTPRequestHandlerFactoryPtr factory_)
|
||||
: context(std::move(context_)), params(params_), factory(factory_)
|
||||
{
|
||||
poco_check_ptr(factory);
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
@ -12,12 +13,12 @@ namespace DB
|
||||
class HTTPServerConnectionFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
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;
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
HTTPContextPtr context;
|
||||
Poco::Net::HTTPServerParams::Ptr params;
|
||||
HTTPRequestHandlerFactoryPtr factory;
|
||||
};
|
||||
|
@ -21,11 +21,11 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
|
||||
: max_uri_size(context->getSettingsRef().http_max_uri_size)
|
||||
, max_fields_number(context->getSettingsRef().http_max_fields)
|
||||
, max_field_name_size(context->getSettingsRef().http_max_field_name_size)
|
||||
, max_field_value_size(context->getSettingsRef().http_max_field_value_size)
|
||||
HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
|
||||
: max_uri_size(context->getMaxUriSize())
|
||||
, max_fields_number(context->getMaxFields())
|
||||
, max_field_name_size(context->getMaxFieldNameSize())
|
||||
, max_field_value_size(context->getMaxFieldValueSize())
|
||||
{
|
||||
response.attachRequest(this);
|
||||
|
||||
@ -34,8 +34,8 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
|
||||
server_address = session.serverAddress();
|
||||
secure = session.socket().secure();
|
||||
|
||||
auto receive_timeout = context->getSettingsRef().http_receive_timeout;
|
||||
auto send_timeout = context->getSettingsRef().http_send_timeout;
|
||||
auto receive_timeout = context->getReceiveTimeout();
|
||||
auto send_timeout = context->getSendTimeout();
|
||||
|
||||
session.socket().setReceiveTimeout(receive_timeout);
|
||||
session.socket().setSendTimeout(send_timeout);
|
||||
@ -46,7 +46,7 @@ HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & re
|
||||
readRequest(*in); /// Try parse according to RFC7230
|
||||
|
||||
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())
|
||||
stream = std::make_unique<LimitReadBuffer>(std::move(in), getContentLength(), false);
|
||||
else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Server/HTTP/HTTPRequest.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include "config.h"
|
||||
|
||||
#include <Poco/Net/HTTPServerSession.h>
|
||||
@ -18,7 +19,7 @@ class ReadBufferFromPocoSocket;
|
||||
class HTTPServerRequest : public HTTPRequest
|
||||
{
|
||||
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
|
||||
/// via unique_ptr - but we can't inherit HTTPServerRequest from ReadBuffer and pass it around,
|
||||
|
@ -144,15 +144,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::
|
||||
else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory")
|
||||
return createInterserverHTTPHandlerFactory(server, name);
|
||||
else if (name == "PrometheusHandler-factory")
|
||||
{
|
||||
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;
|
||||
}
|
||||
return createPrometheusMainHandlerFactory(server, config, async_metrics, name);
|
||||
|
||||
throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/HTTPHandlerRequestFilter.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
@ -148,6 +149,12 @@ createPrometheusHandlerFactory(IServer & server,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
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 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)
|
||||
|
@ -53,4 +53,19 @@ createPrometheusHandlerFactory(IServer & server,
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Server/HTTP/HTTPRequestHandler.h>
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
|
||||
#include "PrometheusMetricsWriter.h"
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user