ISSUES-5436 add integration test

This commit is contained in:
zhang2014 2020-04-26 19:06:14 +08:00
parent 0070f75218
commit 81230944a4
17 changed files with 322 additions and 193 deletions

View File

@ -59,6 +59,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_OPEN_FILE;
extern const int CANNOT_COMPILE_REGEXP;
extern const int UNKNOWN_ELEMENT_IN_AST;
extern const int UNKNOWN_TYPE_OF_AST_NODE;
@ -756,9 +757,9 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request
PredefineQueryHandler::PredefineQueryHandler(
IServer & server, const NameSet & receive_params_, const std::string & predefine_query_
, const std::unordered_map<String, String> & header_name_with_capture_regex_)
, const std::optional<String> & url_regex_, const std::unordered_map<String, String> & header_name_with_regex_)
: HTTPHandler(server, "PredefineQueryHandler"), receive_params(receive_params_), predefine_query(predefine_query_)
, header_name_with_capture_regex(header_name_with_capture_regex_)
, url_regex(url_regex_), header_name_with_capture_regex(header_name_with_regex_)
{
}
@ -802,18 +803,16 @@ void PredefineQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & requ
}
};
if (url_regex)
{
const auto & uri = request.getURI();
set_query_params(uri.data(), find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()), *url_regex);
}
for (const auto & [header_name, regex] : header_name_with_capture_regex)
{
if (header_name == "url")
{
const auto & uri = request.getURI();
set_query_params(uri.data(), find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()), regex);
}
else
{
const auto & header_value = request.get(header_name);
set_query_params(header_value.data(), header_value.data() + header_value.size(), regex);
}
const auto & header_value = request.get(header_name);
set_query_params(header_value.data(), header_value.data() + header_value.size(), regex);
}
}
@ -835,47 +834,64 @@ Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & ser
return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory<DynamicQueryHandler>(server, std::move(query_param_name)), server.config(), config_prefix);
}
static inline bool capturingNamedQueryParam(NameSet receive_params, const std::string & expression)
{
auto compiled_regex = std::make_shared<re2_st::RE2>(expression);
if (!compiled_regex->ok())
throw Exception("Cannot compile re2: " + expression + " for routing_rule, error: " +
compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP);
const auto & capturing_names = compiled_regex->NamedCapturingGroups();
return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator)
{
return std::count_if(receive_params.begin(), receive_params.end(),
[&](const auto & param_name) { return param_name == iterator.first; });
});
}
Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix)
{
if (!server.config().has(config_prefix + ".handler.query"))
Poco::Util::AbstractConfiguration & configuration = server.config();
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);
std::string predefine_query = server.config().getString(config_prefix + ".handler.query");
std::string predefine_query = configuration.getString(config_prefix + ".handler.query");
NameSet analyze_receive_params = analyzeReceiveQueryParams(predefine_query);
std::unordered_map<String, String> type_and_regex;
Poco::Util::AbstractConfiguration::Keys filters_type;
server.config().keys(config_prefix, filters_type);
std::unordered_map<String, String> headers_name_with_regex;
Poco::Util::AbstractConfiguration::Keys headers_name;
configuration.keys(config_prefix + ".headers", headers_name);
for (const auto & filter_type : filters_type)
for (const auto & header_name : headers_name)
{
auto expression = server.config().getString(config_prefix + "." + filter_type);
auto expression = configuration.getString(config_prefix + ".headers." + header_name);
if (startsWith(expression, "regex:"))
{
expression = expression.substr(6);
auto compiled_regex = std::make_shared<re2_st::RE2>(expression);
if (!startsWith(expression, "regex:"))
continue;
if (!compiled_regex->ok())
throw Exception("cannot compile re2: " + expression + " for routing_rule, error: " + compiled_regex->error() +
". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP);
const auto & named_capturing_groups = compiled_regex->NamedCapturingGroups();
const auto & has_capturing_named_query_param = std::count_if(
named_capturing_groups.begin(), named_capturing_groups.end(), [&](const auto & iterator)
{
return std::count_if(analyze_receive_params.begin(), analyze_receive_params.end(), [&](const auto & param_name)
{
return param_name == iterator.first;
});
});
if (has_capturing_named_query_param)
type_and_regex.emplace(std::make_pair(filter_type, expression));
}
expression = expression.substr(6);
if (capturingNamedQueryParam(analyze_receive_params, expression))
headers_name_with_regex.emplace(std::make_pair(header_name, expression));
}
if (configuration.has(config_prefix + ".url"))
{
auto url_expression = configuration.getString(config_prefix + ".url");
if (startsWith(url_expression, "regex:"))
url_expression = url_expression.substr(6);
if (capturingNamedQueryParam(analyze_receive_params, url_expression))
return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory<PredefineQueryHandler>(
server, std::move(analyze_receive_params), std::move(predefine_query), std::optional<String>(url_expression),
std::move(headers_name_with_regex)), configuration, config_prefix);
}
return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory<PredefineQueryHandler>(
server, std::move(analyze_receive_params), std::move(predefine_query), std::move(type_and_regex)), server.config(), config_prefix);
server, std::move(analyze_receive_params), std::move(predefine_query), std::optional<String>{} ,std::move(headers_name_with_regex)),
configuration, config_prefix);
}
}
}

View File

@ -101,11 +101,12 @@ class PredefineQueryHandler : public HTTPHandler
private:
NameSet receive_params;
std::string predefine_query;
std::optional<String> url_regex;
std::unordered_map<String, String> header_name_with_capture_regex;
public:
explicit PredefineQueryHandler(
IServer & server, const NameSet & receive_params_, const std::string & predefine_query_
, const std::unordered_map<String, String> & header_name_with_capture_regex_);
, const std::optional<String> & url_regex_, const std::unordered_map<String, String> & header_name_with_regex_);
virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & context) override;

View File

@ -20,7 +20,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_COMPILE_REGEXP;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
}
@ -89,7 +88,7 @@ static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, con
{
for (const auto & [header_name, header_expression] : headers_expression)
{
const auto & header_value = request.get(header_name);
const auto & header_value = request.get(header_name, "");
if (!checkExpression(StringRef(header_value.data(), header_value.size()), header_expression))
return false;
}

View File

@ -7,6 +7,8 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromString.h>
#include <IO/copyData.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromHTTPServerResponse.h>
#include <Common/Exception.h>
@ -22,55 +24,135 @@ namespace DB
namespace ErrorCodes
{
extern const int INCORRECT_FILE_NAME;
extern const int HTTP_LENGTH_REQUIRED;
extern const int INVALID_CONFIG_PARAMETER;
}
void StaticRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response)
static inline WriteBufferPtr responseWriteBuffer(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, unsigned int 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", "");
CompressionMethod http_response_compression_method = CompressionMethod::None;
if (!http_response_compression_methods.empty())
{
/// If client supports brotli - it's preferred.
/// Both gzip and deflate are supported. If the client supports both, gzip is preferred.
/// NOTE parsing of the list of methods is slightly incorrect.
if (std::string::npos != http_response_compression_methods.find("br"))
http_response_compression_method = CompressionMethod::Brotli;
else if (std::string::npos != http_response_compression_methods.find("gzip"))
http_response_compression_method = CompressionMethod::Gzip;
else if (std::string::npos != http_response_compression_methods.find("deflate"))
http_response_compression_method = CompressionMethod::Zlib;
}
bool client_supports_http_compression = http_response_compression_method != CompressionMethod::None;
return std::make_shared<WriteBufferFromHTTPServerResponse>(
request, response, keep_alive_timeout, client_supports_http_compression, http_response_compression_method);
}
static inline void trySendExceptionToClient(
const std::string & s, int exception_code,
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response , WriteBuffer & out)
{
try
{
setResponseDefaultHeaders(response, server.config().getUInt("keep_alive_timeout", 10));
response.set("X-ClickHouse-Exception-Code", toString<int>(exception_code));
/// 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.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED)
request.stream().ignore(std::numeric_limits<std::streamsize>::max());
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
if (!response.sent())
response.send() << s << std::endl;
else
{
if (out.count() != out.offset())
out.position() = out.buffer().begin();
writeString(s, out);
writeChar('\n', out);
out.next();
out.finalize();
}
}
catch (...)
{
tryLogCurrentException("StaticRequestHandler", "Cannot send exception to client");
}
}
void StaticRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
{
auto keep_alive_timeout = server.config().getUInt("keep_alive_timeout", 10);
const auto & out = responseWriteBuffer(request, response, keep_alive_timeout);
try
{
response.setContentType(content_type);
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status));
response.sendBuffer(response_content.data(), response_content.size());
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("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED);
setResponseDefaultHeaders(response, keep_alive_timeout);
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status));
writeResponse(*out);
}
catch (...)
{
tryLogCurrentException("StaticRequestHandler");
int exception_code = getCurrentExceptionCode();
std::string exception_message = getCurrentExceptionMessage(false, true);
trySendExceptionToClient(exception_message, exception_code, request, response, *out);
}
}
StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_)
: server(server_), status(status_), content_type(content_type_)
void StaticRequestHandler::writeResponse(WriteBuffer & out)
{
static const String file_prefix = "file://";
static const String config_prefix = "config://";
if (startsWith(expression, file_prefix))
if (startsWith(response_expression, file_prefix))
{
std::string config_dir = Poco::Path(server.context().getPath()).parent().toString();
const std::string & file_path = config_dir + expression.substr(file_prefix.size(), expression.size() - file_prefix.size());
const auto & user_files_absolute_path = Poco::Path(server.context().getUserFilesPath()).makeAbsolute().makeDirectory().toString();
const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size());
const auto & file_path = Poco::Path(user_files_absolute_path, file_name).makeAbsolute().toString();
if (!Poco::File(file_path).exists())
throw Exception("Invalid file name for static HTTPHandler." + file_path, ErrorCodes::INCORRECT_FILE_NAME);
throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME);
WriteBufferFromOwnString out;
ReadBufferFromFile in(file_path);
copyData(in, out);
response_content = out.str();
}
else if (startsWith(expression, config_prefix))
else if (startsWith(response_expression, config_prefix))
{
if (expression.size() <= config_prefix.size())
throw Exception("Static routing rule handler must contain a complete configuration path, for example: config://config_key",
if (response_expression.size() <= config_prefix.size())
throw Exception( "Static routing rule handler must contain a complete configuration path, for example: config://config_key",
ErrorCodes::INVALID_CONFIG_PARAMETER);
response_content = server.config().getString(expression.substr(config_prefix.size(), expression.size() - config_prefix.size()), "Ok.\n");
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);
}
else
response_content = expression;
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)
{
}
Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix)
@ -80,7 +162,7 @@ Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & serv
std::string response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory<StaticRequestHandler>(
server, std::move(response_content), status, std::move(response_content_type)), server.config(), config_prefix);
server, std::move(response_content), std::move(status), std::move(response_content_type)), server.config(), config_prefix);
}
}

View File

@ -17,11 +17,13 @@ private:
int status;
String content_type;
String response_content;
String response_expression;
public:
StaticRequestHandler(IServer & server, const String & expression, int status_ = 200, const String & content_type_ = "text/html; charset=UTF-8");
void writeResponse(WriteBuffer & out);
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override;
};

View File

@ -526,18 +526,23 @@
-->
<!-- Uncomment to use custom http routing.
url -
routing_rule are checked from top to bottom, first match runs the handler
url - to match request URL, you can use 'regex:' prefix to use regex match(optional)
methods - to match request method, you can use commas to separate multiple method matches(optional)
headers - to match request headers, match each child element(child element name is header name), you can use 'regex:' prefix to use regex match(optional)
handler is request handler
type - supported types: static, dynamic_query_handler, predefine_query_handler
query - use with predefine_query_handler type, executes query when the handler is called
query_param_name - use with dynamic_query_handler type, extracts and executes the value corresponding to the <query_param_name> value in HTTP request params
status - use with static type, response status code
content_type - use with static type, response content-type
response_content - use with static type, Response content sent to client, when using the prefix 'file://' or 'config://', find the content from the file or configuration send to client.
-->
<!--
<routing_rules>
&lt;!&ndash; rules are checked from top to bottom, first match runs the handler &ndash;&gt;
<routing_rule>
<url>/</url> &lt;!&ndash; match requests on that url &ndash;&gt;
<methods>POST,GET</methods> &lt;!&ndash; and with that method &ndash;&gt;
&lt;!&ndash; other possible request matching rules can be introduced, for headers, ips, etc &ndash;&gt;
<pragma>no-cache</pragma>
<url>/</url>
<methods>POST,GET</methods>
<headers><pragma>no-cache</pragma></headers>
<handler>
<type>dynamic_query_handler</type>
<query_param_name>query</query_param_name>
@ -545,23 +550,22 @@
</routing_rule>
<routing_rule>
<url>/</url>
<url>/predefine_query</url>
<methods>POST,GET</methods>
<handler>
<type>static</type>
<status>200</status>
<content-type>text/plain; charset=UTF-8</content-type>
<response_content>config://http_server_default_response</response_content>
<type>predefine_query_handler</type>
<query>SELECT * FROM system.settings</query>
</handler>
</routing_rule>
<rule> &lt;!&ndash; no conditions, i.e. match everything &ndash;&gt;
<routing_rule>
<handler>
<type>static</type>
<status>404</status>
<content-type>text/html</content-type>
<reponse_content><body><h1>Check your query</h1></body></reponse_content>
<status>200</status>
<content_type>text/plain; charset=UTF-8</content_type>
<response_content>config://http_server_default_response</response_content>
</handler>
</rule>
</routing_rule>
</routing_rules>
-->

View File

@ -1,18 +0,0 @@
<?xml version="1.0"?>
<yandex>
<http_handlers>
<root_handler/>
<ping_handler>/ping_test</ping_handler>
<predefined_query_handler>
<method>POST</method>
<url>/test_one_handler_with_insert_and_select</url>
<queries>
<query>INSERT INTO test.test VALUES(1)</query>
<query>SELECT COUNT() FROM test.test WHERE id = {id:UInt8}</query>
</queries>
</predefined_query_handler>
<no_handler_description>test not found</no_handler_description>
</http_handlers>
</yandex>

View File

@ -45,6 +45,73 @@ def test_predefine_query_handler():
assert 404 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='POST', headers={'XXX': 'xxx'}).status_code
assert 400 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code
assert 500 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code
assert 'max_threads\t1\n' == cluster.instance.http_request('test_predefine_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content
assert 'max_threads\t1\nmax_alter_threads\t1\n' == cluster.instance.http_request(
'query_param_with_url/max_threads?max_threads=1&max_alter_threads=1', headers={'XXX': 'max_alter_threads'}).content
def test_fixed_static_handler():
with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "static_handler", "test_static_handler")) as cluster:
assert 404 == cluster.instance.http_request('', method='GET', headers={'XXX': 'xxx'}).status_code
assert 404 == cluster.instance.http_request('test_get_fixed_static_handler', method='GET', headers={'XXX': 'bad'}).status_code
assert 404 == cluster.instance.http_request('test_get_fixed_static_handler', method='POST', headers={'XXX': 'xxx'}).status_code
assert 402 == cluster.instance.http_request('test_get_fixed_static_handler', method='GET', headers={'XXX': 'xxx'}).status_code
assert 'text/html; charset=UTF-8' == cluster.instance.http_request('test_get_fixed_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type']
assert 'Test get static handler and fix content' == cluster.instance.http_request('test_get_fixed_static_handler', method='GET', headers={'XXX': 'xxx'}).content
def test_config_static_handler():
with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "static_handler", "test_static_handler")) as cluster:
assert 404 == cluster.instance.http_request('', method='GET', headers={'XXX': 'xxx'}).status_code
assert 404 == cluster.instance.http_request('test_get_config_static_handler', method='GET', headers={'XXX': 'bad'}).status_code
assert 404 == cluster.instance.http_request('test_get_config_static_handler', method='POST', headers={'XXX': 'xxx'}).status_code
# check default status code
assert 200 == cluster.instance.http_request('test_get_config_static_handler', method='GET', headers={'XXX': 'xxx'}).status_code
assert 'text/plain; charset=UTF-8' == cluster.instance.http_request('test_get_config_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type']
assert 'Test get static handler and config content' == cluster.instance.http_request('test_get_config_static_handler', method='GET', headers={'XXX': 'xxx'}).content
def test_absolute_path_static_handler():
with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "static_handler", "test_static_handler")) as cluster:
cluster.instance.exec_in_container(
['bash', '-c', 'echo "<html><body>Absolute Path File</body></html>" > /var/lib/clickhouse/user_files/absolute_path_file.html'],
privileged=True, user='root')
assert 404 == cluster.instance.http_request('', method='GET', headers={'XXX': 'xxx'}).status_code
assert 404 == cluster.instance.http_request('test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'bad'}).status_code
assert 404 == cluster.instance.http_request('test_get_absolute_path_static_handler', method='POST', headers={'XXX': 'xxx'}).status_code
# check default status code
assert 200 == cluster.instance.http_request('test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'xxx'}).status_code
assert 'text/html; charset=UTF-8' == cluster.instance.http_request('test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type']
assert '<html><body>Absolute Path File</body></html>\n' == cluster.instance.http_request('test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'xxx'}).content
def test_relative_path_static_handler():
with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "static_handler", "test_static_handler")) as cluster:
cluster.instance.exec_in_container(
['bash', '-c', 'echo "<html><body>Relative Path File</body></html>" > /var/lib/clickhouse/user_files/relative_path_file.html'],
privileged=True, user='root')
assert 404 == cluster.instance.http_request('', method='GET', headers={'XXX': 'xxx'}).status_code
assert 404 == cluster.instance.http_request('test_get_relative_path_static_handler', method='GET', headers={'XXX': 'bad'}).status_code
assert 404 == cluster.instance.http_request('test_get_relative_path_static_handler', method='POST', headers={'XXX': 'xxx'}).status_code
# check default status code
assert 200 == cluster.instance.http_request('test_get_relative_path_static_handler', method='GET', headers={'XXX': 'xxx'}).status_code
assert 'text/html; charset=UTF-8' == cluster.instance.http_request('test_get_relative_path_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type']
assert '<html><body>Relative Path File</body></html>\n' == cluster.instance.http_request('test_get_relative_path_static_handler', method='GET', headers={'XXX': 'xxx'}).content
assert '1\n' == cluster.instance.http_request('test_predefine_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content

View File

@ -1,31 +0,0 @@
<?xml version="1.0"?>
<yandex>
<http_handlers>
<dynamic_query_handler>
<method>POST</method>
<url>/post_query_params_and_settings</url>
<query_param_name>post_query_param</query_param_name>
</dynamic_query_handler>
<dynamic_query_handler>
<method>GET</method>
<url>/get_query_params_and_settings</url>
<query_param_name>get_query_param</query_param_name>
</dynamic_query_handler>
<dynamic_query_handler>
<method>GET</method>
<url><![CDATA[/query_param_with_url/\w+/(?P<param_name_1>[^/]+)(/(?P<param_name_2>[^/]+))?]]></url>
<query_param_name>query_param</query_param_name>
</dynamic_query_handler>
<dynamic_query_handler>
<headers>
<XXX>TEST_HEADER_VALUE</XXX>
<PARAMS_XXX><![CDATA[(?P<param_name_1>[^/]+)(/(?P<param_name_2>[^/]+))?]]></PARAMS_XXX>
</headers>
<query_param_name>query_param</query_param_name>
</dynamic_query_handler>
</http_handlers>
</yandex>

View File

@ -1,3 +0,0 @@
<?xml version="1.0"?>
<yandex>
</yandex>

View File

@ -1,43 +0,0 @@
<?xml version="1.0"?>
<yandex>
<http_handlers>
<predefined_query_handler>
<method>POST</method>
<url>/post_query_params_and_settings</url>
<queries>
<query>SELECT value FROM system.settings WHERE name = {name_1:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {name_2:String}</query>
</queries>
</predefined_query_handler>
<predefined_query_handler>
<method>GET</method>
<url>/get_query_params_and_settings</url>
<queries>
<query>SELECT value FROM system.settings WHERE name = {name_1:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {name_2:String}</query>
</queries>
</predefined_query_handler>
<predefined_query_handler>
<method>GET</method>
<url><![CDATA[/query_param_with_url/\w+/(?P<name_1>[^/]+)(/(?P<name_2>[^/]+))?]]></url>
<queries>
<query>SELECT value FROM system.settings WHERE name = {name_1:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {name_2:String}</query>
</queries>
</predefined_query_handler>
<predefined_query_handler>
<headers>
<XXX>TEST_HEADER_VALUE</XXX>
<PARAMS_XXX><![CDATA[(?P<name_1>[^/]+)(/(?P<name_2>[^/]+))?]]></PARAMS_XXX>
</headers>
<queries>
<query>SELECT value FROM system.settings WHERE name = {name_1:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {name_2:String}</query>
</queries>
</predefined_query_handler>
</http_handlers>
</yandex>

View File

@ -1,3 +0,0 @@
<?xml version="1.0"?>
<yandex>
</yandex>

View File

@ -8,7 +8,17 @@
<url>/test_predefine_handler_get</url>
<handler>
<type>predefine_query_handler</type>
<query>SELECT * FROM system.settings WHERE name = {setting_name:String}</query>
<query>SELECT name, value FROM system.settings WHERE name = {setting_name:String}</query>
</handler>
</routing_rule>
<routing_rule>
<url><![CDATA[regex:/query_param_with_url/(?P<setting_name_1>[^/]+)]]></url>
<headers>
<XXX><![CDATA[regex:(?P<setting_name_2>.+)]]></XXX>
</headers>
<handler>
<type>predefine_query_handler</type>
<query>SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String}</query>
</handler>
</routing_rule>
</routing_rules>

View File

@ -0,0 +1,52 @@
<?xml version="1.0"?>
<yandex>
<test_get_config_static_handler>Test get static handler and config content</test_get_config_static_handler>
<routing_rules>
<routing_rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/test_get_fixed_static_handler</url>
<handler>
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>Test get static handler and fix content</response_content>
</handler>
</routing_rule>
<routing_rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/test_get_config_static_handler</url>
<handler>
<type>static</type>
<response_content>config://test_get_config_static_handler</response_content>
</handler>
</routing_rule>
<routing_rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/test_get_absolute_path_static_handler</url>
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</routing_rule>
<routing_rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/test_get_relative_path_static_handler</url>
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</routing_rule>
</routing_rules>
</yandex>

View File

@ -354,15 +354,6 @@
</default>
</graphite_rollup_example>
<http_handlers>
<root_handler/>
<ping_handler>/ping</ping_handler>
<replicas_status_handler>/replicas_status</replicas_status_handler>
<dynamic_query_handler><query_param_name>query</query_param_name></dynamic_query_handler>
<no_handler_description>Use / or /ping for health checks.&#10;Or /replicas_status for more sophisticated health checks.&#10;Send queries from your program with POST method or GET /?query=...&#10;&#10;Use clickhouse-client:&#10;&#10;For interactive data analysis:&#10;clickhouse-client&#10;&#10;For batch query processing:&#10;clickhouse-client --query='SELECT 1' &gt; result&#10;clickhouse-client &lt; query &gt; result&#10;</no_handler_description>
</http_handlers>
<!-- Directory in <clickhouse-path> containing schema files for various input formats.
The directory will be created if it doesn't exist.
-->

View File

@ -8,3 +8,6 @@ URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/
${CLICKHOUSE_CURL} -vsS ${URL} --data-binary @- <<< "SELECT 1" 2>&1 | perl -lnE 'print if /Keep-Alive/';
${CLICKHOUSE_CURL} -vsS ${URL} --data-binary @- <<< " error here " 2>&1 | perl -lnE 'print if /Keep-Alive/';
${CLICKHOUSE_CURL} -vsS ${URL}ping 2>&1 | perl -lnE 'print if /Keep-Alive/';
# no keep-alive:
${CLICKHOUSE_CURL} -vsS ${URL}404/not/found/ 2>&1 | perl -lnE 'print if /Keep-Alive/';