Fix Static handler

This commit is contained in:
Grigorii Sokolik 2024-06-06 10:50:31 +03:00
parent 979b447451
commit 06383d7a7a
7 changed files with 204 additions and 140 deletions

View File

@ -27,6 +27,7 @@
#include <Server/HTTPHandlerFactory.h>
#include <Server/HTTPHandlerRequestFilter.h>
#include <Server/IServer.h>
#include "Common/logger_useful.h"
#include <Common/SettingsChanges.h>
#include <Common/StringUtils.h>
#include <Common/scope_guard_safe.h>
@ -286,9 +287,6 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util:
return std::chrono::seconds(session_timeout);
}
std::optional<std::unordered_map<String, String>>
parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
void HTTPHandler::pushDelayedResults(Output & used_output)
{
std::vector<WriteBufferPtr> write_buffers;
@ -325,8 +323,7 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
}
HTTPHandler::HTTPHandler(
IServer & server_, const std::string & name, const std::optional<std::unordered_map<String, String>> & http_response_headers_override_)
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_)
: server(server_)
, log(getLogger(name))
, default_settings(server.context()->getSettingsRef())
@ -711,7 +708,8 @@ void HTTPHandler::processQuery(
bool is_in_post_compressed = false;
if (params.getParsed<bool>("decompress", false))
{
in_post_maybe_compressed = std::make_unique<CompressedReadBuffer>(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true);
in_post_maybe_compressed
= std::make_unique<CompressedReadBuffer>(*in_post, /* allow_different_codecs_ = */ false, /* external_data_ = */ true);
is_in_post_compressed = true;
}
else
@ -900,18 +898,14 @@ void HTTPHandler::processQuery(
customizeContext(request, context, *in_post_maybe_compressed);
in = has_external_data ? std::move(in_param) : std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed);
if (http_response_headers_override)
for (auto [header_name, header_value] : *http_response_headers_override)
response.set(header_name, header_value);
applyHTTPResponseHeaders(response, http_response_headers_override);
auto set_query_result = [this, &response](const QueryResultDetails & details)
{
response.add("X-ClickHouse-Query-Id", details.query_id);
if (!(
http_response_headers_override.has_value()
&& http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE)
) && details.content_type)
if (!(http_response_headers_override && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
&& details.content_type)
response.setContentType(*details.content_type);
if (details.format)
@ -1125,10 +1119,8 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
// Setup tracing context for this thread
auto context = session->sessionOrGlobalContext();
thread_trace_context = std::make_unique<OpenTelemetry::TracingContextHolder>("HTTPHandler",
client_trace_context,
context->getSettingsRef(),
context->getOpenTelemetrySpanLog());
thread_trace_context = std::make_unique<OpenTelemetry::TracingContextHolder>(
"HTTPHandler", client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog());
thread_trace_context->root_span.kind = OpenTelemetry::SpanKind::SERVER;
thread_trace_context->root_span.addAttribute("clickhouse.uri", request.getURI());
@ -1204,9 +1196,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
}
DynamicQueryHandler::DynamicQueryHandler(
IServer & server_,
const std::string & param_name_,
const std::optional<std::unordered_map<String, String>> & http_response_headers_override_)
IServer & server_, const std::string & param_name_, const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_)
{
}
@ -1262,7 +1252,7 @@ PredefinedQueryHandler::PredefinedQueryHandler(
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const std::optional<std::unordered_map<String, String>> & http_response_headers_override_)
const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_)
, receive_params(receive_params_)
, predefined_query(predefined_query_)
@ -1349,34 +1339,12 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo
return predefined_query;
}
std::optional<std::unordered_map<String, String>>
parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override;
String http_response_headers_key = config_prefix + ".handler.http_response_headers";
String http_response_headers_key_prefix = http_response_headers_key + ".";
if (config.has(http_response_headers_key))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(http_response_headers_key, keys);
for (const auto & key : keys)
http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key);
}
if (config.has(config_prefix + ".handler.content_type"))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type");
if (http_response_headers_override.empty())
return {};
return std::move(http_response_headers_override);
}
HTTPRequestHandlerFactoryPtr
createDynamicHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
std::optional<std::unordered_map<String, String>> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr<DynamicQueryHandler>
{ return std::make_unique<DynamicQueryHandler>(server, query_param_name, http_response_headers_override); };
@ -1440,7 +1408,7 @@ createPredefinedHandlerFactory(IServer & server, const Poco::Util::AbstractConfi
headers_name_with_regex.emplace(std::make_pair(header_name, regex));
}
std::optional<std::unordered_map<String, String>> http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;

View File

@ -13,6 +13,8 @@
#include <Common/CurrentThread.h>
#include <Common/re2.h>
#include "HTTPResponseHeaderWriter.h"
namespace CurrentMetrics
{
extern const Metric HTTPConnection;
@ -37,10 +39,7 @@ using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
class HTTPHandler : public HTTPRequestHandler
{
public:
HTTPHandler(
IServer & server_,
const std::string & name,
const std::optional<std::unordered_map<String, String>> & http_response_headers_override_);
HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_);
~HTTPHandler() override;
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
@ -86,10 +85,7 @@ private:
bool exception_is_written = false;
std::function<void(WriteBuffer &, const String &)> exception_writer;
bool hasDelayed() const
{
return out_maybe_delayed_and_compressed != out_maybe_compressed.get();
}
bool hasDelayed() const { return out_maybe_delayed_and_compressed != out_maybe_compressed.get(); }
void finalize()
{
@ -103,10 +99,7 @@ private:
out->finalize();
}
bool isFinalized() const
{
return finalized;
}
bool isFinalized() const { return finalized; }
};
IServer & server;
@ -123,7 +116,7 @@ private:
const Settings & default_settings;
/// Overrides for response headers.
std::optional<std::unordered_map<String, String>> http_response_headers_override;
HTTPResponseHeaderSetup http_response_headers_override;
// session is reset at the end of each request/response.
std::unique_ptr<Session> session;
@ -165,7 +158,7 @@ public:
explicit DynamicQueryHandler(
IServer & server_,
const std::string & param_name_ = "query",
const std::optional<std::unordered_map<String, String>> & http_response_headers_override_ = std::nullopt);
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
@ -187,7 +180,7 @@ public:
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const std::optional<std::unordered_map<String, String>> & http_response_headers_override_ = std::nullopt);
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
void customizeContext(HTTPServerRequest & request, ContextMutablePtr context, ReadBuffer & body) override;

View File

@ -1,18 +1,18 @@
#include <memory>
#include <Server/HTTPHandlerFactory.h>
#include <Access/Credentials.h>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <Server/IServer.h>
#include <Access/Credentials.h>
#include <Poco/Util/AbstractConfiguration.h>
#include "HTTPHandler.h"
#include "Server/PrometheusMetricsWriter.h"
#include "StaticRequestHandler.h"
#include "ReplicasStatusHandler.h"
#include "InterserverIOHTTPHandler.h"
#include "PrometheusRequestHandler.h"
#include "ReplicasStatusHandler.h"
#include "Server/PrometheusMetricsWriter.h"
#include "StaticRequestHandler.h"
#include "WebUIRequestHandler.h"
@ -21,9 +21,9 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
namespace
@ -35,10 +35,7 @@ private:
std::string url;
public:
explicit RedirectRequestHandler(std::string url_)
: url(std::move(url_))
{
}
explicit RedirectRequestHandler(std::string url_) : url(std::move(url_)) { }
void handleRequest(HTTPServerRequest &, HTTPServerResponse & response, const ProfileEvents::Event &) override
{
@ -46,9 +43,8 @@ public:
}
};
HTTPRequestHandlerFactoryPtr createRedirectHandlerFactory(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
HTTPRequestHandlerFactoryPtr
createRedirectHandlerFactory(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::string url = config.getString(config_prefix + ".handler.location");
@ -74,7 +70,8 @@ static auto createPingHandlerFactory(IServer & server)
auto creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto ping_response_expression = "Ok.\n";
return std::make_unique<StaticRequestHandler>(server, ping_response_expression);
return std::make_unique<StaticRequestHandler>(
server, ping_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
return std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));
}
@ -102,8 +99,12 @@ static inline auto createHandlersFactoryFromConfig(
const auto & handler_type = config.getString(prefix + "." + key + ".handler.type", "");
if (handler_type.empty())
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Handler type in config is not specified here: "
"{}.{}.handler.type", prefix, key);
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Handler type in config is not specified here: "
"{}.{}.handler.type",
prefix,
key);
if (handler_type == "static")
{
@ -154,19 +155,27 @@ static inline auto createHandlersFactoryFromConfig(
main_handler_factory->addHandler(std::move(handler));
}
else
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Unknown handler type '{}' in config here: {}.{}.handler.type",
handler_type, prefix, key);
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Unknown handler type '{}' in config here: {}.{}.handler.type",
handler_type,
prefix,
key);
}
else
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, "Unknown element in config: "
"{}.{}, must be 'rule' or 'defaults'", prefix, key);
throw Exception(
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"Unknown element in config: "
"{}.{}, must be 'rule' or 'defaults'",
prefix,
key);
}
return main_handler_factory;
}
static inline HTTPRequestHandlerFactoryPtr
createHTTPHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics)
static inline HTTPRequestHandlerFactoryPtr createHTTPHandlerFactory(
IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & name, AsynchronousMetrics & async_metrics)
{
if (config.has("http_handlers"))
{
@ -193,7 +202,8 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I
}
HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, 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")
return createHTTPHandlerFactory(server, config, name, async_metrics);
@ -214,7 +224,8 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS
auto root_creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto root_response_expression = "config://http_server_default_response";
return std::make_unique<StaticRequestHandler>(server, root_response_expression);
return std::make_unique<StaticRequestHandler>(
server, root_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
auto root_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(root_creator));
root_handler->attachStrictPath("/");
@ -265,28 +276,23 @@ void addDefaultHandlersFactory(
{
addCommonDefaultHandlersFactory(factory, server);
auto dynamic_creator = [&server] () -> std::unique_ptr<DynamicQueryHandler>
{
return std::make_unique<DynamicQueryHandler>(server, "query");
};
auto dynamic_creator
= [&server]() -> std::unique_ptr<DynamicQueryHandler> { return std::make_unique<DynamicQueryHandler>(server, "query"); };
auto query_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(std::move(dynamic_creator));
query_handler->addFilter([](const auto & request)
query_handler->addFilter(
[](const auto & request)
{
bool path_matches_get_or_head = startsWith(request.getURI(), "?")
|| startsWith(request.getURI(), "/?")
|| startsWith(request.getURI(), "/query?");
bool is_get_or_head_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD;
bool path_matches_get_or_head
= startsWith(request.getURI(), "?") || startsWith(request.getURI(), "/?") || startsWith(request.getURI(), "/query?");
bool is_get_or_head_request
= request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD;
bool path_matches_post_or_options = path_matches_get_or_head
|| request.getURI() == "/"
|| request.getURI().empty();
bool is_post_or_options_request = request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS;
bool path_matches_post_or_options = path_matches_get_or_head || request.getURI() == "/" || request.getURI().empty();
bool is_post_or_options_request
= request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS;
return (path_matches_get_or_head && is_get_or_head_request) || (path_matches_post_or_options && is_post_or_options_request);
}
);
});
factory.addHandler(query_handler);
/// We check that prometheus handler will be served on current (default) port.
@ -294,10 +300,8 @@ void addDefaultHandlersFactory(
if (config.has("prometheus") && config.getInt("prometheus.port", 0) == 0)
{
auto writer = std::make_shared<PrometheusMetricsWriter>(config, "prometheus", async_metrics);
auto creator = [&server, writer] () -> std::unique_ptr<PrometheusRequestHandler>
{
return std::make_unique<PrometheusRequestHandler>(server, writer);
};
auto creator = [&server, writer]() -> std::unique_ptr<PrometheusRequestHandler>
{ return std::make_unique<PrometheusRequestHandler>(server, writer); };
auto prometheus_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<PrometheusRequestHandler>>(std::move(creator));
prometheus_handler->attachStrictPath(config.getString("prometheus.endpoint", "/metrics"));
prometheus_handler->allowGetAndHeadRequest();

View File

@ -0,0 +1,69 @@
#include "HTTPResponseHeaderWriter.h"
#include <unordered_map>
#include <utility>
#include <Poco/Net/HTTPMessage.h>
namespace DB
{
std::unordered_map<String, String>
baseParseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override;
String http_response_headers_key = config_prefix + ".handler.http_response_headers";
String http_response_headers_key_prefix = http_response_headers_key + ".";
if (config.has(http_response_headers_key))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(http_response_headers_key, keys);
for (const auto & key : keys)
{
http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key);
}
}
if (config.has(config_prefix + ".handler.content_type"))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type");
return http_response_headers_override;
}
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (http_response_headers_override.empty())
return {};
return std::move(http_response_headers_override);
}
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (!http_response_headers_override.contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = default_content_type;
return http_response_headers_override;
}
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type)
{
return {{{Poco::Net::HTTPMessage::CONTENT_TYPE, default_content_type}}};
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup)
{
if (setup)
for (auto [header_name, header_value] : *setup)
response.set(header_name, header_value);
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup)
{
for (auto [header_name, header_value] : setup)
response.set(header_name, header_value);
}
}

View File

@ -0,0 +1,23 @@
#include <optional>
#include <string>
#include <unordered_map>
#include <base/types.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using HTTPResponseHeaderSetup = std::optional<std::unordered_map<String, String>>;
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type);
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup);
}

View File

@ -2,23 +2,24 @@
#include "IServer.h"
#include "HTTPHandlerFactory.h"
#include "HTTPHandlerRequestFilter.h"
#include "HTTPResponseHeaderWriter.h"
#include <IO/HTTPCommon.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromString.h>
#include <IO/copyData.h>
#include <IO/WriteHelpers.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <IO/copyData.h>
#include <Interpreters/Context.h>
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <Common/Exception.h>
#include <filesystem>
#include <unordered_map>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <filesystem>
namespace fs = std::filesystem;
@ -28,15 +29,16 @@ namespace DB
namespace ErrorCodes
{
extern const int INCORRECT_FILE_NAME;
extern const int HTTP_LENGTH_REQUIRED;
extern const int INVALID_CONFIG_PARAMETER;
extern const int INCORRECT_FILE_NAME;
extern const int HTTP_LENGTH_REQUIRED;
extern const int INVALID_CONFIG_PARAMETER;
}
static inline std::unique_ptr<WriteBuffer>
responseWriteBuffer(HTTPServerRequest & request, HTTPServerResponse & response, UInt64 keep_alive_timeout)
{
auto buf = std::unique_ptr<WriteBuffer>(new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout));
auto buf = std::unique_ptr<WriteBuffer>(
new WriteBufferFromHTTPServerResponse(response, request.getMethod() == HTTPRequest::HTTP_HEAD, keep_alive_timeout));
/// The client can pass a HTTP header indicating supported compression method (gzip or deflate).
String http_response_compression_methods = request.get("Accept-Encoding", "");
@ -61,8 +63,8 @@ static inline void trySendExceptionToClient(
/// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body
/// to avoid reading part of the current request body in the next request.
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST
&& response.getKeepAlive() && !request.getStream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED)
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() && !request.getStream().eof()
&& exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED)
request.getStream().ignore(std::numeric_limits<std::streamsize>::max());
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
@ -87,23 +89,26 @@ static inline void trySendExceptionToClient(
}
}
void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/)
void StaticRequestHandler::handleRequest(
HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & /*write_event*/)
{
auto keep_alive_timeout = server.context()->getServerSettings().keep_alive_timeout.totalSeconds();
auto out = responseWriteBuffer(request, response, keep_alive_timeout);
try
{
response.setContentType(content_type);
applyHTTPResponseHeaders(response, http_response_headers_override);
if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1)
response.setChunkedTransferEncoding(true);
/// Workaround. Poco does not detect 411 Length Required case.
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength())
throw Exception(ErrorCodes::HTTP_LENGTH_REQUIRED,
"The Transfer-Encoding is not chunked and there "
"is no Content-Length header for POST request");
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding()
&& !request.hasContentLength())
throw Exception(
ErrorCodes::HTTP_LENGTH_REQUIRED,
"The Transfer-Encoding is not chunked and there "
"is no Content-Length header for POST request");
setResponseDefaultHeaders(response, keep_alive_timeout);
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status));
@ -144,9 +149,10 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out)
else if (startsWith(response_expression, config_prefix))
{
if (response_expression.size() <= config_prefix.size())
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER,
"Static handling rule handler must contain a complete configuration path, for example: "
"config://config_key");
throw Exception(
ErrorCodes::INVALID_CONFIG_PARAMETER,
"Static handling rule handler must contain a complete configuration path, for example: "
"config://config_key");
const auto & config_path = response_expression.substr(config_prefix.size(), response_expression.size() - config_prefix.size());
writeString(server.config().getRawString(config_path, "Ok.\n"), out);
@ -155,23 +161,23 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out)
writeString(response_expression, out);
}
StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_)
: server(server_), status(status_), content_type(content_type_), response_expression(expression)
StaticRequestHandler::StaticRequestHandler(
IServer & server_, const String & expression, const std::unordered_map<String, String> & http_response_headers_override_, int status_)
: server(server_), status(status_), http_response_headers_override(http_response_headers_override_), response_expression(expression)
{
}
HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
HTTPRequestHandlerFactoryPtr
createStaticHandlerFactory(IServer & server, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
int status = config.getInt(config_prefix + ".handler.status", 200);
std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n");
std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
auto creator = [&server, response_content, status, response_content_type]() -> std::unique_ptr<StaticRequestHandler>
{
return std::make_unique<StaticRequestHandler>(server, response_content, status, response_content_type);
};
std::unordered_map<String, String> http_response_headers_override
= parseHTTPResponseHeaders(config, config_prefix, "text/plain; charset=UTF-8");
auto creator = [&server, http_response_headers_override, response_content, status]() -> std::unique_ptr<StaticRequestHandler>
{ return std::make_unique<StaticRequestHandler>(server, response_content, http_response_headers_override, status); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));

View File

@ -1,9 +1,9 @@
#pragma once
#include <unordered_map>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <base/types.h>
namespace DB
{
@ -17,15 +17,16 @@ private:
IServer & server;
int status;
String content_type;
/// Overrides for response headers.
std::unordered_map<String, String> http_response_headers_override;
String response_expression;
public:
StaticRequestHandler(
IServer & server,
const String & expression,
int status_ = 200,
const String & content_type_ = "text/html; charset=UTF-8");
const std::unordered_map<String, String> & http_response_headers_override_,
int status_ = 200);
void writeResponse(WriteBuffer & out);