From 9dc70545c0c7032546fff1c184bb05e490585abb Mon Sep 17 00:00:00 2001 From: tai Date: Tue, 5 Nov 2019 17:46:17 +0800 Subject: [PATCH 01/23] Add TestCase for feature/ISSUE-5436 --- .../integration/test_custom_http/__init__.py | 0 .../test_custom_http/configs/custom_http.xml | 11 +++++++ .../integration/test_custom_http/test.py | 31 +++++++++++++++++++ 3 files changed, 42 insertions(+) create mode 100644 dbms/tests/integration/test_custom_http/__init__.py create mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http.xml create mode 100644 dbms/tests/integration/test_custom_http/test.py diff --git a/dbms/tests/integration/test_custom_http/__init__.py b/dbms/tests/integration/test_custom_http/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http.xml b/dbms/tests/integration/test_custom_http/configs/custom_http.xml new file mode 100644 index 00000000000..22d570b7bca --- /dev/null +++ b/dbms/tests/integration/test_custom_http/configs/custom_http.xml @@ -0,0 +1,11 @@ + + 8123 + + + /${database}/a/${id}/${table} + INSERT INTO ${database:ASTIdentifier}.${table:ASTIdentifier}(id) VALUES + SELECT * FROM ${database:ASTIdenfier}.t + SELECT * FROM a.${table:ASTIdenfier} WHERE id={id:UInt8} + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py new file mode 100644 index 00000000000..d8f79e64e3d --- /dev/null +++ b/dbms/tests/integration/test_custom_http/test.py @@ -0,0 +1,31 @@ +import pytest +import requests + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', main_configs=['configs/custom_http.xml']) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + node.query(''' +CREATE DATABASE `test`; + +CREATE TABLE `test`.`test_custom_http` (`id` UInt8) Engine=Memory; + ''') + + yield cluster + finally: + cluster.shutdown() + +def test(started_cluster): + node_ip = cluster.get_instance_ip(node) + url = 'http://%s:8123/test/a/1/test_custom_http' % node_ip + data="(1)" + params = {'id':1} + response = requests.post(url, params = params, data = data) + + assert response.text == '\n1\n1\n' From 159ba24fcab37981197f50368122842b44227459 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 1 Nov 2019 18:03:21 +0800 Subject: [PATCH 02/23] ISSUES-5436 support custom http --- .../CustomHTTP/HTTPMatchExecutor.cpp | 259 ++++++++++++++++++ .../CustomHTTP/HTTPMatchExecutor.h | 55 ++++ .../CustomHTTP/HTTPMatchExecutorDefault.h | 46 ++++ .../CustomHTTP/HTTPStreamsWithInput.cpp | 73 +++++ .../CustomHTTP/HTTPStreamsWithInput.h | 29 ++ .../CustomHTTP/HTTPStreamsWithOutput.cpp | 217 +++++++++++++++ .../CustomHTTP/HTTPStreamsWithOutput.h | 48 ++++ programs/server/HTTPHandler.cpp | 22 +- programs/server/HTTPHandler.h | 6 +- src/IO/WriteBufferFromHTTPServerResponse.h | 5 + src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 2 + 12 files changed, 755 insertions(+), 13 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp new file mode 100644 index 00000000000..88890755955 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp @@ -0,0 +1,259 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int LOGICAL_ERROR; + extern const int REQUIRED_PASSWORD; + extern const int INVALID_SESSION_TIMEOUT; + extern const int UNKNOWN_COMPRESSION_METHOD; +} + + +namespace +{ + duration parseSessionTimeout(const HTMLForm & params, size_t default_session_timeout, size_t max_session_timeout) + { + size_t session_timeout = default_session_timeout; + + if (params.has("session_timeout")) + { + std::string session_timeout_str = params.get("session_timeout"); + + ReadBufferFromString buf(session_timeout_str); + if (!tryReadIntText(session_timeout, buf) || !buf.eof()) + throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); + + if (session_timeout > max_session_timeout) + throw Exception( + "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); + } + + return std::chrono::seconds(session_timeout); + } +} + + +void HTTPMatchExecutor::execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const +{ + authentication(context, request, params); + + std::shared_ptr session_context; + String session_id = params.get("session_id", ""); + duration session_timeout = parseSessionTimeout(params, 1800, 3600); + + SCOPE_EXIT({ detachSessionContext(session_context, session_id, session_timeout); }); + session_context = attachSessionContext(context, params, session_id, session_timeout); + + initClientInfo(context, request); + used_output.attachRequestAndResponse(context, request, response, params, /* TODO: keep_alive_time_out */ 0); + + HTTPStreamsWithInput used_input(request, params); + collectParamsAndApplySettings(request, params, context); + + Settings & query_settings = context.getSettingsRef(); + used_input.attachSettings(context, query_settings, request); + used_output.attachSettings(context, query_settings, request); + + String execute_query_string = getExecuteQuery(params); + ReadBufferPtr query_in_buffer = std::make_shared(execute_query_string); + + ReadBufferPtr in = query_in_buffer; + if (!needParsePostBody(request, params) || !context.getExternalTables().empty()) + in = std::make_shared(*query_in_buffer, *used_input.in_maybe_internal_compressed); + + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); }); + + used_output.finalize(); +} + +void HTTPMatchExecutor::initClientInfo(Context & context, HTTPServerRequest & request) const +{ + + ClientInfo & client_info = context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); +} + +void HTTPMatchExecutor::authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); + + if (user.empty() && password.empty() && quota_key.empty()) + { + /// User name and password can be passed using query parameters + /// or using HTTP Basic auth (both methods are insecure). + if (request.hasCredentials()) + { + Poco::Net::HTTPBasicCredentials credentials(request); + + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else + { + user = params.get("user", "default"); + password = params.get("password", ""); + } + + quota_key = params.get("quota_key", ""); + } + else + { + /// It is prohibited to mix different authorization schemes. + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) + { + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); + } + } + + std::string query_id = params.get("query_id", ""); + context.setUser(user, password, request.clientAddress(), quota_key); + context.setCurrentQueryId(query_id); +} + +std::shared_ptr HTTPMatchExecutor::attachSessionContext( + Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const +{ + if (!session_id.empty()) + { + std::string session_check = params.get("session_check", ""); + auto session_context = context.acquireSession(session_id, session_timeout, session_check == "1"); + + context = *session_context; + context.setSessionContext(*session_context); + return session_context; + } + return {}; +} + +void HTTPMatchExecutor::detachSessionContext(std::shared_ptr & session_context, const String & session_id, const duration & session_timeout) const +{ + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} + +void HTTPMatchExecutor::collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const +{ + static const NameSet reserved_param_names{ + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", + "session_id", "session_timeout", "session_check"}; + + Names reserved_param_suffixes; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + { + Settings & settings = context.getSettingsRef(); + + if (settings.readonly == 0) + settings.readonly = 2; + } + + bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); + + if (has_multipart || needParsePostBody(request, params)) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + if (has_multipart) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + context.setCurrentDatabase(value); + else if (key == "default_format") + context.setDefaultFormat(value); + else if (!param_could_be_skipped(key) && !acceptQueryParam(context, key, value)) + settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h new file mode 100644 index 00000000000..425605aec07 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class HTTPMatchExecutor; +struct HTTPStreamsWithInput; +struct HTTPStreamsWithOutput; + +using duration = std::chrono::steady_clock::duration; +using HTTPMatchExecutorPtr = std::shared_ptr; + +class HTTPMatchExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + bool match(HTTPServerRequest & request, HTMLForm & params) const { return matchImpl(request, params); }; + + void execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const; + + virtual ~HTTPMatchExecutor() = default; +protected: + + virtual bool matchImpl(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual String getExecuteQuery(HTMLForm & params) const = 0; + + virtual bool needParsePostBody(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual bool acceptQueryParam(Context & context, const String & key, const String & value) const = 0; + + void initClientInfo(Context & context, HTTPServerRequest & request) const; + + void authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const; + + void detachSessionContext(std::shared_ptr & context, const String & session_id, const duration & session_timeout) const; + + std::shared_ptr attachSessionContext(Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const; + + void collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const; + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h new file mode 100644 index 00000000000..54ea20120f5 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class HTTPMatchExecutorDefault : public HTTPMatchExecutor +{ +protected: + + bool matchImpl(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + + bool needParsePostBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } + + String getExecuteQuery(HTMLForm & params) const override + { + String execute_query; + for (const auto & [key, value] : params) + { + if (key == "query") + execute_query += value; + } + + return execute_query; + } + + bool acceptQueryParam(Context &context, const String &key, const String &value) const override + { + if (startsWith(key, "param_")) + { + /// Save name and values of substitution in dictionary. + context.setQueryParameter(key.substr(strlen("param_")), value); + return true; + } + + return key == "query"; + } + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp new file mode 100644 index 00000000000..6ff17217f24 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp @@ -0,0 +1,73 @@ +#include + +#include +#include +#include +#include "HTTPStreamsWithInput.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_COMPRESSION_METHOD; +} + +HTTPStreamsWithInput::HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from) + : in(createRawInBuffer(request)) + , in_maybe_compressed(createCompressedBuffer(request, in)) + , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) +{ +} + +ReadBufferPtr HTTPStreamsWithInput::createRawInBuffer(HTTPServerRequest & request) const +{ + return std::make_unique(request.stream()); +} + +ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +{ + /// Request body can be compressed using algorithm specified in the Content-Encoding header. + String http_compressed_method = request.get("Content-Encoding", ""); + + if (!http_compressed_method.empty()) + { + if (http_compressed_method == "gzip") + return std::make_shared(*raw_buffer, CompressionMethod::Gzip); + else if (http_compressed_method == "deflate") + return std::make_shared(*raw_buffer, CompressionMethod::Zlib); +#if USE_BROTLI + else if (http_compressed_method == "br") + return std::make_shared(*raw_buffer); +#endif + else + throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); + } + + return raw_buffer; +} + +ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +{ + /// The data can also be compressed using incompatible internal algorithm. This is indicated by + /// 'decompress' query parameter. + std::unique_ptr in_post_maybe_compressed; + if (params.getParsed("decompress", false)) + return std::make_unique(*http_maybe_encoding_buffer); + + return http_maybe_encoding_buffer; +} + +void HTTPStreamsWithInput::attachSettings(Context & /*context*/, Settings & settings, HTTPServerRequest & /*request*/) +{ + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (settings.http_native_compression_disable_checksumming_on_decompress) + { + if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + compressed_buffer->disableChecksumming(); + } +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h new file mode 100644 index 00000000000..2c3bd1c2a35 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; + +struct HTTPStreamsWithInput +{ + std::shared_ptr in; + std::shared_ptr in_maybe_compressed; + std::shared_ptr in_maybe_internal_compressed; + + HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from); + + void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + + ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; + ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; + ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; +}; + +} \ No newline at end of file diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp new file mode 100644 index 00000000000..35426315eb3 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp @@ -0,0 +1,217 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + inline void listeningProgress(Context & context, ProgressCallback listener) + { + auto prev = context.getProgressCallback(); + context.setProgressCallback([prev, listener] (const Progress & progress) + { + if (prev) + prev(progress); + + listener(progress); + }); + } + + inline ProgressCallback cancelListener(Context & context, Poco::Net::StreamSocket & socket) + { + /// Assume that at the point this method is called no one is reading data from the socket any more. + /// True for read-only queries. + return [&context, &socket](const Progress &) + { + try + { + char b; + int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); + if (status == 0) + context.killCurrentQuery(); + } + catch (Poco::TimeoutException &) + { + } + catch (...) + { + context.killCurrentQuery(); + } + }; + } +} + +void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & settings, HTTPServerRequest & request) +{ + /// HTTP response compression is turned on only if the client signalled that they support it + /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. + out->setCompression(out->getCompression() && settings.enable_http_compression); + if (out->getCompression()) + out->setCompressionLevel(settings.http_zlib_compression_level); + + out->setSendProgressInterval(settings.http_headers_progress_interval_ms); + + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed + /// Origin header. + out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + + /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. + if (settings.send_progress_in_http_headers) + listeningProgress(context, [this] (const Progress & progress) { out->onProgress(progress); }); + + if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) + { + Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); + + listeningProgress(context, cancelListener(context, socket)); + } +} + +void HTTPStreamsWithOutput::attachRequestAndResponse( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) +{ + out = createEndpoint(request, response, keep_alive_timeout); + out_maybe_compressed = createMaybeCompressionEndpoint(form, out); + out_maybe_delayed_and_compressed = createMaybeDelayedAndCompressionEndpoint(context, form, out_maybe_compressed); +} + +std::shared_ptr HTTPStreamsWithOutput::createEndpoint( + HTTPServerRequest & request, HTTPServerResponse & response, size_t 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", ""); + + if (!http_response_compression_methods.empty()) + { + /// 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("gzip")) + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive_timeout, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared( + request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + +WriteBufferPtr HTTPStreamsWithOutput::createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint) +{ + /// Client can pass a 'compress' flag in the query string. In this case the query result is + /// compressed using internal algorithm. This is not reflected in HTTP headers. + bool internal_compression = form.getParsed("compress", false); + return internal_compression ? std::make_shared(*endpoint) : WriteBufferPtr(endpoint); +} + +WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint) +{ + /// If it is specified, the whole result will be buffered. + /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. + bool buffer_until_eof = form.getParsed("wait_end_of_query", false); + + /// At least, we should postpone sending of first buffer_size result bytes + size_t buffer_size_total = std::max(form.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + + size_t buffer_size_memory = (buffer_size_total > DBMS_DEFAULT_BUFFER_SIZE) ? buffer_size_total : 0; + + if (buffer_size_memory > 0 || buffer_until_eof) + { + CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; + CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; + + if (buffer_size_memory > 0) + cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); + + if (buffer_until_eof) + { + std::string tmp_path_template = context.getTemporaryPath() + "http_buffers/"; + + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; + + cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); + } + else + { + auto push_memory_buffer_and_continue = [next_buffer = endpoint] (const WriteBufferPtr & prev_buf) + { + auto prev_memory_buffer = typeid_cast(prev_buf.get()); + if (!prev_memory_buffer) + throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); + copyData(*rdbuf , *next_buffer); + + return next_buffer; + }; + + cascade_buffer2.emplace_back(push_memory_buffer_and_continue); + } + + return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); + } + + return endpoint; +} + +void HTTPStreamsWithOutput::finalize() const +{ + if (out_maybe_delayed_and_compressed != out_maybe_compressed) + { + /// TODO: set Content-Length if possible + std::vector write_buffers; + std::vector read_buffers; + std::vector read_buffers_raw_ptr; + + auto cascade_buffer = typeid_cast(out_maybe_delayed_and_compressed.get()); + if (!cascade_buffer) + throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + cascade_buffer->getResultBuffers(write_buffers); + + if (write_buffers.empty()) + throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); + + for (auto & write_buf : write_buffers) + { + IReadableWriteBuffer * write_buf_concrete; + ReadBufferPtr reread_buf; + + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) + && (reread_buf = write_buf_concrete->tryGetReadBuffer())) + { + read_buffers.emplace_back(reread_buf); + read_buffers_raw_ptr.emplace_back(reread_buf.get()); + } + } + + ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); + copyData(concat_read_buffer, *out_maybe_compressed); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to + /// the client. + out->finalize(); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h new file mode 100644 index 00000000000..65d87b6744a --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; +using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + +/* Raw data + * ↓ + * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) + * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) + * CompressedWriteBuffer out_maybe_compressed (optional) + * ↓ + * WriteBufferFromHTTPServerResponse out + */ +struct HTTPStreamsWithOutput +{ + using HTTPResponseBufferPtr = std::shared_ptr; + + HTTPResponseBufferPtr out; + /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. + std::shared_ptr out_maybe_compressed; + /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. + std::shared_ptr out_maybe_delayed_and_compressed; + + void finalize() const; + + WriteBufferPtr createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint); + + WriteBufferPtr createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint); + + HTTPResponseBufferPtr createEndpoint(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); + + void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + + void attachRequestAndResponse(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); +}; + +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 77236210741..99afb319ed5 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -216,7 +216,7 @@ void HTTPHandler::processQuery( Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, - Output & used_output) + HTTPStreamsWithOutput & used_output) { Context context = server.context(); @@ -224,6 +224,12 @@ void HTTPHandler::processQuery( LOG_TRACE(log, "Request URI: " << request.getURI()); + if (context.getSettingsRef().allow_experimental_custom_http) + { + context.getHTTPMatchExecutor()->execute(context, request, response, params, used_output); + return; + } + std::istream & istr = request.stream(); /// Part of the query can be passed in the 'query' parameter and the rest in the request body @@ -605,20 +611,14 @@ void HTTPHandler::processQuery( } ); - if (used_output.hasDelayed()) - { - /// TODO: set Content-Length if possible - pushDelayedResults(used_output); - } - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to /// the client. - used_output.out->finalize(); + used_output.finalize(); } void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - Output & used_output) + HTTPStreamsWithOutput & used_output) { try { @@ -655,7 +655,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ else if (used_output.out_maybe_compressed) { /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references - if (used_output.hasDelayed()) + if (used_output.out_maybe_delayed_and_compressed != used_output.out_maybe_compressed) used_output.out_maybe_delayed_and_compressed.reset(); /// Send the error message into already used (and possibly compressed) stream. @@ -691,7 +691,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne setThreadName("HTTPHandler"); ThreadStatus thread_status; - Output used_output; + HTTPStreamsWithOutput used_output; /// In case of exception, send stack trace to client. bool with_stacktrace = false; diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 681a3cce932..603278e11aa 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,6 +7,8 @@ #include #include +#include + namespace CurrentMetrics { @@ -68,14 +70,14 @@ private: Poco::Net::HTTPServerRequest & request, HTMLForm & params, Poco::Net::HTTPServerResponse & response, - Output & used_output); + HTTPStreamsWithOutput & used_output); void trySendExceptionToClient( const std::string & s, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - Output & used_output); + HTTPStreamsWithOutput & used_output); static void pushDelayedResults(Output & used_output); }; diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index ffa36c11c5b..c10288fff9f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -113,6 +113,11 @@ public: compress = enable_compression; } + bool getCompression() + { + return compress; + } + /// Set compression level if the compression is turned on. /// The setting has any effect only if HTTP headers haven't been sent yet. void setCompressionLevel(int level) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 647c3fb8020..03dd311f31b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -348,6 +349,7 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config + mutable std::mutex match_executor_mutex; /// Guards match executor and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -1543,6 +1545,10 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } +HTTPMatchExecutorPtr Context::getHTTPMatchExecutor() +{ + return std::make_shared(); +} void Context::initializeSystemLogs() { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1f81cdbc58b..03859b03348 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,6 +490,8 @@ public: Compiler & getCompiler(); + HTTPMatchExecutorPtr getHTTPMatchExecutor(); + /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 183eb82b7caa012849e2c09f5ffa36c1e5f91d66 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 5 Nov 2019 02:36:28 +0800 Subject: [PATCH 03/23] ISSUES-5436 support custom http [part 2] --- .../Interpreters/CustomHTTP/CustomExecutor.h | 41 ++ .../CustomHTTP/CustomExecutorDefault.h | 50 ++ .../CustomHTTP/ExtractorClientInfo.h | 38 ++ .../CustomHTTP/ExtractorContextChange.h | 98 ++++ ...eamsWithInput.cpp => HTTPInputStreams.cpp} | 33 +- ...PStreamsWithInput.h => HTTPInputStreams.h} | 6 +- .../CustomHTTP/HTTPMatchExecutor.cpp | 259 ---------- .../CustomHTTP/HTTPMatchExecutor.h | 55 -- .../CustomHTTP/HTTPMatchExecutorDefault.h | 46 -- ...msWithOutput.cpp => HTTPOutputStreams.cpp} | 35 +- ...treamsWithOutput.h => HTTPOutputStreams.h} | 16 +- programs/server/HTTPHandler.cpp | 476 +++--------------- programs/server/HTTPHandler.h | 48 +- src/Common/HTMLForm.h | 6 + src/Interpreters/Context.cpp | 12 +- src/Interpreters/Context.h | 2 +- 16 files changed, 359 insertions(+), 862 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithInput.cpp => HTTPInputStreams.cpp} (72%) rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithInput.h => HTTPInputStreams.h} (78%) delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithOutput.cpp => HTTPOutputStreams.cpp} (87%) rename dbms/src/Interpreters/CustomHTTP/{HTTPStreamsWithOutput.h => HTTPOutputStreams.h} (61%) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h new file mode 100644 index 00000000000..0778256b297 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class CustomExecutor; +struct HTTPInputStreams; +struct HTTPOutputStreams; + +using duration = std::chrono::steady_clock::duration; +using HTTPMatchExecutorPtr = std::shared_ptr; + +class CustomExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + virtual ~CustomExecutor() = default; + + virtual bool isQueryParam(const String & param_name) const = 0; + + virtual bool match(HTTPServerRequest & request, HTMLForm & params) const = 0; + + virtual bool canBeParseRequestBody(HTTPServerRequest & request, HTMLForm & params) = 0; + + using QueryExecutor = std::function; + using QueryExecutors = std::vector; + virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h new file mode 100644 index 00000000000..218cb413222 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class CustomExecutorDefault : public CustomExecutor +{ +public: + bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + + bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) override { return false; } + + bool isQueryParam(const String & param_name) const override + { + return param_name == "query" || startsWith(param_name, "param_"); + } + + QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override + { + ReadBufferPtr in = prepareAndGetQueryInput(context, request, params, input_streams); + + return {[&, shared_in = in](HTTPOutputStreams & output, HTTPServerResponse & response) + { + executeQuery( + *shared_in, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + }}; + } + +private: + ReadBufferPtr prepareAndGetQueryInput(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const + { + for (const auto & [key, value] : params) + { + + } + } +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h b/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h new file mode 100644 index 00000000000..76840f3d682 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ExtractorClientInfo +{ +public: + ExtractorClientInfo(ClientInfo & info_) : client_info(info_) {} + + void extract(Poco::Net::HTTPServerRequest & request) + { + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); + } + +private: + ClientInfo & client_info; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h new file mode 100644 index 00000000000..a228f75391f --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -0,0 +1,98 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ExtractorContextChange +{ +public: + ExtractorContextChange(Context & context_, const HTTPMatchExecutorPtr & executor_) : context(context_), executor(executor_) {} + + void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + { + Names reserved_param_suffixes; + static const NameSet reserved_param_names{ + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", + "session_id", "session_timeout", "session_check"}; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + { + Settings & settings = context.getSettingsRef(); + + if (settings.readonly == 0) + settings.readonly = 2; + } + + bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); + + if (has_multipart || executor->canBeParseRequestBody(request, params)) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + if (has_multipart) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + context.setCurrentDatabase(value); + else if (key == "default_format") + context.setDefaultFormat(value); + else if (!param_could_be_skipped(key) && !executor->isQueryParam(key)) + settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); + } + +private: + Context & context; + const HTTPMatchExecutorPtr & executor; + +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp similarity index 72% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp rename to dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index 6ff17217f24..e6985621b00 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -1,9 +1,10 @@ -#include +#include + +#include #include #include #include -#include "HTTPStreamsWithInput.h" namespace DB @@ -14,19 +15,26 @@ namespace ErrorCodes extern const int UNKNOWN_COMPRESSION_METHOD; } -HTTPStreamsWithInput::HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from) +HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) : in(createRawInBuffer(request)) , in_maybe_compressed(createCompressedBuffer(request, in)) , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) { + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) + { + if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + compressed_buffer->disableChecksumming(); + } } -ReadBufferPtr HTTPStreamsWithInput::createRawInBuffer(HTTPServerRequest & request) const +ReadBufferPtr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -38,7 +46,7 @@ ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & r else if (http_compressed_method == "deflate") return std::make_shared(*raw_buffer, CompressionMethod::Zlib); #if USE_BROTLI - else if (http_compressed_method == "br") + else if (http_compressed_method == "br") return std::make_shared(*raw_buffer); #endif else @@ -48,7 +56,7 @@ ReadBufferPtr HTTPStreamsWithInput::createCompressedBuffer(HTTPServerRequest & r return raw_buffer; } -ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -59,15 +67,4 @@ ReadBufferPtr HTTPStreamsWithInput::createInternalCompressedBuffer(HTMLForm & pa return http_maybe_encoding_buffer; } -void HTTPStreamsWithInput::attachSettings(Context & /*context*/, Settings & settings, HTTPServerRequest & /*request*/) -{ - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (settings.http_native_compression_disable_checksumming_on_decompress) - { - if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) - compressed_buffer->disableChecksumming(); - } -} - } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h similarity index 78% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h rename to dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h index 2c3bd1c2a35..88f0438398d 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithInput.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h @@ -11,15 +11,13 @@ namespace DB using HTTPServerRequest = Poco::Net::HTTPServerRequest; -struct HTTPStreamsWithInput +struct HTTPInputStreams { std::shared_ptr in; std::shared_ptr in_maybe_compressed; std::shared_ptr in_maybe_internal_compressed; - HTTPStreamsWithInput(HTTPServerRequest & request, HTMLForm & from); - - void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); + HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp deleted file mode 100644 index 88890755955..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.cpp +++ /dev/null @@ -1,259 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; - extern const int LOGICAL_ERROR; - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; - extern const int UNKNOWN_COMPRESSION_METHOD; -} - - -namespace -{ - duration parseSessionTimeout(const HTMLForm & params, size_t default_session_timeout, size_t max_session_timeout) - { - size_t session_timeout = default_session_timeout; - - if (params.has("session_timeout")) - { - std::string session_timeout_str = params.get("session_timeout"); - - ReadBufferFromString buf(session_timeout_str); - if (!tryReadIntText(session_timeout, buf) || !buf.eof()) - throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); - - if (session_timeout > max_session_timeout) - throw Exception( - "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + - ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); - } - - return std::chrono::seconds(session_timeout); - } -} - - -void HTTPMatchExecutor::execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const -{ - authentication(context, request, params); - - std::shared_ptr session_context; - String session_id = params.get("session_id", ""); - duration session_timeout = parseSessionTimeout(params, 1800, 3600); - - SCOPE_EXIT({ detachSessionContext(session_context, session_id, session_timeout); }); - session_context = attachSessionContext(context, params, session_id, session_timeout); - - initClientInfo(context, request); - used_output.attachRequestAndResponse(context, request, response, params, /* TODO: keep_alive_time_out */ 0); - - HTTPStreamsWithInput used_input(request, params); - collectParamsAndApplySettings(request, params, context); - - Settings & query_settings = context.getSettingsRef(); - used_input.attachSettings(context, query_settings, request); - used_output.attachSettings(context, query_settings, request); - - String execute_query_string = getExecuteQuery(params); - ReadBufferPtr query_in_buffer = std::make_shared(execute_query_string); - - ReadBufferPtr in = query_in_buffer; - if (!needParsePostBody(request, params) || !context.getExternalTables().empty()) - in = std::make_shared(*query_in_buffer, *used_input.in_maybe_internal_compressed); - - executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); }); - - used_output.finalize(); -} - -void HTTPMatchExecutor::initClientInfo(Context & context, HTTPServerRequest & request) const -{ - - ClientInfo & client_info = context.getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); -} - -void HTTPMatchExecutor::authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); - - if (user.empty() && password.empty() && quota_key.empty()) - { - /// User name and password can be passed using query parameters - /// or using HTTP Basic auth (both methods are insecure). - if (request.hasCredentials()) - { - Poco::Net::HTTPBasicCredentials credentials(request); - - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else - { - user = params.get("user", "default"); - password = params.get("password", ""); - } - - quota_key = params.get("quota_key", ""); - } - else - { - /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() - || params.has("user") - || params.has("password") - || params.has("quota_key")) - { - throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); - } - } - - std::string query_id = params.get("query_id", ""); - context.setUser(user, password, request.clientAddress(), quota_key); - context.setCurrentQueryId(query_id); -} - -std::shared_ptr HTTPMatchExecutor::attachSessionContext( - Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const -{ - if (!session_id.empty()) - { - std::string session_check = params.get("session_check", ""); - auto session_context = context.acquireSession(session_id, session_timeout, session_check == "1"); - - context = *session_context; - context.setSessionContext(*session_context); - return session_context; - } - return {}; -} - -void HTTPMatchExecutor::detachSessionContext(std::shared_ptr & session_context, const String & session_id, const duration & session_timeout) const -{ - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} - -void HTTPMatchExecutor::collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const -{ - static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", - "session_id", "session_timeout", "session_check"}; - - Names reserved_param_suffixes; - - auto param_could_be_skipped = [&] (const String & name) - { - if (reserved_param_names.count(name)) - return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } - - return false; - }; - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - { - Settings & settings = context.getSettingsRef(); - - if (settings.readonly == 0) - settings.readonly = 2; - } - - bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); - - if (has_multipart || needParsePostBody(request, params)) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - if (has_multipart) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - } - - SettingsChanges settings_changes; - for (const auto & [key, value] : params) - { - if (key == "database") - context.setCurrentDatabase(value); - else if (key == "default_format") - context.setDefaultFormat(value); - else if (!param_could_be_skipped(key) && !acceptQueryParam(context, key, value)) - settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h deleted file mode 100644 index 425605aec07..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutor.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class Context; -class HTTPMatchExecutor; -struct HTTPStreamsWithInput; -struct HTTPStreamsWithOutput; - -using duration = std::chrono::steady_clock::duration; -using HTTPMatchExecutorPtr = std::shared_ptr; - -class HTTPMatchExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - - bool match(HTTPServerRequest & request, HTMLForm & params) const { return matchImpl(request, params); }; - - void execute(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & params, HTTPStreamsWithOutput & used_output) const; - - virtual ~HTTPMatchExecutor() = default; -protected: - - virtual bool matchImpl(HTTPServerRequest & request, HTMLForm & params) const = 0; - - virtual String getExecuteQuery(HTMLForm & params) const = 0; - - virtual bool needParsePostBody(HTTPServerRequest & request, HTMLForm & params) const = 0; - - virtual bool acceptQueryParam(Context & context, const String & key, const String & value) const = 0; - - void initClientInfo(Context & context, HTTPServerRequest & request) const; - - void authentication(Context & context, HTTPServerRequest & request, HTMLForm & params) const; - - void detachSessionContext(std::shared_ptr & context, const String & session_id, const duration & session_timeout) const; - - std::shared_ptr attachSessionContext(Context & context, HTMLForm & params, const String & session_id, const duration & session_timeout) const; - - void collectParamsAndApplySettings(HTTPServerRequest & request, HTMLForm & params, Context & context) const; - -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h deleted file mode 100644 index 54ea20120f5..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPMatchExecutorDefault.h +++ /dev/null @@ -1,46 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -class HTTPMatchExecutorDefault : public HTTPMatchExecutor -{ -protected: - - bool matchImpl(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } - - bool needParsePostBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } - - String getExecuteQuery(HTMLForm & params) const override - { - String execute_query; - for (const auto & [key, value] : params) - { - if (key == "query") - execute_query += value; - } - - return execute_query; - } - - bool acceptQueryParam(Context &context, const String &key, const String &value) const override - { - if (startsWith(key, "param_")) - { - /// Save name and values of substitution in dictionary. - context.setQueryParameter(key.substr(strlen("param_")), value); - return true; - } - - return key == "query"; - } - -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp similarity index 87% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp rename to dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 35426315eb3..156aa6ba975 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,6 +8,7 @@ #include #include #include +#include "HTTPOutputStreams.h" namespace DB @@ -51,8 +52,14 @@ namespace } } -void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & settings, HTTPServerRequest & request) +HTTPOutputStreams::HTTPOutputStreams( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) + : out(createResponseOut(request, response, keep_alive_timeout)) + , out_maybe_compressed(createMaybeCompressionOut(form, out)) + , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { + Settings & settings = context.getSettingsRef(); + /// HTTP response compression is turned on only if the client signalled that they support it /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. out->setCompression(out->getCompression() && settings.enable_http_compression); @@ -77,16 +84,8 @@ void HTTPStreamsWithOutput::attachSettings(Context & context, Settings & setting } } -void HTTPStreamsWithOutput::attachRequestAndResponse( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) -{ - out = createEndpoint(request, response, keep_alive_timeout); - out_maybe_compressed = createMaybeCompressionEndpoint(form, out); - out_maybe_delayed_and_compressed = createMaybeDelayedAndCompressionEndpoint(context, form, out_maybe_compressed); -} - -std::shared_ptr HTTPStreamsWithOutput::createEndpoint( - HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout) +std::shared_ptr HTTPOutputStreams::createResponseOut( + HTTPServerRequest &request, HTTPServerResponse &response, size_t 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", ""); @@ -112,15 +111,15 @@ std::shared_ptr HTTPStreamsWithOutput::create request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); } -WriteBufferPtr HTTPStreamsWithOutput::createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint) +WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. bool internal_compression = form.getParsed("compress", false); - return internal_compression ? std::make_shared(*endpoint) : WriteBufferPtr(endpoint); + return internal_compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } -WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint) +WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) { /// If it is specified, the whole result will be buffered. /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. @@ -152,7 +151,7 @@ WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(C } else { - auto push_memory_buffer_and_continue = [next_buffer = endpoint] (const WriteBufferPtr & prev_buf) + auto push_memory_buffer_and_continue = [next_buffer = out_] (const WriteBufferPtr & prev_buf) { auto prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) @@ -170,10 +169,10 @@ WriteBufferPtr HTTPStreamsWithOutput::createMaybeDelayedAndCompressionEndpoint(C return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); } - return endpoint; + return out_; } -void HTTPStreamsWithOutput::finalize() const +void HTTPOutputStreams::finalize() const { if (out_maybe_delayed_and_compressed != out_maybe_compressed) { diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h similarity index 61% rename from dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h rename to dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index 65d87b6744a..b8edaad0b37 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPStreamsWithOutput.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -22,7 +22,7 @@ using HTTPServerResponse = Poco::Net::HTTPServerResponse; * ↓ * WriteBufferFromHTTPServerResponse out */ -struct HTTPStreamsWithOutput +struct HTTPOutputStreams { using HTTPResponseBufferPtr = std::shared_ptr; @@ -32,17 +32,17 @@ struct HTTPStreamsWithOutput /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. std::shared_ptr out_maybe_delayed_and_compressed; + HTTPOutputStreams() = default; + + HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + void finalize() const; - WriteBufferPtr createMaybeDelayedAndCompressionEndpoint(Context & context, HTMLForm & form, WriteBufferPtr & endpoint); + WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); - WriteBufferPtr createMaybeCompressionEndpoint(HTMLForm & form, std::shared_ptr & endpoint); + WriteBufferPtr createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_); - HTTPResponseBufferPtr createEndpoint(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); - - void attachSettings(Context & context, Settings & settings, HTTPServerRequest & request); - - void attachRequestAndResponse(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); }; } diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 99afb319ed5..fb018979938 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -170,80 +170,40 @@ static std::chrono::steady_clock::duration parseSessionTimeout( } -void HTTPHandler::pushDelayedResults(Output & used_output) -{ - std::vector write_buffers; - std::vector read_buffers; - std::vector read_buffers_raw_ptr; - - auto cascade_buffer = typeid_cast(used_output.out_maybe_delayed_and_compressed.get()); - if (!cascade_buffer) - throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - cascade_buffer->getResultBuffers(write_buffers); - - if (write_buffers.empty()) - throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); - - for (auto & write_buf : write_buffers) - { - IReadableWriteBuffer * write_buf_concrete; - ReadBufferPtr reread_buf; - - if (write_buf - && (write_buf_concrete = dynamic_cast(write_buf.get())) - && (reread_buf = write_buf_concrete->tryGetReadBuffer())) - { - read_buffers.emplace_back(reread_buf); - read_buffers_raw_ptr.emplace_back(reread_buf.get()); - } - } - - ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); - copyData(concat_read_buffer, *used_output.out_maybe_compressed); -} - - HTTPHandler::HTTPHandler(IServer & server_) - : server(server_) - , log(&Logger::get("HTTPHandler")) + : server(server_), log(&Logger::get("HTTPHandler")) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } -void HTTPHandler::processQuery( - Poco::Net::HTTPServerRequest & request, - HTMLForm & params, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output) +HTTPHandler::SessionContextHolder::~SessionContextHolder() { - Context context = server.context(); + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} - CurrentThread::QueryScope query_scope(context); - LOG_TRACE(log, "Request URI: " << request.getURI()); +HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_server, HTMLForm & params) +{ + session_id = params.get("session_id", ""); + context = std::make_unique(accepted_server.context()); - if (context.getSettingsRef().allow_experimental_custom_http) + if (!session_id.empty()) { - context.getHTTPMatchExecutor()->execute(context, request, response, params, used_output); - return; + session_timeout = parseSessionTimeout(accepted_server.config(), params); + session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + context = std::make_unique(*session_context); + context->setSessionContext(*session_context); } +} - std::istream & istr = request.stream(); - - /// Part of the query can be passed in the 'query' parameter and the rest in the request body - /// (http method need not necessarily be POST). In this case the entire query consists of the - /// contents of the 'query' parameter, a line break and the request body. - std::string query_param = params.get("query", ""); - if (!query_param.empty()) - query_param += '\n'; - - /// The user and password can be passed by headers (similar to X-Auth-*), - /// which is used by load balancers to pass authentication information. - std::string user = request.get("X-ClickHouse-User", ""); - std::string password = request.get("X-ClickHouse-Key", ""); - std::string quota_key = request.get("X-ClickHouse-Quota", ""); +void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & request, HTMLForm & params) +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); if (user.empty() && password.empty() && quota_key.empty()) { @@ -277,348 +237,35 @@ void HTTPHandler::processQuery( } std::string query_id = params.get("query_id", ""); - context.setUser(user, password, request.clientAddress(), quota_key); - context.setCurrentQueryId(query_id); - - /// The user could specify session identifier and session timeout. - /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. - - std::shared_ptr session; - String session_id; - std::chrono::steady_clock::duration session_timeout; - bool session_is_set = params.has("session_id"); - const auto & config = server.config(); - - if (session_is_set) - { - session_id = params.get("session_id"); - session_timeout = parseSessionTimeout(config, params); - std::string session_check = params.get("session_check", ""); - - session = context.acquireNamedSession(session_id, session_timeout, session_check == "1"); - - context = session->context; - context.setSessionContext(session->context); - } - - SCOPE_EXIT({ - if (session) - session->release(); - }); - - /// 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; - - /// Client can pass a 'compress' flag in the query string. In this case the query result is - /// compressed using internal algorithm. This is not reflected in HTTP headers. - bool internal_compression = params.getParsed("compress", false); - - /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total = std::max( - params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); - - /// If it is specified, the whole result will be buffered. - /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. - bool buffer_until_eof = params.getParsed("wait_end_of_query", false); - - size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; - size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; - - unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); - - used_output.out = std::make_shared( - request, response, keep_alive_timeout, client_supports_http_compression, http_response_compression_method); - - if (internal_compression) - used_output.out_maybe_compressed = std::make_shared(*used_output.out); - else - used_output.out_maybe_compressed = used_output.out; - - if (buffer_size_memory > 0 || buffer_until_eof) - { - CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; - CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; - - if (buffer_size_memory > 0) - cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); - - if (buffer_until_eof) - { - const std::string tmp_path(context.getTemporaryVolume()->getNextDisk()->getPath()); - const std::string tmp_path_template(tmp_path + "http_buffers/"); - - auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) - { - return WriteBufferFromTemporaryFile::create(tmp_path_template); - }; - - cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); - } - else - { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) - { - auto prev_memory_buffer = typeid_cast(prev_buf.get()); - if (!prev_memory_buffer) - throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); - - return next_buffer; - }; - - cascade_buffer2.emplace_back(push_memory_buffer_and_continue); - } - - used_output.out_maybe_delayed_and_compressed = std::make_shared( - std::move(cascade_buffer1), std::move(cascade_buffer2)); - } - else - { - used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; - } - - std::unique_ptr in_param = std::make_unique(query_param); - - std::unique_ptr in_post_raw = std::make_unique(istr); - - /// Request body can be compressed using algorithm specified in the Content-Encoding header. - String http_request_compression_method_str = request.get("Content-Encoding", ""); - std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( - std::make_unique(istr), chooseCompressionMethod({}, http_request_compression_method_str)); - - /// The data can also be compressed using incompatible internal algorithm. This is indicated by - /// 'decompress' query parameter. - std::unique_ptr in_post_maybe_compressed; - bool in_post_compressed = false; - if (params.getParsed("decompress", false)) - { - in_post_maybe_compressed = std::make_unique(*in_post); - in_post_compressed = true; - } - else - in_post_maybe_compressed = std::move(in_post); - - std::unique_ptr in; - - static const NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; - - Names reserved_param_suffixes; - - auto param_could_be_skipped = [&] (const String & name) - { - if (reserved_param_names.count(name)) - return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } - - return false; - }; - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - const auto & settings = context.getSettingsRef(); - - /// Only readonly queries are allowed for HTTP GET requests. - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - { - if (settings.readonly == 0) - context.setSetting("readonly", 2); - } - - bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); - - if (has_external_data) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - - SettingsChanges settings_changes; - for (const auto & [key, value] : params) - { - if (key == "database") - { - context.setCurrentDatabase(value); - } - else if (key == "default_format") - { - context.setDefaultFormat(value); - } - else if (param_could_be_skipped(key)) - { - } - else if (startsWith(key, "param_")) - { - /// Save name and values of substitution in dictionary. - const String parameter_name = key.substr(strlen("param_")); - context.setQueryParameter(parameter_name, value); - } - else - { - /// All other query parameters are treated as settings. - settings_changes.push_back({key, value}); - } - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); - - /// Used in case of POST request with form-data, but it isn't expected to be deleted after that scope. - std::string full_query; - - /// Support for "external data for query processing". - if (has_external_data) - { - ExternalTablesHandler handler(context, params); - params.load(request, istr, handler); - - /// Params are of both form params POST and uri (GET params) - for (const auto & it : params) - if (it.first == "query") - full_query += it.second; - - in = std::make_unique(full_query); - } - else - in = std::make_unique(*in_param, *in_post_maybe_compressed); - - - /// HTTP response compression is turned on only if the client signalled that they support it - /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. - used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression); - if (client_supports_http_compression) - used_output.out->setCompressionLevel(settings.http_zlib_compression_level); - - used_output.out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) - static_cast(*in_post_maybe_compressed).disableChecksumming(); - - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed - /// Origin header. - used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); - - ClientInfo & client_info = context.getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - - auto append_callback = [&context] (ProgressCallback callback) - { - auto prev = context.getProgressCallback(); - - context.setProgressCallback([prev, callback] (const Progress & progress) - { - if (prev) - prev(progress); - - callback(progress); - }); - }; - - /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. - if (settings.send_progress_in_http_headers) - append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); - - if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) - { - Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - - append_callback([&context, &socket](const Progress &) - { - /// Assume that at the point this method is called no one is reading data from the socket any more. - /// True for read-only queries. - try - { - char b; - //FIXME looks like MSG_DONTWAIT is useless because of POCO_BROKEN_TIMEOUTS - int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); - if (status == 0) - context.killCurrentQuery(); - } - catch (Poco::TimeoutException &) - { - } - catch (...) - { - context.killCurrentQuery(); - } - }); - } - - customizeContext(context); - - executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) - { - response.setContentType(content_type); - response.add("X-ClickHouse-Query-Id", current_query_id); - response.add("X-ClickHouse-Format", format); - response.add("X-ClickHouse-Timezone", timezone); - } - ); - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to - /// the client. - used_output.finalize(); + context->setUser(user, password, request.clientAddress(), quota_key); + context->setCurrentQueryId(query_id); } -void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, +void HTTPHandler::processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder) +{ + const auto & [name, custom_executor] = holder.context->getCustomExecutor(request/*, params*/); + + LOG_TRACE(log, "Using " << name << " to execute URI: " << request.getURI()); + + ExtractorClientInfo{holder.context->getClientInfo()}.extract(request); + ExtractorContextChange{*holder.context.get(), custom_executor}.extract(request, params); + + auto & config = server.config(); + HTTPInputStreams input_streams{*holder.context, request, params}; + HTTPOutputStreams output_streams(*holder.context, request, response, params, config.getUInt("keep_alive_timeout", 10)); + + const auto & query_executors = custom_executor->getQueryExecutor(*holder.context, request, params, input_streams); + for (const auto & query_executor : query_executors) + query_executor(output_streams, response); + + output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + + LOG_INFO(log, "Done processing query"); +} + +void HTTPHandler::trySendExceptionToClient(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output) + HTTPOutputStreams & used_output) { try { @@ -650,7 +297,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ if (!response.sent() && !used_output.out_maybe_compressed) { /// If nothing was sent yet and we don't even know if we must compress the response. - response.send() << s << std::endl; + response.send() << message << std::endl; } else if (used_output.out_maybe_compressed) { @@ -671,7 +318,7 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ used_output.out->position() = used_output.out->buffer().begin(); } - writeString(s, *used_output.out_maybe_compressed); + writeString(message, *used_output.out_maybe_compressed); writeChar('\n', *used_output.out_maybe_compressed); used_output.out_maybe_compressed->next(); @@ -685,21 +332,21 @@ void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_ } } - void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) { setThreadName("HTTPHandler"); ThreadStatus thread_status; - HTTPStreamsWithOutput used_output; + HTTPOutputStreams used_output; /// In case of exception, send stack trace to client. bool with_stacktrace = false; try { - response.setContentType("text/plain; charset=UTF-8"); + response.set("Content-Type", "text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); + /// For keep-alive to work. if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); @@ -708,14 +355,16 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne with_stacktrace = params.getParsed("stacktrace", false); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && - !request.hasContentLength()) - { + 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); - } - processQuery(request, params, response, used_output); - LOG_INFO(log, "Done processing query"); + { + SessionContextHolder holder{server, params}; + CurrentThread::QueryScope query_scope(*holder.context); + + holder.authentication(request, params); + processQuery(request, params, response, holder); + } } catch (...) { @@ -724,12 +373,11 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne /** If exception is received from remote server, then stack trace is embedded in message. * If exception is thrown on local server, then stack trace is in separate field. */ - std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); int exception_code = getCurrentExceptionCode(); + std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, used_output); + trySendExceptionToClient(exception_message, exception_code, request, response, HTTPOutputStreams{}); } } - } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 603278e11aa..10e1f63071e 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace CurrentMetrics @@ -30,31 +30,22 @@ public: void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - /// This method is called right before the query execution. - virtual void customizeContext(DB::Context& /* context */) {} - private: - struct Output + using HTTPRequest = Poco::Net::HTTPServerRequest; + using HTTPResponse = Poco::Net::HTTPServerResponse; + + struct SessionContextHolder { - /* Raw data - * ↓ - * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) - * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) - * CompressedWriteBuffer out_maybe_compressed (optional) - * ↓ - * WriteBufferFromHTTPServerResponse out - */ + ~SessionContextHolder(); - std::shared_ptr out; - /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. - std::shared_ptr out_maybe_compressed; - /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. - std::shared_ptr out_maybe_delayed_and_compressed; + SessionContextHolder(IServer & accepted_server, HTMLForm & params); - inline bool hasDelayed() const - { - return out_maybe_delayed_and_compressed != out_maybe_compressed; - } + void authentication(HTTPServerRequest & request, HTMLForm & params); + + String session_id; + std::unique_ptr context = nullptr; + std::shared_ptr session_context = nullptr; + std::chrono::steady_clock::duration session_timeout; }; IServer & server; @@ -66,20 +57,11 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; /// Also initializes 'used_output'. - void processQuery( - Poco::Net::HTTPServerRequest & request, - HTMLForm & params, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output); + void processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder); void trySendExceptionToClient( - const std::string & s, - int exception_code, - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, - HTTPStreamsWithOutput & used_output); + const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, HTTPOutputStreams & used_output); - static void pushDelayedResults(Output & used_output); }; } diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index 2490d613160..fa6f31bf5d2 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -26,6 +26,12 @@ struct HTMLForm : public Poco::Net::HTMLForm readUrl(istr); } + template + bool check(const std::string & key, T check_value) + { + const auto & value = getParsed(key, T()); + return value == check_value; + } template T getParsed(const std::string & key, T default_value) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 03dd311f31b..d5a26966a47 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -44,7 +44,7 @@ #include #include #include -#include +#include #include #include #include @@ -1545,11 +1545,6 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } -HTTPMatchExecutorPtr Context::getHTTPMatchExecutor() -{ - return std::make_shared(); -} - void Context::initializeSystemLogs() { auto lock = getLock(); @@ -2049,6 +2044,11 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +{ + return std::pair("Default", std::shared_ptr()); +} + StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 03859b03348..3cc348260b8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,7 +490,7 @@ public: Compiler & getCompiler(); - HTTPMatchExecutorPtr getHTTPMatchExecutor(); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 60abff3341484dcaf5e858bb12f4935751927de7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 15:02:10 +0800 Subject: [PATCH 04/23] ISSUES-5436 support custom http [part 3] --- .../Interpreters/CustomHTTP/CustomExecutor.h | 6 +- .../CustomHTTP/CustomExecutorDefault.h | 32 +++++-- .../CustomHTTP/ExtractorContextChange.h | 4 +- .../CustomHTTP/HTTPOutputStreams.cpp | 49 +++++++---- .../CustomHTTP/HTTPOutputStreams.h | 18 ++-- programs/server/HTTPHandler.cpp | 88 ++++++------------- programs/server/HTTPHandler.h | 8 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 8 +- src/IO/WriteBufferFromHTTPServerResponse.h | 4 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 2 +- 11 files changed, 114 insertions(+), 109 deletions(-) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 0778256b297..b2014886d93 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -12,13 +12,9 @@ namespace DB { class Context; -class CustomExecutor; struct HTTPInputStreams; struct HTTPOutputStreams; -using duration = std::chrono::steady_clock::duration; -using HTTPMatchExecutorPtr = std::shared_ptr; - class CustomExecutor { public: @@ -38,4 +34,6 @@ public: virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; }; +using CustomExecutorPtr = std::shared_ptr; + } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h index 218cb413222..9bde5279a5e 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -6,6 +6,7 @@ #include #include #include +#include "HTTPInputStreams.h" namespace DB @@ -25,12 +26,20 @@ public: QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override { - ReadBufferPtr in = prepareAndGetQueryInput(context, request, params, input_streams); - - return {[&, shared_in = in](HTTPOutputStreams & output, HTTPServerResponse & response) + return {[&](HTTPOutputStreams & output, HTTPServerResponse & response) { + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + ReadBufferPtr temp_query_buf; + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + executeQuery( - *shared_in, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + *execute_query_buf, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, [&response] (const String & content_type) { response.setContentType(content_type); }, [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } ); @@ -38,12 +47,21 @@ public: } private: - ReadBufferPtr prepareAndGetQueryInput(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const + String prepareQuery(Context & context, HTMLForm & params) const { - for (const auto & [key, value] : params) - { + const static size_t prefix_size = strlen("param_"); + std::stringstream query_stream; + for (const auto & param : params) + { + if (param.first == "query") + query_stream << param.second; + else if (startsWith(param.first, "param_")) + context.setQueryParameter(param.first.substr(prefix_size), param.second); } + + query_stream << "\n"; + return query_stream.str(); } }; diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h index a228f75391f..7b63df33783 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -12,7 +12,7 @@ namespace DB class ExtractorContextChange { public: - ExtractorContextChange(Context & context_, const HTTPMatchExecutorPtr & executor_) : context(context_), executor(executor_) {} + ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) { @@ -91,7 +91,7 @@ public: private: Context & context; - const HTTPMatchExecutorPtr & executor; + const CustomExecutorPtr & executor; }; diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 156aa6ba975..113523494cf 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -52,10 +52,17 @@ namespace } } +HTTPOutputStreams::HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout) + : out(createResponseOut(request, response, keep_alive_timeout)) + , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) + , out_maybe_delayed_and_compressed(out_maybe_compressed) +{ +} + HTTPOutputStreams::HTTPOutputStreams( Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) : out(createResponseOut(request, response, keep_alive_timeout)) - , out_maybe_compressed(createMaybeCompressionOut(form, out)) + , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { Settings & settings = context.getSettingsRef(); @@ -68,8 +75,7 @@ HTTPOutputStreams::HTTPOutputStreams( out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed - /// Origin header. + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed Origin header. out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. @@ -84,8 +90,7 @@ HTTPOutputStreams::HTTPOutputStreams( } } -std::shared_ptr HTTPOutputStreams::createResponseOut( - HTTPServerRequest &request, HTTPServerResponse &response, size_t keep_alive_timeout) +HTTPResponseBufferPtr HTTPOutputStreams::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive) { /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -96,27 +101,26 @@ std::shared_ptr HTTPOutputStreams::createResp /// NOTE parsing of the list of methods is slightly incorrect. if (std::string::npos != http_response_compression_methods.find("gzip")) return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); else if (std::string::npos != http_response_compression_methods.find("deflate")) return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); #if USE_BROTLI else if (http_response_compression_methods == "br") return std::make_shared( - request, response, keep_alive_timeout, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); #endif } - return std::make_shared( - request, response, keep_alive_timeout, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); } -WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_) +WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. - bool internal_compression = form.getParsed("compress", false); - return internal_compression ? std::make_shared(*out_) : WriteBufferPtr(out_); +// bool internal_compression = form.getParsed("compress", false); + return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) @@ -172,6 +176,20 @@ WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & return out_; } +HTTPOutputStreams::~HTTPOutputStreams() +{ + /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + if (out_maybe_delayed_and_compressed != out_maybe_compressed) + out_maybe_delayed_and_compressed.reset(); + + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + if (out->count() == out->offset()) + { + out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); + out->position() = out->buffer().begin(); + } +} + void HTTPOutputStreams::finalize() const { if (out_maybe_delayed_and_compressed != out_maybe_compressed) @@ -208,8 +226,9 @@ void HTTPOutputStreams::finalize() const copyData(concat_read_buffer, *out_maybe_compressed); } - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to - /// the client. + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + out_maybe_compressed->next(); + out->next(); out->finalize(); } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index b8edaad0b37..614acb6fcf3 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -12,7 +12,7 @@ namespace DB using HTTPServerRequest = Poco::Net::HTTPServerRequest; using HTTPServerResponse = Poco::Net::HTTPServerResponse; - +using HTTPResponseBufferPtr = std::shared_ptr; /* Raw data * ↓ @@ -24,25 +24,27 @@ using HTTPServerResponse = Poco::Net::HTTPServerResponse; */ struct HTTPOutputStreams { - using HTTPResponseBufferPtr = std::shared_ptr; - HTTPResponseBufferPtr out; /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. std::shared_ptr out_maybe_compressed; /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. std::shared_ptr out_maybe_delayed_and_compressed; - HTTPOutputStreams() = default; - - HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + ~HTTPOutputStreams(); void finalize() const; WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); - WriteBufferPtr createMaybeCompressionOut(HTMLForm & form, std::shared_ptr & out_); + WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive_timeout); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive); + + HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout); + + HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); }; +using HTTPOutputStreamsPtr = std::unique_ptr; + } diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index fb018979938..b8bc3c9820c 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -192,7 +192,7 @@ HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_serve if (!session_id.empty()) { session_timeout = parseSessionTimeout(accepted_server.config(), params); - session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); context = std::make_unique(*session_context); context->setSessionContext(*session_context); @@ -241,31 +241,26 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque context->setCurrentQueryId(query_id); } -void HTTPHandler::processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder) +void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { - const auto & [name, custom_executor] = holder.context->getCustomExecutor(request/*, params*/); + const auto & name_with_custom_executor = context.getCustomExecutor(request/*, params*/); + LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); - LOG_TRACE(log, "Using " << name << " to execute URI: " << request.getURI()); + ExtractorClientInfo{context.getClientInfo()}.extract(request); + ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); - ExtractorClientInfo{holder.context->getClientInfo()}.extract(request); - ExtractorContextChange{*holder.context.get(), custom_executor}.extract(request, params); + HTTPInputStreams input_streams{context, request, params}; + HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); - auto & config = server.config(); - HTTPInputStreams input_streams{*holder.context, request, params}; - HTTPOutputStreams output_streams(*holder.context, request, response, params, config.getUInt("keep_alive_timeout", 10)); - - const auto & query_executors = custom_executor->getQueryExecutor(*holder.context, request, params, input_streams); + const auto & query_executors = name_with_custom_executor.second->getQueryExecutor(context, request, params, input_streams); for (const auto & query_executor : query_executors) query_executor(output_streams, response); output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - - LOG_INFO(log, "Done processing query"); } -void HTTPHandler::trySendExceptionToClient(const std::string & message, int exception_code, - Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTTPOutputStreams & used_output) +void HTTPHandler::trySendExceptionToClient( + const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, bool compression) { try { @@ -273,57 +268,25 @@ void HTTPHandler::trySendExceptionToClient(const std::string & message, int exce /// 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) - { + 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::max()); - } - bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || - exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD; - - if (auth_fail) + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { response.requireAuthentication("ClickHouse server HTTP API"); + response.send() << message << std::endl; } else { response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - } + HTTPOutputStreams output_streams(request, response, compression, getKeepAliveTimeout()); - if (!response.sent() && !used_output.out_maybe_compressed) - { - /// If nothing was sent yet and we don't even know if we must compress the response. - response.send() << message << std::endl; - } - else if (used_output.out_maybe_compressed) - { - /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references - if (used_output.out_maybe_delayed_and_compressed != used_output.out_maybe_compressed) - used_output.out_maybe_delayed_and_compressed.reset(); + writeString(message, *output_streams.out_maybe_compressed); + writeChar('\n', *output_streams.out_maybe_compressed); - /// Send the error message into already used (and possibly compressed) stream. - /// Note that the error message will possibly be sent after some data. - /// Also HTTP code 200 could have already been sent. - - /// If buffer has data, and that data wasn't sent yet, then no need to send that data - bool data_sent = used_output.out->count() != used_output.out->offset(); - - if (!data_sent) - { - used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin(); - used_output.out->position() = used_output.out->buffer().begin(); - } - - writeString(message, *used_output.out_maybe_compressed); - writeChar('\n', *used_output.out_maybe_compressed); - - used_output.out_maybe_compressed->next(); - used_output.out->next(); - used_output.out->finalize(); + output_streams.finalize(); } } catch (...) @@ -337,10 +300,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne setThreadName("HTTPHandler"); ThreadStatus thread_status; - HTTPOutputStreams used_output; - /// In case of exception, send stack trace to client. - bool with_stacktrace = false; + bool with_stacktrace = false, internal_compression = false; try { @@ -353,6 +314,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne HTMLForm params(request); with_stacktrace = params.getParsed("stacktrace", false); + internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) @@ -363,7 +325,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne CurrentThread::QueryScope query_scope(*holder.context); holder.authentication(request, params); - processQuery(request, params, response, holder); + processQuery(*holder.context, request, params, response); + LOG_INFO(log, "Done processing query"); } } catch (...) @@ -375,8 +338,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne */ int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - - trySendExceptionToClient(exception_message, exception_code, request, response, HTTPOutputStreams{}); + trySendExceptionToClient(exception_message, exception_code, request, response, internal_compression); } } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 10e1f63071e..47d63af3740 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -56,11 +56,11 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - /// Also initializes 'used_output'. - void processQuery(HTTPRequest & request, HTMLForm & params, HTTPResponse & response, SessionContextHolder & holder); + size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - void trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, HTTPOutputStreams & used_output); + void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response); + + void trySendExceptionToClient(const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, bool compression); }; diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 0f30f1352e3..252a32a5d7c 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -153,13 +153,17 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_, - CompressionMethod compression_method_) - : BufferWithOwnMemory(DBMS_DEFAULT_BUFFER_SIZE) + CompressionMethod compression_method_, + size_t size, + bool finish_send_headers_) + : BufferWithOwnMemory(size) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) , compress(compress_) , compression_method(compression_method_) + , headers_started_sending(finish_send_headers_) + , headers_finished_sending(finish_send_headers_) { } diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index c10288fff9f..638a1f69dee 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -95,7 +95,9 @@ public: Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. - CompressionMethod compression_method_ = CompressionMethod::None); + CompressionMethod compression_method_ = CompressionMethod::Gzip, + size_t size = DBMS_DEFAULT_BUFFER_SIZE, + bool finish_send_headers_ = false); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d5a26966a47..06b34cccd35 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2044,9 +2044,9 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) { - return std::pair("Default", std::shared_ptr()); + return std::pair("Default", std::make_shared()); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3cc348260b8..6e0b78250e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -490,7 +490,7 @@ public: Compiler & getCompiler(); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From fd00757178323ee948cd660209c93cd7f7611355 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 19:01:02 +0800 Subject: [PATCH 05/23] ISSUES-5436 support custom http [part 4] --- .../CustomHTTP/CustomExecutor.cpp | 121 ++++++++++++++++++ .../Interpreters/CustomHTTP/CustomExecutor.h | 77 +++++++++-- .../CustomHTTP/CustomExecutorDefault.h | 50 ++++---- .../CustomHTTP/HTTPInputStreams.cpp | 2 +- .../CustomHTTP/HTTPInputStreams.h | 2 +- programs/server/HTTPHandler.cpp | 9 +- programs/server/Server.cpp | 1 + src/Interpreters/Context.cpp | 31 ++++- src/Interpreters/Context.h | 4 +- 9 files changed, 249 insertions(+), 48 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp new file mode 100644 index 00000000000..7767082555e --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -0,0 +1,121 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +bool CustomExecutor::match(HTTPRequest & request, HTMLForm & params) const +{ + for (const auto & matcher : matchers) + { + if (!matcher->match(request, params)) + return false; + } + + return true; +} + +bool CustomExecutor::isQueryParam(const String & param_name) const +{ + for (const auto & query_executor : query_executors) + { + if (!query_executor->isQueryParam(param_name)) + return false; + } + + return true; +} + +bool CustomExecutor::canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const +{ + for (const auto & query_executor : query_executors) + { + if (!query_executor->canBeParseRequestBody(request, params)) + return false; + } + + return true; +} + +void CustomExecutor::executeQuery( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) +{ + for (const auto & query_executor : query_executors) + query_executor->executeQueryImpl(context, request, response, params, input_streams, output_streams); + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + output_streams.finalize(); +} + +CustomExecutor::CustomExecutor( + const std::vector & matchers_, const std::vector & query_executors_) + : matchers(matchers_), query_executors(query_executors_) +{ +} + +CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + updateCustomExecutors(config, settings, config_prefix); +} + +void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + Configuration::Keys custom_executors_keys; + config.keys(config_prefix, custom_executors_keys); + + std::unordered_map new_custom_executors; + + for (const auto & custom_executor_key : custom_executors_keys) + { + if (custom_executor_key == "Default") + throw Exception("CustomExecutor cannot be 'Default'.", ErrorCodes::SYNTAX_ERROR); + else if (custom_executor_key.find('.') != String::npos) + throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); + + new_custom_executors[custom_executor_key] = createCustomExecutor(config, settings, config_prefix + "." + custom_executor_key); + } + + new_custom_executors["Default"] = CustomExecutorDefault::createDefaultCustomExecutor(); + + std::unique_lock lock(rwlock); + custom_executors = new_custom_executors; +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::Configuration & config, const Settings & /*settings*/, const String & config_prefix) +{ + Configuration::Keys matchers_or_query_executors_type; + config.keys(config_prefix, matchers_or_query_executors_type); + + for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) + { + if (matcher_or_query_executor_type.find('.') != String::npos) + throw Exception( + "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", + ErrorCodes::SYNTAX_ERROR); + +// throw Exception("", ErrorCodes::NOT_IMPLEMENTED); +// new_custom_executors[matcher_or_query_executor_type] = createCustomExecutor(config, settings, config_prefix + "." + matcher_or_query_executor_type); + } + return DB::CustomExecutorPtr(); +} + +std::pair CustomExecutors::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +{ + std::shared_lock lock(rwlock); + + for (const auto & custom_executor : custom_executors) + if (custom_executor.second->match(request, params)) + return custom_executor; + + throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index b2014886d93..f6584e249be 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -1,39 +1,92 @@ #pragma once +#include + #include +#include +#include #include #include #include #include #include -#include namespace DB { class Context; +class CustomExecutor; struct HTTPInputStreams; struct HTTPOutputStreams; +using HTTPRequest = Poco::Net::HTTPServerRequest; +using HTTPResponse = Poco::Net::HTTPServerResponse; +using CustomExecutorPtr = std::shared_ptr; + +class CustomExecutors +{ +public: + using Configuration = Poco::Util::AbstractConfiguration; + CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "CustomHTTP"); + + CustomExecutors(const CustomExecutors &) = delete; + CustomExecutors & operator=(const CustomExecutors &) = delete; + + void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); + + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; +private: + mutable std::shared_mutex rwlock; + std::unordered_map custom_executors; + + CustomExecutorPtr createCustomExecutor(const Configuration & config, const Settings & settings, const String & config_prefix); +}; + class CustomExecutor { public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; + bool isQueryParam(const String & param_name) const; - virtual ~CustomExecutor() = default; + bool match(HTTPRequest & request, HTMLForm & params) const; - virtual bool isQueryParam(const String & param_name) const = 0; + bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; - virtual bool match(HTTPServerRequest & request, HTMLForm & params) const = 0; + void executeQuery( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams + ); - virtual bool canBeParseRequestBody(HTTPServerRequest & request, HTMLForm & params) = 0; +public: + class CustomMatcher + { + public: + virtual ~CustomMatcher() = default; - using QueryExecutor = std::function; - using QueryExecutors = std::vector; - virtual QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const = 0; + virtual bool match(HTTPRequest & request, HTMLForm & params) const = 0; + }; + + class CustomQueryExecutor + { + public: + virtual ~CustomQueryExecutor() = default; + + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody(HTTPRequest &, HTMLForm &) const = 0; + + virtual void executeQueryImpl( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; + }; + +public: + using CustomMatcherPtr = std::shared_ptr; + using CustomQueryExecutorPtr = std::shared_ptr; + + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + +private: + std::vector matchers; + std::vector query_executors; }; -using CustomExecutorPtr = std::shared_ptr; - } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h index 9bde5279a5e..20b0fdb4eb5 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h @@ -5,45 +5,51 @@ #include #include #include +#include #include -#include "HTTPInputStreams.h" namespace DB { -class CustomExecutorDefault : public CustomExecutor +class CustomExecutorDefault : public CustomExecutor::CustomMatcher, public CustomExecutor::CustomQueryExecutor { public: bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } - bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) override { return false; } + bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } - bool isQueryParam(const String & param_name) const override + bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } + + void executeQueryImpl( + Context & context, HTTPRequest & request, HTTPResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override { - return param_name == "query" || startsWith(param_name, "param_"); + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + ReadBufferPtr temp_query_buf; + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); } - QueryExecutors getQueryExecutor(Context & context, HTTPServerRequest & request, HTMLForm & params, const HTTPInputStreams & input_streams) const override + static CustomExecutorPtr createDefaultCustomExecutor() { - return {[&](HTTPOutputStreams & output, HTTPServerResponse & response) - { - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + const auto & default_custom_executor = std::make_shared(); - ReadBufferPtr temp_query_buf; - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } + std::vector custom_matchers{default_custom_executor}; + std::vector custom_query_executors{default_custom_executor}; - executeQuery( - *execute_query_buf, *output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - }}; + return std::make_shared(custom_matchers, custom_query_executors); } private: diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index e6985621b00..7c441891b33 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -24,7 +24,7 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques /// checksums of client data compressed with internal algorithm are not checked. if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) { - if(CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) + if (CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) compressed_buffer->disableChecksumming(); } } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h index 88f0438398d..389d4312362 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h @@ -24,4 +24,4 @@ struct HTTPInputStreams ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; }; -} \ No newline at end of file +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index b8bc3c9820c..8f460b50db7 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -243,7 +243,7 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { - const auto & name_with_custom_executor = context.getCustomExecutor(request/*, params*/); + const auto & name_with_custom_executor = context.getCustomExecutor(request, params); LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); ExtractorClientInfo{context.getClientInfo()}.extract(request); @@ -251,12 +251,7 @@ void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLFor HTTPInputStreams input_streams{context, request, params}; HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); - - const auto & query_executors = name_with_custom_executor.second->getQueryExecutor(context, request, params, input_streams); - for (const auto & query_executor : query_executors) - query_executor(output_streams, response); - - output_streams.finalize(); /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); } void HTTPHandler::trySendExceptionToClient( diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3490ff6a445..710506151c4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -466,6 +466,7 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); + global_context->setCustomExecutorConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b34cccd35..39c7953b29a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -44,7 +45,7 @@ #include #include #include -#include +#include #include #include #include @@ -349,7 +350,9 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config - mutable std::mutex match_executor_mutex; /// Guards match executor and their config + std::unique_ptr custom_executors; + ConfigurationPtr custom_executors_config; + mutable std::mutex custom_executors_mutex; /// Guards custom executors and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -2044,9 +2047,29 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & /*request*/) +void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix) { - return std::pair("Default", std::make_shared()); + std::lock_guard lock(shared->custom_executors_mutex); + + shared->custom_executors_config = config; + + if (!shared->custom_executors) + shared->custom_executors = std::make_unique(*shared->custom_executors_config, settings, config_prefix); + else + shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); +} + +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +{ + std::lock_guard lock(shared->custom_executors_mutex); + + if (!shared->custom_executors) + { + auto & config = shared->custom_executors_config ? *shared->custom_executors_config : getConfigRef(); + shared->custom_executors = std::make_unique(config, settings); + } + + return shared->custom_executors->getCustomExecutor(request, params); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6e0b78250e4..c73d03179c9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,6 +42,7 @@ namespace zkutil class ZooKeeper; } +class HTMLForm; namespace DB { @@ -490,7 +491,8 @@ public: Compiler & getCompiler(); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request/*, HTMLForm & params*/); + void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params) const; /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 7aef95b0820d446a5ab3df504030fcb0bae7f6c2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 6 Nov 2019 21:40:38 +0800 Subject: [PATCH 06/23] ISSUES-5436 support custom http [part 5] --- .../CustomHTTP/CustomExecutor.cpp | 83 +++++++++++++--- .../Interpreters/CustomHTTP/CustomExecutor.h | 57 ++++------- .../CustomHTTP/CustomExecutorMatchers.h | 97 +++++++++++++++++++ ...ecutorDefault.h => CustomQueryExecutors.h} | 37 ++++--- programs/server/HTTPHandler.cpp | 2 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 4 +- 7 files changed, 211 insertions(+), 73 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h rename dbms/src/Interpreters/CustomHTTP/{CustomExecutorDefault.h => CustomQueryExecutors.h} (67%) diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index 7767082555e..c37da93bcaf 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include "CustomExecutor.h" namespace DB @@ -11,11 +11,11 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -bool CustomExecutor::match(HTTPRequest & request, HTMLForm & params) const +bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const { for (const auto & matcher : matchers) { - if (!matcher->match(request, params)) + if (!matcher->match(context, request, params)) return false; } @@ -56,14 +56,17 @@ void CustomExecutor::executeQuery( } CustomExecutor::CustomExecutor( - const std::vector & matchers_, const std::vector & query_executors_) + const std::vector & matchers_, const std::vector & query_executors_) : matchers(matchers_), query_executors(query_executors_) { } -CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +static CustomExecutorPtr createDefaultCustomExecutor() { - updateCustomExecutors(config, settings, config_prefix); + std::vector custom_matchers{std::make_shared()}; + std::vector custom_query_executors{std::make_shared()}; + + return std::make_shared(custom_matchers, custom_query_executors); } void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) @@ -71,7 +74,7 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const Configuration::Keys custom_executors_keys; config.keys(config_prefix, custom_executors_keys); - std::unordered_map new_custom_executors; + std::vector> new_custom_executors; for (const auto & custom_executor_key : custom_executors_keys) { @@ -80,20 +83,48 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const else if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - new_custom_executors[custom_executor_key] = createCustomExecutor(config, settings, config_prefix + "." + custom_executor_key); + new_custom_executors.push_back( + std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); } - new_custom_executors["Default"] = CustomExecutorDefault::createDefaultCustomExecutor(); + new_custom_executors.push_back(std::make_pair("Default", createDefaultCustomExecutor())); std::unique_lock lock(rwlock); custom_executors = new_custom_executors; } -CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::Configuration & config, const Settings & /*settings*/, const String & config_prefix) +void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) +{ + const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); + const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); + + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR CustomQueryExecutor name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", + ErrorCodes::LOGICAL_ERROR); + + query_executor_creators[query_executor_name] = creator; +} + +void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) +{ + const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); + const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); + + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR CustomExecutorMatcher name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", + ErrorCodes::LOGICAL_ERROR); + + custom_matcher_creators[matcher_name] = creator; +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix) { Configuration::Keys matchers_or_query_executors_type; config.keys(config_prefix, matchers_or_query_executors_type); + std::vector custom_query_executors; + std::vector custom_executor_matchers; + for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) { if (matcher_or_query_executor_type.find('.') != String::npos) @@ -101,21 +132,43 @@ CustomExecutorPtr CustomExecutors::createCustomExecutor(const CustomExecutors::C "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", ErrorCodes::SYNTAX_ERROR); -// throw Exception("", ErrorCodes::NOT_IMPLEMENTED); -// new_custom_executors[matcher_or_query_executor_type] = createCustomExecutor(config, settings, config_prefix + "." + matcher_or_query_executor_type); + const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); + const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); + + if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) + throw Exception("CustomMatcher or CustomQueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", + ErrorCodes::NOT_IMPLEMENTED); + + if (matcher_creator_it != custom_matcher_creators.end()) + custom_executor_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + + if (query_executor_creator_it != query_executor_creators.end()) + custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); } - return DB::CustomExecutorPtr(); + + for (const auto & custom_executor_matcher : custom_executor_matchers) + custom_executor_matcher->checkQueryExecutor(custom_query_executors); + + return std::make_shared(custom_executor_matchers, custom_query_executors); } -std::pair CustomExecutors::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const { std::shared_lock lock(rwlock); for (const auto & custom_executor : custom_executors) - if (custom_executor.second->match(request, params)) + if (custom_executor.second->match(context, request, params)) return custom_executor; throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); } +CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +{ + registerCustomMatcher("URL", [&](const auto & config, const auto & prefix) + { return std::make_shared(config, prefix); }); + + updateCustomExecutors(config, settings, config_prefix); +} + } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index f6584e249be..5b5ccc53850 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -10,14 +10,15 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB { - -class Context; class CustomExecutor; -struct HTTPInputStreams; -struct HTTPOutputStreams; using HTTPRequest = Poco::Net::HTTPServerRequest; using HTTPResponse = Poco::Net::HTTPServerResponse; @@ -32,14 +33,22 @@ public: CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; + using QueryExecutorCreator = std::function; + void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); + + using CustomMatcherCreator = const std::function; + void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); + void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; + std::pair getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; private: mutable std::shared_mutex rwlock; - std::unordered_map custom_executors; + std::vector> custom_executors; + std::unordered_map query_executor_creators; + std::unordered_map custom_matcher_creators; - CustomExecutorPtr createCustomExecutor(const Configuration & config, const Settings & settings, const String & config_prefix); + CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); }; class CustomExecutor @@ -47,45 +56,19 @@ class CustomExecutor public: bool isQueryParam(const String & param_name) const; - bool match(HTTPRequest & request, HTMLForm & params) const; - bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; + bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; + void executeQuery( Context & context, HTTPRequest & request, HTTPResponse & response, HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams ); -public: - class CustomMatcher - { - public: - virtual ~CustomMatcher() = default; - - virtual bool match(HTTPRequest & request, HTMLForm & params) const = 0; - }; - - class CustomQueryExecutor - { - public: - virtual ~CustomQueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody(HTTPRequest &, HTMLForm &) const = 0; - - virtual void executeQueryImpl( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; - }; - -public: - using CustomMatcherPtr = std::shared_ptr; - using CustomQueryExecutorPtr = std::shared_ptr; - - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); private: - std::vector matchers; + std::vector matchers; std::vector query_executors; }; diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h new file mode 100644 index 00000000000..79a48f069ca --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h @@ -0,0 +1,97 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +#if USE_RE2_ST +# include +#else +# define re2_st re2 +#endif + +namespace DB +{ + +class CustomExecutorMatcher +{ +public: + virtual ~CustomExecutorMatcher() = default; + + virtual bool checkQueryExecutor(const std::vector & check_executors) const = 0; + + virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; +}; + +using CustomExecutorMatcherPtr = std::shared_ptr; + + +class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + bool checkQueryExecutor(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } +}; + +class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) + : url_match_searcher(analyzeURLPatten(configuration.getString(url_config_key, ""), params_name_extract_from_url)) + { + } + + bool checkQueryExecutor(const std::vector & custom_query_executors) const override + { + for (const auto & param_name_from_url : params_name_extract_from_url) + { + bool found_param_name = false; + for (const auto & custom_query_executor : custom_query_executors) + { + if (custom_query_executor->isQueryParam(param_name_from_url)) + { + found_param_name = true; + break; + } + } + + if (!found_param_name) + throw Exception("The param name '" + param_name_from_url + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + } + + return true; + } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + const String request_uri = request.getURI(); + re2_st::StringPiece query_params_matches[params_name_extract_from_url.size()]; + +// re2_st::StringPiece input; +// if (url_match_searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, query_params_matches, num_captures)) +// { +// +// } + return false; + } + +private: + re2_st::RE2 url_match_searcher; + std::vector params_name_extract_from_url; + + String analyzeURLPatten(const String & /*url_patten*/, std::vector & /*matches*/) + { + return ".+"; + /// TODO: first we replace all capture group + /// TODO: second we replace all ${identifier} + } +}; + + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h similarity index 67% rename from dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h rename to dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h index 20b0fdb4eb5..0473148b89c 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorDefault.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h @@ -1,28 +1,43 @@ #pragma once -#include +#include +#include #include #include #include #include #include #include - +#include +#include namespace DB { -class CustomExecutorDefault : public CustomExecutor::CustomMatcher, public CustomExecutor::CustomQueryExecutor +class CustomQueryExecutor { public: - bool match(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } + virtual ~CustomQueryExecutor() = default; - bool canBeParseRequestBody(HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return false; } + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody(Poco::Net::HTTPServerRequest &, HTMLForm &) const = 0; + virtual void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; +}; + +using CustomQueryExecutorPtr = std::shared_ptr; + +class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor +{ +public: bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } + bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return false; } + void executeQueryImpl( - Context & context, HTTPRequest & request, HTTPResponse & response, + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override { const auto & execute_query = prepareQuery(context, params); @@ -42,16 +57,6 @@ public: ); } - static CustomExecutorPtr createDefaultCustomExecutor() - { - const auto & default_custom_executor = std::make_shared(); - - std::vector custom_matchers{default_custom_executor}; - std::vector custom_query_executors{default_custom_executor}; - - return std::make_shared(custom_matchers, custom_query_executors); - } - private: String prepareQuery(Context & context, HTMLForm & params) const { diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 8f460b50db7..64138a6d7d0 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -244,7 +244,7 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) { const auto & name_with_custom_executor = context.getCustomExecutor(request, params); - LOG_TRACE(log, "Using " << name_with_custom_executor.first << " to execute URI: " << request.getURI()); + LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); ExtractorClientInfo{context.getClientInfo()}.extract(request); ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 39c7953b29a..b479a7bc46d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2059,7 +2059,7 @@ void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const Str shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); } -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) const +std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) { std::lock_guard lock(shared->custom_executors_mutex); @@ -2069,7 +2069,7 @@ std::pair Context::getCustomExecutor(Poco::Net::HTTPS shared->custom_executors = std::make_unique(config, settings); } - return shared->custom_executors->getCustomExecutor(request, params); + return shared->custom_executors->getCustomExecutor(*this, request, params); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c73d03179c9..2d3ce2f9962 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,7 +42,7 @@ namespace zkutil class ZooKeeper; } -class HTMLForm; +struct HTMLForm; namespace DB { @@ -492,7 +492,7 @@ public: Compiler & getCompiler(); void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params) const; + std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); From 6b716e511bab5983475e101dd975c2ff4d478b7d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 11:51:11 +0800 Subject: [PATCH 07/23] ISSUES-5436 support custom http [part 6] --- .../CustomHTTP/CustomExecutor.cpp | 36 +++++- .../Interpreters/CustomHTTP/CustomExecutor.h | 7 +- .../CustomHTTP/CustomExecutorMatchers.h | 91 +++++++++------ .../CustomHTTP/CustomQueryExecutors.h | 52 +++++++++ dbms/src/Interpreters/QueryParameterVisitor.h | 35 ++++++ programs/server/HTTPHandler.cpp | 10 +- programs/server/config.xml | 10 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 107 +++++++++++------- src/IO/WriteBufferFromHTTPServerResponse.h | 2 + src/Interpreters/Context.h | 2 +- .../ReplaceQueryParameterVisitor.cpp | 6 +- 11 files changed, 271 insertions(+), 87 deletions(-) create mode 100644 dbms/src/Interpreters/QueryParameterVisitor.h diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index c37da93bcaf..981d243ef35 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -9,6 +9,8 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; + extern const int DUPLICATE_CUSTOM_EXECUTOR; + extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; } bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const @@ -69,7 +71,7 @@ static CustomExecutorPtr createDefaultCustomExecutor() return std::make_shared(custom_matchers, custom_query_executors); } -void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) +void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) { Configuration::Keys custom_executors_keys; config.keys(config_prefix, custom_executors_keys); @@ -83,6 +85,11 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const else if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); + const auto & exists_executor = [&](auto & ele) { return ele.first == custom_executor_key; }; + if (std::count_if(new_custom_executors.begin(), new_custom_executors.end(), exists_executor)) + throw Exception("CustomExecutor name '" + custom_executor_key + "' already exists in system.", + ErrorCodes::DUPLICATE_CUSTOM_EXECUTOR); + new_custom_executors.push_back( std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); } @@ -146,12 +153,23 @@ CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & co custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); } - for (const auto & custom_executor_matcher : custom_executor_matchers) - custom_executor_matcher->checkQueryExecutor(custom_query_executors); - + checkCustomMatchersAndQueryExecutors(custom_executor_matchers, custom_query_executors); return std::make_shared(custom_executor_matchers, custom_query_executors); } +void CustomExecutors::checkCustomMatchersAndQueryExecutors( + std::vector & matchers, std::vector & query_executors) +{ + const auto & sum_func = [&](auto & ele) { return !ele->canBeParseRequestBody(); }; + const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); + + if (need_post_data_count > 1) + throw Exception("The CustomExecutor can only contain one insert query.", ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); + + for (const auto & matcher : matchers) + matcher->checkQueryExecutors(query_executors); +} + std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const { std::shared_lock lock(rwlock); @@ -165,8 +183,14 @@ std::pair CustomExecutors::getCustomExecutor(Context CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) { - registerCustomMatcher("URL", [&](const auto & config, const auto & prefix) - { return std::make_shared(config, prefix); }); + registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); + + registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); + + registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); updateCustomExecutors(config, settings, config_prefix); } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 5b5ccc53850..94d1fe28923 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -28,7 +28,7 @@ class CustomExecutors { public: using Configuration = Poco::Util::AbstractConfiguration; - CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "CustomHTTP"); + CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "custom_http"); CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; @@ -36,7 +36,7 @@ public: using QueryExecutorCreator = std::function; void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - using CustomMatcherCreator = const std::function; + using CustomMatcherCreator = std::function; void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); @@ -49,6 +49,9 @@ private: std::unordered_map custom_matcher_creators; CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); + + void checkCustomMatchersAndQueryExecutors(std::vector & matchers, std::vector & query_executors); + }; class CustomExecutor diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h index 79a48f069ca..825b3c0b268 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h @@ -22,7 +22,7 @@ class CustomExecutorMatcher public: virtual ~CustomExecutorMatcher() = default; - virtual bool checkQueryExecutor(const std::vector & check_executors) const = 0; + virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; }; @@ -33,63 +33,88 @@ using CustomExecutorMatcherPtr = std::shared_ptr; class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher { public: - bool checkQueryExecutor(const std::vector & /*check_executors*/) const override { return true; } + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } }; +class HTTPMethodCustomExecutorMatcher : public CustomExecutorMatcher +{ +public: + + HTTPMethodCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & method_config_key) + { + match_method = Poco::toLower(configuration.getString(method_config_key)); + } + + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + return Poco::toLower(request.getMethod()) == match_method; + } + +private: + String match_method; +}; + class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher { public: HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) - : url_match_searcher(analyzeURLPatten(configuration.getString(url_config_key, ""), params_name_extract_from_url)) { + regex_matcher = std::make_unique(configuration.getString(url_config_key)); } - bool checkQueryExecutor(const std::vector & custom_query_executors) const override + bool checkQueryExecutors(const std::vector & custom_query_executors) const override { - for (const auto & param_name_from_url : params_name_extract_from_url) - { - bool found_param_name = false; - for (const auto & custom_query_executor : custom_query_executors) - { - if (custom_query_executor->isQueryParam(param_name_from_url)) - { - found_param_name = true; - break; - } - } - - if (!found_param_name) - throw Exception("The param name '" + param_name_from_url + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - } + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + if (!checkQueryExecutors(named_capturing_group.first, custom_query_executors)) + throw Exception("The param name '" + named_capturing_group.first + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return true; } - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override { const String request_uri = request.getURI(); - re2_st::StringPiece query_params_matches[params_name_extract_from_url.size()]; + int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; -// re2_st::StringPiece input; -// if (url_match_searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, query_params_matches, num_captures)) -// { -// -// } + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(request_uri.data(), request_uri.size()); + if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + const auto & full_match = matches[0]; + const char * url_end = request_uri.data() + request_uri.size(); + const char * not_matched_begin = request_uri.data() + full_match.size(); + + if (not_matched_begin != url_end && *not_matched_begin == '/') + ++not_matched_begin; + + if (not_matched_begin == url_end || *not_matched_begin == '?') + { + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + { + const auto & capturing_value = matches[named_capturing_group.second]; + context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); + } + + return true; + } + } return false; } private: - re2_st::RE2 url_match_searcher; - std::vector params_name_extract_from_url; + std::unique_ptr regex_matcher; - String analyzeURLPatten(const String & /*url_patten*/, std::vector & /*matches*/) + bool checkQueryExecutors(const String & param_name, const std::vector & custom_query_executors) const { - return ".+"; - /// TODO: first we replace all capture group - /// TODO: second we replace all ${identifier} + for (const auto & custom_query_executor : custom_query_executors) + if (custom_query_executor->isQueryParam(param_name)) + return true; + + return false; } }; diff --git a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h index 0473148b89c..f986db6d41d 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h @@ -6,10 +6,14 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include namespace DB { @@ -29,6 +33,54 @@ public: using CustomQueryExecutorPtr = std::shared_ptr; +class ConstQueryCustomQueryExecutor : public CustomQueryExecutor +{ +public: + ConstQueryCustomQueryExecutor(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) + { + execute_query = configuration.getString(config_key, ""); + + const char * query_begin = execute_query.data(); + const char * query_end = execute_query.data() + execute_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + + QueryParameterVisitor{query_params_name}.visit(extract_query_ast); + can_be_parse_request_body = !extract_query_ast->as(); + } + + bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } + + bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return can_be_parse_request_body; } + + void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & /*input_streams*/, const HTTPOutputStreams & output_streams) const override + { + prepareQueryParams(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + } + +private: + String execute_query; + NameSet query_params_name; + bool can_be_parse_request_body{false}; + + void prepareQueryParams(Context & context, HTMLForm & params) const + { + for (const auto & param : params) + if (isQueryParam(param.first)) + context.setQueryParameter(param.first, param.second); + } +}; + class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor { public: diff --git a/dbms/src/Interpreters/QueryParameterVisitor.h b/dbms/src/Interpreters/QueryParameterVisitor.h new file mode 100644 index 00000000000..d765aa00bea --- /dev/null +++ b/dbms/src/Interpreters/QueryParameterVisitor.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class QueryParameterVisitor +{ +public: + QueryParameterVisitor(NameSet & parameters_name) : query_parameters(parameters_name) {} + + void visit(const ASTPtr & ast) + { + for (const auto & child : ast->children) + { + if (const auto & query_parameter = child->as()) + visitQueryParameter(*query_parameter); + else + visit(child); + } + } + +private: + NameSet & query_parameters; + + void visitQueryParameter(const ASTQueryParameter & query_parameter) + { + query_parameters.insert(query_parameter.name); + } +}; + +} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 64138a6d7d0..6ceaf7d7d3f 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -267,10 +267,14 @@ void HTTPHandler::trySendExceptionToClient( && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) request.stream().ignore(std::numeric_limits::max()); - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD + || exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) { - response.requireAuthentication("ClickHouse server HTTP API"); + if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); + else + response.requireAuthentication("ClickHouse server HTTP API"); + response.send() << message << std::endl; } else diff --git a/programs/server/config.xml b/programs/server/config.xml index fb2f9be6e24..5111e6102c2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -512,7 +512,6 @@ --> /var/lib/clickhouse/format_schemas/ - + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 252a32a5d7c..c7b3abb008b 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -31,11 +31,6 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() response.set("Access-Control-Allow-Origin", "*"); setResponseDefaultHeaders(response, keep_alive_timeout); - -#if defined(POCO_CLICKHOUSE_PATCH) - if (request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) - std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); -#endif } } @@ -96,46 +91,79 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders() } +void WriteBufferFromHTTPServerResponse::choiceSendEncode() +{ + if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (!compress) + { +#if defined(POCO_CLICKHOUSE_PATCH) + *response_header_ostr << "Content-Encoding: gzip\r\n"; +#else + response.set("Content-Encoding", "gzip"); + response_body_ostr = &(response.send()); +#endif + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*deflating_buf; + } + else if (compression_method == CompressionMethod::Zlib) + { +#if defined(POCO_CLICKHOUSE_PATCH) + *response_header_ostr << "Content-Encoding: deflate\r\n"; +#else + /// Newline autosent by response.send() + response_body_ostr = &(response.send()); +#endif + out_raw = std::make_unique(*response_body_ostr); + deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*deflating_buf; + } +#if USE_BROTLI + else if (compression_method == CompressionMethod::Brotli) + { +#if defined(POCO_CLICKHOUSE_PATCH) + std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); + if (headers_started_sending && !headers_finished_sending) + *response_header_ostr << "Content-Encoding: " << encoding_type << "\r\n"; +#else + response.set("Content-Encoding", content_encoding_name); +#endif + + /// Newline autosent by response.send() + /// This may result in an extra empty line in the response body + response_body_ostr = &(response.send()); +#endif + }; + + out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out = &*out_raw; + } +#if USE_BROTLI + else if (compression_method == CompressionMethod::Brotli) + { + set_encoding_type("br"); + out_raw.emplace(*response_body_ostr); + brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); + out = &*brotli_buf; + } +#endif + else + throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", + ErrorCodes::LOGICAL_ERROR); + /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. + } + } +} + + void WriteBufferFromHTTPServerResponse::nextImpl() { { std::lock_guard lock(mutex); startSendHeaders(); - - if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (compress) - { - auto content_encoding_name = toContentEncodingName(compression_method); - -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: " << content_encoding_name << "\r\n"; -#else - response.set("Content-Encoding", content_encoding_name); -#endif - } - -#if !defined(POCO_CLICKHOUSE_PATCH) - response_body_ostr = &(response.send()); -#endif - - /// We reuse our buffer in "out" to avoid extra allocations and copies. - - if (compress) - out = wrapWriteBufferWithCompressionMethod( - std::make_unique(*response_body_ostr), - compress ? compression_method : CompressionMethod::None, - compression_level, - working_buffer.size(), - working_buffer.begin()); - else - out = std::make_unique( - *response_body_ostr, - working_buffer.size(), - working_buffer.begin()); - } - + choiceSendEncode(); finishSendHeaders(); } @@ -207,7 +235,6 @@ void WriteBufferFromHTTPServerResponse::finalize() } } - WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { try diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 638a1f69dee..fd57279646f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -87,6 +87,8 @@ private: /// This method finish headers with \r\n, allowing to start to send body. void finishSendHeaders(); + void choiceSendEncode(); + void nextImpl() override; public: diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2d3ce2f9962..bef448a2389 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -491,7 +491,7 @@ public: Compiler & getCompiler(); - void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "CustomHTTP"); + void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "custom_http"); std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); /// Call after initialization before using system logs. Call for global context. diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 5c29c722f88..98dc90d9dda 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -55,7 +56,10 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsTextEscaped(temp_column, read_buffer, format_settings); + + skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides + data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); + skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From 07ed4ba4f4ff44d5985803f720d6965f22524d28 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 16:30:47 +0800 Subject: [PATCH 08/23] ISSUES-5436 add integration test for custom http --- .../test_custom_http/configs/custom_http.xml | 11 ---- .../configs/custom_http_config.xml | 54 +++++++++++++++++++ .../integration/test_custom_http/test.py | 22 +++----- 3 files changed, 60 insertions(+), 27 deletions(-) delete mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http.xml create mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http_config.xml diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http.xml b/dbms/tests/integration/test_custom_http/configs/custom_http.xml deleted file mode 100644 index 22d570b7bca..00000000000 --- a/dbms/tests/integration/test_custom_http/configs/custom_http.xml +++ /dev/null @@ -1,11 +0,0 @@ - - 8123 - - - /${database}/a/${id}/${table} - INSERT INTO ${database:ASTIdentifier}.${table:ASTIdentifier}(id) VALUES - SELECT * FROM ${database:ASTIdenfier}.t - SELECT * FROM a.${table:ASTIdenfier} WHERE id={id:UInt8} - - - diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml new file mode 100644 index 00000000000..f91e196abab --- /dev/null +++ b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml @@ -0,0 +1,54 @@ + + 8123 + + + + /test_for_single_insert + INSERT INTO test.test VALUES + + + + /test_for_single_select + SELECT * FROM test.test ORDER BY id + + + + /test_for_multiple_select + SELECT * FROM test.test ORDER BY id + SELECT * FROM test.test ORDER BY id + + + + /test_for_multiple_select + INSERT INTO test.test VALUES + SELECT 'test_for_hybrid_insert_and_select_with_one_insert first' + SELECT 'test_for_hybrid_insert_and_select_with_one_insert second' + + + + + + /test_for_url_match + SELECT 'Matched test_for_url_match' + + + + PUT + SELECT 'Matched test_for_method_match' + + + + /test_for_multiple_match + GET + SELECT 'Matched test_for_multiple_match' + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py index d8f79e64e3d..805c6163839 100644 --- a/dbms/tests/integration/test_custom_http/test.py +++ b/dbms/tests/integration/test_custom_http/test.py @@ -1,31 +1,21 @@ import pytest -import requests from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', main_configs=['configs/custom_http.xml']) +test_instance = cluster.add_instance('node', main_configs=['configs/custom_http_config.xml']) + @pytest.fixture(scope="module") def start_cluster(): try: cluster.start() - node.query(''' -CREATE DATABASE `test`; - -CREATE TABLE `test`.`test_custom_http` (`id` UInt8) Engine=Memory; - ''') - + test_instance.query('CREATE DATABASE `test`') + test_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') yield cluster finally: cluster.shutdown() -def test(started_cluster): - node_ip = cluster.get_instance_ip(node) - url = 'http://%s:8123/test/a/1/test_custom_http' % node_ip - data="(1)" - params = {'id':1} - response = requests.post(url, params = params, data = data) - assert response.text == '\n1\n1\n' +def test_for_single_insert(started_cluster): + assert test_instance.http_query('/test_for_single_insert', data='(1)(2)(3)') == '\n' From 847f7ab4396f4e21ac3c07f13ab9629d01011b33 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 7 Nov 2019 18:24:30 +0800 Subject: [PATCH 09/23] ISSUES-5436 fix build failure & fix test failure --- .../CustomHTTP/AlwaysQueryMatcher.h | 16 +++ .../CustomHTTP/ConstQueryExecutor.cpp | 73 ++++++++++ .../CustomHTTP/ConstQueryExecutor.h | 30 ++++ .../CustomHTTP/CustomExecutor.cpp | 135 +++++++++--------- .../Interpreters/CustomHTTP/CustomExecutor.h | 27 ++-- .../CustomHTTP/CustomExecutorMatchers.h | 122 ---------------- .../CustomHTTP/CustomQueryExecutors.h | 131 ----------------- .../CustomHTTP/DynamicQueryExecutor.cpp | 60 ++++++++ .../CustomHTTP/DynamicQueryExecutor.h | 37 +++++ .../CustomHTTP/ExtractorContextChange.h | 117 +++++++++------ .../CustomHTTP/HTTPInputStreams.cpp | 2 + .../CustomHTTP/HTTPOutputStreams.cpp | 44 ++---- .../CustomHTTP/HTTPOutputStreams.h | 8 +- .../CustomHTTP/MethodQueryMatcher.h | 30 ++++ .../CustomHTTP/QueryExecutorAndMatcher.h | 47 ++++++ .../CustomHTTP/URLQueryMatcher.cpp | 73 ++++++++++ .../Interpreters/CustomHTTP/URLQueryMatcher.h | 32 +++++ .../configs/custom_http_config.xml | 4 +- programs/server/HTTPHandler.cpp | 103 +++++++++---- programs/server/HTTPHandler.h | 14 +- programs/server/config.xml | 11 +- src/IO/WriteBufferFromHTTPServerResponse.cpp | 64 ++++----- src/IO/WriteBufferFromHTTPServerResponse.h | 5 +- .../ReplaceQueryParameterVisitor.cpp | 2 - 24 files changed, 691 insertions(+), 496 deletions(-) create mode 100644 dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h create mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h create mode 100644 dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h create mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp create mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h diff --git a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h new file mode 100644 index 00000000000..56fa001d57c --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ + +class AlwaysQueryMatcher : public QueryMatcher +{ +public: + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp new file mode 100644 index 00000000000..e6ed579b696 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp @@ -0,0 +1,73 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_CUSTOM_EXECUTOR_PARAM; +} + +void prepareQueryParams(Context & context, HTMLForm & params, const NameSet & query_params_name) +{ + for (const auto & param : params) + if (query_params_name.count(param.first)) + context.setQueryParameter(param.first, param.second); +} + +QueryExecutorConst::QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) +{ + execute_query = configuration.getString(config_key); + + const char * query_begin = execute_query.data(); + const char * query_end = execute_query.data() + execute_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + + QueryParameterVisitor{query_params_name}.visit(extract_query_ast); + can_be_parse_request_body = !extract_query_ast->as(); + + const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); + for (const auto & prepared_param_name : query_params_name) + { + if (Settings::findIndex(prepared_param_name) != Settings::npos || reserved_params_name.count(prepared_param_name)) + throw Exception( + "Illegal custom executor query param name '" + prepared_param_name + "', Because it's a reserved name or Settings name", + ErrorCodes::ILLEGAL_CUSTOM_EXECUTOR_PARAM); + } +} + +void QueryExecutorConst::executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const +{ + ReadBufferPtr temp_query_buf; + prepareQueryParams(context, params, query_params_name); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (!canBeParseRequestBody() && !startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h new file mode 100644 index 00000000000..170d2a959ef --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +namespace DB +{ + +class QueryExecutorConst : public QueryExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + + bool canBeParseRequestBody() const override { return can_be_parse_request_body; } + + bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } + + QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); + + void executeQueryImpl( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; + +private: + String execute_query; + NameSet query_params_name; + bool can_be_parse_request_body{false}; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index 981d243ef35..d3d0bfc5aeb 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -1,6 +1,10 @@ #include #include -#include "CustomExecutor.h" +#include +#include +#include +#include +#include namespace DB @@ -9,10 +13,17 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; - extern const int DUPLICATE_CUSTOM_EXECUTOR; + extern const int UNKNOW_QUERY_EXECUTOR; extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; } +CustomExecutor::CustomExecutor( + const std::vector & matchers_, + const std::vector & query_executors_) + : matchers(matchers_), query_executors(query_executors_) +{ +} + bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const { for (const auto & matcher : matchers) @@ -35,11 +46,11 @@ bool CustomExecutor::isQueryParam(const String & param_name) const return true; } -bool CustomExecutor::canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const +bool CustomExecutor::canBeParseRequestBody() const { for (const auto & query_executor : query_executors) { - if (!query_executor->canBeParseRequestBody(request, params)) + if (!query_executor->canBeParseRequestBody()) return false; } @@ -57,20 +68,6 @@ void CustomExecutor::executeQuery( output_streams.finalize(); } -CustomExecutor::CustomExecutor( - const std::vector & matchers_, const std::vector & query_executors_) - : matchers(matchers_), query_executors(query_executors_) -{ -} - -static CustomExecutorPtr createDefaultCustomExecutor() -{ - std::vector custom_matchers{std::make_shared()}; - std::vector custom_query_executors{std::make_shared()}; - - return std::make_shared(custom_matchers, custom_query_executors); -} - void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) { Configuration::Keys custom_executors_keys; @@ -80,91 +77,91 @@ void CustomExecutors::updateCustomExecutors(const Configuration & config, const for (const auto & custom_executor_key : custom_executors_keys) { - if (custom_executor_key == "Default") - throw Exception("CustomExecutor cannot be 'Default'.", ErrorCodes::SYNTAX_ERROR); - else if (custom_executor_key.find('.') != String::npos) + if (custom_executor_key.find('.') != String::npos) throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - const auto & exists_executor = [&](auto & ele) { return ele.first == custom_executor_key; }; - if (std::count_if(new_custom_executors.begin(), new_custom_executors.end(), exists_executor)) - throw Exception("CustomExecutor name '" + custom_executor_key + "' already exists in system.", - ErrorCodes::DUPLICATE_CUSTOM_EXECUTOR); - - new_custom_executors.push_back( - std::make_pair(custom_executor_key, createCustomExecutor(config, config_prefix + "." + custom_executor_key))); + new_custom_executors.push_back({custom_executor_key, createCustomExecutor(config, config_prefix, custom_executor_key)}); } - new_custom_executors.push_back(std::make_pair("Default", createDefaultCustomExecutor())); - std::unique_lock lock(rwlock); custom_executors = new_custom_executors; } -void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); - const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR CustomQueryExecutor name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", - ErrorCodes::LOGICAL_ERROR); - - query_executor_creators[query_executor_name] = creator; -} - void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) { const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR CustomExecutorMatcher name must be unique between the CustomQueryExecutor and CustomExecutorMatcher.", - ErrorCodes::LOGICAL_ERROR); + throw Exception("LOGICAL_ERROR QueryMatcher name must be unique between the QueryExecutor and QueryMatcher.", + ErrorCodes::LOGICAL_ERROR); custom_matcher_creators[matcher_name] = creator; } -CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix) +void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) { - Configuration::Keys matchers_or_query_executors_type; - config.keys(config_prefix, matchers_or_query_executors_type); + const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); + const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - std::vector custom_query_executors; - std::vector custom_executor_matchers; + if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) + throw Exception("LOGICAL_ERROR QueryExecutor name must be unique between the QueryExecutor and QueryMatcher.", + ErrorCodes::LOGICAL_ERROR); - for (const auto & matcher_or_query_executor_type : matchers_or_query_executors_type) + query_executor_creators[query_executor_name] = creator; +} + +String fixMatcherOrExecutorTypeName(const String & matcher_or_executor_type_name) +{ + auto type_name_end_pos = matcher_or_executor_type_name.find('['); + return type_name_end_pos == String::npos ? matcher_or_executor_type_name : matcher_or_executor_type_name.substr(0, type_name_end_pos); +} + +CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name) +{ + Configuration::Keys matchers_key; + config.keys(config_prefix + "." + name, matchers_key); + + std::vector query_matchers; + std::vector query_executors; + + for (const auto & matcher_key : matchers_key) { + String matcher_or_query_executor_type = fixMatcherOrExecutorTypeName(matcher_key); + if (matcher_or_query_executor_type.find('.') != String::npos) - throw Exception( - "CustomMatcher or CustomQueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", + throw Exception("CustomMatcher or QueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", ErrorCodes::SYNTAX_ERROR); const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) - throw Exception("CustomMatcher or CustomQueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", - ErrorCodes::NOT_IMPLEMENTED); + throw Exception("CustomMatcher or QueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", + ErrorCodes::NOT_IMPLEMENTED); if (matcher_creator_it != custom_matcher_creators.end()) - custom_executor_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + query_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); if (query_executor_creator_it != query_executor_creators.end()) - custom_query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + matcher_or_query_executor_type)); + query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); } - checkCustomMatchersAndQueryExecutors(custom_executor_matchers, custom_query_executors); - return std::make_shared(custom_executor_matchers, custom_query_executors); + checkQueryMatchersAndExecutors(name, query_matchers, query_executors); + return std::make_shared(query_matchers, query_executors); } -void CustomExecutors::checkCustomMatchersAndQueryExecutors( - std::vector & matchers, std::vector & query_executors) +void CustomExecutors::checkQueryMatchersAndExecutors( + const String & name, std::vector & matchers, std::vector & query_executors) { - const auto & sum_func = [&](auto & ele) { return !ele->canBeParseRequestBody(); }; + if (matchers.empty() || query_executors.empty()) + throw Exception("The CustomExecutor '" + name + "' must contain a Matcher and a QueryExecutor.", ErrorCodes::SYNTAX_ERROR); + + const auto & sum_func = [&](auto & ele) -> bool { return !ele->canBeParseRequestBody(); }; const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); if (need_post_data_count > 1) - throw Exception("The CustomExecutor can only contain one insert query.", ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); + throw Exception("The CustomExecutor '" + name + "' can only contain one insert query." + toString(need_post_data_count), ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); for (const auto & matcher : matchers) matcher->checkQueryExecutors(query_executors); @@ -178,19 +175,25 @@ std::pair CustomExecutors::getCustomExecutor(Context if (custom_executor.second->match(context, request, params)) return custom_executor; - throw Exception("LOGICAL_ERROR not found custom executor.", ErrorCodes::LOGICAL_ERROR); + throw Exception("No query executors matched", ErrorCodes::UNKNOW_QUERY_EXECUTOR); } CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) { registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); + + registerCustomMatcher("always_matched", [&](const auto & /*matcher_config*/, const auto & /*prefix*/) + { return std::make_shared(); }); registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); + { return std::make_shared(matcher_config, prefix); }); + + registerQueryExecutor("dynamic_query", [&](const auto & matcher_config, const auto & prefix) + { return std::make_shared(matcher_config, prefix); }); updateCustomExecutors(config, settings, config_prefix); } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h index 94d1fe28923..7b37a6afdf5 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h @@ -7,14 +7,13 @@ #include #include #include -#include -#include -#include #include #include #include -#include -#include +#include +#include +#include +#include namespace DB { @@ -33,10 +32,10 @@ public: CustomExecutors(const CustomExecutors &) = delete; CustomExecutors & operator=(const CustomExecutors &) = delete; - using QueryExecutorCreator = std::function; + using QueryExecutorCreator = std::function; void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - using CustomMatcherCreator = std::function; + using CustomMatcherCreator = std::function; void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); @@ -48,18 +47,18 @@ private: std::unordered_map query_executor_creators; std::unordered_map custom_matcher_creators; - CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix); + CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name); - void checkCustomMatchersAndQueryExecutors(std::vector & matchers, std::vector & query_executors); + void checkQueryMatchersAndExecutors(const String & name, std::vector & matchers, std::vector & query_executors); }; class CustomExecutor { public: - bool isQueryParam(const String & param_name) const; + bool canBeParseRequestBody() const; - bool canBeParseRequestBody(HTTPRequest & request, HTMLForm & params) const; + bool isQueryParam(const String & param_name) const; bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; @@ -68,11 +67,11 @@ public: HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams ); - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); + CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); private: - std::vector matchers; - std::vector query_executors; + std::vector matchers; + std::vector query_executors; }; } diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h deleted file mode 100644 index 825b3c0b268..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutorMatchers.h +++ /dev/null @@ -1,122 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - - -#if USE_RE2_ST -# include -#else -# define re2_st re2 -#endif - -namespace DB -{ - -class CustomExecutorMatcher -{ -public: - virtual ~CustomExecutorMatcher() = default; - - virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; - - virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; -}; - -using CustomExecutorMatcherPtr = std::shared_ptr; - - -class AlwaysMatchedCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } -}; - -class HTTPMethodCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - - HTTPMethodCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & method_config_key) - { - match_method = Poco::toLower(configuration.getString(method_config_key)); - } - - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - return Poco::toLower(request.getMethod()) == match_method; - } - -private: - String match_method; -}; - -class HTTPURLCustomExecutorMatcher : public CustomExecutorMatcher -{ -public: - HTTPURLCustomExecutorMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & url_config_key) - { - regex_matcher = std::make_unique(configuration.getString(url_config_key)); - } - - bool checkQueryExecutors(const std::vector & custom_query_executors) const override - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - if (!checkQueryExecutors(named_capturing_group.first, custom_query_executors)) - throw Exception("The param name '" + named_capturing_group.first + "' is uselessed.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return true; - } - - bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - const String request_uri = request.getURI(); - int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(request_uri.data(), request_uri.size()); - if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - const auto & full_match = matches[0]; - const char * url_end = request_uri.data() + request_uri.size(); - const char * not_matched_begin = request_uri.data() + full_match.size(); - - if (not_matched_begin != url_end && *not_matched_begin == '/') - ++not_matched_begin; - - if (not_matched_begin == url_end || *not_matched_begin == '?') - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - { - const auto & capturing_value = matches[named_capturing_group.second]; - context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); - } - - return true; - } - } - return false; - } - -private: - std::unique_ptr regex_matcher; - - bool checkQueryExecutors(const String & param_name, const std::vector & custom_query_executors) const - { - for (const auto & custom_query_executor : custom_query_executors) - if (custom_query_executor->isQueryParam(param_name)) - return true; - - return false; - } -}; - - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h b/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h deleted file mode 100644 index f986db6d41d..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomQueryExecutors.h +++ /dev/null @@ -1,131 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class CustomQueryExecutor -{ -public: - virtual ~CustomQueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody(Poco::Net::HTTPServerRequest &, HTMLForm &) const = 0; - - virtual void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; -}; - -using CustomQueryExecutorPtr = std::shared_ptr; - -class ConstQueryCustomQueryExecutor : public CustomQueryExecutor -{ -public: - ConstQueryCustomQueryExecutor(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) - { - execute_query = configuration.getString(config_key, ""); - - const char * query_begin = execute_query.data(); - const char * query_end = execute_query.data() + execute_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - - QueryParameterVisitor{query_params_name}.visit(extract_query_ast); - can_be_parse_request_body = !extract_query_ast->as(); - } - - bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } - - bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return can_be_parse_request_body; } - - void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & /*input_streams*/, const HTTPOutputStreams & output_streams) const override - { - prepareQueryParams(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - -private: - String execute_query; - NameSet query_params_name; - bool can_be_parse_request_body{false}; - - void prepareQueryParams(Context & context, HTMLForm & params) const - { - for (const auto & param : params) - if (isQueryParam(param.first)) - context.setQueryParameter(param.first, param.second); - } -}; - -class ExtractQueryParamCustomQueryExecutor : public CustomQueryExecutor -{ -public: - bool isQueryParam(const String & param_name) const override { return param_name == "query" || startsWith(param_name, "param_"); } - - bool canBeParseRequestBody(Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*form*/) const override { return false; } - - void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override - { - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - ReadBufferPtr temp_query_buf; - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - -private: - String prepareQuery(Context & context, HTMLForm & params) const - { - const static size_t prefix_size = strlen("param_"); - - std::stringstream query_stream; - for (const auto & param : params) - { - if (param.first == "query") - query_stream << param.second; - else if (startsWith(param.first, "param_")) - context.setQueryParameter(param.first.substr(prefix_size), param.second); - } - - query_stream << "\n"; - return query_stream.str(); - } -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp new file mode 100644 index 00000000000..9b4a4006556 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp @@ -0,0 +1,60 @@ +#include + +#include +#include +#include + +namespace DB +{ + +bool QueryExecutorDynamic::isQueryParam(const String & param_name) const +{ + return param_name == dynamic_param_name || startsWith(param_name, "param_"); +} + +QueryExecutorDynamic::QueryExecutorDynamic(const Configuration & configuration, const String & config_key) +{ + dynamic_param_name = configuration.getString(config_key + "." + "param_name", "query"); +} + +void QueryExecutorDynamic::executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const +{ + ReadBufferPtr temp_query_buf; + const auto & execute_query = prepareQuery(context, params); + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (!startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, + context, [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); +} + +String QueryExecutorDynamic::prepareQuery(Context & context, HTMLForm & params) const +{ + const static size_t prefix_size = strlen("param_"); + + WriteBufferFromOwnString query_buffer; + for (const auto & param : params) + { + if (param.first == dynamic_param_name) + writeString(param.second, query_buffer); + else if (startsWith(param.first, "param_")) + context.setQueryParameter(param.first.substr(prefix_size), param.second); + } + + if (query_buffer.offset()) + writeString("\n", query_buffer); + + return query_buffer.str(); +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h new file mode 100644 index 00000000000..a09146246d4 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class QueryExecutorDynamic : public QueryExecutor +{ +public: + using HTTPServerRequest = Poco::Net::HTTPServerRequest; + using HTTPServerResponse = Poco::Net::HTTPServerResponse; + using Configuration = Poco::Util::AbstractConfiguration; + + bool canBeParseRequestBody() const override { return false; } + + bool isQueryParam(const String & param_name) const override; + + QueryExecutorDynamic(const Configuration & configuration, const String & config_key); + + void executeQueryImpl( + Context & context, HTTPServerRequest & request, HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; + +private: + String dynamic_param_name{"query"}; + + String prepareQuery(Context & context, HTMLForm & params) const; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h index 7b63df33783..56411cb122a 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h @@ -9,44 +9,81 @@ namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_HTTP_PARAM; +} + class ExtractorContextChange { public: ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} - void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + static const NameSet & getReservedParamNames() { - Names reserved_param_suffixes; static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "buffer_size", "wait_end_of_query", - "session_id", "session_timeout", "session_check"}; + "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check" + }; - auto param_could_be_skipped = [&] (const String & name) + return reserved_param_names; + } + + static std::function reservedParamSuffixesFilter(bool reserved) + { + if (!reserved) + return [&](const String &) { return false; }; + + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + return [&](const String & param_name) { - if (reserved_param_names.count(name)) + if (endsWith(param_name, "_format")) + return true; + else if (endsWith(param_name, "_types")) + return true; + else if (endsWith(param_name, "_structure")) return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } return false; }; + } + + void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) + { + bool is_multipart_data = startsWith(request.getContentType().data(), "multipart/form-data"); /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not /// belong to the Settings class. + becomeReadonlyIfNeed(request); + changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + if (is_multipart_data || executor->canBeParseRequestBody()) + { + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. + /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` + if (executor->canBeParseRequestBody()) + changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); + } + } + +private: + Context & context; + const CustomExecutorPtr & executor; + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + /// Only readonly queries are allowed for HTTP GET requests. + void becomeReadonlyIfNeed(HTTPRequest & request) + { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { Settings & settings = context.getSettingsRef(); @@ -54,45 +91,33 @@ public: if (settings.readonly == 0) settings.readonly = 2; } + } - bool has_multipart = startsWith(request.getContentType().data(), "multipart/form-data"); - - if (has_multipart || executor->canBeParseRequestBody(request, params)) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - if (has_multipart) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - } + void changeSettingsFromParams(HTMLForm & params, const std::function & reserved_param_suffixes) + { SettingsChanges settings_changes; - for (const auto & [key, value] : params) + const auto & reserved_param_names = getReservedParamNames(); + + for (const auto & [name, value] : params) { - if (key == "database") + if (name == "database") context.setCurrentDatabase(value); - else if (key == "default_format") + else if (name == "default_format") context.setDefaultFormat(value); - else if (!param_could_be_skipped(key) && !executor->isQueryParam(key)) - settings_changes.push_back({key, value}); /// All other query parameters are treated as settings. + else if (!reserved_param_names.count(name) && !reserved_param_suffixes(name)) + { + if (Settings::findIndex(name) != Settings::npos) + settings_changes.push_back({name, value}); + else if (!executor->isQueryParam(name)) + throw Exception("Unknown HTTP param name: '" + name + "'", ErrorCodes::UNKNOWN_HTTP_PARAM); + } } /// For external data we also want settings context.checkSettingsConstraints(settings_changes); context.applySettingsChanges(settings_changes); } - -private: - Context & context; - const CustomExecutorPtr & executor; - }; } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp index 7c441891b33..3321b57b38d 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp @@ -1,7 +1,9 @@ +#include #include #include +#include #include #include #include diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp index 113523494cf..8ac682af5e4 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp @@ -52,16 +52,15 @@ namespace } } -HTTPOutputStreams::HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout) - : out(createResponseOut(request, response, keep_alive_timeout)) +HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress) + : out(raw_out) , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) , out_maybe_delayed_and_compressed(out_maybe_compressed) { } -HTTPOutputStreams::HTTPOutputStreams( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout) - : out(createResponseOut(request, response, keep_alive_timeout)) +HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form) + : out(raw_out) , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) { @@ -90,36 +89,10 @@ HTTPOutputStreams::HTTPOutputStreams( } } -HTTPResponseBufferPtr HTTPOutputStreams::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive) -{ - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); - - if (!http_response_compression_methods.empty()) - { - /// 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("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -#endif - } - - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE, response.sent()); -} - WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) { /// Client can pass a 'compress' flag in the query string. In this case the query result is /// compressed using internal algorithm. This is not reflected in HTTP headers. -// bool internal_compression = form.getParsed("compress", false); return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); } @@ -178,13 +151,18 @@ WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & HTTPOutputStreams::~HTTPOutputStreams() { - /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + /// This could be a broken HTTP Request + /// Because it does not call finalize or writes some data to output stream after call finalize + /// In this case we need to clean up its broken state to ensure that they are not sent to the client + + /// For delayed stream, we destory CascadeBuffer and without sending any data to client. if (out_maybe_delayed_and_compressed != out_maybe_compressed) out_maybe_delayed_and_compressed.reset(); - /// If buffer has data, and that data wasn't sent yet, then no need to send that data if (out->count() == out->offset()) { + /// If buffer has data and server never sends data to client + /// no need to send that data out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); out->position() = out->buffer().begin(); } diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h index 614acb6fcf3..fba122ea294 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h +++ b/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h @@ -34,15 +34,13 @@ struct HTTPOutputStreams void finalize() const; - WriteBufferPtr createMaybeDelayedAndCompressionOut(Context &context, HTMLForm &form, WriteBufferPtr &out_); + WriteBufferPtr createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_); WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response, size_t keep_alive); + HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress); - HTTPOutputStreams(HTTPServerRequest & request, HTTPServerResponse & response, bool internal_compress, size_t keep_alive_timeout); - - HTTPOutputStreams(Context & context, HTTPServerRequest & request, HTTPServerResponse & response, HTMLForm & form, size_t keep_alive_timeout); + HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form); }; using HTTPOutputStreamsPtr = std::unique_ptr; diff --git a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h new file mode 100644 index 00000000000..689aff67dc7 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include "QueryExecutorAndMatcher.h" + +namespace DB +{ + +class MethodQueryMatcher : public QueryMatcher +{ +public: + using Config = Poco::Util::AbstractConfiguration; + + MethodQueryMatcher(const Config & configuration, const String & config_key) + : method(Poco::toLower(configuration.getString(config_key))) + { + } + + bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override + { + return Poco::toLower(request.getMethod()) == method; + } + + bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } + +private: + String method; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h new file mode 100644 index 00000000000..7f32c4c2ddb --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; +} + +class QueryExecutor +{ +public: + virtual ~QueryExecutor() = default; + + virtual bool isQueryParam(const String &) const = 0; + virtual bool canBeParseRequestBody() const = 0; + + virtual void executeQueryImpl( + Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; +}; + +using QueryExecutorPtr = std::shared_ptr; + + +class QueryMatcher +{ +public: + virtual ~QueryMatcher() = default; + + virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; + + virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; +}; + +using QueryMatcherPtr = std::shared_ptr; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp new file mode 100644 index 00000000000..82bb864c926 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp @@ -0,0 +1,73 @@ +#include +#include "URLQueryMatcher.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; + extern const int UNDEFINED_CUSTOM_EXECUTOR_PARAM; +} + +URLQueryMatcher::URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) +{ + const auto & regex_str = configuration.getString(config_key); + regex_matcher = std::make_unique(regex_str); + + if (!regex_matcher->ok()) + throw Exception("cannot compile re2: " + regex_str + ", error: " + regex_matcher->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); +} + +bool checkQueryOneQueryParam(const String & param_name, const std::vector & custom_query_executors) +{ + for (const auto & custom_query_executor : custom_query_executors) + if (custom_query_executor->isQueryParam(param_name)) + return true; + + return false; +} + +bool URLQueryMatcher::checkQueryExecutors(const std::vector & custom_query_executors) const +{ + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + if (!checkQueryOneQueryParam(named_capturing_group.first, custom_query_executors)) + throw Exception("The param name '" + named_capturing_group.first + "' is not defined in the QueryExecutor.", + ErrorCodes::UNDEFINED_CUSTOM_EXECUTOR_PARAM); + + return true; +} + +bool URLQueryMatcher::match(Context & context, Poco::Net::HTTPServerRequest &request, HTMLForm &) const +{ + const String request_uri = request.getURI(); + int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(request_uri.data(), request_uri.size()); + if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + const auto & full_match = matches[0]; + const char * url_end = request_uri.data() + request_uri.size(); + const char * not_matched_begin = request_uri.data() + full_match.size(); + + if (not_matched_begin != url_end && *not_matched_begin == '/') + ++not_matched_begin; + + if (not_matched_begin == url_end || *not_matched_begin == '?') + { + for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) + { + const auto & capturing_value = matches[named_capturing_group.second]; + context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); + } + + return true; + } + } + return false; +} + +} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h new file mode 100644 index 00000000000..61323db3e53 --- /dev/null +++ b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + +#include +#include + +#if USE_RE2_ST +# include +#else +# define re2_st re2 +#endif + + +namespace DB +{ + +class URLQueryMatcher : public QueryMatcher +{ +public: + URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); + + bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm &) const override; + + bool checkQueryExecutors(const std::vector & custom_query_executors) const override; + +private: + std::unique_ptr regex_matcher; +}; + +} diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml index f91e196abab..eb56cd6c18c 100644 --- a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml +++ b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml @@ -41,13 +41,13 @@ - PUT + PUT SELECT 'Matched test_for_method_match' - /test_for_multiple_match GET + /test_for_multiple_match SELECT 'Matched test_for_multiple_match' diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 6ceaf7d7d3f..cfbefb94ee4 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -90,6 +90,7 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; + extern const int UNKNOW_QUERY_EXECUTOR; } @@ -117,7 +118,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH) return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + else if (exception_code == ErrorCodes::UNKNOW_QUERY_EXECUTOR || + exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE || @@ -184,18 +186,22 @@ HTTPHandler::SessionContextHolder::~SessionContextHolder() } -HTTPHandler::SessionContextHolder::SessionContextHolder(IServer & accepted_server, HTMLForm & params) +HTTPHandler::SessionContextHolder::SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params) + : query_context(query_context_) { - session_id = params.get("session_id", ""); - context = std::make_unique(accepted_server.context()); + authentication(request, params); - if (!session_id.empty()) { - session_timeout = parseSessionTimeout(accepted_server.config(), params); - session_context = context->acquireSession(session_id, session_timeout, params.check("session_check", "1")); + session_id = params.get("session_id", ""); - context = std::make_unique(*session_context); - context->setSessionContext(*session_context); + if (!session_id.empty()) + { + session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); + session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + query_context = *session_context; + query_context.setSessionContext(*session_context); + } } } @@ -237,11 +243,11 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque } std::string query_id = params.get("query_id", ""); - context->setUser(user, password, request.clientAddress(), quota_key); - context->setCurrentQueryId(query_id); + query_context.setUser(user, password, request.clientAddress(), quota_key); + query_context.setCurrentQueryId(query_id); } -void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response) +void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out) { const auto & name_with_custom_executor = context.getCustomExecutor(request, params); LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); @@ -250,12 +256,13 @@ void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLFor ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(context, request, response, params, getKeepAliveTimeout()); + HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); } void HTTPHandler::trySendExceptionToClient( - const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, bool compression) + const std::string & message, int exception_code, HTTPRequest & request, + HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression) { try { @@ -265,22 +272,25 @@ void HTTPHandler::trySendExceptionToClient( /// 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::max()); - - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD - || exception_code == ErrorCodes::REQUIRED_PASSWORD || exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) { - if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - else - response.requireAuthentication("ClickHouse server HTTP API"); + request.stream().ignore(std::numeric_limits::max()); + } - response.send() << message << std::endl; + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + response.requireAuthentication("ClickHouse server HTTP API"); } else { response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - HTTPOutputStreams output_streams(request, response, compression, getKeepAliveTimeout()); + } + + if (!response_out && !response.sent()) + response.send() << message << std::endl; + else + { + HTTPOutputStreams output_streams(response_out, compression); writeString(message, *output_streams.out_maybe_compressed); writeChar('\n', *output_streams.out_maybe_compressed); @@ -300,10 +310,12 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne ThreadStatus thread_status; /// In case of exception, send stack trace to client. + HTTPResponseBufferPtr response_out; bool with_stacktrace = false, internal_compression = false; try { + response_out = createResponseOut(request, response); response.set("Content-Type", "text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); @@ -316,15 +328,22 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. +<<<<<<< HEAD:programs/server/HTTPHandler.cpp 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); +======= + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && !request.getChunkedTransferEncoding() + && !request.hasContentLength()) + throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); +>>>>>>> ISSUES-5436 fix build failure & fix test failure:dbms/programs/server/HTTPHandler.cpp { - SessionContextHolder holder{server, params}; - CurrentThread::QueryScope query_scope(*holder.context); + Context query_context = server.context(); + CurrentThread::QueryScope query_scope(query_context); - holder.authentication(request, params); - processQuery(*holder.context, request, params, response); + SessionContextHolder holder{query_context, request, params}; + processQuery(holder.query_context, request, params, response, response_out); LOG_INFO(log, "Done processing query"); } } @@ -337,8 +356,34 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne */ int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, internal_compression); + trySendExceptionToClient(exception_message, exception_code, request, response, response_out, internal_compression); } } +HTTPResponseBufferPtr HTTPHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +{ + size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + + if (!http_response_compression_methods.empty()) + { + /// 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("gzip")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 47d63af3740..1faa0efe907 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -38,12 +38,12 @@ private: { ~SessionContextHolder(); - SessionContextHolder(IServer & accepted_server, HTMLForm & params); - void authentication(HTTPServerRequest & request, HTMLForm & params); + SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params); + String session_id; - std::unique_ptr context = nullptr; + Context & query_context; std::shared_ptr session_context = nullptr; std::chrono::steady_clock::duration session_timeout; }; @@ -58,9 +58,13 @@ private: size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response); + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - void trySendExceptionToClient(const std::string & message, int exception_code, HTTPRequest & request, HTTPResponse & response, bool compression); + void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out); + + void trySendExceptionToClient( + const std::string & message, int exception_code, HTTPRequest & request, + HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression); }; diff --git a/programs/server/config.xml b/programs/server/config.xml index 5111e6102c2..1c04dee6c3a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,11 +525,18 @@ --> + + + + + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index c7b3abb008b..25c2974056c 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -31,6 +31,11 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() response.set("Access-Control-Allow-Origin", "*"); setResponseDefaultHeaders(response, keep_alive_timeout); + +#if defined(POCO_CLICKHOUSE_PATCH) + if (request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) + std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); +#endif } } @@ -91,12 +96,19 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders() } -void WriteBufferFromHTTPServerResponse::choiceSendEncode() +void WriteBufferFromHTTPServerResponse::nextImpl() { - if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) { - if (!compress) + std::lock_guard lock(mutex); + + startSendHeaders(); + + if (!out && request.getMethod() != Poco::Net::HTTPRequest::HTTP_HEAD) { + if (compress) + { + if (compression_method == CompressionMethod::Gzip) + { #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: gzip\r\n"; #else @@ -112,8 +124,8 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: deflate\r\n"; #else - /// Newline autosent by response.send() - response_body_ostr = &(response.send()); + response.set("Content-Encoding", "gzip"); + response_body_ostr = &(response.send()); #endif out_raw = std::make_unique(*response_body_ostr); deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); @@ -123,9 +135,7 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() else if (compression_method == CompressionMethod::Brotli) { #if defined(POCO_CLICKHOUSE_PATCH) - std::tie(response_header_ostr, response_body_ostr) = response.beginSend(); - if (headers_started_sending && !headers_finished_sending) - *response_header_ostr << "Content-Encoding: " << encoding_type << "\r\n"; + *response_header_ostr << "Content-Encoding: deflate\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif @@ -134,37 +144,23 @@ void WriteBufferFromHTTPServerResponse::choiceSendEncode() /// This may result in an extra empty line in the response body response_body_ostr = &(response.send()); #endif - }; out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out = &*out_raw; } -#if USE_BROTLI - else if (compression_method == CompressionMethod::Brotli) - { - set_encoding_type("br"); - out_raw.emplace(*response_body_ostr); - brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*brotli_buf; - } -#endif else - throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", - ErrorCodes::LOGICAL_ERROR); - /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. + { +#if !defined(POCO_CLICKHOUSE_PATCH) + response_body_ostr = &(response.send()); +#endif + + out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out = &*out_raw; + } } - } -} - -void WriteBufferFromHTTPServerResponse::nextImpl() -{ - { - std::lock_guard lock(mutex); - - startSendHeaders(); - choiceSendEncode(); finishSendHeaders(); + } if (out) @@ -182,16 +178,13 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( unsigned keep_alive_timeout_, bool compress_, CompressionMethod compression_method_, - size_t size, - bool finish_send_headers_) + size_t size) : BufferWithOwnMemory(size) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) , compress(compress_) , compression_method(compression_method_) - , headers_started_sending(finish_send_headers_) - , headers_finished_sending(finish_send_headers_) { } @@ -235,6 +228,7 @@ void WriteBufferFromHTTPServerResponse::finalize() } } + WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { try diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index fd57279646f..528ba8ffe23 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -87,8 +87,6 @@ private: /// This method finish headers with \r\n, allowing to start to send body. void finishSendHeaders(); - void choiceSendEncode(); - void nextImpl() override; public: @@ -98,8 +96,7 @@ public: unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. CompressionMethod compression_method_ = CompressionMethod::Gzip, - size_t size = DBMS_DEFAULT_BUFFER_SIZE, - bool finish_send_headers_ = false); + size_t size = DBMS_DEFAULT_BUFFER_SIZE); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 98dc90d9dda..e2231e35f03 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -57,9 +57,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); - skipWhitespaceIfAny(read_buffer); /// Skip white space on both sides if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From dafba9d66a4af0f476399fe1bcfc3d1f0db53184 Mon Sep 17 00:00:00 2001 From: tai Date: Fri, 8 Nov 2019 20:37:31 +0800 Subject: [PATCH 10/23] ISSUES-5436 add integration test for custom http --- dbms/programs/server/HTTPHandlerFactory.h | 64 +++ .../CustomHTTP/CustomExecutor.cpp | 2 +- .../common_configs/common_config.xml | 415 ++++++++++++++++++ .../common_configs/common_users.xml | 138 ++++++ .../configs/custom_http_config.xml | 54 --- .../normally_configs/config.xml | 30 ++ .../normally_configs/users.xml | 3 + .../integration/test_custom_http/test.py | 29 +- programs/server/config.xml | 33 +- tests/integration/helpers/cluster.py | 8 +- .../0_stateless/00408_http_keep_alive.sh | 3 - 11 files changed, 702 insertions(+), 77 deletions(-) create mode 100644 dbms/programs/server/HTTPHandlerFactory.h create mode 100644 dbms/tests/integration/test_custom_http/common_configs/common_config.xml create mode 100644 dbms/tests/integration/test_custom_http/common_configs/common_users.xml delete mode 100644 dbms/tests/integration/test_custom_http/configs/custom_http_config.xml create mode 100644 dbms/tests/integration/test_custom_http/normally_configs/config.xml create mode 100644 dbms/tests/integration/test_custom_http/normally_configs/users.xml diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h new file mode 100644 index 00000000000..73e0a2b7339 --- /dev/null +++ b/dbms/programs/server/HTTPHandlerFactory.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include +#include "IServer.h" +#include "HTTPHandler.h" +#include "InterserverIOHTTPHandler.h" +#include "NotFoundHandler.h" +#include "PingRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "RootRequestHandler.h" + + +namespace DB +{ + +template +class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +private: + IServer & server; + Logger * log; + std::string name; + +public: + HTTPRequestHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) + { + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override + { + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " + << request.getMethod() + << ", Address: " + << request.clientAddress().toString() + << ", User-Agent: " + << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (uri == "/") + return new RootRequestHandler(server); + if (uri == "/ping") + return new PingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new ReplicasStatusHandler(server.context()); + } + + return new HandlerType(server); + } +}; + +using HTTPHandlerFactory = HTTPRequestHandlerFactory; +using InterserverIOHTTPHandlerFactory = HTTPRequestHandlerFactory; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp index d3d0bfc5aeb..3e984deb8aa 100644 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp @@ -175,7 +175,7 @@ std::pair CustomExecutors::getCustomExecutor(Context if (custom_executor.second->match(context, request, params)) return custom_executor; - throw Exception("No query executors matched", ErrorCodes::UNKNOW_QUERY_EXECUTOR); + throw Exception("No CustomExecutor match " + request.getURI(), ErrorCodes::UNKNOW_QUERY_EXECUTOR); } CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_config.xml b/dbms/tests/integration/test_custom_http/common_configs/common_config.xml new file mode 100644 index 00000000000..154ebf6c35e --- /dev/null +++ b/dbms/tests/integration/test_custom_http/common_configs/common_config.xml @@ -0,0 +1,415 @@ + + + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + + 8123 + 9000 + + + + + + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + /etc/clickhouse-server/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + + + + + 9009 + + + + + + + + + + + + + + + + + + + + 4096 + 3 + + + 100 + + + + + + 8589934592 + + + 5368709120 + + + + /var/lib/clickhouse/ + + + /var/lib/clickhouse/tmp/ + + + /var/lib/clickhouse/user_files/ + + + users.xml + + + default + + + + + + default + + + + + + + + + false + + + + + + + + localhost + 9000 + + + + + + + localhost + 9000 + + + + + localhost + 9000 + + + + + + + localhost + 9440 + 1 + + + + + + + localhost + 9000 + + + + + localhost + 1 + + + + + + + + + + + + + + + + + 3600 + + + + 3600 + + + 60 + + + + + + + + + + system + query_log
+ + toYYYYMM(event_date) + + 7500 +
+ + + + system + query_thread_log
+ toYYYYMM(event_date) + 7500 +
+ + + + + + + + + + + + + + + *_dictionary.xml + + + + + + + + + + /clickhouse/task_queue/ddl + + + + + + + + + + + + + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + + + /var/lib/clickhouse/format_schemas/ + + + +
diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_users.xml b/dbms/tests/integration/test_custom_http/common_configs/common_users.xml new file mode 100644 index 00000000000..9755c29d480 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/common_configs/common_users.xml @@ -0,0 +1,138 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + + + + + a = 1 + + + + + a + b < 1 or c - d > 5 + + + + + c = 1 + + + + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml b/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml deleted file mode 100644 index eb56cd6c18c..00000000000 --- a/dbms/tests/integration/test_custom_http/configs/custom_http_config.xml +++ /dev/null @@ -1,54 +0,0 @@ - - 8123 - - - - /test_for_single_insert - INSERT INTO test.test VALUES - - - - /test_for_single_select - SELECT * FROM test.test ORDER BY id - - - - /test_for_multiple_select - SELECT * FROM test.test ORDER BY id - SELECT * FROM test.test ORDER BY id - - - - /test_for_multiple_select - INSERT INTO test.test VALUES - SELECT 'test_for_hybrid_insert_and_select_with_one_insert first' - SELECT 'test_for_hybrid_insert_and_select_with_one_insert second' - - - - - - /test_for_url_match - SELECT 'Matched test_for_url_match' - - - - PUT - SELECT 'Matched test_for_method_match' - - - - GET - /test_for_multiple_match - SELECT 'Matched test_for_multiple_match' - - - diff --git a/dbms/tests/integration/test_custom_http/normally_configs/config.xml b/dbms/tests/integration/test_custom_http/normally_configs/config.xml new file mode 100644 index 00000000000..6fab88bcf80 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/normally_configs/config.xml @@ -0,0 +1,30 @@ + + + + + + PUT + /test_for_only_insert_queries + INSERT INTO test.test VALUES + + + + POST + /test_for_only_select_queries + SELECT value FROM system.settings WHERE name = 'max_threads' + SELECT value FROM system.settings WHERE name = 'max_alter_threads' + + + + /test_for_hybrid_insert_and_select_queries + INSERT INTO test.test VALUES + SELECT * FROM test.test ORDER BY id + + + + /test_for_throw_exception_when_after_select + SELECT * FROM test.test ORDER BY id + SELECT throwIf(number = 2, 'Throw Exception') FROM numbers(3) + + + diff --git a/dbms/tests/integration/test_custom_http/normally_configs/users.xml b/dbms/tests/integration/test_custom_http/normally_configs/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_custom_http/normally_configs/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py index 805c6163839..08f8d37586c 100644 --- a/dbms/tests/integration/test_custom_http/test.py +++ b/dbms/tests/integration/test_custom_http/test.py @@ -1,21 +1,38 @@ +import os + import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -test_instance = cluster.add_instance('node', main_configs=['configs/custom_http_config.xml']) +SCRIPT_PATH = os.path.dirname(os.path.realpath(__file__)) + + +def add_instance(name, config_dir): + print os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml') + print os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml') + return cluster.add_instance(name, config_dir=os.path.join(SCRIPT_PATH, config_dir), + main_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml')], + user_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml')]) + + +normally_instance = add_instance("normally_node", "normally_configs") @pytest.fixture(scope="module") -def start_cluster(): +def started_cluster(): try: cluster.start() - test_instance.query('CREATE DATABASE `test`') - test_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') + normally_instance.query('CREATE DATABASE `test`') + normally_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') yield cluster finally: cluster.shutdown() -def test_for_single_insert(started_cluster): - assert test_instance.http_query('/test_for_single_insert', data='(1)(2)(3)') == '\n' +def test_normally_match(started_cluster): + assert normally_instance.http_request('test_for_only_insert_queries', method='PUT', data='(1)(2)(3)') == '' + assert normally_instance.http_request(url='test_for_only_select_queries', + params='max_threads=1', method='POST', data='max_alter_threads=2') == '1\n2\n' + assert normally_instance.http_request('test_for_hybrid_insert_and_select_queries', method='POST', data='(4)') == '1\n2\n3\n4\n' + assert 'Throw Exception' in normally_instance.http_request('test_for_throw_exception_when_after_select') diff --git a/programs/server/config.xml b/programs/server/config.xml index 1c04dee6c3a..706149b66b1 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,21 +525,30 @@ --> + - - - + + + + + query + + - diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a9fd572a8b0..efe47da9f46 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -11,6 +11,7 @@ import subprocess import time import urllib import httplib +import requests import xml.dom.minidom import logging import docker @@ -687,7 +688,7 @@ class ClickHouseInstance: def http_code_and_message(): return str(open_result.getcode()) + " " + httplib.responses[open_result.getcode()] + ": " + open_result.read() - + if expect_fail_and_get_error: if open_result.getcode() == 200: raise Exception("ClickHouse HTTP server is expected to fail, but succeeded: " + open_result.read()) @@ -697,6 +698,11 @@ class ClickHouseInstance: raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) return open_result.read() + # Connects to the instance via HTTP interface, sends a query and returns the answer + def http_request(self, url, method='GET', params=None, data=None): + url = "http://" + self.ip_address + ":8123/"+url + return requests.request(method=method, url=url, params=params, data=data).content + # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): return self.http_query(sql=sql, data=data, params=params, user=user, password=password, expect_fail_and_get_error=True) diff --git a/tests/queries/0_stateless/00408_http_keep_alive.sh b/tests/queries/0_stateless/00408_http_keep_alive.sh index 79e39d83704..83ac8c55f24 100755 --- a/tests/queries/0_stateless/00408_http_keep_alive.sh +++ b/tests/queries/0_stateless/00408_http_keep_alive.sh @@ -8,6 +8,3 @@ 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/'; From 8105a9bbe0cfc5831150be9f3f94a61348576ba7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 14 Nov 2019 10:20:06 +0800 Subject: [PATCH 11/23] ISSUES-5436 fix review suggestions --- dbms/programs/server/HTTPHandlerFactory.cpp | 140 ++++++++ dbms/programs/server/HTTPHandlerFactory.h | 73 ++-- .../HTTPRequestHandler}/ExtractorClientInfo.h | 0 .../ExtractorContextChange.h | 16 +- .../HTTPExceptionHandler.cpp | 154 ++++++++ .../HTTPRequestHandler/HTTPExceptionHandler.h | 19 + .../HTTPPingRequestHandler.cpp | 45 +++ .../HTTPPingRequestHandler.h | 26 ++ .../HTTPQueryRequestHandler.cpp | 179 ++++++++++ .../HTTPQueryRequestHandler.h | 48 +++ ...TPQueryRequestHandlerMatcherAndCreator.cpp | 330 ++++++++++++++++++ ...HTTPQueryRequestHandlerMatcherAndCreator.h | 92 +++++ .../HTTPReplicasStatusRequestHandler.cpp | 118 +++++++ .../HTTPReplicasStatusRequestHandler.h | 25 ++ .../HTTPRootRequestHandler.cpp | 45 +++ .../HTTPRootRequestHandler.h | 26 ++ .../HTTPSessionContextHolder.cpp | 105 ++++++ .../HTTPSessionContextHolder.h | 24 ++ .../HTTPInputStreams.cpp | 19 +- dbms/src/DataStreams/HTTPInputStreams.h | 29 ++ .../HTTPOutputStreams.cpp | 2 +- .../HTTPOutputStreams.h | 0 .../CustomHTTP/AlwaysQueryMatcher.h | 16 - .../CustomHTTP/ConstQueryExecutor.cpp | 73 ---- .../CustomHTTP/ConstQueryExecutor.h | 30 -- .../CustomHTTP/CustomExecutor.cpp | 201 ----------- .../Interpreters/CustomHTTP/CustomExecutor.h | 77 ---- .../CustomHTTP/DynamicQueryExecutor.cpp | 60 ---- .../CustomHTTP/DynamicQueryExecutor.h | 37 -- .../CustomHTTP/HTTPInputStreams.h | 27 -- .../CustomHTTP/MethodQueryMatcher.h | 30 -- .../CustomHTTP/QueryExecutorAndMatcher.h | 47 --- .../CustomHTTP/URLQueryMatcher.cpp | 73 ---- .../Interpreters/CustomHTTP/URLQueryMatcher.h | 32 -- .../normally_configs/config.xml | 30 -- .../integration/test_custom_http/test.py | 38 -- .../__init__.py | 0 .../common_configs/common_config.xml | 0 .../common_configs/common_users.xml | 0 .../other_tests_configs/config.xml | 18 + .../other_tests_configs}/users.xml | 0 .../test_http_handlers_config/test.py | 87 +++++ .../test_insert_and_select_dynamic/config.xml | 23 ++ .../test_insert_and_select_dynamic/users.xml | 3 + .../config.xml | 26 ++ .../users.xml | 3 + .../config.xml | 31 ++ .../test_param_and_settings_dynamic/users.xml | 3 + .../config.xml | 43 +++ .../users.xml | 3 + programs/server/CMakeLists.txt | 10 +- programs/server/NotFoundHandler.cpp | 18 +- programs/server/NotFoundHandler.h | 9 +- programs/server/Server.cpp | 1 - programs/server/config.xml | 48 +-- src/IO/WriteBufferFromHTTPServerResponse.cpp | 17 +- src/Interpreters/Context.cpp | 31 +- src/Interpreters/Context.h | 5 - .../ReplaceQueryParameterVisitor.cpp | 2 - tests/integration/helpers/cluster.py | 6 +- .../test_https_replication/configs/config.xml | 9 + 61 files changed, 1760 insertions(+), 922 deletions(-) create mode 100644 dbms/programs/server/HTTPHandlerFactory.cpp rename dbms/{src/Interpreters/CustomHTTP => programs/server/HTTPRequestHandler}/ExtractorClientInfo.h (100%) rename dbms/{src/Interpreters/CustomHTTP => programs/server/HTTPRequestHandler}/ExtractorContextChange.h (87%) create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp create mode 100644 dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPInputStreams.cpp (71%) create mode 100644 dbms/src/DataStreams/HTTPInputStreams.h rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPOutputStreams.cpp (99%) rename dbms/src/{Interpreters/CustomHTTP => DataStreams}/HTTPOutputStreams.h (100%) delete mode 100644 dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/CustomExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h delete mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp delete mode 100644 dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h delete mode 100644 dbms/tests/integration/test_custom_http/normally_configs/config.xml delete mode 100644 dbms/tests/integration/test_custom_http/test.py rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/__init__.py (100%) rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/common_configs/common_config.xml (100%) rename dbms/tests/integration/{test_custom_http => test_http_handlers_config}/common_configs/common_users.xml (100%) create mode 100644 dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml rename dbms/tests/integration/{test_custom_http/normally_configs => test_http_handlers_config/other_tests_configs}/users.xml (100%) create mode 100644 dbms/tests/integration/test_http_handlers_config/test.py create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml create mode 100644 dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp new file mode 100644 index 00000000000..9a2d1e52f09 --- /dev/null +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -0,0 +1,140 @@ +#include "HTTPHandlerFactory.h" + +#include "NotFoundHandler.h" +#include "HTTPRequestHandler/HTTPRootRequestHandler.h" +#include "HTTPRequestHandler/HTTPPingRequestHandler.h" +#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int UNKNOW_HTTP_HANDLER_TYPE; + extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; +} + +InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ +} + +Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + { + if (uri == "/") + return new HTTPRootRequestHandler(server); + if (uri == "/ping") + return new HTTPPingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new HTTPReplicasStatusRequestHandler(server.context()); + } + + if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new InterserverIOHTTPHandler(server); + } + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new NotFoundHandler( + "Use / or /ping for health checks.\n" + "Or /replicas_status for more sophisticated health checks.\n" + "Send queries from your program with POST method or GET /?query=...\n\n" + " Use clickhouse-client:\n\n" + " For interactive data analysis:\n" + " clickhouse-client\n\n" + " For batch query processing:\n" + " clickhouse-client --query='SELECT 1' > result\n" + " clickhouse-client < query > result" + ); + } + + return nullptr; +} + +HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ + updateHTTPHandlersCreator(server.config()); + + if (handlers_creator.empty()) + throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); +} + +Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + for (const auto & [matcher, creator] : handlers_creator) + { + if (matcher(request)) + return creator(); + } + + return new NotFoundHandler(no_handler_description); +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer &, const String &); + +HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer &, const String &); + +void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; + configuration.keys(key, http_handlers_item_key); + + handlers_creator.reserve(http_handlers_item_key.size()); + for (const auto & http_handler_type_name : http_handlers_item_key) + { + if (http_handler_type_name.find('.') != String::npos) + throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + + const auto & handler_key = key + "." + http_handler_type_name; + + if (startsWith(http_handler_type_name, "root_handler")) + handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "ping_handler")) + handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "dynamic_query_handler")) + handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefine_query_handler")) + handlers_creator.push_back({createPredefineQueryHandlerMatcher(server, handler_key), createPredefineQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "replicas_status_handler")) + handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); + else if (http_handler_type_name == "no_handler_description") + no_handler_description = configuration.getString(key + ".no_handler_description"); + else + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOW_HTTP_HANDLER_TYPE); + } +} + +} diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h index 73e0a2b7339..5add7b619af 100644 --- a/dbms/programs/server/HTTPHandlerFactory.h +++ b/dbms/programs/server/HTTPHandlerFactory.h @@ -1,64 +1,51 @@ #pragma once -#include #include #include +#include +#include #include #include "IServer.h" -#include "HTTPHandler.h" #include "InterserverIOHTTPHandler.h" -#include "NotFoundHandler.h" -#include "PingRequestHandler.h" -#include "ReplicasStatusHandler.h" -#include "RootRequestHandler.h" namespace DB { -template -class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { +public: + InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + +private: + IServer & server; + Logger * log; + std::string name; +}; + +using HTTPHandlerCreator = std::function; +using HTTPHandlerMatcher = std::function; +using HTTPHandlerMatcherAndCreator = std::pair; +using HTTPHandlersMatcherAndCreator = std::vector; + +class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + HTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + + void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); + private: IServer & server; Logger * log; std::string name; -public: - HTTPRequestHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) - { - } - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " - << request.getMethod() - << ", Address: " - << request.clientAddress().toString() - << ", User-Agent: " - << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (uri == "/") - return new RootRequestHandler(server); - if (uri == "/ping") - return new PingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new ReplicasStatusHandler(server.context()); - } - - return new HandlerType(server); - } + String no_handler_description; + HTTPHandlersMatcherAndCreator handlers_creator; }; -using HTTPHandlerFactory = HTTPRequestHandlerFactory; -using InterserverIOHTTPHandlerFactory = HTTPRequestHandlerFactory; - } diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h b/dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h similarity index 100% rename from dbms/src/Interpreters/CustomHTTP/ExtractorClientInfo.h rename to dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h diff --git a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h similarity index 87% rename from dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h rename to dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h index 56411cb122a..0fdfd73b4f6 100644 --- a/dbms/src/Interpreters/CustomHTTP/ExtractorContextChange.h +++ b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -1,10 +1,10 @@ #pragma once -#include #include +#include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ namespace ErrorCodes class ExtractorContextChange { public: - ExtractorContextChange(Context & context_, const CustomExecutorPtr & executor_) : context(context_), executor(executor_) {} + ExtractorContextChange(Context & context_, bool settings_may_in_post_) : context(context_), settings_may_in_post(settings_may_in_post_) {} static const NameSet & getReservedParamNames() { @@ -59,20 +59,20 @@ public: becomeReadonlyIfNeed(request); changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - if (is_multipart_data || executor->canBeParseRequestBody()) + if (is_multipart_data || settings_may_in_post) { ExternalTablesHandler handler(context, params); params.load(request, request.stream(), handler); /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` - if (executor->canBeParseRequestBody()) + if (settings_may_in_post) changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); } } private: Context & context; - const CustomExecutorPtr & executor; + bool settings_may_in_post; /// 'readonly' setting values mean: /// readonly = 0 - any query is allowed, client can change any setting. @@ -82,7 +82,7 @@ private: /// In theory if initially readonly = 0, the client can change any setting and then set readonly /// to some other value. /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeed(HTTPRequest & request) + void becomeReadonlyIfNeed(Poco::Net::HTTPServerRequest & request) { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { @@ -109,8 +109,6 @@ private: { if (Settings::findIndex(name) != Settings::npos) settings_changes.push_back({name, value}); - else if (!executor->isQueryParam(name)) - throw Exception("Unknown HTTP param name: '" + name + "'", ErrorCodes::UNKNOWN_HTTP_PARAM); } } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp new file mode 100644 index 00000000000..8bfa351aab2 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp @@ -0,0 +1,154 @@ +#include "HTTPExceptionHandler.h" + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int READONLY; + extern const int UNKNOWN_COMPRESSION_METHOD; + + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_OPEN_FILE; + + extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int TOO_DEEP_AST; + extern const int TOO_BIG_AST; + extern const int UNEXPECTED_AST_STRUCTURE; + + extern const int SYNTAX_ERROR; + + extern const int INCORRECT_DATA; + extern const int TYPE_MISMATCH; + + extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_SETTING; + extern const int UNKNOWN_DIRECTION_OF_SORTING; + extern const int UNKNOWN_AGGREGATE_FUNCTION; + extern const int UNKNOWN_FORMAT; + extern const int UNKNOWN_DATABASE_ENGINE; + extern const int UNKNOWN_TYPE_OF_QUERY; + + extern const int QUERY_IS_TOO_LARGE; + + extern const int NOT_IMPLEMENTED; + extern const int SOCKET_TIMEOUT; + + extern const int UNKNOWN_USER; + extern const int WRONG_PASSWORD; + extern const int REQUIRED_PASSWORD; + + extern const int INVALID_SESSION_TIMEOUT; + extern const int HTTP_LENGTH_REQUIRED; +} + +static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) +{ + using namespace Poco::Net; + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + return HTTPResponse::HTTP_UNAUTHORIZED; + else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + + exception_code == ErrorCodes::SYNTAX_ERROR || + + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH) + return HTTPResponse::HTTP_BAD_REQUEST; + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) + return HTTPResponse::HTTP_NOT_FOUND; + else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) + return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; + else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) + return HTTPResponse::HTTP_NOT_IMPLEMENTED; + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + return HTTPResponse::HTTP_LENGTH_REQUIRED; + + return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; +} + + +void HTTPExceptionHandler::handle( + const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, + bool compression, Poco::Logger * log) +{ + try + { + /// 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::max()); + } + + if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + response.requireAuthentication("ClickHouse server HTTP API"); + } + else + { + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); + } + + if (!response_out && !response.sent()) + response.send() << message << std::endl; + else + { + HTTPOutputStreams output_streams(response_out, compression); + + writeString(message, *output_streams.out_maybe_compressed); + writeChar('\n', *output_streams.out_maybe_compressed); + + output_streams.finalize(); + } + } + catch (...) + { + tryLogCurrentException(log, "Cannot send exception to client"); + } +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h new file mode 100644 index 00000000000..d2a5383ed4a --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class HTTPExceptionHandler +{ +public: + static void handle(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, bool compression, + Poco::Logger * log); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp new file mode 100644 index 00000000000..292f98ba0eb --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp @@ -0,0 +1,45 @@ +#include "HTTPPingRequestHandler.h" + +#include + +#include + +#include +#include + +namespace DB +{ + +void HTTPPingRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + const char * data = "Ok.\n"; + response.sendBuffer(data, strlen(data)); + } + catch (...) + { + tryLogCurrentException("HTTPPingRequestHandler"); + } +} + +HTTPHandlerMatcher createPingHandlerMatcher(IServer & server, const String & key) +{ + const auto & path = server.config().getString(key, "/ping"); + + return [&, path = path](const Poco::Net::HTTPServerRequest & request) + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && + request.getURI() == path; + }; +} + +HTTPHandlerCreator createPingHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPPingRequestHandler(server); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h new file mode 100644 index 00000000000..7d524dd638d --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h @@ -0,0 +1,26 @@ +#pragma once + +#include "../IServer.h" +#include "../HTTPHandlerFactory.h" + +#include + + +namespace DB +{ + +/// Response with "Ok.\n". Used for availability checks. +class HTTPPingRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPPingRequestHandler(const IServer & server_) : server(server_) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp new file mode 100644 index 00000000000..1bb93908638 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -0,0 +1,179 @@ +#include "HTTPQueryRequestHandler.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "ExtractorClientInfo.h" +#include "ExtractorContextChange.h" +#include "HTTPQueryRequestHandlerMatcherAndCreator.h" +#include "HTTPSessionContextHolder.h" +#include "HTTPExceptionHandler.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int HTTP_LENGTH_REQUIRED; +} + +template +HTTPQueryRequestHandler::HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_) + : server(server_), log(&Logger::get("HTTPQueryRequestHandler")), extractor(extractor_) +{ + server_display_name = server.config().getString("display_name", getFQDNOrHostName()); +} + +template +void HTTPQueryRequestHandler::processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, + Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out) +{ + ExtractorClientInfo{context.getClientInfo()}.extract(request); + ExtractorContextChange{context, extractor.loadSettingsFromPost()}.extract(request, params); + + HTTPInputStreams input_streams{context, request, params}; + HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); + + const auto & queries = extractor.extract(context, request, params); + + for (const auto & [execute_query, not_touch_post] : queries) + { + ReadBufferPtr temp_query_buf; + ReadBufferPtr execute_query_buf = std::make_shared(execute_query); + + if (not_touch_post && !startsWith(request.getContentType().data(), "multipart/form-data")) + { + temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed + execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); + } + + executeQuery( + *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, + context, [&response] (const String & content_type) { response.setContentType(content_type); }, + [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } + ); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. + output_streams.finalize(); +} + +template +void HTTPQueryRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) +{ + setThreadName("HTTPHandler"); + ThreadStatus thread_status; + + /// In case of exception, send stack trace to client. + HTTPResponseBufferPtr response_out; + bool with_stacktrace = false, internal_compression = false; + + try + { + response_out = createResponseOut(request, response); + response.set("Content-Type", "text/plain; charset=UTF-8"); + response.set("X-ClickHouse-Server-Display-Name", server_display_name); + + /// For keep-alive to work. + if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) + response.setChunkedTransferEncoding(true); + + HTMLForm params(request); + with_stacktrace = params.getParsed("stacktrace", false); + internal_compression = params.getParsed("compress", false); + + /// Workaround. Poco does not detect 411 Length Required case. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) + throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); + + { + Context query_context = server.context(); + CurrentThread::QueryScope query_scope(query_context); + + HTTPSessionContextHolder holder{query_context, request, params}; + processQuery(holder.query_context, request, params, response, response_out); + LOG_INFO(log, "Done processing query"); + } + } + catch (...) + { + tryLogCurrentException(log); + + /** If exception is received from remote server, then stack trace is embedded in message. + * If exception is thrown on local server, then stack trace is in separate field. + */ + int exception_code = getCurrentExceptionCode(); + std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); + HTTPExceptionHandler::handle(exception_message, exception_code, request, response, response_out, internal_compression, log); + } +} + +template +HTTPResponseBufferPtr HTTPQueryRequestHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +{ + size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); + /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). + String http_response_compression_methods = request.get("Accept-Encoding", ""); + + if (!http_response_compression_methods.empty()) + { + /// 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("gzip")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + else if (std::string::npos != http_response_compression_methods.find("deflate")) + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); +#if USE_BROTLI + else if (http_response_compression_methods == "br") + return std::make_shared( + request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); +#endif + } + + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); +} + + +template class HTTPQueryRequestHandler; +template class HTTPQueryRequestHandler; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h new file mode 100644 index 00000000000..5303b0cda89 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h @@ -0,0 +1,48 @@ +#pragma once + +#include "../IServer.h" + +#include + +#include +#include + +#include + + +namespace CurrentMetrics +{ + extern const Metric HTTPConnection; +} + +namespace Poco { class Logger; } + +namespace DB +{ + +template +class HTTPQueryRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; + Poco::Logger * log; + QueryParamExtractor extractor; + + /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. + String server_display_name; + + CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; + + HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); + + void processQuery( + Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, + Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp new file mode 100644 index 00000000000..321a585d939 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp @@ -0,0 +1,330 @@ +#include "HTTPQueryRequestHandlerMatcherAndCreator.h" + +#include "../HTTPHandlerFactory.h" +#include "HTTPQueryRequestHandler.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EMPTY_PREDEFINE_QUERY; + extern const int CANNOT_COMPILE_REGEXP; + extern const int UNKNOWN_QUERY_PARAMETER; + extern const int DUPLICATE_CAPTURE_QUERY_PARAM; + extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; + extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY; +} + +ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) + : url_regex(url_regex_), headers_regex(headers_regex_) +{ + dynamic_param_name = configuration.getString(key + ".query_param_name", "query"); + + NameSet extracted_names; + + if (url_regex) + { + for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) + { + if (startsWith(capturing_name, "param_")) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_url[capturing_name] = capturing_index; + } + } + } + + if (!headers_regex.empty()) + { + for (const auto & [header_name, header_regex] : headers_regex) + { + for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) + { + if (startsWith(capturing_name, "param_")) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_headers[header_name][capturing_name] = capturing_index; + } + } + } + } +} + +template +void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) +{ + int num_captures = regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(value.data(), value.size()); + + if (regex->Match(input, 0, value.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + { + for (const auto & [capturing_name, capturing_index] : extract_params) + { + String param_name = capturing_name; + const auto & capturing_value = matches[capturing_index]; + + if constexpr (remove_prefix_for_param) + { + const static size_t prefix_size = strlen("param_"); + param_name = capturing_name.substr(prefix_size); + } + + context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + } + } +} + +ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + if (!extract_from_url.empty()) + extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); + + + if (!extract_from_headers.empty()) + for (const auto & [header_name, extract_params] : extract_from_headers) + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + + String extracted_query_from_params; + const static size_t prefix_size = strlen("param_"); + + for (const auto & [param_name, param_value] : params) + { + if (param_name == dynamic_param_name) + extracted_query_from_params += param_value; + else if (startsWith(param_name, "param_")) + context.setQueryParameter(param_name.substr(prefix_size), param_value); + } + + if (!extracted_query_from_params.empty()) + extracted_query_from_params += "\n"; + + return {{extracted_query_from_params, true}}; +} + +ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) + : url_regex(url_regex_), headers_regex(headers_regex_) +{ + Poco::Util::AbstractConfiguration::Keys queries_key; + configuration.keys(key + ".queries", queries_key); + + if (queries_key.empty()) + throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINE_QUERY); + + for (const auto & query_key : queries_key) + { + const auto & predefine_query = configuration.getString(key + ".queries." + query_key); + + const char * query_begin = predefine_query.data(); + const char * query_end = predefine_query.data() + predefine_query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); + QueryParameterVisitor{queries_names}.visit(extract_query_ast); + + bool is_insert_query = extract_query_ast->as(); + + if (has_insert_query && is_insert_query) + throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY); + + has_insert_query |= is_insert_query; + predefine_queries.push_back({predefine_query, is_insert_query}); + } + + const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); + for (const auto & predefine_query_name : queries_names) + { + if (Settings::findIndex(predefine_query_name) != Settings::npos || reserved_params_name.count(predefine_query_name)) + throw Exception("Illegal http_handler param name '" + predefine_query_name + + "', Because it's reserved name or Settings name", ErrorCodes::ILLEGAL_HTTP_HANDLER_PARAM_NAME); + } + + NameSet extracted_names; + + if (url_regex) + { + for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) + { + if (queries_names.count(capturing_name)) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_url[capturing_name] = capturing_index; + } + } + } + + if (!headers_regex.empty()) + { + for (const auto & [header_name, header_regex] : headers_regex) + { + for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) + { + if (queries_names.count(capturing_name)) + { + if (extracted_names.count(capturing_name)) + throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); + + extracted_names.emplace(capturing_name); + extract_from_headers[header_name][capturing_name] = capturing_index; + } + } + } + } +} + +ExtractRes ExtractorPredefineQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + if (!extract_from_url.empty()) + extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); + + if (!extract_from_headers.empty()) + for (const auto & [header_name, extract_params] : extract_from_headers) + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + + for (const auto & param : params) + if (queries_names.count(param.first)) + context.setQueryParameter(param.first, param.second); + + return predefine_queries; +} + +RegexRule HTTPQueryRequestHandlerMatcherAndCreator::createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + if (!configuration.has(key)) + return {}; + + const auto & regex_str = configuration.getString(key); + const auto & url_regex_rule = std::make_shared(regex_str); + + if (!url_regex_rule->ok()) + throw Exception("cannot compile re2: " + regex_str + " for HTTPHandler url, error: " + url_regex_rule->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + return url_regex_rule; +} + +HeadersRegexRule HTTPQueryRequestHandlerMatcherAndCreator::createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + if (!configuration.has(key)) + return {}; + + Poco::Util::AbstractConfiguration::Keys headers_names; + configuration.keys(key, headers_names); + + HeadersRegexRule headers_regex_rule; + for (const auto & header_name : headers_names) + { + if (headers_regex_rule.count(header_name)) + throw Exception("Duplicate header match declaration '" + header_name + "'", ErrorCodes::LOGICAL_ERROR); + + headers_regex_rule[header_name] = createRegexRule(configuration, key + "." + header_name); + } + + return headers_regex_rule; +} + +size_t findFirstMissingMatchPos(const re2_st::RE2 & regex_rule, const String & match_content) +{ + int num_captures = regex_rule.NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(match_content.data(), match_content.size()); + if (regex_rule.Match(input, 0, match_content.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + return matches[0].size(); + + return size_t(0); +} + +HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher( + const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) +{ + return [method = Poco::toLower(method), url_rule = url_rule, headers_rule = headers_rule](const Poco::Net::HTTPServerRequest & request) + { + if (!method.empty() && Poco::toLower(request.getMethod()) != method) + return false; + + if (url_rule) + { + Poco::URI uri(request.getURI()); + const auto & request_uri = uri.getPath(); + size_t first_missing_pos = findFirstMissingMatchPos(*url_rule, request_uri); + + const char * url_end = request_uri.data() + request_uri.size(); + const char * first_missing = request_uri.data() + first_missing_pos; + + if (first_missing != url_end && *first_missing == '/') + ++first_missing; + + if (first_missing != url_end && *first_missing != '?') + return false; /// Not full matched + } + + if (!headers_rule.empty()) + { + for (const auto & [header_name, header_rule] : headers_rule) + { + const String & header_value = request.get(header_name); + if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) + return false; + } + } + + return true; + }; +} + +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, + HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); +} + + +HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, + HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); +} + +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( + server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) + { + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + + return [&, query_extract = extract]() + { + return new HTTPQueryRequestHandler(server, *query_extract); + }; + }); +} + +HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer & server, const String & key) +{ + return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( + server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) + { + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + + return [&, query_extract = extract]() + { + return new HTTPQueryRequestHandler(server, *query_extract); + }; + }); +} +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h new file mode 100644 index 00000000000..456689d1ff5 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h @@ -0,0 +1,92 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "ExtractorContextChange.h" +#include "../HTTPHandlerFactory.h" + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif + +namespace DB +{ + +using RegexRule = std::shared_ptr; +using HeadersRegexRule = std::map; +using ExtractRes = std::vector>; + +class ExtractorDynamicQueryParameters +{ +public: + ExtractorDynamicQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, + const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ + ); + + bool loadSettingsFromPost() const { return false; } + + ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + +private: + const RegexRule url_regex; + const HeadersRegexRule headers_regex; + + String dynamic_param_name; + std::map extract_from_url; + std::map> extract_from_headers; +}; + +class ExtractorPredefineQueryParameters +{ +public: + ExtractorPredefineQueryParameters( + Poco::Util::AbstractConfiguration & configuration, const String & key, + const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ + ); + + bool loadSettingsFromPost() const { return !has_insert_query; } + + ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + +private: + const RegexRule url_regex; + const HeadersRegexRule headers_regex; + + NameSet queries_names; + bool has_insert_query{false}; + ExtractRes predefine_queries; + std::map extract_from_url; + std::map> extract_from_headers; +}; + +class HTTPQueryRequestHandlerMatcherAndCreator +{ +public: + template + static auto invokeWithParsedRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key, const NestedFunction & fun) + { + return fun(configuration.getString(key + ".method", ""), createRegexRule(configuration, key + ".url"), + createHeadersRegexRule(configuration, key + ".headers")); + } + + static HTTPHandlerMatcher createHandlerMatcher(const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule); + +private: + static RegexRule createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); + + static HeadersRegexRule createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp new file mode 100644 index 00000000000..ea70abbcc6f --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp @@ -0,0 +1,118 @@ +#include "HTTPReplicasStatusRequestHandler.h" + +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + + +HTTPReplicasStatusRequestHandler::HTTPReplicasStatusRequestHandler(Context & context_) + : context(context_) +{ +} + +void HTTPReplicasStatusRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) +{ + try + { + HTMLForm params(request); + + /// Even if lag is small, output detailed information about the lag. + bool verbose = params.get("verbose", "") == "1"; + + const MergeTreeSettings & settings = context.getMergeTreeSettings(); + + bool ok = true; + std::stringstream message; + + auto databases = context.getDatabases(); + + /// Iterate through all the replicated tables. + for (const auto & db : databases) + { + /// Lazy database can not contain replicated tables + if (db.second->getEngineName() == "Lazy") + continue; + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + auto & table = iterator->table(); + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + + if (!table_replicated) + continue; + + time_t absolute_delay = 0; + time_t relative_delay = 0; + + table_replicated->getReplicaDelays(absolute_delay, relative_delay); + + if ((settings.min_absolute_delay_to_close && absolute_delay >= static_cast(settings.min_absolute_delay_to_close)) + || (settings.min_relative_delay_to_close && relative_delay >= static_cast(settings.min_relative_delay_to_close))) + ok = false; + + message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name()) + << ":\tAbsolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".\n"; + } + } + + const auto & config = context.getConfigRef(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + if (ok && !verbose) + { + const char * data = "Ok.\n"; + response.sendBuffer(data, strlen(data)); + } + else + { + response.send() << message.rdbuf(); + } + } + catch (...) + { + tryLogCurrentException("HTTPReplicasStatusRequestHandler"); + + try + { + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); + + if (!response.sent()) + { + /// We have not sent anything yet and we don't even know if we need to compress response. + response.send() << getCurrentExceptionMessage(false) << std::endl; + } + } + catch (...) + { + LOG_ERROR((&Logger::get("HTTPReplicasStatusRequestHandler")), "Cannot send exception to client"); + } + } +} + +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer & server, const String & key) +{ + const auto & prefix = server.config().getString(key, "/replicas_status"); + + return [&, prefix = prefix](const Poco::Net::HTTPServerRequest & request) + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && + startsWith(request.getURI(), prefix); + }; +} + +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPReplicasStatusRequestHandler(server.context()); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h new file mode 100644 index 00000000000..277e59eb02c --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h @@ -0,0 +1,25 @@ +#pragma once + +#include "../HTTPHandlerFactory.h" +#include + + +namespace DB +{ + +class Context; + +/// Replies "Ok.\n" if all replicas on this server don't lag too much. Otherwise output lag information. +class HTTPReplicasStatusRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPReplicasStatusRequestHandler(Context & context_); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + Context & context; +}; + + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp new file mode 100644 index 00000000000..032b51d5b7a --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp @@ -0,0 +1,45 @@ +#include "HTTPRootRequestHandler.h" + +#include + +#include + +#include +#include + +namespace DB +{ + +void HTTPRootRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + const auto & config = server.config(); + setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); + + response.setContentType("text/html; charset=UTF-8"); + + const std::string data = config.getString("http_server_default_response", "Ok.\n"); + response.sendBuffer(data.data(), data.size()); + } + catch (...) + { + tryLogCurrentException("HTTPRootRequestHandler"); + } +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &) +{ + return [&](const Poco::Net::HTTPServerRequest & request) -> bool + { + return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + && request.getURI() == "/"; + }; +} + +HTTPHandlerCreator createRootHandlerCreator(IServer & server, const String &) +{ + return [&]() { return new HTTPRootRequestHandler(server); }; +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h new file mode 100644 index 00000000000..136f3292385 --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h @@ -0,0 +1,26 @@ +#pragma once + +#include "../IServer.h" +#include "../HTTPHandlerFactory.h" + +#include + + +namespace DB +{ + +/// Response with custom string. Can be used for browser. +class HTTPRootRequestHandler : public Poco::Net::HTTPRequestHandler +{ +public: + explicit HTTPRootRequestHandler(const IServer & server_) : server(server_) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const IServer & server; +}; + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp new file mode 100644 index 00000000000..840044634aa --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp @@ -0,0 +1,105 @@ +#include "HTTPSessionContextHolder.h" +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int REQUIRED_PASSWORD; + extern const int INVALID_SESSION_TIMEOUT; +} + +static std::chrono::steady_clock::duration parseSessionTimeout( + const Poco::Util::AbstractConfiguration & config, + const HTMLForm & params) +{ + unsigned session_timeout = config.getInt("default_session_timeout", 60); + + if (params.has("session_timeout")) + { + unsigned max_session_timeout = config.getUInt("max_session_timeout", 3600); + std::string session_timeout_str = params.get("session_timeout"); + + ReadBufferFromString buf(session_timeout_str); + if (!tryReadIntText(session_timeout, buf) || !buf.eof()) + throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); + + if (session_timeout > max_session_timeout) + throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", + ErrorCodes::INVALID_SESSION_TIMEOUT); + } + + return std::chrono::seconds(session_timeout); +} + +HTTPSessionContextHolder::~HTTPSessionContextHolder() +{ + if (session_context) + session_context->releaseSession(session_id, session_timeout); +} + +void HTTPSessionContextHolder::authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params) +{ + auto user = request.get("X-ClickHouse-User", ""); + auto password = request.get("X-ClickHouse-Key", ""); + auto quota_key = request.get("X-ClickHouse-Quota", ""); + + if (user.empty() && password.empty() && quota_key.empty()) + { + /// User name and password can be passed using query parameters + /// or using HTTP Basic auth (both methods are insecure). + if (request.hasCredentials()) + { + Poco::Net::HTTPBasicCredentials credentials(request); + + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else + { + user = params.get("user", "default"); + password = params.get("password", ""); + } + + quota_key = params.get("quota_key", ""); + } + else + { + /// It is prohibited to mix different authorization schemes. + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) + { + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); + } + } + + std::string query_id = params.get("query_id", ""); + query_context.setUser(user, password, request.clientAddress(), quota_key); + query_context.setCurrentQueryId(query_id); +} + +HTTPSessionContextHolder::HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params) + : query_context(query_context_) +{ + authentication(request, params); + + { + session_id = params.get("session_id", ""); + + if (!session_id.empty()) + { + session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); + session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); + + query_context = *session_context; + query_context.setSessionContext(*session_context); + } + } +} + +} diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h new file mode 100644 index 00000000000..9ef8b8f7abe --- /dev/null +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct HTTPSessionContextHolder +{ + ~HTTPSessionContextHolder(); + + void authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params); + + HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params); + + String session_id; + Context & query_context; + std::shared_ptr session_context = nullptr; + std::chrono::steady_clock::duration session_timeout; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp b/dbms/src/DataStreams/HTTPInputStreams.cpp similarity index 71% rename from dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp rename to dbms/src/DataStreams/HTTPInputStreams.cpp index 3321b57b38d..3e5d6c01ab3 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.cpp +++ b/dbms/src/DataStreams/HTTPInputStreams.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include @@ -31,12 +31,12 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques } } -ReadBufferPtr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const +std::unique_ptr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const +std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, std::unique_ptr & raw_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -44,21 +44,22 @@ ReadBufferPtr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & reque if (!http_compressed_method.empty()) { if (http_compressed_method == "gzip") - return std::make_shared(*raw_buffer, CompressionMethod::Gzip); + return std::make_unique(std::move(raw_buffer), CompressionMethod::Gzip); else if (http_compressed_method == "deflate") - return std::make_shared(*raw_buffer, CompressionMethod::Zlib); + return std::make_unique(std::move(raw_buffer), CompressionMethod::Zlib); #if USE_BROTLI else if (http_compressed_method == "br") - return std::make_shared(*raw_buffer); + return std::make_unique(std::move(raw_buffer)); #endif else throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } - return raw_buffer; + return std::move(raw_buffer); } -ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const +std::unique_ptr HTTPInputStreams::createInternalCompressedBuffer( + HTMLForm & params, std::unique_ptr & http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. @@ -66,7 +67,7 @@ ReadBufferPtr HTTPInputStreams::createInternalCompressedBuffer(HTMLForm & params if (params.getParsed("decompress", false)) return std::make_unique(*http_maybe_encoding_buffer); - return http_maybe_encoding_buffer; + return std::move(http_maybe_encoding_buffer); } } diff --git a/dbms/src/DataStreams/HTTPInputStreams.h b/dbms/src/DataStreams/HTTPInputStreams.h new file mode 100644 index 00000000000..652f5370a5a --- /dev/null +++ b/dbms/src/DataStreams/HTTPInputStreams.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using HTTPServerRequest = Poco::Net::HTTPServerRequest; + +struct HTTPInputStreams +{ + using ReadBufferUniquePtr = std::unique_ptr; + + ReadBufferUniquePtr in; + ReadBufferUniquePtr in_maybe_compressed; + ReadBufferUniquePtr in_maybe_internal_compressed; + + HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); + + ReadBufferUniquePtr createRawInBuffer(HTTPServerRequest & request) const; + ReadBufferUniquePtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & raw_buffer) const; + ReadBufferUniquePtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; +}; + +} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp b/dbms/src/DataStreams/HTTPOutputStreams.cpp similarity index 99% rename from dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp rename to dbms/src/DataStreams/HTTPOutputStreams.cpp index 8ac682af5e4..31f4929bef1 100644 --- a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.cpp +++ b/dbms/src/DataStreams/HTTPOutputStreams.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h b/dbms/src/DataStreams/HTTPOutputStreams.h similarity index 100% rename from dbms/src/Interpreters/CustomHTTP/HTTPOutputStreams.h rename to dbms/src/DataStreams/HTTPOutputStreams.h diff --git a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h deleted file mode 100644 index 56fa001d57c..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/AlwaysQueryMatcher.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class AlwaysQueryMatcher : public QueryMatcher -{ -public: - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & /*request*/, HTMLForm & /*params*/) const override { return true; } -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp deleted file mode 100644 index e6ed579b696..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_CUSTOM_EXECUTOR_PARAM; -} - -void prepareQueryParams(Context & context, HTMLForm & params, const NameSet & query_params_name) -{ - for (const auto & param : params) - if (query_params_name.count(param.first)) - context.setQueryParameter(param.first, param.second); -} - -QueryExecutorConst::QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) -{ - execute_query = configuration.getString(config_key); - - const char * query_begin = execute_query.data(); - const char * query_end = execute_query.data() + execute_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - - QueryParameterVisitor{query_params_name}.visit(extract_query_ast); - can_be_parse_request_body = !extract_query_ast->as(); - - const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); - for (const auto & prepared_param_name : query_params_name) - { - if (Settings::findIndex(prepared_param_name) != Settings::npos || reserved_params_name.count(prepared_param_name)) - throw Exception( - "Illegal custom executor query param name '" + prepared_param_name + "', Because it's a reserved name or Settings name", - ErrorCodes::ILLEGAL_CUSTOM_EXECUTOR_PARAM); - } -} - -void QueryExecutorConst::executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const -{ - ReadBufferPtr temp_query_buf; - prepareQueryParams(context, params, query_params_name); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (!canBeParseRequestBody() && !startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, - [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h deleted file mode 100644 index 170d2a959ef..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/ConstQueryExecutor.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class QueryExecutorConst : public QueryExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - - bool canBeParseRequestBody() const override { return can_be_parse_request_body; } - - bool isQueryParam(const String & param_name) const override { return query_params_name.count(param_name); } - - QueryExecutorConst(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); - - void executeQueryImpl( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; - -private: - String execute_query; - NameSet query_params_name; - bool can_be_parse_request_body{false}; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp deleted file mode 100644 index 3e984deb8aa..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.cpp +++ /dev/null @@ -1,201 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; - extern const int UNKNOW_QUERY_EXECUTOR; - extern const int TOO_MANY_INPUT_CUSTOM_EXECUTOR; -} - -CustomExecutor::CustomExecutor( - const std::vector & matchers_, - const std::vector & query_executors_) - : matchers(matchers_), query_executors(query_executors_) -{ -} - -bool CustomExecutor::match(Context & context, HTTPRequest & request, HTMLForm & params) const -{ - for (const auto & matcher : matchers) - { - if (!matcher->match(context, request, params)) - return false; - } - - return true; -} - -bool CustomExecutor::isQueryParam(const String & param_name) const -{ - for (const auto & query_executor : query_executors) - { - if (!query_executor->isQueryParam(param_name)) - return false; - } - - return true; -} - -bool CustomExecutor::canBeParseRequestBody() const -{ - for (const auto & query_executor : query_executors) - { - if (!query_executor->canBeParseRequestBody()) - return false; - } - - return true; -} - -void CustomExecutor::executeQuery( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) -{ - for (const auto & query_executor : query_executors) - query_executor->executeQueryImpl(context, request, response, params, input_streams, output_streams); - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - output_streams.finalize(); -} - -void CustomExecutors::updateCustomExecutors(const Configuration & config, const Settings & /*settings*/, const String & config_prefix) -{ - Configuration::Keys custom_executors_keys; - config.keys(config_prefix, custom_executors_keys); - - std::vector> new_custom_executors; - - for (const auto & custom_executor_key : custom_executors_keys) - { - if (custom_executor_key.find('.') != String::npos) - throw Exception("CustomExecutor names with dots are not supported: '" + custom_executor_key + "'", ErrorCodes::SYNTAX_ERROR); - - new_custom_executors.push_back({custom_executor_key, createCustomExecutor(config, config_prefix, custom_executor_key)}); - } - - std::unique_lock lock(rwlock); - custom_executors = new_custom_executors; -} - -void CustomExecutors::registerCustomMatcher(const String & matcher_name, const CustomExecutors::CustomMatcherCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(matcher_name); - const auto & query_executor_creator_it = query_executor_creators.find(matcher_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR QueryMatcher name must be unique between the QueryExecutor and QueryMatcher.", - ErrorCodes::LOGICAL_ERROR); - - custom_matcher_creators[matcher_name] = creator; -} - -void CustomExecutors::registerQueryExecutor(const String & query_executor_name, const CustomExecutors::QueryExecutorCreator & creator) -{ - const auto & matcher_creator_it = custom_matcher_creators.find(query_executor_name); - const auto & query_executor_creator_it = query_executor_creators.find(query_executor_name); - - if (matcher_creator_it != custom_matcher_creators.end() && query_executor_creator_it != query_executor_creators.end()) - throw Exception("LOGICAL_ERROR QueryExecutor name must be unique between the QueryExecutor and QueryMatcher.", - ErrorCodes::LOGICAL_ERROR); - - query_executor_creators[query_executor_name] = creator; -} - -String fixMatcherOrExecutorTypeName(const String & matcher_or_executor_type_name) -{ - auto type_name_end_pos = matcher_or_executor_type_name.find('['); - return type_name_end_pos == String::npos ? matcher_or_executor_type_name : matcher_or_executor_type_name.substr(0, type_name_end_pos); -} - -CustomExecutorPtr CustomExecutors::createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name) -{ - Configuration::Keys matchers_key; - config.keys(config_prefix + "." + name, matchers_key); - - std::vector query_matchers; - std::vector query_executors; - - for (const auto & matcher_key : matchers_key) - { - String matcher_or_query_executor_type = fixMatcherOrExecutorTypeName(matcher_key); - - if (matcher_or_query_executor_type.find('.') != String::npos) - throw Exception("CustomMatcher or QueryExecutor names with dots are not supported: '" + matcher_or_query_executor_type + "'", - ErrorCodes::SYNTAX_ERROR); - - const auto & matcher_creator_it = custom_matcher_creators.find(matcher_or_query_executor_type); - const auto & query_executor_creator_it = query_executor_creators.find(matcher_or_query_executor_type); - - if (matcher_creator_it == custom_matcher_creators.end() && query_executor_creator_it == query_executor_creators.end()) - throw Exception("CustomMatcher or QueryExecutor '" + matcher_or_query_executor_type + "' is not implemented.", - ErrorCodes::NOT_IMPLEMENTED); - - if (matcher_creator_it != custom_matcher_creators.end()) - query_matchers.push_back(matcher_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); - - if (query_executor_creator_it != query_executor_creators.end()) - query_executors.push_back(query_executor_creator_it->second(config, config_prefix + "." + name + "." + matcher_key)); - } - - checkQueryMatchersAndExecutors(name, query_matchers, query_executors); - return std::make_shared(query_matchers, query_executors); -} - -void CustomExecutors::checkQueryMatchersAndExecutors( - const String & name, std::vector & matchers, std::vector & query_executors) -{ - if (matchers.empty() || query_executors.empty()) - throw Exception("The CustomExecutor '" + name + "' must contain a Matcher and a QueryExecutor.", ErrorCodes::SYNTAX_ERROR); - - const auto & sum_func = [&](auto & ele) -> bool { return !ele->canBeParseRequestBody(); }; - const auto & need_post_data_count = std::count_if(query_executors.begin(), query_executors.end(), sum_func); - - if (need_post_data_count > 1) - throw Exception("The CustomExecutor '" + name + "' can only contain one insert query." + toString(need_post_data_count), ErrorCodes::TOO_MANY_INPUT_CUSTOM_EXECUTOR); - - for (const auto & matcher : matchers) - matcher->checkQueryExecutors(query_executors); -} - -std::pair CustomExecutors::getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const -{ - std::shared_lock lock(rwlock); - - for (const auto & custom_executor : custom_executors) - if (custom_executor.second->match(context, request, params)) - return custom_executor; - - throw Exception("No CustomExecutor match " + request.getURI(), ErrorCodes::UNKNOW_QUERY_EXECUTOR); -} - -CustomExecutors::CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix) -{ - registerCustomMatcher("URL", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerCustomMatcher("method", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerCustomMatcher("always_matched", [&](const auto & /*matcher_config*/, const auto & /*prefix*/) - { return std::make_shared(); }); - - registerQueryExecutor("query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - registerQueryExecutor("dynamic_query", [&](const auto & matcher_config, const auto & prefix) - { return std::make_shared(matcher_config, prefix); }); - - updateCustomExecutors(config, settings, config_prefix); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h b/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h deleted file mode 100644 index 7b37a6afdf5..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/CustomExecutor.h +++ /dev/null @@ -1,77 +0,0 @@ -#pragma once - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -class CustomExecutor; - -using HTTPRequest = Poco::Net::HTTPServerRequest; -using HTTPResponse = Poco::Net::HTTPServerResponse; -using CustomExecutorPtr = std::shared_ptr; - -class CustomExecutors -{ -public: - using Configuration = Poco::Util::AbstractConfiguration; - CustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix = "custom_http"); - - CustomExecutors(const CustomExecutors &) = delete; - CustomExecutors & operator=(const CustomExecutors &) = delete; - - using QueryExecutorCreator = std::function; - void registerQueryExecutor(const String & query_executor_name, const QueryExecutorCreator & creator); - - using CustomMatcherCreator = std::function; - void registerCustomMatcher(const String & matcher_name, const CustomMatcherCreator & creator); - - void updateCustomExecutors(const Configuration & config, const Settings & settings, const String & config_prefix); - - std::pair getCustomExecutor(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const; -private: - mutable std::shared_mutex rwlock; - std::vector> custom_executors; - std::unordered_map query_executor_creators; - std::unordered_map custom_matcher_creators; - - CustomExecutorPtr createCustomExecutor(const Configuration & config, const String & config_prefix, const String & name); - - void checkQueryMatchersAndExecutors(const String & name, std::vector & matchers, std::vector & query_executors); - -}; - -class CustomExecutor -{ -public: - bool canBeParseRequestBody() const; - - bool isQueryParam(const String & param_name) const; - - bool match(Context & context, HTTPRequest & request, HTMLForm & params) const; - - void executeQuery( - Context & context, HTTPRequest & request, HTTPResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams - ); - - CustomExecutor(const std::vector & matchers_, const std::vector & query_executors_); - -private: - std::vector matchers; - std::vector query_executors; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp deleted file mode 100644 index 9b4a4006556..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include - -#include -#include -#include - -namespace DB -{ - -bool QueryExecutorDynamic::isQueryParam(const String & param_name) const -{ - return param_name == dynamic_param_name || startsWith(param_name, "param_"); -} - -QueryExecutorDynamic::QueryExecutorDynamic(const Configuration & configuration, const String & config_key) -{ - dynamic_param_name = configuration.getString(config_key + "." + "param_name", "query"); -} - -void QueryExecutorDynamic::executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const -{ - ReadBufferPtr temp_query_buf; - const auto & execute_query = prepareQuery(context, params); - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (!startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, - context, [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); -} - -String QueryExecutorDynamic::prepareQuery(Context & context, HTMLForm & params) const -{ - const static size_t prefix_size = strlen("param_"); - - WriteBufferFromOwnString query_buffer; - for (const auto & param : params) - { - if (param.first == dynamic_param_name) - writeString(param.second, query_buffer); - else if (startsWith(param.first, "param_")) - context.setQueryParameter(param.first.substr(prefix_size), param.second); - } - - if (query_buffer.offset()) - writeString("\n", query_buffer); - - return query_buffer.str(); -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h b/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h deleted file mode 100644 index a09146246d4..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/DynamicQueryExecutor.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class QueryExecutorDynamic : public QueryExecutor -{ -public: - using HTTPServerRequest = Poco::Net::HTTPServerRequest; - using HTTPServerResponse = Poco::Net::HTTPServerResponse; - using Configuration = Poco::Util::AbstractConfiguration; - - bool canBeParseRequestBody() const override { return false; } - - bool isQueryParam(const String & param_name) const override; - - QueryExecutorDynamic(const Configuration & configuration, const String & config_key); - - void executeQueryImpl( - Context & context, HTTPServerRequest & request, HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const override; - -private: - String dynamic_param_name{"query"}; - - String prepareQuery(Context & context, HTMLForm & params) const; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h b/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h deleted file mode 100644 index 389d4312362..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/HTTPInputStreams.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; - -struct HTTPInputStreams -{ - std::shared_ptr in; - std::shared_ptr in_maybe_compressed; - std::shared_ptr in_maybe_internal_compressed; - - HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - - ReadBufferPtr createRawInBuffer(HTTPServerRequest & request) const; - ReadBufferPtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferPtr & raw_buffer) const; - ReadBufferPtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferPtr & http_maybe_encoding_buffer) const; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h deleted file mode 100644 index 689aff67dc7..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/MethodQueryMatcher.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include "QueryExecutorAndMatcher.h" - -namespace DB -{ - -class MethodQueryMatcher : public QueryMatcher -{ -public: - using Config = Poco::Util::AbstractConfiguration; - - MethodQueryMatcher(const Config & configuration, const String & config_key) - : method(Poco::toLower(configuration.getString(config_key))) - { - } - - bool match(Context & /*context*/, Poco::Net::HTTPServerRequest & request, HTMLForm & /*params*/) const override - { - return Poco::toLower(request.getMethod()) == method; - } - - bool checkQueryExecutors(const std::vector & /*check_executors*/) const override { return true; } - -private: - String method; -}; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h b/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h deleted file mode 100644 index 7f32c4c2ddb..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/QueryExecutorAndMatcher.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_COMPILE_REGEXP; -} - -class QueryExecutor -{ -public: - virtual ~QueryExecutor() = default; - - virtual bool isQueryParam(const String &) const = 0; - virtual bool canBeParseRequestBody() const = 0; - - virtual void executeQueryImpl( - Context & context, Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, - HTMLForm & params, const HTTPInputStreams & input_streams, const HTTPOutputStreams & output_streams) const = 0; -}; - -using QueryExecutorPtr = std::shared_ptr; - - -class QueryMatcher -{ -public: - virtual ~QueryMatcher() = default; - - virtual bool checkQueryExecutors(const std::vector &check_executors) const = 0; - - virtual bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) const = 0; -}; - -using QueryMatcherPtr = std::shared_ptr; - -} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp deleted file mode 100644 index 82bb864c926..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include "URLQueryMatcher.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_COMPILE_REGEXP; - extern const int UNDEFINED_CUSTOM_EXECUTOR_PARAM; -} - -URLQueryMatcher::URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key) -{ - const auto & regex_str = configuration.getString(config_key); - regex_matcher = std::make_unique(regex_str); - - if (!regex_matcher->ok()) - throw Exception("cannot compile re2: " + regex_str + ", error: " + regex_matcher->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); -} - -bool checkQueryOneQueryParam(const String & param_name, const std::vector & custom_query_executors) -{ - for (const auto & custom_query_executor : custom_query_executors) - if (custom_query_executor->isQueryParam(param_name)) - return true; - - return false; -} - -bool URLQueryMatcher::checkQueryExecutors(const std::vector & custom_query_executors) const -{ - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - if (!checkQueryOneQueryParam(named_capturing_group.first, custom_query_executors)) - throw Exception("The param name '" + named_capturing_group.first + "' is not defined in the QueryExecutor.", - ErrorCodes::UNDEFINED_CUSTOM_EXECUTOR_PARAM); - - return true; -} - -bool URLQueryMatcher::match(Context & context, Poco::Net::HTTPServerRequest &request, HTMLForm &) const -{ - const String request_uri = request.getURI(); - int num_captures = regex_matcher->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(request_uri.data(), request_uri.size()); - if (regex_matcher->Match(input, 0, request_uri.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - const auto & full_match = matches[0]; - const char * url_end = request_uri.data() + request_uri.size(); - const char * not_matched_begin = request_uri.data() + full_match.size(); - - if (not_matched_begin != url_end && *not_matched_begin == '/') - ++not_matched_begin; - - if (not_matched_begin == url_end || *not_matched_begin == '?') - { - for (const auto & named_capturing_group : regex_matcher->NamedCapturingGroups()) - { - const auto & capturing_value = matches[named_capturing_group.second]; - context.setQueryParameter(named_capturing_group.first, String(capturing_value.data(), capturing_value.size())); - } - - return true; - } - } - return false; -} - -} diff --git a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h b/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h deleted file mode 100644 index 61323db3e53..00000000000 --- a/dbms/src/Interpreters/CustomHTTP/URLQueryMatcher.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - -#if USE_RE2_ST -# include -#else -# define re2_st re2 -#endif - - -namespace DB -{ - -class URLQueryMatcher : public QueryMatcher -{ -public: - URLQueryMatcher(const Poco::Util::AbstractConfiguration & configuration, const String & config_key); - - bool match(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm &) const override; - - bool checkQueryExecutors(const std::vector & custom_query_executors) const override; - -private: - std::unique_ptr regex_matcher; -}; - -} diff --git a/dbms/tests/integration/test_custom_http/normally_configs/config.xml b/dbms/tests/integration/test_custom_http/normally_configs/config.xml deleted file mode 100644 index 6fab88bcf80..00000000000 --- a/dbms/tests/integration/test_custom_http/normally_configs/config.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - - - PUT - /test_for_only_insert_queries - INSERT INTO test.test VALUES - - - - POST - /test_for_only_select_queries - SELECT value FROM system.settings WHERE name = 'max_threads' - SELECT value FROM system.settings WHERE name = 'max_alter_threads' - - - - /test_for_hybrid_insert_and_select_queries - INSERT INTO test.test VALUES - SELECT * FROM test.test ORDER BY id - - - - /test_for_throw_exception_when_after_select - SELECT * FROM test.test ORDER BY id - SELECT throwIf(number = 2, 'Throw Exception') FROM numbers(3) - - - diff --git a/dbms/tests/integration/test_custom_http/test.py b/dbms/tests/integration/test_custom_http/test.py deleted file mode 100644 index 08f8d37586c..00000000000 --- a/dbms/tests/integration/test_custom_http/test.py +++ /dev/null @@ -1,38 +0,0 @@ -import os - -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -SCRIPT_PATH = os.path.dirname(os.path.realpath(__file__)) - - -def add_instance(name, config_dir): - print os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml') - print os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml') - return cluster.add_instance(name, config_dir=os.path.join(SCRIPT_PATH, config_dir), - main_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_config.xml')], - user_configs=[os.path.join(SCRIPT_PATH, 'common_configs', 'common_users.xml')]) - - -normally_instance = add_instance("normally_node", "normally_configs") - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - normally_instance.query('CREATE DATABASE `test`') - normally_instance.query('CREATE TABLE `test`.`test` (`id` UInt8) Engine = Memory') - yield cluster - finally: - cluster.shutdown() - - -def test_normally_match(started_cluster): - assert normally_instance.http_request('test_for_only_insert_queries', method='PUT', data='(1)(2)(3)') == '' - assert normally_instance.http_request(url='test_for_only_select_queries', - params='max_threads=1', method='POST', data='max_alter_threads=2') == '1\n2\n' - assert normally_instance.http_request('test_for_hybrid_insert_and_select_queries', method='POST', data='(4)') == '1\n2\n3\n4\n' - assert 'Throw Exception' in normally_instance.http_request('test_for_throw_exception_when_after_select') diff --git a/dbms/tests/integration/test_custom_http/__init__.py b/dbms/tests/integration/test_http_handlers_config/__init__.py similarity index 100% rename from dbms/tests/integration/test_custom_http/__init__.py rename to dbms/tests/integration/test_http_handlers_config/__init__.py diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_config.xml b/dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/common_configs/common_config.xml rename to dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml diff --git a/dbms/tests/integration/test_custom_http/common_configs/common_users.xml b/dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/common_configs/common_users.xml rename to dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml new file mode 100644 index 00000000000..f117eaba2f7 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml @@ -0,0 +1,18 @@ + + + + + + /ping_test + + POST + /test_one_handler_with_insert_and_select + + INSERT INTO test.test VALUES(1) + SELECT COUNT() FROM test.test WHERE id = {id:UInt8} + + + + test not found + + diff --git a/dbms/tests/integration/test_custom_http/normally_configs/users.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml similarity index 100% rename from dbms/tests/integration/test_custom_http/normally_configs/users.xml rename to dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test.py b/dbms/tests/integration/test_http_handlers_config/test.py new file mode 100644 index 00000000000..9e332dd8675 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test.py @@ -0,0 +1,87 @@ +import os +import urllib +import contextlib + +from helpers.cluster import ClickHouseCluster + + +class SimpleCluster: + def close(self): + self.cluster.shutdown() + + def __init__(self, cluster, name, config_dir): + self.cluster = cluster + self.instance = self.add_instance(name, config_dir) + cluster.start() + + def add_instance(self, name, config_dir): + script_path = os.path.dirname(os.path.realpath(__file__)) + return self.cluster.add_instance(name, config_dir=os.path.join(script_path, config_dir), + main_configs=[os.path.join(script_path, 'common_configs', 'common_config.xml')], + user_configs=[os.path.join(script_path, 'common_configs', 'common_users.xml')]) + + +def test_dynamic_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_insert_and_select", "test_insert_and_select_dynamic")) as cluster: + insert_data_query = urllib.quote_plus('INSERT INTO test.test VALUES') + select_data_query = urllib.quote_plus('SELECT * FROM test.test ORDER BY id') + create_database_query = urllib.quote_plus('CREATE DATABASE test') + create_test_table_query = 'CREATE TABLE test.test (id UInt8) Engine = Memory' + assert cluster.instance.http_request('create_test_table?max_threads=1&test_create_query_param=' + create_database_query, method='PUT') == '' + assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT', data=create_test_table_query) == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query + '(1)', method='POST') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query, method='POST', data='(2)') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='INSERT INTO test.test VALUES(3)(4)') == '' + assert cluster.instance.http_request('query_data_from_test?max_threads=1&test_select_query_param=' + select_data_query, method='GET') == '1\n2\n3\n4\n' + + +def test_predefine_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_insert_and_select", "test_insert_and_select_predefine")) as cluster: + assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT') == '' + assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='(1)(2)(3)(4)') == '' + assert cluster.instance.http_request('query_data_from_test?max_threads=1', method='GET') == '1\n2\n3\n4\n' + + +def test_dynamic_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_params_and_settings", "test_param_and_settings_dynamic")) as cluster: + settings = 'max_threads=1&max_alter_threads=2' + query_param = 'param_name_1=max_threads¶m_name_2=max_alter_threads' + test_query = 'SELECT value FROM system.settings where name = {name_1:String} OR name = {name_2:String}' + quoted_test_query = urllib.quote_plus(test_query) + assert cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + quoted_test_query + '&' + query_param + '&' + settings, method='POST') == '1\n2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST', data=test_query) == '1\n2\n' + assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + settings, method='POST', data=query_param) + assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + query_param, method='POST', data=settings) + + assert cluster.instance.http_request('get_query_params_and_settings?get_query_param=' + quoted_test_query + '&' + query_param + '&' + settings) == '1\n2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?query_param=' + quoted_test_query + '&' + settings) == '1\n2\n' + assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?query_param=' + quoted_test_query + '&' + query_param + '&' + settings) + + assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\n2\n' + + +def test_predefine_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_params_and_settings", "test_param_and_settings_predefine")) as cluster: + settings = 'max_threads=1&max_alter_threads=2' + query_param = 'name_1=max_threads&name_2=max_alter_threads' + assert cluster.instance.http_request('get_query_params_and_settings?' + query_param + '&' + settings, method='GET') == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?' + settings) == '1\nmax_alter_threads\t2\n' + assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?' + query_param + '&' + settings) + + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param, method='POST', data=settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + settings, method='POST', data=query_param) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST') == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('post_query_params_and_settings', method='POST', data=query_param + '&' + settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\nmax_alter_threads\t2\n' + + +def test_other_configs(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "test_other_configs", "other_tests_configs")) as cluster: + assert cluster.instance.http_request('', method='GET') == 'Ok.\n' + assert cluster.instance.http_request('ping_test', method='GET') == 'Ok.\n' + assert cluster.instance.http_request('404/NOT_FOUND', method='GET') == 'There is no handle /404/NOT_FOUND\n\ntest not found\n' + + cluster.instance.query('CREATE DATABASE test') + cluster.instance.query('CREATE TABLE test.test (id UInt8) Engine = Memory') + assert cluster.instance.http_request('test_one_handler_with_insert_and_select?id=1', method='POST', data='(1)(2)') == '2\n' + assert 'Cannot parse input' in cluster.instance.http_request('test_one_handler_with_insert_and_select', method='POST', data='id=1') diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml new file mode 100644 index 00000000000..a4ec94fe62e --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml @@ -0,0 +1,23 @@ + + + + + + PUT + /create_test_table + test_create_query_param + + + + POST + /insert_data_to_test + test_insert_query_param + + + + GET + /query_data_from_test + test_select_query_param + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml new file mode 100644 index 00000000000..0f0ab81d139 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml @@ -0,0 +1,26 @@ + + + + + + PUT + /create_test_table + + CREATE DATABASE test + CREATE TABLE test.test (id UInt8) Engine = Memory + + + + + POST + /insert_data_to_test + INSERT INTO test.test VALUES + + + + GET + /query_data_from_test + SELECT * FROM test.test ORDER BY id + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml new file mode 100644 index 00000000000..06c7143f3e8 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml @@ -0,0 +1,31 @@ + + + + + + POST + /post_query_params_and_settings + post_query_param + + + + GET + /get_query_params_and_settings + get_query_param + + + + GET + .+)/(?P.+)]]> + query_param + + + + + TEST_HEADER_VALUE + .+)/(?P.+)]]> + + query_param + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml new file mode 100644 index 00000000000..cca4bc04a69 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml @@ -0,0 +1,43 @@ + + + + + + POST + /post_query_params_and_settings + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + GET + /get_query_params_and_settings + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + GET + .+)/(?P.+)]]> + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + + + TEST_HEADER_VALUE + .+)/(?P.+)]]> + + + SELECT value FROM system.settings WHERE name = {name_1:String} + SELECT name, value FROM system.settings WHERE name = {name_2:String} + + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index f6e9d1ff702..064cda4847a 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -11,7 +11,15 @@ set(CLICKHOUSE_SERVER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/RootRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/Server.cpp ${CMAKE_CURRENT_SOURCE_DIR}/TCPHandler.cpp - ) + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandlerFactory.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPPingRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPRootRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPSessionContextHolder.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPExceptionHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +) set(CLICKHOUSE_SERVER_SOURCES ${CLICKHOUSE_SERVER_SOURCES} diff --git a/programs/server/NotFoundHandler.cpp b/programs/server/NotFoundHandler.cpp index 766e8895784..26df1a1b7d4 100644 --- a/programs/server/NotFoundHandler.cpp +++ b/programs/server/NotFoundHandler.cpp @@ -17,16 +17,14 @@ void NotFoundHandler::handleRequest( try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - response.send() << "There is no handle " << request.getURI() << "\n\n" - << "Use / or /ping for health checks.\n" - << "Or /replicas_status for more sophisticated health checks.\n\n" - << "Send queries from your program with POST method or GET /?query=...\n\n" - << "Use clickhouse-client:\n\n" - << "For interactive data analysis:\n" - << " clickhouse-client\n\n" - << "For batch query processing:\n" - << " clickhouse-client --query='SELECT 1' > result\n" - << " clickhouse-client < query > result\n"; + + std::stringstream output_description; + output_description << "There is no handle " << request.getURI() << "\n\n"; + + if (!no_handler_description.empty()) + output_description << no_handler_description << "\n"; + + response.send() << output_description.str(); } catch (...) { diff --git a/programs/server/NotFoundHandler.h b/programs/server/NotFoundHandler.h index 7f758e49d0d..caf527726c4 100644 --- a/programs/server/NotFoundHandler.h +++ b/programs/server/NotFoundHandler.h @@ -10,9 +10,12 @@ namespace DB class NotFoundHandler : public Poco::Net::HTTPRequestHandler { public: - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; + NotFoundHandler(const std::string & no_handler_description_) : no_handler_description(no_handler_description_) {} + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + const std::string no_handler_description; }; } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 710506151c4..3490ff6a445 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -466,7 +466,6 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); - global_context->setCustomExecutorConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/programs/server/config.xml b/programs/server/config.xml index 706149b66b1..c4ba4e66ab3 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,30 +525,30 @@ --> - - - - - - - query - - - - + + + /ping + /replicas_status + query + + Use / or /ping for health checks. Or /replicas_status for more sophisticated health checks. Send queries from your program with POST method or GET /?query=... Use clickhouse-client: For interactive data analysis: clickhouse-client For batch query processing: clickhouse-client --query='SELECT 1' > result clickhouse-client < query > result + + + diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index 25c2974056c..c61572d7097 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -124,7 +124,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() #if defined(POCO_CLICKHOUSE_PATCH) *response_header_ostr << "Content-Encoding: deflate\r\n"; #else - response.set("Content-Encoding", "gzip"); + response.set("Content-Encoding", "deflate"); response_body_ostr = &(response.send()); #endif out_raw = std::make_unique(*response_body_ostr); @@ -135,18 +135,15 @@ void WriteBufferFromHTTPServerResponse::nextImpl() else if (compression_method == CompressionMethod::Brotli) { #if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: deflate\r\n"; + *response_header_ostr << "Content-Encoding: br\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif - /// Newline autosent by response.send() - /// This may result in an extra empty line in the response body - response_body_ostr = &(response.send()); -#endif - - out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); - out = &*out_raw; + else + throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", + ErrorCodes::LOGICAL_ERROR); + /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. } else { @@ -154,7 +151,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl() response_body_ostr = &(response.send()); #endif - out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); + out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out = &*out_raw; } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b479a7bc46d..647c3fb8020 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -45,7 +44,6 @@ #include #include #include -#include #include #include #include @@ -350,9 +348,6 @@ struct ContextShared std::unique_ptr clusters; ConfigurationPtr clusters_config; /// Stores updated configs mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config - std::unique_ptr custom_executors; - ConfigurationPtr custom_executors_config; - mutable std::mutex custom_executors_mutex; /// Guards custom executors and their config #if USE_EMBEDDED_COMPILER std::shared_ptr compiled_expression_cache; @@ -1548,6 +1543,7 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptrclusters->setCluster(cluster_name, cluster); } + void Context::initializeSystemLogs() { auto lock = getLock(); @@ -2047,31 +2043,6 @@ void Context::resetInputCallbacks() input_blocks_reader = {}; } -void Context::setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix) -{ - std::lock_guard lock(shared->custom_executors_mutex); - - shared->custom_executors_config = config; - - if (!shared->custom_executors) - shared->custom_executors = std::make_unique(*shared->custom_executors_config, settings, config_prefix); - else - shared->custom_executors->updateCustomExecutors(*shared->custom_executors_config, settings, config_prefix); -} - -std::pair Context::getCustomExecutor(Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - std::lock_guard lock(shared->custom_executors_mutex); - - if (!shared->custom_executors) - { - auto & config = shared->custom_executors_config ? *shared->custom_executors_config : getConfigRef(); - shared->custom_executors = std::make_unique(config, settings); - } - - return shared->custom_executors->getCustomExecutor(*this, request, params); -} - StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bef448a2389..c9213c806ff 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -42,7 +42,6 @@ namespace zkutil class ZooKeeper; } -struct HTMLForm; namespace DB { @@ -107,7 +106,6 @@ class Volume; using VolumePtr = std::shared_ptr; struct NamedSession; - #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; #endif @@ -491,9 +489,6 @@ public: Compiler & getCompiler(); - void setCustomExecutorConfig(const ConfigurationPtr & config, const String & config_prefix = "custom_http"); - std::pair getCustomExecutor(Poco::Net::HTTPServerRequest &request, HTMLForm & params); - /// Call after initialization before using system logs. Call for global context. void initializeSystemLogs(); diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e2231e35f03..e55c0b05a2d 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -56,7 +55,6 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); if (!read_buffer.eof()) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index efe47da9f46..76d9e9238be 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -698,10 +698,10 @@ class ClickHouseInstance: raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) return open_result.read() - # Connects to the instance via HTTP interface, sends a query and returns the answer - def http_request(self, url, method='GET', params=None, data=None): + # Connects to the instance via HTTP interface, sends a query and returns the answer + def http_request(self, url, method='GET', params=None, data=None, headers=None): url = "http://" + self.ip_address + ":8123/"+url - return requests.request(method=method, url=url, params=params, data=data).content + return requests.request(method=method, url=url, params=params, data=data, headers=headers).content # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): diff --git a/tests/integration/test_https_replication/configs/config.xml b/tests/integration/test_https_replication/configs/config.xml index 35a43b2fc54..be81ccfc145 100644 --- a/tests/integration/test_https_replication/configs/config.xml +++ b/tests/integration/test_https_replication/configs/config.xml @@ -354,6 +354,15 @@ + + + /ping + /replicas_status + query + + Use / or /ping for health checks. Or /replicas_status for more sophisticated health checks. Send queries from your program with POST method or GET /?query=... Use clickhouse-client: For interactive data analysis: clickhouse-client For batch query processing: clickhouse-client --query='SELECT 1' > result clickhouse-client < query > result + + From b82eee5255dc88b83605ba9d46b7c9b995a135ff Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 29 Nov 2019 13:46:46 +0800 Subject: [PATCH 12/23] ISSUES-5436 fix review suggestions & add some '?' re2 test --- dbms/programs/server/HTTPHandlerFactory.cpp | 12 ++--- .../ExtractorContextChange.h | 4 +- .../HTTPQueryRequestHandler.cpp | 2 +- ...TPQueryRequestHandlerMatcherAndCreator.cpp | 45 +++++++++++-------- ...HTTPQueryRequestHandlerMatcherAndCreator.h | 4 +- .../HTTPSessionContextHolder.h | 1 + .../other_tests_configs/config.xml | 4 +- .../test_http_handlers_config/test.py | 18 ++++++-- .../config.xml | 12 ++--- .../users.xml | 0 .../config.xml | 4 +- .../config.xml | 20 ++++----- .../users.xml | 0 programs/server/config.xml | 6 +-- 14 files changed, 76 insertions(+), 56 deletions(-) rename dbms/tests/integration/test_http_handlers_config/{test_insert_and_select_predefine => test_insert_and_select_predefined}/config.xml (75%) rename dbms/tests/integration/test_http_handlers_config/{test_insert_and_select_predefine => test_insert_and_select_predefined}/users.xml (100%) rename dbms/tests/integration/test_http_handlers_config/{test_param_and_settings_predefine => test_param_and_settings_predefined}/config.xml (73%) rename dbms/tests/integration/test_http_handlers_config/{test_param_and_settings_predefine => test_param_and_settings_predefined}/users.xml (100%) diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp index 9a2d1e52f09..b2c07075d54 100644 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { extern const int SYNTAX_ERROR; - extern const int UNKNOW_HTTP_HANDLER_TYPE; + extern const int UNKNOWN_HTTP_HANDLER_TYPE; extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; } @@ -99,13 +99,13 @@ HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) { @@ -126,14 +126,14 @@ void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfigura handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); else if (startsWith(http_handler_type_name, "dynamic_query_handler")) handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefine_query_handler")) - handlers_creator.push_back({createPredefineQueryHandlerMatcher(server, handler_key), createPredefineQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefined_query_handler")) + handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); else if (startsWith(http_handler_type_name, "replicas_status_handler")) handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); else if (http_handler_type_name == "no_handler_description") no_handler_description = configuration.getString(key + ".no_handler_description"); else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOW_HTTP_HANDLER_TYPE); + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); } } diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h index 0fdfd73b4f6..707e86a217d 100644 --- a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ b/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -56,7 +56,7 @@ public: /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not /// belong to the Settings class. - becomeReadonlyIfNeed(request); + becomeReadonlyIfNeeded(request); changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); if (is_multipart_data || settings_may_in_post) @@ -82,7 +82,7 @@ private: /// In theory if initially readonly = 0, the client can change any setting and then set readonly /// to some other value. /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeed(Poco::Net::HTTPServerRequest & request) + void becomeReadonlyIfNeeded(Poco::Net::HTTPServerRequest & request) { if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) { diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index 1bb93908638..ca22496665c 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -174,6 +174,6 @@ HTTPResponseBufferPtr HTTPQueryRequestHandler::createRespon template class HTTPQueryRequestHandler; -template class HTTPQueryRequestHandler; +template class HTTPQueryRequestHandler; } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp index 321a585d939..81f25f0bd08 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp @@ -9,12 +9,12 @@ namespace DB namespace ErrorCodes { - extern const int EMPTY_PREDEFINE_QUERY; + extern const int EMPTY_PREDEFINED_QUERY; extern const int CANNOT_COMPILE_REGEXP; extern const int UNKNOWN_QUERY_PARAMETER; extern const int DUPLICATE_CAPTURE_QUERY_PARAM; extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; - extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY; + extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY; } ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( @@ -62,6 +62,9 @@ ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( template void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) { + if (value.empty()) + return; + int num_captures = regex->NumberOfCapturingGroups() + 1; re2_st::StringPiece matches[num_captures]; @@ -71,16 +74,19 @@ void extractParamWithRegex(Context & context, const RegexRule & regex, const std { for (const auto & [capturing_name, capturing_index] : extract_params) { - String param_name = capturing_name; const auto & capturing_value = matches[capturing_index]; - if constexpr (remove_prefix_for_param) + if (capturing_value.data()) { - const static size_t prefix_size = strlen("param_"); - param_name = capturing_name.substr(prefix_size); - } + String param_name = capturing_name; + if constexpr (remove_prefix_for_param) + { + const static size_t prefix_size = strlen("param_"); + param_name = capturing_name.substr(prefix_size); + } - context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); + } } } } @@ -93,7 +99,7 @@ ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net if (!extract_from_headers.empty()) for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); String extracted_query_from_params; const static size_t prefix_size = strlen("param_"); @@ -112,7 +118,7 @@ ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net return {{extracted_query_from_params, true}}; } -ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( +ExtractorPredefinedQueryParameters::ExtractorPredefinedQueryParameters( Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) : url_regex(url_regex_), headers_regex(headers_regex_) { @@ -120,7 +126,7 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( configuration.keys(key + ".queries", queries_key); if (queries_key.empty()) - throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINE_QUERY); + throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINED_QUERY); for (const auto & query_key : queries_key) { @@ -136,7 +142,7 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( bool is_insert_query = extract_query_ast->as(); if (has_insert_query && is_insert_query) - throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINE_QUERY); + throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY); has_insert_query |= is_insert_query; predefine_queries.push_back({predefine_query, is_insert_query}); @@ -186,14 +192,14 @@ ExtractorPredefineQueryParameters::ExtractorPredefineQueryParameters( } } -ExtractRes ExtractorPredefineQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) +ExtractRes ExtractorPredefinedQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) { if (!extract_from_url.empty()) extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); if (!extract_from_headers.empty()) for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name)); + extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); for (const auto & param : params) if (queries_names.count(param.first)) @@ -277,6 +283,9 @@ HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatche { for (const auto & [header_name, header_rule] : headers_rule) { + if (!request.has(header_name)) + return false; + const String & header_value = request.get(header_name); if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) return false; @@ -294,7 +303,7 @@ HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const Stri } -HTTPHandlerMatcher createPredefineQueryHandlerMatcher(IServer & server, const String & key) +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer & server, const String & key) { return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); @@ -314,16 +323,16 @@ HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const Stri }); } -HTTPHandlerCreator createPredefineQueryHandlerCreator(IServer & server, const String & key) +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer & server, const String & key) { return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); + const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); return [&, query_extract = extract]() { - return new HTTPQueryRequestHandler(server, *query_extract); + return new HTTPQueryRequestHandler(server, *query_extract); }; }); } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h index 456689d1ff5..0d7bc8356bb 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h @@ -48,10 +48,10 @@ private: std::map> extract_from_headers; }; -class ExtractorPredefineQueryParameters +class ExtractorPredefinedQueryParameters { public: - ExtractorPredefineQueryParameters( + ExtractorPredefinedQueryParameters( Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ ); diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h index 9ef8b8f7abe..deadd3910ad 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h +++ b/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h @@ -7,6 +7,7 @@ namespace DB { +/// Manage the lifetime of the session context. struct HTTPSessionContextHolder { ~HTTPSessionContextHolder(); diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml index f117eaba2f7..0c908a4f877 100644 --- a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml @@ -4,14 +4,14 @@ /ping_test - + POST /test_one_handler_with_insert_and_select INSERT INTO test.test VALUES(1) SELECT COUNT() FROM test.test WHERE id = {id:UInt8} - + test not found diff --git a/dbms/tests/integration/test_http_handlers_config/test.py b/dbms/tests/integration/test_http_handlers_config/test.py index 9e332dd8675..a996fbdc908 100644 --- a/dbms/tests/integration/test_http_handlers_config/test.py +++ b/dbms/tests/integration/test_http_handlers_config/test.py @@ -35,8 +35,8 @@ def test_dynamic_query_handler_with_insert_and_select(): assert cluster.instance.http_request('query_data_from_test?max_threads=1&test_select_query_param=' + select_data_query, method='GET') == '1\n2\n3\n4\n' -def test_predefine_query_handler_with_insert_and_select(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_insert_and_select", "test_insert_and_select_predefine")) as cluster: +def test_predefined_query_handler_with_insert_and_select(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_insert_and_select", "test_insert_and_select_predefined")) as cluster: assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT') == '' assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='(1)(2)(3)(4)') == '' assert cluster.instance.http_request('query_data_from_test?max_threads=1', method='GET') == '1\n2\n3\n4\n' @@ -54,18 +54,25 @@ def test_dynamic_query_handler_with_params_and_settings(): assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + query_param, method='POST', data=settings) assert cluster.instance.http_request('get_query_params_and_settings?get_query_param=' + quoted_test_query + '&' + query_param + '&' + settings) == '1\n2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads') == '1\n2\n' assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?query_param=' + quoted_test_query + '&' + settings) == '1\n2\n' + assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings) assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?query_param=' + quoted_test_query + '&' + query_param + '&' + settings) assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\n2\n' + assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\n2\n' + assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) + assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings) -def test_predefine_query_handler_with_params_and_settings(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefine_params_and_settings", "test_param_and_settings_predefine")) as cluster: +def test_predefined_query_handler_with_params_and_settings(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_params_and_settings", "test_param_and_settings_predefined")) as cluster: settings = 'max_threads=1&max_alter_threads=2' query_param = 'name_1=max_threads&name_2=max_alter_threads' assert cluster.instance.http_request('get_query_params_and_settings?' + query_param + '&' + settings, method='GET') == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?' + settings) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings + '&name_2=max_alter_threads') == '1\nmax_alter_threads\t2\n' + assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings) assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?' + query_param + '&' + settings) assert cluster.instance.http_request('post_query_params_and_settings?' + query_param, method='POST', data=settings) == '1\nmax_alter_threads\t2\n' @@ -73,6 +80,9 @@ def test_predefine_query_handler_with_params_and_settings(): assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST') == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('post_query_params_and_settings', method='POST', data=query_param + '&' + settings) == '1\nmax_alter_threads\t2\n' assert cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\nmax_alter_threads\t2\n' + assert cluster.instance.http_request('test_match_headers?' + settings + '&name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\nmax_alter_threads\t2\n' + assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) + assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?' + settings) def test_other_configs(): diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml similarity index 75% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml rename to dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml index 0f0ab81d139..6319bbf327f 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml @@ -2,25 +2,25 @@ - + PUT /create_test_table CREATE DATABASE test CREATE TABLE test.test (id UInt8) Engine = Memory - + - + POST /insert_data_to_test INSERT INTO test.test VALUES - + - + GET /query_data_from_test SELECT * FROM test.test ORDER BY id - + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefine/users.xml rename to dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml index 06c7143f3e8..f9c576e5546 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml @@ -16,14 +16,14 @@ GET - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> query_param TEST_HEADER_VALUE - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> query_param diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml similarity index 73% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml rename to dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml index cca4bc04a69..1ad1fecb2e5 100644 --- a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/config.xml +++ b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml @@ -2,42 +2,42 @@ - + POST /post_query_params_and_settings SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + GET /get_query_params_and_settings SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + GET - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + - + TEST_HEADER_VALUE - .+)/(?P.+)]]> + [^/]+)(/(?P[^/]+))?]]> SELECT value FROM system.settings WHERE name = {name_1:String} SELECT name, value FROM system.settings WHERE name = {name_2:String} - + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml b/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefine/users.xml rename to dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml diff --git a/programs/server/config.xml b/programs/server/config.xml index c4ba4e66ab3..74557a6ab7d 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -537,8 +537,8 @@ url - RE2 compatible regular expression (optional) method - HTTP method(optional) headers - HTTP Header(optional) - queries - predefine queries (mandatory) - + queries - predefined queries (mandatory) + /test_simple_predefine GET default @@ -546,7 +546,7 @@ SELECT 1, {query_prepared_param_1:String} SELECT 1, {query_prepared_param_2:String} - + --> From 1eda48b4b136a8f12520377f3948a8964a284c0d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 9 Dec 2019 12:47:26 +0800 Subject: [PATCH 13/23] fix bad git rebase --- .../server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index ca22496665c..77e72972551 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include #include From 835dc4c4452ca3ec2a3dc39947fa6df13d0f21ad Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 3 Apr 2020 17:09:19 +0800 Subject: [PATCH 14/23] After merge upsream master fix --- .../DataStreams/HTTPInputStreams.cpp | 22 +-- dbms/{src => }/DataStreams/HTTPInputStreams.h | 6 +- .../DataStreams/HTTPOutputStreams.cpp | 0 .../{src => }/DataStreams/HTTPOutputStreams.h | 0 .../Interpreters/QueryParameterVisitor.h | 0 dbms/programs/server/HTTPHandlerFactory.cpp | 140 ------------------ dbms/programs/server/HTTPHandlerFactory.h | 51 ------- programs/server/HTTPHandler.cpp | 7 - programs/server/HTTPHandlerFactory.cpp | 135 ++++++++++++++--- programs/server/HTTPHandlerFactory.h | 136 ++++------------- .../HTTPRequestHandler/ExtractorClientInfo.h | 0 .../ExtractorContextChange.h | 5 - .../HTTPExceptionHandler.cpp | 0 .../HTTPRequestHandler/HTTPExceptionHandler.h | 0 .../HTTPPingRequestHandler.cpp | 0 .../HTTPPingRequestHandler.h | 0 .../HTTPQueryRequestHandler.cpp | 11 +- .../HTTPQueryRequestHandler.h | 0 ...TPQueryRequestHandlerMatcherAndCreator.cpp | 0 ...HTTPQueryRequestHandlerMatcherAndCreator.h | 0 .../HTTPReplicasStatusRequestHandler.cpp | 0 .../HTTPReplicasStatusRequestHandler.h | 0 .../HTTPRootRequestHandler.cpp | 0 .../HTTPRootRequestHandler.h | 0 .../HTTPSessionContextHolder.cpp | 0 .../HTTPSessionContextHolder.h | 0 src/IO/WriteBufferFromHTTPServerResponse.cpp | 64 +++----- src/IO/WriteBufferFromHTTPServerResponse.h | 3 +- src/Interpreters/Context.h | 1 + .../ReplaceQueryParameterVisitor.cpp | 2 +- 30 files changed, 188 insertions(+), 395 deletions(-) rename dbms/{src => }/DataStreams/HTTPInputStreams.cpp (74%) rename dbms/{src => }/DataStreams/HTTPInputStreams.h (62%) rename dbms/{src => }/DataStreams/HTTPOutputStreams.cpp (100%) rename dbms/{src => }/DataStreams/HTTPOutputStreams.h (100%) rename dbms/{src => }/Interpreters/QueryParameterVisitor.h (100%) delete mode 100644 dbms/programs/server/HTTPHandlerFactory.cpp delete mode 100644 dbms/programs/server/HTTPHandlerFactory.h rename {dbms/programs => programs}/server/HTTPRequestHandler/ExtractorClientInfo.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/ExtractorContextChange.h (98%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPExceptionHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPExceptionHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPPingRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp (94%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPRootRequestHandler.h (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp (100%) rename {dbms/programs => programs}/server/HTTPRequestHandler/HTTPSessionContextHolder.h (100%) diff --git a/dbms/src/DataStreams/HTTPInputStreams.cpp b/dbms/DataStreams/HTTPInputStreams.cpp similarity index 74% rename from dbms/src/DataStreams/HTTPInputStreams.cpp rename to dbms/DataStreams/HTTPInputStreams.cpp index 3e5d6c01ab3..e760069a040 100644 --- a/dbms/src/DataStreams/HTTPInputStreams.cpp +++ b/dbms/DataStreams/HTTPInputStreams.cpp @@ -18,9 +18,9 @@ namespace ErrorCodes } HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) - : in(createRawInBuffer(request)) - , in_maybe_compressed(createCompressedBuffer(request, in)) - , in_maybe_internal_compressed(createInternalCompressedBuffer(from, in_maybe_compressed)) + : in(plainBuffer(request)) + , in_maybe_compressed(compressedBuffer(request, in)) + , in_maybe_internal_compressed(internalCompressedBuffer(from, in_maybe_compressed)) { /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, /// checksums of client data compressed with internal algorithm are not checked. @@ -31,12 +31,12 @@ HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & reques } } -std::unique_ptr HTTPInputStreams::createRawInBuffer(HTTPServerRequest & request) const +std::unique_ptr HTTPInputStreams::plainBuffer(HTTPServerRequest & request) const { return std::make_unique(request.stream()); } -std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerRequest & request, std::unique_ptr & raw_buffer) const +std::unique_ptr HTTPInputStreams::compressedBuffer(HTTPServerRequest & request, std::unique_ptr & plain_buffer) const { /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_compressed_method = request.get("Content-Encoding", ""); @@ -44,22 +44,22 @@ std::unique_ptr HTTPInputStreams::createCompressedBuffer(HTTPServerR if (!http_compressed_method.empty()) { if (http_compressed_method == "gzip") - return std::make_unique(std::move(raw_buffer), CompressionMethod::Gzip); + return std::make_unique(std::move(plain_buffer), CompressionMethod::Gzip); else if (http_compressed_method == "deflate") - return std::make_unique(std::move(raw_buffer), CompressionMethod::Zlib); + return std::make_unique(std::move(plain_buffer), CompressionMethod::Zlib); #if USE_BROTLI else if (http_compressed_method == "br") - return std::make_unique(std::move(raw_buffer)); + return std::make_unique(std::move(plain_buffer)); #endif else throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } - return std::move(raw_buffer); + return std::move(plain_buffer); } -std::unique_ptr HTTPInputStreams::createInternalCompressedBuffer( - HTMLForm & params, std::unique_ptr & http_maybe_encoding_buffer) const +std::unique_ptr HTTPInputStreams::internalCompressedBuffer( + HTMLForm ¶ms, std::unique_ptr &http_maybe_encoding_buffer) const { /// The data can also be compressed using incompatible internal algorithm. This is indicated by /// 'decompress' query parameter. diff --git a/dbms/src/DataStreams/HTTPInputStreams.h b/dbms/DataStreams/HTTPInputStreams.h similarity index 62% rename from dbms/src/DataStreams/HTTPInputStreams.h rename to dbms/DataStreams/HTTPInputStreams.h index 652f5370a5a..f2325da676f 100644 --- a/dbms/src/DataStreams/HTTPInputStreams.h +++ b/dbms/DataStreams/HTTPInputStreams.h @@ -21,9 +21,9 @@ struct HTTPInputStreams HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - ReadBufferUniquePtr createRawInBuffer(HTTPServerRequest & request) const; - ReadBufferUniquePtr createCompressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & raw_buffer) const; - ReadBufferUniquePtr createInternalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; + ReadBufferUniquePtr plainBuffer(HTTPServerRequest & request) const; + ReadBufferUniquePtr compressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & plain_buffer) const; + ReadBufferUniquePtr internalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; }; } diff --git a/dbms/src/DataStreams/HTTPOutputStreams.cpp b/dbms/DataStreams/HTTPOutputStreams.cpp similarity index 100% rename from dbms/src/DataStreams/HTTPOutputStreams.cpp rename to dbms/DataStreams/HTTPOutputStreams.cpp diff --git a/dbms/src/DataStreams/HTTPOutputStreams.h b/dbms/DataStreams/HTTPOutputStreams.h similarity index 100% rename from dbms/src/DataStreams/HTTPOutputStreams.h rename to dbms/DataStreams/HTTPOutputStreams.h diff --git a/dbms/src/Interpreters/QueryParameterVisitor.h b/dbms/Interpreters/QueryParameterVisitor.h similarity index 100% rename from dbms/src/Interpreters/QueryParameterVisitor.h rename to dbms/Interpreters/QueryParameterVisitor.h diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp deleted file mode 100644 index b2c07075d54..00000000000 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ /dev/null @@ -1,140 +0,0 @@ -#include "HTTPHandlerFactory.h" - -#include "NotFoundHandler.h" -#include "HTTPRequestHandler/HTTPRootRequestHandler.h" -#include "HTTPRequestHandler/HTTPPingRequestHandler.h" -#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; - extern const int UNKNOWN_HTTP_HANDLER_TYPE; - extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; -} - -InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) -{ -} - -Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) -{ - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - { - if (uri == "/") - return new HTTPRootRequestHandler(server); - if (uri == "/ping") - return new HTTPPingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new HTTPReplicasStatusRequestHandler(server.context()); - } - - if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new InterserverIOHTTPHandler(server); - } - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new NotFoundHandler( - "Use / or /ping for health checks.\n" - "Or /replicas_status for more sophisticated health checks.\n" - "Send queries from your program with POST method or GET /?query=...\n\n" - " Use clickhouse-client:\n\n" - " For interactive data analysis:\n" - " clickhouse-client\n\n" - " For batch query processing:\n" - " clickhouse-client --query='SELECT 1' > result\n" - " clickhouse-client < query > result" - ); - } - - return nullptr; -} - -HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) -{ - updateHTTPHandlersCreator(server.config()); - - if (handlers_creator.empty()) - throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); -} - -Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) -{ - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - for (const auto & [matcher, creator] : handlers_creator) - { - if (matcher(request)) - return creator(); - } - - return new NotFoundHandler(no_handler_description); -} - -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); - -HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); - -void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; - configuration.keys(key, http_handlers_item_key); - - handlers_creator.reserve(http_handlers_item_key.size()); - for (const auto & http_handler_type_name : http_handlers_item_key) - { - if (http_handler_type_name.find('.') != String::npos) - throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); - - const auto & handler_key = key + "." + http_handler_type_name; - - if (startsWith(http_handler_type_name, "root_handler")) - handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "ping_handler")) - handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "dynamic_query_handler")) - handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefined_query_handler")) - handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "replicas_status_handler")) - handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); - else if (http_handler_type_name == "no_handler_description") - no_handler_description = configuration.getString(key + ".no_handler_description"); - else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); - } -} - -} diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h deleted file mode 100644 index 5add7b619af..00000000000 --- a/dbms/programs/server/HTTPHandlerFactory.h +++ /dev/null @@ -1,51 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include "IServer.h" -#include "InterserverIOHTTPHandler.h" - - -namespace DB -{ - -class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - -private: - IServer & server; - Logger * log; - std::string name; -}; - -using HTTPHandlerCreator = std::function; -using HTTPHandlerMatcher = std::function; -using HTTPHandlerMatcherAndCreator = std::pair; -using HTTPHandlersMatcherAndCreator = std::vector; - -class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - HTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); - -private: - IServer & server; - Logger * log; - std::string name; - - String no_handler_description; - HTTPHandlersMatcherAndCreator handlers_creator; -}; - -} diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index cfbefb94ee4..593a156ca52 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -328,15 +328,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne internal_compression = params.getParsed("compress", false); /// Workaround. Poco does not detect 411 Length Required case. -<<<<<<< HEAD:programs/server/HTTPHandler.cpp 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); -======= - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST - && !request.getChunkedTransferEncoding() - && !request.hasContentLength()) - throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); ->>>>>>> ISSUES-5436 fix build failure & fix test failure:dbms/programs/server/HTTPHandler.cpp { Context query_context = server.context(); diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 7a1bcaf4fc6..b2c07075d54 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -1,43 +1,140 @@ #include "HTTPHandlerFactory.h" +#include "NotFoundHandler.h" +#include "HTTPRequestHandler/HTTPRootRequestHandler.h" +#include "HTTPRequestHandler/HTTPPingRequestHandler.h" +#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" + namespace DB { -HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int UNKNOWN_HTTP_HANDLER_TYPE; + extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; +} + +InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) { } -Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler( - const Poco::Net::HTTPServerRequest & request) // override +Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) { LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " - << request.getMethod() - << ", Address: " - << request.clientAddress().toString() - << ", User-Agent: " - << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); - for (auto & handler_factory : child_handler_factories) + const auto & uri = request.getURI(); + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) { - auto handler = handler_factory->createRequestHandler(request); - if (handler != nullptr) - return handler; + if (uri == "/") + return new HTTPRootRequestHandler(server); + if (uri == "/ping") + return new HTTPPingRequestHandler(server); + else if (startsWith(uri, "/replicas_status")) + return new HTTPReplicasStatusRequestHandler(server.context()); } - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + { + return new InterserverIOHTTPHandler(server); + } + + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return new NotFoundHandler; + return new NotFoundHandler( + "Use / or /ping for health checks.\n" + "Or /replicas_status for more sophisticated health checks.\n" + "Send queries from your program with POST method or GET /?query=...\n\n" + " Use clickhouse-client:\n\n" + " For interactive data analysis:\n" + " clickhouse-client\n\n" + " For batch query processing:\n" + " clickhouse-client --query='SELECT 1' > result\n" + " clickhouse-client < query > result" + ); } return nullptr; } +HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) + : server(server_), log(&Logger::get(name_)), name(name_) +{ + updateHTTPHandlersCreator(server.config()); + + if (handlers_creator.empty()) + throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); +} + +Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +{ + LOG_TRACE(log, "HTTP Request for " << name << ". " + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); + + for (const auto & [matcher, creator] : handlers_creator) + { + if (matcher(request)) + return creator(); + } + + return new NotFoundHandler(no_handler_description); +} + +HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); +HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); + +HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); +HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); + +void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +{ + Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; + configuration.keys(key, http_handlers_item_key); + + handlers_creator.reserve(http_handlers_item_key.size()); + for (const auto & http_handler_type_name : http_handlers_item_key) + { + if (http_handler_type_name.find('.') != String::npos) + throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + + const auto & handler_key = key + "." + http_handler_type_name; + + if (startsWith(http_handler_type_name, "root_handler")) + handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "ping_handler")) + handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "dynamic_query_handler")) + handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "predefined_query_handler")) + handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); + else if (startsWith(http_handler_type_name, "replicas_status_handler")) + handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); + else if (http_handler_type_name == "no_handler_description") + no_handler_description = configuration.getString(key + ".no_handler_description"); + else + throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); + } +} + } diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index fcd7fb5d4a2..5add7b619af 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -1,127 +1,51 @@ #pragma once -#include #include #include +#include +#include #include #include "IServer.h" -#include "HTTPHandler.h" #include "InterserverIOHTTPHandler.h" -#include "NotFoundHandler.h" -#include "PingRequestHandler.h" -#include "PrometheusRequestHandler.h" -#include "ReplicasStatusHandler.h" -#include "RootRequestHandler.h" namespace DB { -/// Handle request using child handlers -class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory +class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { -private: - using TThis = HTTPRequestHandlerFactoryMain; +public: + InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + +private: + IServer & server; + Logger * log; + std::string name; +}; + +using HTTPHandlerCreator = std::function; +using HTTPHandlerMatcher = std::function; +using HTTPHandlerMatcherAndCreator = std::pair; +using HTTPHandlersMatcherAndCreator = std::vector; + +class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + HTTPHandlerFactory(IServer & server_, const std::string & name_); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; + + void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); + +private: IServer & server; Logger * log; std::string name; - std::vector> child_handler_factories; - -public: - HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - template - TThis * addHandler(TArgs &&... args) - { - child_handler_factories.emplace_back(std::make_unique(server, std::forward(args)...)); - return this; - } + String no_handler_description; + HTTPHandlersMatcherAndCreator handlers_creator; }; - -/// Handle POST or GET with params -template -class HTTPQueryRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -private: - IServer & server; - -public: - HTTPQueryRequestHandlerFactory(IServer & server_) : server(server_) {} - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - if (request.getURI().find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - return new HandleType(server); - return nullptr; - } -}; - - -/// Handle GET or HEAD endpoint on specified path -template -class HTTPGetRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -private: - IServer & server; -public: - HTTPGetRequestHandlerFactory(IServer & server_) : server(server_) {} - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - auto & method = request.getMethod(); - if (!(method == Poco::Net::HTTPRequest::HTTP_GET || method == Poco::Net::HTTPRequest::HTTP_HEAD)) - return nullptr; - - auto & uri = request.getURI(); - bool uri_match = TGetEndpoint::strict_path ? uri == TGetEndpoint::path : startsWith(uri, TGetEndpoint::path); - if (uri_match) - return new typename TGetEndpoint::HandleType(server); - - return nullptr; - } -}; - - -struct RootEndpoint -{ - static constexpr auto path = "/"; - static constexpr auto strict_path = true; - using HandleType = RootRequestHandler; -}; - -struct PingEndpoint -{ - static constexpr auto path = "/ping"; - static constexpr auto strict_path = true; - using HandleType = PingRequestHandler; -}; - -struct ReplicasStatusEndpoint -{ - static constexpr auto path = "/replicas_status"; - static constexpr auto strict_path = false; - using HandleType = ReplicasStatusHandler; -}; - -using HTTPRootRequestHandlerFactory = HTTPGetRequestHandlerFactory; -using HTTPPingRequestHandlerFactory = HTTPGetRequestHandlerFactory; -using HTTPReplicasStatusRequestHandlerFactory = HTTPGetRequestHandlerFactory; - -template -HTTPRequestHandlerFactoryMain * createDefaultHandlerFatory(IServer & server, const std::string & name) -{ - auto handlerFactory = new HTTPRequestHandlerFactoryMain(server, name); - handlerFactory->addHandler() - ->addHandler() - ->addHandler() - ->addHandler>(); - return handlerFactory; -} - - } diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h b/programs/server/HTTPRequestHandler/ExtractorClientInfo.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/ExtractorClientInfo.h rename to programs/server/HTTPRequestHandler/ExtractorClientInfo.h diff --git a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/programs/server/HTTPRequestHandler/ExtractorContextChange.h similarity index 98% rename from dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h rename to programs/server/HTTPRequestHandler/ExtractorContextChange.h index 707e86a217d..1d4b4e5f58a 100644 --- a/dbms/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ b/programs/server/HTTPRequestHandler/ExtractorContextChange.h @@ -9,11 +9,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_HTTP_PARAM; -} - class ExtractorContextChange { public: diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h rename to programs/server/HTTPRequestHandler/HTTPExceptionHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp similarity index 94% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp index 77e72972551..723ff85abe3 100644 --- a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp @@ -156,19 +156,16 @@ HTTPResponseBufferPtr HTTPQueryRequestHandler::createRespon /// 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("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Gzip); else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Zlib); #if USE_BROTLI else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, true, CompressionMethod::Brotli); #endif } - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return std::make_shared(request, response, keep_alive, false, CompressionMethod{}); } diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h rename to programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp rename to programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h rename to programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp rename to programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp diff --git a/dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h similarity index 100% rename from dbms/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h rename to programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h diff --git a/src/IO/WriteBufferFromHTTPServerResponse.cpp b/src/IO/WriteBufferFromHTTPServerResponse.cpp index c61572d7097..0f30f1352e3 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.cpp +++ b/src/IO/WriteBufferFromHTTPServerResponse.cpp @@ -107,57 +107,36 @@ void WriteBufferFromHTTPServerResponse::nextImpl() { if (compress) { - if (compression_method == CompressionMethod::Gzip) - { + auto content_encoding_name = toContentEncodingName(compression_method); + #if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: gzip\r\n"; -#else - response.set("Content-Encoding", "gzip"); - response_body_ostr = &(response.send()); -#endif - out_raw = std::make_unique(*response_body_ostr); - deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*deflating_buf; - } - else if (compression_method == CompressionMethod::Zlib) - { -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: deflate\r\n"; -#else - response.set("Content-Encoding", "deflate"); - response_body_ostr = &(response.send()); -#endif - out_raw = std::make_unique(*response_body_ostr); - deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin()); - out = &*deflating_buf; - } -#if USE_BROTLI - else if (compression_method == CompressionMethod::Brotli) - { -#if defined(POCO_CLICKHOUSE_PATCH) - *response_header_ostr << "Content-Encoding: br\r\n"; + *response_header_ostr << "Content-Encoding: " << content_encoding_name << "\r\n"; #else response.set("Content-Encoding", content_encoding_name); #endif - - else - throw Exception("Logical error: unknown compression method passed to WriteBufferFromHTTPServerResponse", - ErrorCodes::LOGICAL_ERROR); - /// Use memory allocated for the outer buffer in the buffer pointed to by out. This avoids extra allocation and copy. } - else - { + #if !defined(POCO_CLICKHOUSE_PATCH) - response_body_ostr = &(response.send()); + response_body_ostr = &(response.send()); #endif - out_raw = std::make_unique(*response_body_ostr, working_buffer.size(), working_buffer.begin()); - out = &*out_raw; - } + /// We reuse our buffer in "out" to avoid extra allocations and copies. + + if (compress) + out = wrapWriteBufferWithCompressionMethod( + std::make_unique(*response_body_ostr), + compress ? compression_method : CompressionMethod::None, + compression_level, + working_buffer.size(), + working_buffer.begin()); + else + out = std::make_unique( + *response_body_ostr, + working_buffer.size(), + working_buffer.begin()); } finishSendHeaders(); - } if (out) @@ -174,9 +153,8 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_, - CompressionMethod compression_method_, - size_t size) - : BufferWithOwnMemory(size) + CompressionMethod compression_method_) + : BufferWithOwnMemory(DBMS_DEFAULT_BUFFER_SIZE) , request(request_) , response(response_) , keep_alive_timeout(keep_alive_timeout_) diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index 528ba8ffe23..c10288fff9f 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -95,8 +95,7 @@ public: Poco::Net::HTTPServerResponse & response_, unsigned keep_alive_timeout_, bool compress_ = false, /// If true - set Content-Encoding header and compress the result. - CompressionMethod compression_method_ = CompressionMethod::Gzip, - size_t size = DBMS_DEFAULT_BUFFER_SIZE); + CompressionMethod compression_method_ = CompressionMethod::None); /// Writes progess in repeating HTTP headers. void onProgress(const Progress & progress); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c9213c806ff..1f81cdbc58b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -106,6 +106,7 @@ class Volume; using VolumePtr = std::shared_ptr; struct NamedSession; + #if USE_EMBEDDED_COMPILER class CompiledExpressionCache; #endif diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e55c0b05a2d..5c29c722f88 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -55,7 +55,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) IColumn & temp_column = *temp_column_ptr; ReadBufferFromString read_buffer{value}; FormatSettings format_settings; - data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings); + data_type->deserializeAsTextEscaped(temp_column, read_buffer, format_settings); if (!read_buffer.eof()) throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'" From 57cbecf93511e36e2fef1bfa6612e05e0445872c Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 4 Apr 2020 16:57:16 +0800 Subject: [PATCH 15/23] ISSUES-5436 reworking predefine http --- dbms/DataStreams/HTTPInputStreams.cpp | 73 --- dbms/DataStreams/HTTPInputStreams.h | 29 - dbms/DataStreams/HTTPOutputStreams.cpp | 213 ------ dbms/DataStreams/HTTPOutputStreams.h | 48 -- programs/server/CMakeLists.txt | 11 +- programs/server/HTTPHandler.cpp | 612 +++++++++++++++--- programs/server/HTTPHandler.h | 85 ++- programs/server/HTTPHandlerFactory.cpp | 200 +++--- programs/server/HTTPHandlerFactory.h | 126 +++- programs/server/HTTPHandlerRequestFilter.h | 98 +++ .../HTTPRequestHandler/ExtractorClientInfo.h | 38 -- .../ExtractorContextChange.h | 116 ---- .../HTTPExceptionHandler.cpp | 154 ----- .../HTTPRequestHandler/HTTPExceptionHandler.h | 19 - .../HTTPPingRequestHandler.cpp | 45 -- .../HTTPPingRequestHandler.h | 26 - .../HTTPQueryRequestHandler.cpp | 175 ----- .../HTTPQueryRequestHandler.h | 48 -- ...TPQueryRequestHandlerMatcherAndCreator.cpp | 339 ---------- ...HTTPQueryRequestHandlerMatcherAndCreator.h | 92 --- .../HTTPReplicasStatusRequestHandler.cpp | 118 ---- .../HTTPReplicasStatusRequestHandler.h | 25 - .../HTTPRootRequestHandler.cpp | 45 -- .../HTTPRootRequestHandler.h | 26 - .../HTTPSessionContextHolder.cpp | 105 --- .../HTTPSessionContextHolder.h | 25 - programs/server/NotFoundHandler.cpp | 20 +- programs/server/NotFoundHandler.h | 11 +- programs/server/PingRequestHandler.cpp | 31 - programs/server/PingRequestHandler.h | 27 - programs/server/ReplicasStatusHandler.h | 2 +- programs/server/RootRequestHandler.cpp | 33 - programs/server/RootRequestHandler.h | 27 - programs/server/Server.cpp | 44 +- programs/server/StaticRequestHandler.cpp | 86 +++ programs/server/StaticRequestHandler.h | 28 + src/Common/HTMLForm.h | 6 - .../Interpreters/QueryParameterVisitor.h | 14 + 38 files changed, 1043 insertions(+), 2177 deletions(-) delete mode 100644 dbms/DataStreams/HTTPInputStreams.cpp delete mode 100644 dbms/DataStreams/HTTPInputStreams.h delete mode 100644 dbms/DataStreams/HTTPOutputStreams.cpp delete mode 100644 dbms/DataStreams/HTTPOutputStreams.h create mode 100644 programs/server/HTTPHandlerRequestFilter.h delete mode 100644 programs/server/HTTPRequestHandler/ExtractorClientInfo.h delete mode 100644 programs/server/HTTPRequestHandler/ExtractorContextChange.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPExceptionHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h delete mode 100644 programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp delete mode 100644 programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h delete mode 100644 programs/server/PingRequestHandler.cpp delete mode 100644 programs/server/PingRequestHandler.h delete mode 100644 programs/server/RootRequestHandler.cpp delete mode 100644 programs/server/RootRequestHandler.h create mode 100644 programs/server/StaticRequestHandler.cpp create mode 100644 programs/server/StaticRequestHandler.h rename {dbms => src}/Interpreters/QueryParameterVisitor.h (59%) diff --git a/dbms/DataStreams/HTTPInputStreams.cpp b/dbms/DataStreams/HTTPInputStreams.cpp deleted file mode 100644 index e760069a040..00000000000 --- a/dbms/DataStreams/HTTPInputStreams.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include -#include - -#include - -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_COMPRESSION_METHOD; -} - -HTTPInputStreams::HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from) - : in(plainBuffer(request)) - , in_maybe_compressed(compressedBuffer(request, in)) - , in_maybe_internal_compressed(internalCompressedBuffer(from, in_maybe_compressed)) -{ - /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, - /// checksums of client data compressed with internal algorithm are not checked. - if (context.getSettingsRef().http_native_compression_disable_checksumming_on_decompress) - { - if (CompressedReadBuffer * compressed_buffer = typeid_cast(in_maybe_internal_compressed.get())) - compressed_buffer->disableChecksumming(); - } -} - -std::unique_ptr HTTPInputStreams::plainBuffer(HTTPServerRequest & request) const -{ - return std::make_unique(request.stream()); -} - -std::unique_ptr HTTPInputStreams::compressedBuffer(HTTPServerRequest & request, std::unique_ptr & plain_buffer) const -{ - /// Request body can be compressed using algorithm specified in the Content-Encoding header. - String http_compressed_method = request.get("Content-Encoding", ""); - - if (!http_compressed_method.empty()) - { - if (http_compressed_method == "gzip") - return std::make_unique(std::move(plain_buffer), CompressionMethod::Gzip); - else if (http_compressed_method == "deflate") - return std::make_unique(std::move(plain_buffer), CompressionMethod::Zlib); -#if USE_BROTLI - else if (http_compressed_method == "br") - return std::make_unique(std::move(plain_buffer)); -#endif - else - throw Exception("Unknown Content-Encoding of HTTP request: " + http_compressed_method, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - } - - return std::move(plain_buffer); -} - -std::unique_ptr HTTPInputStreams::internalCompressedBuffer( - HTMLForm ¶ms, std::unique_ptr &http_maybe_encoding_buffer) const -{ - /// The data can also be compressed using incompatible internal algorithm. This is indicated by - /// 'decompress' query parameter. - std::unique_ptr in_post_maybe_compressed; - if (params.getParsed("decompress", false)) - return std::make_unique(*http_maybe_encoding_buffer); - - return std::move(http_maybe_encoding_buffer); -} - -} diff --git a/dbms/DataStreams/HTTPInputStreams.h b/dbms/DataStreams/HTTPInputStreams.h deleted file mode 100644 index f2325da676f..00000000000 --- a/dbms/DataStreams/HTTPInputStreams.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; - -struct HTTPInputStreams -{ - using ReadBufferUniquePtr = std::unique_ptr; - - ReadBufferUniquePtr in; - ReadBufferUniquePtr in_maybe_compressed; - ReadBufferUniquePtr in_maybe_internal_compressed; - - HTTPInputStreams(Context & context, HTTPServerRequest & request, HTMLForm & from); - - ReadBufferUniquePtr plainBuffer(HTTPServerRequest & request) const; - ReadBufferUniquePtr compressedBuffer(HTTPServerRequest & request, ReadBufferUniquePtr & plain_buffer) const; - ReadBufferUniquePtr internalCompressedBuffer(HTMLForm & params, ReadBufferUniquePtr & http_maybe_encoding_buffer) const; -}; - -} diff --git a/dbms/DataStreams/HTTPOutputStreams.cpp b/dbms/DataStreams/HTTPOutputStreams.cpp deleted file mode 100644 index 31f4929bef1..00000000000 --- a/dbms/DataStreams/HTTPOutputStreams.cpp +++ /dev/null @@ -1,213 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "HTTPOutputStreams.h" - - -namespace DB -{ - -namespace -{ - inline void listeningProgress(Context & context, ProgressCallback listener) - { - auto prev = context.getProgressCallback(); - context.setProgressCallback([prev, listener] (const Progress & progress) - { - if (prev) - prev(progress); - - listener(progress); - }); - } - - inline ProgressCallback cancelListener(Context & context, Poco::Net::StreamSocket & socket) - { - /// Assume that at the point this method is called no one is reading data from the socket any more. - /// True for read-only queries. - return [&context, &socket](const Progress &) - { - try - { - char b; - int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); - if (status == 0) - context.killCurrentQuery(); - } - catch (Poco::TimeoutException &) - { - } - catch (...) - { - context.killCurrentQuery(); - } - }; - } -} - -HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress) - : out(raw_out) - , out_maybe_compressed(createMaybeCompressionOut(internal_compress, out)) - , out_maybe_delayed_and_compressed(out_maybe_compressed) -{ -} - -HTTPOutputStreams::HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form) - : out(raw_out) - , out_maybe_compressed(createMaybeCompressionOut(form.getParsed("compress", false), out)) - , out_maybe_delayed_and_compressed(createMaybeDelayedAndCompressionOut(context, form, out_maybe_compressed)) -{ - Settings & settings = context.getSettingsRef(); - - /// HTTP response compression is turned on only if the client signalled that they support it - /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. - out->setCompression(out->getCompression() && settings.enable_http_compression); - if (out->getCompression()) - out->setCompressionLevel(settings.http_zlib_compression_level); - - out->setSendProgressInterval(settings.http_headers_progress_interval_ms); - - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed Origin header. - out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); - - /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. - if (settings.send_progress_in_http_headers) - listeningProgress(context, [this] (const Progress & progress) { out->onProgress(progress); }); - - if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) - { - Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - - listeningProgress(context, cancelListener(context, socket)); - } -} - -WriteBufferPtr HTTPOutputStreams::createMaybeCompressionOut(bool compression, HTTPResponseBufferPtr & out_) -{ - /// Client can pass a 'compress' flag in the query string. In this case the query result is - /// compressed using internal algorithm. This is not reflected in HTTP headers. - return compression ? std::make_shared(*out_) : WriteBufferPtr(out_); -} - -WriteBufferPtr HTTPOutputStreams::createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_) -{ - /// If it is specified, the whole result will be buffered. - /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. - bool buffer_until_eof = form.getParsed("wait_end_of_query", false); - - /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total = std::max(form.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); - - size_t buffer_size_memory = (buffer_size_total > DBMS_DEFAULT_BUFFER_SIZE) ? buffer_size_total : 0; - - if (buffer_size_memory > 0 || buffer_until_eof) - { - CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; - CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; - - if (buffer_size_memory > 0) - cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); - - if (buffer_until_eof) - { - std::string tmp_path_template = context.getTemporaryPath() + "http_buffers/"; - - auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) - { - return WriteBufferFromTemporaryFile::create(tmp_path_template); - }; - - cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); - } - else - { - auto push_memory_buffer_and_continue = [next_buffer = out_] (const WriteBufferPtr & prev_buf) - { - auto prev_memory_buffer = typeid_cast(prev_buf.get()); - if (!prev_memory_buffer) - throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); - - return next_buffer; - }; - - cascade_buffer2.emplace_back(push_memory_buffer_and_continue); - } - - return std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); - } - - return out_; -} - -HTTPOutputStreams::~HTTPOutputStreams() -{ - /// This could be a broken HTTP Request - /// Because it does not call finalize or writes some data to output stream after call finalize - /// In this case we need to clean up its broken state to ensure that they are not sent to the client - - /// For delayed stream, we destory CascadeBuffer and without sending any data to client. - if (out_maybe_delayed_and_compressed != out_maybe_compressed) - out_maybe_delayed_and_compressed.reset(); - - if (out->count() == out->offset()) - { - /// If buffer has data and server never sends data to client - /// no need to send that data - out_maybe_compressed->position() = out_maybe_compressed->buffer().begin(); - out->position() = out->buffer().begin(); - } -} - -void HTTPOutputStreams::finalize() const -{ - if (out_maybe_delayed_and_compressed != out_maybe_compressed) - { - /// TODO: set Content-Length if possible - std::vector write_buffers; - std::vector read_buffers; - std::vector read_buffers_raw_ptr; - - auto cascade_buffer = typeid_cast(out_maybe_delayed_and_compressed.get()); - if (!cascade_buffer) - throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); - - cascade_buffer->getResultBuffers(write_buffers); - - if (write_buffers.empty()) - throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); - - for (auto & write_buf : write_buffers) - { - IReadableWriteBuffer * write_buf_concrete; - ReadBufferPtr reread_buf; - - if (write_buf - && (write_buf_concrete = dynamic_cast(write_buf.get())) - && (reread_buf = write_buf_concrete->tryGetReadBuffer())) - { - read_buffers.emplace_back(reread_buf); - read_buffers_raw_ptr.emplace_back(reread_buf.get()); - } - } - - ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); - copyData(concat_read_buffer, *out_maybe_compressed); - } - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - out_maybe_compressed->next(); - out->next(); - out->finalize(); -} - -} diff --git a/dbms/DataStreams/HTTPOutputStreams.h b/dbms/DataStreams/HTTPOutputStreams.h deleted file mode 100644 index fba122ea294..00000000000 --- a/dbms/DataStreams/HTTPOutputStreams.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -using HTTPServerRequest = Poco::Net::HTTPServerRequest; -using HTTPServerResponse = Poco::Net::HTTPServerResponse; -using HTTPResponseBufferPtr = std::shared_ptr; - -/* Raw data - * ↓ - * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) - * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) - * CompressedWriteBuffer out_maybe_compressed (optional) - * ↓ - * WriteBufferFromHTTPServerResponse out - */ -struct HTTPOutputStreams -{ - HTTPResponseBufferPtr out; - /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. - std::shared_ptr out_maybe_compressed; - /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. - std::shared_ptr out_maybe_delayed_and_compressed; - - ~HTTPOutputStreams(); - - void finalize() const; - - WriteBufferPtr createMaybeDelayedAndCompressionOut(Context & context, HTMLForm & form, WriteBufferPtr & out_); - - WriteBufferPtr createMaybeCompressionOut(bool compression, std::shared_ptr & out_); - - HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, bool internal_compress); - - HTTPOutputStreams(HTTPResponseBufferPtr & raw_out, Context & context, HTTPServerRequest & request, HTMLForm & form); -}; - -using HTTPOutputStreamsPtr = std::unique_ptr; - -} diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 064cda4847a..a252310cc93 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -4,21 +4,12 @@ set(CLICKHOUSE_SERVER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/InterserverIOHTTPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/MetricsTransmitter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/NotFoundHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/PingRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusMetricsWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/ReplicasStatusHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/RootRequestHandler.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/StaticRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/Server.cpp ${CMAKE_CURRENT_SOURCE_DIR}/TCPHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandlerFactory.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPPingRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPRootRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPSessionContextHolder.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPExceptionHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp ) set(CLICKHOUSE_SERVER_SOURCES diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 593a156ca52..98093e94c79 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -1,5 +1,8 @@ #include "HTTPHandler.h" +#include "HTTPHandlerFactory.h" +#include "HTTPHandlerRequestFilter.h" + #include #include #include @@ -7,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -32,6 +36,7 @@ #include #include #include +#include #include #include @@ -78,6 +83,7 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT; extern const int UNKNOWN_DATABASE_ENGINE; extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int NO_ELEMENTS_IN_CONFIG; extern const int QUERY_IS_TOO_LARGE; @@ -90,7 +96,6 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; - extern const int UNKNOW_QUERY_EXECUTOR; } @@ -118,8 +123,7 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::INCORRECT_DATA || exception_code == ErrorCodes::TYPE_MISMATCH) return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOW_QUERY_EXECUTOR || - exception_code == ErrorCodes::UNKNOWN_TABLE || + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE || @@ -172,44 +176,67 @@ static std::chrono::steady_clock::duration parseSessionTimeout( } -HTTPHandler::HTTPHandler(IServer & server_) - : server(server_), log(&Logger::get("HTTPHandler")) +void HTTPHandler::pushDelayedResults(Output & used_output) +{ + std::vector write_buffers; + std::vector read_buffers; + std::vector read_buffers_raw_ptr; + + auto cascade_buffer = typeid_cast(used_output.out_maybe_delayed_and_compressed.get()); + if (!cascade_buffer) + throw Exception("Expected CascadeWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + cascade_buffer->getResultBuffers(write_buffers); + + if (write_buffers.empty()) + throw Exception("At least one buffer is expected to overwrite result into HTTP response", ErrorCodes::LOGICAL_ERROR); + + for (auto & write_buf : write_buffers) + { + IReadableWriteBuffer * write_buf_concrete; + ReadBufferPtr reread_buf; + + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) + && (reread_buf = write_buf_concrete->tryGetReadBuffer())) + { + read_buffers.emplace_back(reread_buf); + read_buffers_raw_ptr.emplace_back(reread_buf.get()); + } + } + + ConcatReadBuffer concat_read_buffer(read_buffers_raw_ptr); + copyData(concat_read_buffer, *used_output.out_maybe_compressed); +} + + +HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) + : server(server_) + , log(&Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } -HTTPHandler::SessionContextHolder::~SessionContextHolder() +void HTTPHandler::processQuery( + Poco::Net::HTTPServerRequest & request, + HTMLForm & params, + Poco::Net::HTTPServerResponse & response, + Output & used_output) { - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} + Context context = server.context(); + CurrentThread::QueryScope query_scope(context); -HTTPHandler::SessionContextHolder::SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params) - : query_context(query_context_) -{ - authentication(request, params); + LOG_TRACE(log, "Request URI: " << request.getURI()); - { - session_id = params.get("session_id", ""); + std::istream & istr = request.stream(); - if (!session_id.empty()) - { - session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); - session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); - - query_context = *session_context; - query_context.setSessionContext(*session_context); - } - } -} - -void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & request, HTMLForm & params) -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); + /// The user and password can be passed by headers (similar to X-Auth-*), + /// which is used by load balancers to pass authentication information. + std::string user = request.get("X-ClickHouse-User", ""); + std::string password = request.get("X-ClickHouse-Key", ""); + std::string quota_key = request.get("X-ClickHouse-Quota", ""); if (user.empty() && password.empty() && quota_key.empty()) { @@ -243,26 +270,331 @@ void HTTPHandler::SessionContextHolder::authentication(HTTPServerRequest & reque } std::string query_id = params.get("query_id", ""); - query_context.setUser(user, password, request.clientAddress(), quota_key); - query_context.setCurrentQueryId(query_id); + context.setUser(user, password, request.clientAddress(), quota_key); + context.setCurrentQueryId(query_id); + + /// The user could specify session identifier and session timeout. + /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. + + std::shared_ptr session; + String session_id; + std::chrono::steady_clock::duration session_timeout; + bool session_is_set = params.has("session_id"); + const auto & config = server.config(); + + if (session_is_set) + { + session_id = params.get("session_id"); + session_timeout = parseSessionTimeout(config, params); + std::string session_check = params.get("session_check", ""); + + session = context.acquireNamedSession(session_id, session_timeout, session_check == "1"); + + context = session->context; + context.setSessionContext(session->context); + } + + SCOPE_EXIT({ + if (session) + session->release(); + }); + + /// 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; + + /// Client can pass a 'compress' flag in the query string. In this case the query result is + /// compressed using internal algorithm. This is not reflected in HTTP headers. + bool internal_compression = params.getParsed("compress", false); + + /// At least, we should postpone sending of first buffer_size result bytes + size_t buffer_size_total = std::max( + params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + + /// If it is specified, the whole result will be buffered. + /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. + bool buffer_until_eof = params.getParsed("wait_end_of_query", false); + + size_t buffer_size_http = DBMS_DEFAULT_BUFFER_SIZE; + size_t buffer_size_memory = (buffer_size_total > buffer_size_http) ? buffer_size_total : 0; + + unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", 10); + + used_output.out = std::make_shared( + request, response, keep_alive_timeout, client_supports_http_compression, http_response_compression_method); + + if (internal_compression) + used_output.out_maybe_compressed = std::make_shared(*used_output.out); + else + used_output.out_maybe_compressed = used_output.out; + + if (buffer_size_memory > 0 || buffer_until_eof) + { + CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; + CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; + + if (buffer_size_memory > 0) + cascade_buffer1.emplace_back(std::make_shared(buffer_size_memory)); + + if (buffer_until_eof) + { + const std::string tmp_path(context.getTemporaryVolume()->getNextDisk()->getPath()); + const std::string tmp_path_template(tmp_path + "http_buffers/"); + + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; + + cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); + } + else + { + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) + { + auto prev_memory_buffer = typeid_cast(prev_buf.get()); + if (!prev_memory_buffer) + throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); + + auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); + copyData(*rdbuf , *next_buffer); + + return next_buffer; + }; + + cascade_buffer2.emplace_back(push_memory_buffer_and_continue); + } + + used_output.out_maybe_delayed_and_compressed = std::make_shared( + std::move(cascade_buffer1), std::move(cascade_buffer2)); + } + else + { + used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; + } + + std::unique_ptr in_post_raw = std::make_unique(istr); + + /// Request body can be compressed using algorithm specified in the Content-Encoding header. + String http_request_compression_method_str = request.get("Content-Encoding", ""); + std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( + std::make_unique(istr), chooseCompressionMethod({}, http_request_compression_method_str)); + + /// The data can also be compressed using incompatible internal algorithm. This is indicated by + /// 'decompress' query parameter. + std::unique_ptr in_post_maybe_compressed; + bool in_post_compressed = false; + if (params.getParsed("decompress", false)) + { + in_post_maybe_compressed = std::make_unique(*in_post); + in_post_compressed = true; + } + else + in_post_maybe_compressed = std::move(in_post); + + std::unique_ptr in; + + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; + + Names reserved_param_suffixes; + + auto param_could_be_skipped = [&] (const String & name) + { + if (reserved_param_names.count(name)) + return true; + + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } + + return false; + }; + + /// Settings can be overridden in the query. + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + + /// In theory if initially readonly = 0, the client can change any setting and then set readonly + /// to some other value. + const auto & settings = context.getSettingsRef(); + + /// Only readonly queries are allowed for HTTP GET requests. + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + { + if (settings.readonly == 0) + context.setSetting("readonly", 2); + } + + bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); + + if (has_external_data) + { + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + reserved_param_suffixes.reserve(3); + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + reserved_param_suffixes.emplace_back("_format"); + reserved_param_suffixes.emplace_back("_types"); + reserved_param_suffixes.emplace_back("_structure"); + } + + SettingsChanges settings_changes; + for (const auto & [key, value] : params) + { + if (key == "database") + { + context.setCurrentDatabase(value); + } + else if (key == "default_format") + { + context.setDefaultFormat(value); + } + else if (param_could_be_skipped(key)) + { + } + else + { + /// Other than query parameters are treated as settings. + if (!customizeQueryParam(context, key, value)) + settings_changes.push_back({key, value}); + } + } + + /// For external data we also want settings + context.checkSettingsConstraints(settings_changes); + context.applySettingsChanges(settings_changes); + + const auto & query = getQuery(request, params, context); + std::unique_ptr in_param = std::make_unique(query); + in = has_external_data ? std::move(in_param) : std::make_unique(*in_param, *in_post_maybe_compressed); + + /// HTTP response compression is turned on only if the client signalled that they support it + /// (using Accept-Encoding header) and 'enable_http_compression' setting is turned on. + used_output.out->setCompression(client_supports_http_compression && settings.enable_http_compression); + if (client_supports_http_compression) + used_output.out->setCompressionLevel(settings.http_zlib_compression_level); + + used_output.out->setSendProgressInterval(settings.http_headers_progress_interval_ms); + + /// If 'http_native_compression_disable_checksumming_on_decompress' setting is turned on, + /// checksums of client data compressed with internal algorithm are not checked. + if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) + static_cast(*in_post_maybe_compressed).disableChecksumming(); + + /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed + /// Origin header. + used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + + ClientInfo & client_info = context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::HTTP; + + /// Query sent through HTTP interface is initial. + client_info.initial_user = client_info.current_user; + client_info.initial_query_id = client_info.current_query_id; + client_info.initial_address = client_info.current_address; + + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + client_info.http_method = http_method; + client_info.http_user_agent = request.get("User-Agent", ""); + + auto append_callback = [&context] (ProgressCallback callback) + { + auto prev = context.getProgressCallback(); + + context.setProgressCallback([prev, callback] (const Progress & progress) + { + if (prev) + prev(progress); + + callback(progress); + }); + }; + + /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. + if (settings.send_progress_in_http_headers) + append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); + + if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) + { + Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); + + append_callback([&context, &socket](const Progress &) + { + /// Assume that at the point this method is called no one is reading data from the socket any more. + /// True for read-only queries. + try + { + char b; + //FIXME looks like MSG_DONTWAIT is useless because of POCO_BROKEN_TIMEOUTS + int status = socket.receiveBytes(&b, 1, MSG_DONTWAIT | MSG_PEEK); + if (status == 0) + context.killCurrentQuery(); + } + catch (Poco::TimeoutException &) + { + } + catch (...) + { + context.killCurrentQuery(); + } + }); + } + + customizeContext(context); + + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) + { + response.setContentType(content_type); + response.add("X-ClickHouse-Query-Id", current_query_id); + response.add("X-ClickHouse-Format", format); + response.add("X-ClickHouse-Timezone", timezone); + } + ); + + if (used_output.hasDelayed()) + { + /// TODO: set Content-Length if possible + pushDelayedResults(used_output); + } + + /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to + /// the client. + used_output.out->finalize(); } -void HTTPHandler::processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out) -{ - const auto & name_with_custom_executor = context.getCustomExecutor(request, params); - LOG_TRACE(log, "Using '" << name_with_custom_executor.first << "' CustomExecutor to execute URI: " << request.getURI()); - - ExtractorClientInfo{context.getClientInfo()}.extract(request); - ExtractorContextChange{context, name_with_custom_executor.second}.extract(request, params); - - HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); - name_with_custom_executor.second->executeQuery(context, request, response, params, input_streams, output_streams); -} - -void HTTPHandler::trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, - HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression) +void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, + Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, + Output & used_output) { try { @@ -270,14 +602,19 @@ void HTTPHandler::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.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + 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::max()); } - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD) + bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD; + + if (auth_fail) { response.requireAuthentication("ClickHouse server HTTP API"); } @@ -286,16 +623,36 @@ void HTTPHandler::trySendExceptionToClient( response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); } - if (!response_out && !response.sent()) - response.send() << message << std::endl; - else + if (!response.sent() && !used_output.out_maybe_compressed) { - HTTPOutputStreams output_streams(response_out, compression); + /// If nothing was sent yet and we don't even know if we must compress the response. + response.send() << s << std::endl; + } + else if (used_output.out_maybe_compressed) + { + /// Destroy CascadeBuffer to actualize buffers' positions and reset extra references + if (used_output.hasDelayed()) + used_output.out_maybe_delayed_and_compressed.reset(); - writeString(message, *output_streams.out_maybe_compressed); - writeChar('\n', *output_streams.out_maybe_compressed); + /// Send the error message into already used (and possibly compressed) stream. + /// Note that the error message will possibly be sent after some data. + /// Also HTTP code 200 could have already been sent. - output_streams.finalize(); + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + bool data_sent = used_output.out->count() != used_output.out->offset(); + + if (!data_sent) + { + used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin(); + used_output.out->position() = used_output.out->buffer().begin(); + } + + writeString(s, *used_output.out_maybe_compressed); + writeChar('\n', *used_output.out_maybe_compressed); + + used_output.out_maybe_compressed->next(); + used_output.out->next(); + used_output.out->finalize(); } } catch (...) @@ -304,41 +661,37 @@ void HTTPHandler::trySendExceptionToClient( } } + void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) { setThreadName("HTTPHandler"); ThreadStatus thread_status; + Output used_output; + /// In case of exception, send stack trace to client. - HTTPResponseBufferPtr response_out; - bool with_stacktrace = false, internal_compression = false; + bool with_stacktrace = false; try { - response_out = createResponseOut(request, response); - response.set("Content-Type", "text/plain; charset=UTF-8"); + response.setContentType("text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); - /// For keep-alive to work. if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) response.setChunkedTransferEncoding(true); HTMLForm params(request); with_stacktrace = params.getParsed("stacktrace", false); - internal_compression = params.getParsed("compress", false); /// 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); - + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && + !request.hasContentLength()) { - Context query_context = server.context(); - CurrentThread::QueryScope query_scope(query_context); - - SessionContextHolder holder{query_context, request, params}; - processQuery(holder.query_context, request, params, response, response_out); - LOG_INFO(log, "Done processing query"); + throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); } + + processQuery(request, params, response, used_output); + LOG_INFO(log, "Done processing query"); } catch (...) { @@ -347,36 +700,103 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne /** If exception is received from remote server, then stack trace is embedded in message. * If exception is thrown on local server, then stack trace is in separate field. */ - int exception_code = getCurrentExceptionCode(); std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - trySendExceptionToClient(exception_message, exception_code, request, response, response_out, internal_compression); + int exception_code = getCurrentExceptionCode(); + + trySendExceptionToClient(exception_message, exception_code, request, response, used_output); } } -HTTPResponseBufferPtr HTTPHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) +DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_) + : HTTPHandler(server_, "DynamicQueryHandler"), param_name(param_name_) { - size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); +} - if (!http_response_compression_methods.empty()) +bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) +{ + if (key == param_name) + return true; /// do nothing + + if (startsWith(key, "param_")) { - /// 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("gzip")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Gzip, DBMS_DEFAULT_BUFFER_SIZE); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Zlib, DBMS_DEFAULT_BUFFER_SIZE); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared( - request, response, keep_alive, true, CompressionMethod::Brotli, DBMS_DEFAULT_BUFFER_SIZE); -#endif + /// Save name and values of substitution in dictionary. + const String parameter_name = key.substr(strlen("param_")); + context.setQueryParameter(parameter_name, value); + return true; } - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}, DBMS_DEFAULT_BUFFER_SIZE); + return false; } +std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) +{ + + if (likely(!startsWith(request.getContentType(), "multipart/form-data"))) + { + /// Part of the query can be passed in the 'query' parameter and the rest in the request body + /// (http method need not necessarily be POST). In this case the entire query consists of the + /// contents of the 'query' parameter, a line break and the request body. + std::string query_param = params.get(param_name, ""); + return query_param.empty() ? query_param : query_param + "\n"; + } + + /// Support for "external data for query processing". + /// Used in case of POST request with form-data, but it isn't expected to be deleted after that scope. + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + + std::string full_query; + /// Params are of both form params POST and uri (GET params) + for (const auto & it : params) + if (it.first == param_name) + full_query += it.second; + + return full_query; } + +PredefineQueryHandler::PredefineQueryHandler(IServer & server, const NameSet & receive_params_, const std::string & predefine_query_) + : HTTPHandler(server, "PredefineQueryHandler"), receive_params(receive_params_), predefine_query(predefine_query_) +{ +} + +bool PredefineQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) +{ + if (receive_params.count(key)) + { + context.setQueryParameter(key, value); + return true; + } + + return false; +} + +std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) +{ + if (unlikely(startsWith(request.getContentType(), "multipart/form-data"))) + { + /// Support for "external data for query processing". + ExternalTablesHandler handler(context, params); + params.load(request, request.stream(), handler); + } + + return predefine_query; +} + +Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) +{ + const auto & query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory(server, query_param_name), server.config(), config_prefix); +} + + +Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix) +{ + if (!server.config().has(config_prefix + ".handler.query")) + throw Exception("There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + const auto & predefine_query = server.config().getString(config_prefix + ".handler.query"); + + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( + server, analyzeReceiveQueryParams(predefine_query), predefine_query), server.config(), config_prefix); +} +} \ No newline at end of file diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 1faa0efe907..f5139b6a9cc 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,8 +7,6 @@ #include #include -#include - namespace CurrentMetrics { @@ -26,26 +24,39 @@ class WriteBufferFromHTTPServerResponse; class HTTPHandler : public Poco::Net::HTTPRequestHandler { public: - explicit HTTPHandler(IServer & server_); + explicit HTTPHandler(IServer & server_, const std::string & name); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + /// This method is called right before the query execution. + virtual void customizeContext(Context & /* context */) {} + + virtual bool customizeQueryParam(Context & context, const std::string & key, const std::string & value) = 0; + + virtual std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) = 0; + private: - using HTTPRequest = Poco::Net::HTTPServerRequest; - using HTTPResponse = Poco::Net::HTTPServerResponse; - - struct SessionContextHolder + struct Output { - ~SessionContextHolder(); + /* Raw data + * ↓ + * CascadeWriteBuffer out_maybe_delayed_and_compressed (optional) + * ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults) + * CompressedWriteBuffer out_maybe_compressed (optional) + * ↓ + * WriteBufferFromHTTPServerResponse out + */ - void authentication(HTTPServerRequest & request, HTMLForm & params); + std::shared_ptr out; + /// Points to 'out' or to CompressedWriteBuffer(*out), depending on settings. + std::shared_ptr out_maybe_compressed; + /// Points to 'out' or to CompressedWriteBuffer(*out) or to CascadeWriteBuffer. + std::shared_ptr out_maybe_delayed_and_compressed; - SessionContextHolder(Context & query_context_, HTTPRequest & request, HTMLForm & params); - - String session_id; - Context & query_context; - std::shared_ptr session_context = nullptr; - std::chrono::steady_clock::duration session_timeout; + inline bool hasDelayed() const + { + return out_maybe_delayed_and_compressed != out_maybe_compressed; + } }; IServer & server; @@ -56,16 +67,46 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - size_t getKeepAliveTimeout() { return server.config().getUInt("keep_alive_timeout", 10); } - - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - - void processQuery(Context & context, HTTPRequest & request, HTMLForm & params, HTTPResponse & response, HTTPResponseBufferPtr & response_out); + /// Also initializes 'used_output'. + void processQuery( + Poco::Net::HTTPServerRequest & request, + HTMLForm & params, + Poco::Net::HTTPServerResponse & response, + Output & used_output); void trySendExceptionToClient( - const std::string & message, int exception_code, HTTPRequest & request, - HTTPResponse & response, HTTPResponseBufferPtr response_out, bool compression); + const std::string & s, + int exception_code, + Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response, + Output & used_output); + static void pushDelayedResults(Output & used_output); +}; + +class DynamicQueryHandler : public HTTPHandler +{ +private: + std::string param_name; +public: + explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query"); + + std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) override; + + bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; +}; + +class PredefineQueryHandler : public HTTPHandler +{ +private: + NameSet receive_params; + std::string predefine_query; +public: + explicit PredefineQueryHandler(IServer & server, const NameSet & receive_params, const std::string & predefine_query_); + + std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) override; + + bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; }; } diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index b2c07075d54..8dee745d0e3 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -1,9 +1,21 @@ #include "HTTPHandlerFactory.h" +#include +#include +#include +#include + +#include "HTTPHandler.h" #include "NotFoundHandler.h" -#include "HTTPRequestHandler/HTTPRootRequestHandler.h" -#include "HTTPRequestHandler/HTTPPingRequestHandler.h" -#include "HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h" +#include "StaticRequestHandler.h" +#include "ReplicasStatusHandler.h" +#include "InterserverIOHTTPHandler.h" + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif namespace DB @@ -11,130 +23,128 @@ namespace DB namespace ErrorCodes { - extern const int SYNTAX_ERROR; - extern const int UNKNOWN_HTTP_HANDLER_TYPE; - extern const int EMPTY_HTTP_HANDLER_IN_CONFIG; + extern const int CANNOT_COMPILE_REGEXP; + extern const int NO_ELEMENTS_IN_CONFIG; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -InterserverIOHTTPHandlerFactory::InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_) +HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) : server(server_), log(&Logger::get(name_)), name(name_) { } -Poco::Net::HTTPRequestHandler * InterserverIOHTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler(const Poco::Net::HTTPServerRequest & request) // override { LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); + << "Method: " + << request.getMethod() + << ", Address: " + << request.clientAddress().toString() + << ", User-Agent: " + << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) + << ", Content Type: " << request.getContentType() + << ", Transfer Encoding: " << request.getTransferEncoding()); - const auto & uri = request.getURI(); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) + for (auto & handler_factory : child_handler_factories) { - if (uri == "/") - return new HTTPRootRequestHandler(server); - if (uri == "/ping") - return new HTTPPingRequestHandler(server); - else if (startsWith(uri, "/replicas_status")) - return new HTTPReplicasStatusRequestHandler(server.context()); + auto handler = handler_factory->createRequestHandler(request); + if (handler != nullptr) + return handler; } - if (uri.find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - return new InterserverIOHTTPHandler(server); - } - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return new NotFoundHandler( - "Use / or /ping for health checks.\n" - "Or /replicas_status for more sophisticated health checks.\n" - "Send queries from your program with POST method or GET /?query=...\n\n" - " Use clickhouse-client:\n\n" - " For interactive data analysis:\n" - " clickhouse-client\n\n" - " For batch query processing:\n" - " clickhouse-client --query='SELECT 1' > result\n" - " clickhouse-client < query > result" - ); + return new NotFoundHandler; } return nullptr; } -HTTPHandlerFactory::HTTPHandlerFactory(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) +HTTPRequestHandlerFactoryMain::~HTTPRequestHandlerFactoryMain() { - updateHTTPHandlersCreator(server.config()); - - if (handlers_creator.empty()) - throw Exception("The HTTPHandlers does not exist in the config.xml", ErrorCodes::EMPTY_HTTP_HANDLER_IN_CONFIG); + while (!child_handler_factories.empty()) + delete child_handler_factories.back(), child_handler_factories.pop_back(); } -Poco::Net::HTTPRequestHandler * HTTPHandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) +HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory) { - LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " << request.getMethod() - << ", Address: " << request.clientAddress().toString() - << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") - << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) - << ", Content Type: " << request.getContentType() - << ", Transfer Encoding: " << request.getTransferEncoding()); - - for (const auto & [matcher, creator] : handlers_creator) - { - if (matcher(request)) - return creator(); - } - - return new NotFoundHandler(no_handler_description); + child_handler_factories.emplace_back(child_factory); + return this; } -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPingHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer &, const String &); -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer &, const String &); - -HTTPHandlerCreator createRootHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPingHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer &, const String &); -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer &, const String &); - -void HTTPHandlerFactory::updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key) +static inline auto createHandlersFactoryFromConfig(IServer & server, const std::string & name, const String & prefix) { - Poco::Util::AbstractConfiguration::Keys http_handlers_item_key; - configuration.keys(key, http_handlers_item_key); + auto main_handler_factory = new HTTPRequestHandlerFactoryMain(server, name); - handlers_creator.reserve(http_handlers_item_key.size()); - for (const auto & http_handler_type_name : http_handlers_item_key) + Poco::Util::AbstractConfiguration::Keys keys; + server.config().keys(prefix, keys); + + for (const auto & key : keys) { - if (http_handler_type_name.find('.') != String::npos) - throw Exception("HTTPHandler type name with dots are not supported: '" + http_handler_type_name + "'", ErrorCodes::SYNTAX_ERROR); + if (!startsWith(key, "routing_rule")) + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - const auto & handler_key = key + "." + http_handler_type_name; + const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); - if (startsWith(http_handler_type_name, "root_handler")) - handlers_creator.push_back({createRootHandlerMatcher(server, handler_key), createRootHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "ping_handler")) - handlers_creator.push_back({createPingHandlerMatcher(server, handler_key), createPingHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "dynamic_query_handler")) - handlers_creator.push_back({createDynamicQueryHandlerMatcher(server, handler_key), createDynamicQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "predefined_query_handler")) - handlers_creator.push_back({createPredefinedQueryHandlerMatcher(server, handler_key), createPredefinedQueryHandlerCreator(server, handler_key)}); - else if (startsWith(http_handler_type_name, "replicas_status_handler")) - handlers_creator.push_back({createReplicasStatusHandlerMatcher(server, handler_key), createReplicasStatusHandlerCreator(server, handler_key)}); - else if (http_handler_type_name == "no_handler_description") - no_handler_description = configuration.getString(key + ".no_handler_description"); + if (handler_type == "static") + main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); + else if (handler_type == "dynamic_query_handler") + main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); + else if (handler_type == "predefine_query_handler") + main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); else - throw Exception("Unknown HTTPHandler type name: " + http_handler_type_name, ErrorCodes::UNKNOWN_HTTP_HANDLER_TYPE); + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + return main_handler_factory; +} + +static const auto ping_response_expression = "Ok.\n"; +static const auto root_response_expression = "config://http_server_default_response"; + +static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IServer & server, const std::string & name) +{ + if (server.config().has("routing_rules")) + return createHandlersFactoryFromConfig(server, name, "routing_rules"); + else + { + return (new HTTPRequestHandlerFactoryMain(server, name)) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->attachStrictPath("/")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->attachStrictPath("/ping")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, "query"))->allowPostAndGetParamsRequest()); + /// TODO: +// if (configuration.has("prometheus") && configuration.getInt("prometheus.port", 0) == 0) +// handler_factory->addHandler(async_metrics); } } +static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandlerFactory(IServer & server, const std::string & name) +{ + return (new HTTPRequestHandlerFactoryMain(server, name)) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->attachStrictPath("/")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->attachStrictPath("/ping")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) + ->addHandler((new RoutingRuleHTTPHandlerFactory(server))->allowPostAndGetParamsRequest()); } + +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name) +{ + if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") + return createHTTPHandlerFactory(server, name); + else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") + return createInterserverHTTPHandlerFactory(server, name); + + throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); +} + +} \ No newline at end of file diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index 5add7b619af..15f15db9768 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -1,51 +1,113 @@ #pragma once +#include "IServer.h" +#include +#include #include #include #include -#include -#include -#include "IServer.h" -#include "InterserverIOHTTPHandler.h" - namespace DB { -class InterserverIOHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +/// Handle request using child handlers +class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory { -public: - InterserverIOHTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - private: - IServer & server; - Logger * log; - std::string name; -}; + using TThis = HTTPRequestHandlerFactoryMain; -using HTTPHandlerCreator = std::function; -using HTTPHandlerMatcher = std::function; -using HTTPHandlerMatcherAndCreator = std::pair; -using HTTPHandlersMatcherAndCreator = std::vector; - -class HTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - HTTPHandlerFactory(IServer & server_, const std::string & name_); - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - - void updateHTTPHandlersCreator(Poco::Util::AbstractConfiguration & configuration, const String & key = "http_handlers"); - -private: IServer & server; Logger * log; std::string name; - String no_handler_description; - HTTPHandlersMatcherAndCreator handlers_creator; + std::vector child_handler_factories; +public: + + ~HTTPRequestHandlerFactoryMain(); + + HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); + + TThis * addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory); + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; }; +template +class RoutingRuleHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +{ +public: + using TThis = RoutingRuleHTTPHandlerFactory; + using Filter = std::function; + + template + RoutingRuleHTTPHandlerFactory(TArgs &&... args) + { + creator = [args = std::tuple(std::forward(args) ...)]() + { + return std::apply([&](auto && ... endpoint_args) + { + return new TEndpoint(std::forward(endpoint_args)...); + }, std::move(args)); + }; + } + + TThis * addFilter(Filter cur_filter) + { + Filter prev_filter = filter; + filter = [prev_filter, cur_filter](const auto & request) + { + return prev_filter ? prev_filter(request) && cur_filter(request) : cur_filter(request); + }; + + return this; + } + + TThis * attachStrictPath(const String & strict_path) + { + return addFilter([strict_path](const auto & request) { return request.getURI() == strict_path; }); + } + + TThis * attachNonStrictPath(const String & non_strict_path) + { + return addFilter([non_strict_path](const auto & request) { return startsWith(request.getURI(), non_strict_path); }); + } + + /// Handle GET or HEAD endpoint on specified path + TThis * allowGetAndHeadRequest() + { + return addFilter([](const auto & request) + { + return request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD; + }); + } + + /// Handle POST or GET with params + TThis * allowPostAndGetParamsRequest() + { + return addFilter([](const auto & request) + { + return request.getURI().find('?') != std::string::npos + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; + }); + } + + Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override + { + return filter(request) ? creator() : nullptr; + } + +private: + Filter filter; + std::function creator; +}; + +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name); + +Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix); + +Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix); + +Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix); + } diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h new file mode 100644 index 00000000000..5f794884239 --- /dev/null +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -0,0 +1,98 @@ +#pragma once + +#include "HTTPHandlerFactory.h" + +#include +#include +#include +#include + +#include + +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; +} + +static inline std::string uriPathGetter(const Poco::Net::HTTPServerRequest & request) +{ + const auto & uri = request.getURI(); + const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); + + return std::string(uri.data(), end - uri.data()); +} + +static inline std::function headerGetter(const std::string & header_name) +{ + return [header_name](const Poco::Net::HTTPServerRequest & request) { return request.get(header_name, ""); }; +} + +static inline auto methodsExpressionFilter(const std::string &methods_expression) +{ + Poco::StringTokenizer tokenizer(Poco::toUpper(Poco::trim(methods_expression)), ","); + return [methods = std::vector(tokenizer.begin(), tokenizer.end())](const Poco::Net::HTTPServerRequest & request) + { + return std::count(methods.begin(), methods.end(), request.getMethod()); + }; +} + +template +static inline auto regularExpressionFilter(const std::string & regular_expression, const GetFunction & get) +{ + auto compiled_regex = std::make_shared(regular_expression); + + if (!compiled_regex->ok()) + throw Exception("cannot compile re2: " + regular_expression + " for routing_rule, error: " + compiled_regex->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + return std::make_pair(compiled_regex, [get = std::move(get), compiled_regex](const Poco::Net::HTTPServerRequest & request) + { + const auto & test_content = get(request); + int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(test_content.data(), test_content.size()); + return compiled_regex->Match(input, 0, test_content.size(), re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); + }); +} + +template +static inline std::function expressionFilter(const std::string & expression, const GetFunction & get) +{ + if (startsWith(expression, "regex:")) + return regularExpressionFilter(expression, get).second; + + return [expression, get = std::move(get)](const Poco::Net::HTTPServerRequest & request) { return get(request) == expression; }; +} + +template +static inline Poco::Net::HTTPRequestHandlerFactory * addFiltersFromConfig( + RoutingRuleHTTPHandlerFactory * factory, Poco::Util::AbstractConfiguration & config, const std::string & prefix) +{ + Poco::Util::AbstractConfiguration::Keys filters_type; + config.keys(prefix, filters_type); + + for (const auto & filter_type : filters_type) + { + if (filter_type == "handler") + continue; /// Skip handler config + else if (filter_type == "method") + factory->addFilter(methodsExpressionFilter(config.getString(prefix + "." + filter_type))); + else + factory->addFilter(expressionFilter(config.getString(prefix + "." + filter_type), filter_type == "url" + ? uriPathGetter : headerGetter(filter_type))); + } + + return factory; +} + +} diff --git a/programs/server/HTTPRequestHandler/ExtractorClientInfo.h b/programs/server/HTTPRequestHandler/ExtractorClientInfo.h deleted file mode 100644 index 76840f3d682..00000000000 --- a/programs/server/HTTPRequestHandler/ExtractorClientInfo.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ExtractorClientInfo -{ -public: - ExtractorClientInfo(ClientInfo & info_) : client_info(info_) {} - - void extract(Poco::Net::HTTPServerRequest & request) - { - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::HTTP; - - /// Query sent through HTTP interface is initial. - client_info.initial_user = client_info.current_user; - client_info.initial_query_id = client_info.current_query_id; - client_info.initial_address = client_info.current_address; - - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - } - -private: - ClientInfo & client_info; -}; - -} diff --git a/programs/server/HTTPRequestHandler/ExtractorContextChange.h b/programs/server/HTTPRequestHandler/ExtractorContextChange.h deleted file mode 100644 index 1d4b4e5f58a..00000000000 --- a/programs/server/HTTPRequestHandler/ExtractorContextChange.h +++ /dev/null @@ -1,116 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -class ExtractorContextChange -{ -public: - ExtractorContextChange(Context & context_, bool settings_may_in_post_) : context(context_), settings_may_in_post(settings_may_in_post_) {} - - static const NameSet & getReservedParamNames() - { - static const NameSet reserved_param_names{ - "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check" - }; - - return reserved_param_names; - } - - static std::function reservedParamSuffixesFilter(bool reserved) - { - if (!reserved) - return [&](const String &) { return false; }; - - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - return [&](const String & param_name) - { - if (endsWith(param_name, "_format")) - return true; - else if (endsWith(param_name, "_types")) - return true; - else if (endsWith(param_name, "_structure")) - return true; - - return false; - }; - } - - void extract(Poco::Net::HTTPServerRequest & request, HTMLForm & params) - { - bool is_multipart_data = startsWith(request.getContentType().data(), "multipart/form-data"); - - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - becomeReadonlyIfNeeded(request); - changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - - if (is_multipart_data || settings_may_in_post) - { - ExternalTablesHandler handler(context, params); - params.load(request, request.stream(), handler); - - /// We use the `Post Request Body Settings` to override the `Qeruy String Param settings` - if (settings_may_in_post) - changeSettingsFromParams(params, reservedParamSuffixesFilter(is_multipart_data)); - } - } - -private: - Context & context; - bool settings_may_in_post; - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - /// Only readonly queries are allowed for HTTP GET requests. - void becomeReadonlyIfNeeded(Poco::Net::HTTPServerRequest & request) - { - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - { - Settings & settings = context.getSettingsRef(); - - if (settings.readonly == 0) - settings.readonly = 2; - } - } - - - void changeSettingsFromParams(HTMLForm & params, const std::function & reserved_param_suffixes) - { - SettingsChanges settings_changes; - const auto & reserved_param_names = getReservedParamNames(); - - for (const auto & [name, value] : params) - { - if (name == "database") - context.setCurrentDatabase(value); - else if (name == "default_format") - context.setDefaultFormat(value); - else if (!reserved_param_names.count(name) && !reserved_param_suffixes(name)) - { - if (Settings::findIndex(name) != Settings::npos) - settings_changes.push_back({name, value}); - } - } - - /// For external data we also want settings - context.checkSettingsConstraints(settings_changes); - context.applySettingsChanges(settings_changes); - } -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp deleted file mode 100644 index 8bfa351aab2..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.cpp +++ /dev/null @@ -1,154 +0,0 @@ -#include "HTTPExceptionHandler.h" - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int READONLY; - extern const int UNKNOWN_COMPRESSION_METHOD; - - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_OPEN_FILE; - - extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int UNEXPECTED_AST_STRUCTURE; - - extern const int SYNTAX_ERROR; - - extern const int INCORRECT_DATA; - extern const int TYPE_MISMATCH; - - extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_FUNCTION; - extern const int UNKNOWN_IDENTIFIER; - extern const int UNKNOWN_TYPE; - extern const int UNKNOWN_STORAGE; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_SETTING; - extern const int UNKNOWN_DIRECTION_OF_SORTING; - extern const int UNKNOWN_AGGREGATE_FUNCTION; - extern const int UNKNOWN_FORMAT; - extern const int UNKNOWN_DATABASE_ENGINE; - extern const int UNKNOWN_TYPE_OF_QUERY; - - extern const int QUERY_IS_TOO_LARGE; - - extern const int NOT_IMPLEMENTED; - extern const int SOCKET_TIMEOUT; - - extern const int UNKNOWN_USER; - extern const int WRONG_PASSWORD; - extern const int REQUIRED_PASSWORD; - - extern const int INVALID_SESSION_TIMEOUT; - extern const int HTTP_LENGTH_REQUIRED; -} - -static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) -{ - using namespace Poco::Net; - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - return HTTPResponse::HTTP_UNAUTHORIZED; - else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || - exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || - exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_DATE || - exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || - exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || - - exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || - exception_code == ErrorCodes::TOO_DEEP_AST || - exception_code == ErrorCodes::TOO_BIG_AST || - exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || - - exception_code == ErrorCodes::SYNTAX_ERROR || - - exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH) - return HTTPResponse::HTTP_BAD_REQUEST; - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || - exception_code == ErrorCodes::UNKNOWN_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || - exception_code == ErrorCodes::UNKNOWN_TYPE || - exception_code == ErrorCodes::UNKNOWN_STORAGE || - exception_code == ErrorCodes::UNKNOWN_DATABASE || - exception_code == ErrorCodes::UNKNOWN_SETTING || - exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || - exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_FORMAT || - exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || - - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) - return HTTPResponse::HTTP_NOT_FOUND; - else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) - return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; - else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) - return HTTPResponse::HTTP_NOT_IMPLEMENTED; - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || - exception_code == ErrorCodes::CANNOT_OPEN_FILE) - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - return HTTPResponse::HTTP_LENGTH_REQUIRED; - - return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; -} - - -void HTTPExceptionHandler::handle( - const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, - bool compression, Poco::Logger * log) -{ - try - { - /// 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::max()); - } - - if (exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::REQUIRED_PASSWORD) - { - response.requireAuthentication("ClickHouse server HTTP API"); - } - else - { - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); - } - - if (!response_out && !response.sent()) - response.send() << message << std::endl; - else - { - HTTPOutputStreams output_streams(response_out, compression); - - writeString(message, *output_streams.out_maybe_compressed); - writeChar('\n', *output_streams.out_maybe_compressed); - - output_streams.finalize(); - } - } - catch (...) - { - tryLogCurrentException(log, "Cannot send exception to client"); - } -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h b/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h deleted file mode 100644 index d2a5383ed4a..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPExceptionHandler.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class HTTPExceptionHandler -{ -public: - static void handle(const std::string & message, int exception_code, Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, std::shared_ptr response_out, bool compression, - Poco::Logger * log); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp deleted file mode 100644 index 292f98ba0eb..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include "HTTPPingRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void HTTPPingRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - catch (...) - { - tryLogCurrentException("HTTPPingRequestHandler"); - } -} - -HTTPHandlerMatcher createPingHandlerMatcher(IServer & server, const String & key) -{ - const auto & path = server.config().getString(key, "/ping"); - - return [&, path = path](const Poco::Net::HTTPServerRequest & request) - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && - request.getURI() == path; - }; -} - -HTTPHandlerCreator createPingHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPPingRequestHandler(server); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h deleted file mode 100644 index 7d524dd638d..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPPingRequestHandler.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include "../IServer.h" -#include "../HTTPHandlerFactory.h" - -#include - - -namespace DB -{ - -/// Response with "Ok.\n". Used for availability checks. -class HTTPPingRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPPingRequestHandler(const IServer & server_) : server(server_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp deleted file mode 100644 index 723ff85abe3..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.cpp +++ /dev/null @@ -1,175 +0,0 @@ -#include "HTTPQueryRequestHandler.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "ExtractorClientInfo.h" -#include "ExtractorContextChange.h" -#include "HTTPQueryRequestHandlerMatcherAndCreator.h" -#include "HTTPSessionContextHolder.h" -#include "HTTPExceptionHandler.h" - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int HTTP_LENGTH_REQUIRED; -} - -template -HTTPQueryRequestHandler::HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_) - : server(server_), log(&Logger::get("HTTPQueryRequestHandler")), extractor(extractor_) -{ - server_display_name = server.config().getString("display_name", getFQDNOrHostName()); -} - -template -void HTTPQueryRequestHandler::processQuery( - Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, - Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out) -{ - ExtractorClientInfo{context.getClientInfo()}.extract(request); - ExtractorContextChange{context, extractor.loadSettingsFromPost()}.extract(request, params); - - HTTPInputStreams input_streams{context, request, params}; - HTTPOutputStreams output_streams = HTTPOutputStreams(response_out, context, request, params); - - const auto & queries = extractor.extract(context, request, params); - - for (const auto & [execute_query, not_touch_post] : queries) - { - ReadBufferPtr temp_query_buf; - ReadBufferPtr execute_query_buf = std::make_shared(execute_query); - - if (not_touch_post && !startsWith(request.getContentType().data(), "multipart/form-data")) - { - temp_query_buf = execute_query_buf; /// we create a temporary reference for not to be destroyed - execute_query_buf = std::make_unique(*temp_query_buf, *input_streams.in_maybe_internal_compressed); - } - - executeQuery( - *execute_query_buf, *output_streams.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, - context, [&response] (const String & content_type) { response.setContentType(content_type); }, - [&response] (const String & current_query_id) { response.add("X-ClickHouse-Query-Id", current_query_id); } - ); - } - - /// Send HTTP headers with code 200 if no exception happened and the data is still not sent to the client. - output_streams.finalize(); -} - -template -void HTTPQueryRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) -{ - setThreadName("HTTPHandler"); - ThreadStatus thread_status; - - /// In case of exception, send stack trace to client. - HTTPResponseBufferPtr response_out; - bool with_stacktrace = false, internal_compression = false; - - try - { - response_out = createResponseOut(request, response); - response.set("Content-Type", "text/plain; charset=UTF-8"); - response.set("X-ClickHouse-Server-Display-Name", server_display_name); - - /// For keep-alive to work. - if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1) - response.setChunkedTransferEncoding(true); - - HTMLForm params(request); - with_stacktrace = params.getParsed("stacktrace", false); - internal_compression = params.getParsed("compress", false); - - /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && !request.hasContentLength()) - throw Exception("There is neither Transfer-Encoding header nor Content-Length header", ErrorCodes::HTTP_LENGTH_REQUIRED); - - { - Context query_context = server.context(); - CurrentThread::QueryScope query_scope(query_context); - - HTTPSessionContextHolder holder{query_context, request, params}; - processQuery(holder.query_context, request, params, response, response_out); - LOG_INFO(log, "Done processing query"); - } - } - catch (...) - { - tryLogCurrentException(log); - - /** If exception is received from remote server, then stack trace is embedded in message. - * If exception is thrown on local server, then stack trace is in separate field. - */ - int exception_code = getCurrentExceptionCode(); - std::string exception_message = getCurrentExceptionMessage(with_stacktrace, true); - HTTPExceptionHandler::handle(exception_message, exception_code, request, response, response_out, internal_compression, log); - } -} - -template -HTTPResponseBufferPtr HTTPQueryRequestHandler::createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response) -{ - size_t keep_alive = server.config().getUInt("keep_alive_timeout", 10); - /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). - String http_response_compression_methods = request.get("Accept-Encoding", ""); - - if (!http_response_compression_methods.empty()) - { - /// 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("gzip")) - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Gzip); - else if (std::string::npos != http_response_compression_methods.find("deflate")) - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Zlib); -#if USE_BROTLI - else if (http_response_compression_methods == "br") - return std::make_shared(request, response, keep_alive, true, CompressionMethod::Brotli); -#endif - } - - return std::make_shared(request, response, keep_alive, false, CompressionMethod{}); -} - - -template class HTTPQueryRequestHandler; -template class HTTPQueryRequestHandler; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h deleted file mode 100644 index 5303b0cda89..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandler.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include "../IServer.h" - -#include - -#include -#include - -#include - - -namespace CurrentMetrics -{ - extern const Metric HTTPConnection; -} - -namespace Poco { class Logger; } - -namespace DB -{ - -template -class HTTPQueryRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPQueryRequestHandler(const IServer & server_, const QueryParamExtractor & extractor_); - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; - Poco::Logger * log; - QueryParamExtractor extractor; - - /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. - String server_display_name; - - CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - - HTTPResponseBufferPtr createResponseOut(HTTPServerRequest & request, HTTPServerResponse & response); - - void processQuery( - Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params, - Poco::Net::HTTPServerResponse & response, HTTPResponseBufferPtr & response_out); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp deleted file mode 100644 index 81f25f0bd08..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.cpp +++ /dev/null @@ -1,339 +0,0 @@ -#include "HTTPQueryRequestHandlerMatcherAndCreator.h" - -#include "../HTTPHandlerFactory.h" -#include "HTTPQueryRequestHandler.h" -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int EMPTY_PREDEFINED_QUERY; - extern const int CANNOT_COMPILE_REGEXP; - extern const int UNKNOWN_QUERY_PARAMETER; - extern const int DUPLICATE_CAPTURE_QUERY_PARAM; - extern const int ILLEGAL_HTTP_HANDLER_PARAM_NAME; - extern const int TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY; -} - -ExtractorDynamicQueryParameters::ExtractorDynamicQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) - : url_regex(url_regex_), headers_regex(headers_regex_) -{ - dynamic_param_name = configuration.getString(key + ".query_param_name", "query"); - - NameSet extracted_names; - - if (url_regex) - { - for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) - { - if (startsWith(capturing_name, "param_")) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_url[capturing_name] = capturing_index; - } - } - } - - if (!headers_regex.empty()) - { - for (const auto & [header_name, header_regex] : headers_regex) - { - for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) - { - if (startsWith(capturing_name, "param_")) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_headers[header_name][capturing_name] = capturing_index; - } - } - } - } -} - -template -void extractParamWithRegex(Context & context, const RegexRule & regex, const std::map & extract_params, const String & value) -{ - if (value.empty()) - return; - - int num_captures = regex->NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(value.data(), value.size()); - - if (regex->Match(input, 0, value.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - { - for (const auto & [capturing_name, capturing_index] : extract_params) - { - const auto & capturing_value = matches[capturing_index]; - - if (capturing_value.data()) - { - String param_name = capturing_name; - if constexpr (remove_prefix_for_param) - { - const static size_t prefix_size = strlen("param_"); - param_name = capturing_name.substr(prefix_size); - } - - context.setQueryParameter(param_name, String(capturing_value.data(), capturing_value.size())); - } - } - } -} - -ExtractRes ExtractorDynamicQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - if (!extract_from_url.empty()) - extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); - - - if (!extract_from_headers.empty()) - for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); - - String extracted_query_from_params; - const static size_t prefix_size = strlen("param_"); - - for (const auto & [param_name, param_value] : params) - { - if (param_name == dynamic_param_name) - extracted_query_from_params += param_value; - else if (startsWith(param_name, "param_")) - context.setQueryParameter(param_name.substr(prefix_size), param_value); - } - - if (!extracted_query_from_params.empty()) - extracted_query_from_params += "\n"; - - return {{extracted_query_from_params, true}}; -} - -ExtractorPredefinedQueryParameters::ExtractorPredefinedQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_) - : url_regex(url_regex_), headers_regex(headers_regex_) -{ - Poco::Util::AbstractConfiguration::Keys queries_key; - configuration.keys(key + ".queries", queries_key); - - if (queries_key.empty()) - throw Exception("There must be at least one predefined query in the predefined HTTPHandler.", ErrorCodes::EMPTY_PREDEFINED_QUERY); - - for (const auto & query_key : queries_key) - { - const auto & predefine_query = configuration.getString(key + ".queries." + query_key); - - const char * query_begin = predefine_query.data(); - const char * query_end = predefine_query.data() + predefine_query.size(); - - ParserQuery parser(query_end, false); - ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "", 0); - QueryParameterVisitor{queries_names}.visit(extract_query_ast); - - bool is_insert_query = extract_query_ast->as(); - - if (has_insert_query && is_insert_query) - throw Exception("Too many insert queries in predefined queries.", ErrorCodes::TOO_MANY_INSERT_QUERY_WITH_PREDEFINED_QUERY); - - has_insert_query |= is_insert_query; - predefine_queries.push_back({predefine_query, is_insert_query}); - } - - const auto & reserved_params_name = ExtractorContextChange::getReservedParamNames(); - for (const auto & predefine_query_name : queries_names) - { - if (Settings::findIndex(predefine_query_name) != Settings::npos || reserved_params_name.count(predefine_query_name)) - throw Exception("Illegal http_handler param name '" + predefine_query_name + - "', Because it's reserved name or Settings name", ErrorCodes::ILLEGAL_HTTP_HANDLER_PARAM_NAME); - } - - NameSet extracted_names; - - if (url_regex) - { - for (const auto & [capturing_name, capturing_index] : url_regex->NamedCapturingGroups()) - { - if (queries_names.count(capturing_name)) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_url[capturing_name] = capturing_index; - } - } - } - - if (!headers_regex.empty()) - { - for (const auto & [header_name, header_regex] : headers_regex) - { - for (const auto & [capturing_name, capturing_index] : header_regex->NamedCapturingGroups()) - { - if (queries_names.count(capturing_name)) - { - if (extracted_names.count(capturing_name)) - throw Exception("Duplicate capture query parameter '" + capturing_name + "'", ErrorCodes::DUPLICATE_CAPTURE_QUERY_PARAM); - - extracted_names.emplace(capturing_name); - extract_from_headers[header_name][capturing_name] = capturing_index; - } - } - } - } -} - -ExtractRes ExtractorPredefinedQueryParameters::extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - if (!extract_from_url.empty()) - extractParamWithRegex(context, url_regex, extract_from_url, Poco::URI{request.getURI()}.getPath()); - - if (!extract_from_headers.empty()) - for (const auto & [header_name, extract_params] : extract_from_headers) - extractParamWithRegex(context, headers_regex.at(header_name), extract_params, request.get(header_name, "")); - - for (const auto & param : params) - if (queries_names.count(param.first)) - context.setQueryParameter(param.first, param.second); - - return predefine_queries; -} - -RegexRule HTTPQueryRequestHandlerMatcherAndCreator::createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - if (!configuration.has(key)) - return {}; - - const auto & regex_str = configuration.getString(key); - const auto & url_regex_rule = std::make_shared(regex_str); - - if (!url_regex_rule->ok()) - throw Exception("cannot compile re2: " + regex_str + " for HTTPHandler url, error: " + url_regex_rule->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); - - return url_regex_rule; -} - -HeadersRegexRule HTTPQueryRequestHandlerMatcherAndCreator::createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key) -{ - if (!configuration.has(key)) - return {}; - - Poco::Util::AbstractConfiguration::Keys headers_names; - configuration.keys(key, headers_names); - - HeadersRegexRule headers_regex_rule; - for (const auto & header_name : headers_names) - { - if (headers_regex_rule.count(header_name)) - throw Exception("Duplicate header match declaration '" + header_name + "'", ErrorCodes::LOGICAL_ERROR); - - headers_regex_rule[header_name] = createRegexRule(configuration, key + "." + header_name); - } - - return headers_regex_rule; -} - -size_t findFirstMissingMatchPos(const re2_st::RE2 & regex_rule, const String & match_content) -{ - int num_captures = regex_rule.NumberOfCapturingGroups() + 1; - - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(match_content.data(), match_content.size()); - if (regex_rule.Match(input, 0, match_content.size(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) - return matches[0].size(); - - return size_t(0); -} - -HTTPHandlerMatcher HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher( - const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) -{ - return [method = Poco::toLower(method), url_rule = url_rule, headers_rule = headers_rule](const Poco::Net::HTTPServerRequest & request) - { - if (!method.empty() && Poco::toLower(request.getMethod()) != method) - return false; - - if (url_rule) - { - Poco::URI uri(request.getURI()); - const auto & request_uri = uri.getPath(); - size_t first_missing_pos = findFirstMissingMatchPos(*url_rule, request_uri); - - const char * url_end = request_uri.data() + request_uri.size(); - const char * first_missing = request_uri.data() + first_missing_pos; - - if (first_missing != url_end && *first_missing == '/') - ++first_missing; - - if (first_missing != url_end && *first_missing != '?') - return false; /// Not full matched - } - - if (!headers_rule.empty()) - { - for (const auto & [header_name, header_rule] : headers_rule) - { - if (!request.has(header_name)) - return false; - - const String & header_value = request.get(header_name); - if (header_value.size() != findFirstMissingMatchPos(*header_rule, header_value)) - return false; - } - } - - return true; - }; -} - -HTTPHandlerMatcher createDynamicQueryHandlerMatcher(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, - HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); -} - - -HTTPHandlerMatcher createPredefinedQueryHandlerMatcher(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule(server.config(), key, - HTTPQueryRequestHandlerMatcherAndCreator::createHandlerMatcher); -} - -HTTPHandlerCreator createDynamicQueryHandlerCreator(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( - server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) - { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); - - return [&, query_extract = extract]() - { - return new HTTPQueryRequestHandler(server, *query_extract); - }; - }); -} - -HTTPHandlerCreator createPredefinedQueryHandlerCreator(IServer & server, const String & key) -{ - return HTTPQueryRequestHandlerMatcherAndCreator::invokeWithParsedRegexRule( - server.config(), key, [&](const String &, const RegexRule & url_rule, const HeadersRegexRule & headers_rule) - { - const auto & extract = std::make_shared(server.config(), key, url_rule, headers_rule); - - return [&, query_extract = extract]() - { - return new HTTPQueryRequestHandler(server, *query_extract); - }; - }); -} -} diff --git a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h b/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h deleted file mode 100644 index 0d7bc8356bb..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPQueryRequestHandlerMatcherAndCreator.h +++ /dev/null @@ -1,92 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include "ExtractorContextChange.h" -#include "../HTTPHandlerFactory.h" - -#if USE_RE2_ST -#include -#else -#define re2_st re2 -#endif - -namespace DB -{ - -using RegexRule = std::shared_ptr; -using HeadersRegexRule = std::map; -using ExtractRes = std::vector>; - -class ExtractorDynamicQueryParameters -{ -public: - ExtractorDynamicQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, - const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ - ); - - bool loadSettingsFromPost() const { return false; } - - ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - -private: - const RegexRule url_regex; - const HeadersRegexRule headers_regex; - - String dynamic_param_name; - std::map extract_from_url; - std::map> extract_from_headers; -}; - -class ExtractorPredefinedQueryParameters -{ -public: - ExtractorPredefinedQueryParameters( - Poco::Util::AbstractConfiguration & configuration, const String & key, - const RegexRule & url_regex_, const HeadersRegexRule & headers_regex_ - ); - - bool loadSettingsFromPost() const { return !has_insert_query; } - - ExtractRes extract(Context & context, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - -private: - const RegexRule url_regex; - const HeadersRegexRule headers_regex; - - NameSet queries_names; - bool has_insert_query{false}; - ExtractRes predefine_queries; - std::map extract_from_url; - std::map> extract_from_headers; -}; - -class HTTPQueryRequestHandlerMatcherAndCreator -{ -public: - template - static auto invokeWithParsedRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key, const NestedFunction & fun) - { - return fun(configuration.getString(key + ".method", ""), createRegexRule(configuration, key + ".url"), - createHeadersRegexRule(configuration, key + ".headers")); - } - - static HTTPHandlerMatcher createHandlerMatcher(const String & method, const RegexRule & url_rule, const HeadersRegexRule & headers_rule); - -private: - static RegexRule createRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); - - static HeadersRegexRule createHeadersRegexRule(Poco::Util::AbstractConfiguration & configuration, const String & key); -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp deleted file mode 100644 index ea70abbcc6f..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.cpp +++ /dev/null @@ -1,118 +0,0 @@ -#include "HTTPReplicasStatusRequestHandler.h" - -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - - -HTTPReplicasStatusRequestHandler::HTTPReplicasStatusRequestHandler(Context & context_) - : context(context_) -{ -} - -void HTTPReplicasStatusRequestHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) -{ - try - { - HTMLForm params(request); - - /// Even if lag is small, output detailed information about the lag. - bool verbose = params.get("verbose", "") == "1"; - - const MergeTreeSettings & settings = context.getMergeTreeSettings(); - - bool ok = true; - std::stringstream message; - - auto databases = context.getDatabases(); - - /// Iterate through all the replicated tables. - for (const auto & db : databases) - { - /// Lazy database can not contain replicated tables - if (db.second->getEngineName() == "Lazy") - continue; - - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) - { - auto & table = iterator->table(); - StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); - - if (!table_replicated) - continue; - - time_t absolute_delay = 0; - time_t relative_delay = 0; - - table_replicated->getReplicaDelays(absolute_delay, relative_delay); - - if ((settings.min_absolute_delay_to_close && absolute_delay >= static_cast(settings.min_absolute_delay_to_close)) - || (settings.min_relative_delay_to_close && relative_delay >= static_cast(settings.min_relative_delay_to_close))) - ok = false; - - message << backQuoteIfNeed(db.first) << "." << backQuoteIfNeed(iterator->name()) - << ":\tAbsolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".\n"; - } - } - - const auto & config = context.getConfigRef(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - if (ok && !verbose) - { - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - else - { - response.send() << message.rdbuf(); - } - } - catch (...) - { - tryLogCurrentException("HTTPReplicasStatusRequestHandler"); - - try - { - response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); - - if (!response.sent()) - { - /// We have not sent anything yet and we don't even know if we need to compress response. - response.send() << getCurrentExceptionMessage(false) << std::endl; - } - } - catch (...) - { - LOG_ERROR((&Logger::get("HTTPReplicasStatusRequestHandler")), "Cannot send exception to client"); - } - } -} - -HTTPHandlerMatcher createReplicasStatusHandlerMatcher(IServer & server, const String & key) -{ - const auto & prefix = server.config().getString(key, "/replicas_status"); - - return [&, prefix = prefix](const Poco::Net::HTTPServerRequest & request) - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) && - startsWith(request.getURI(), prefix); - }; -} - -HTTPHandlerCreator createReplicasStatusHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPReplicasStatusRequestHandler(server.context()); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h deleted file mode 100644 index 277e59eb02c..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPReplicasStatusRequestHandler.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include "../HTTPHandlerFactory.h" -#include - - -namespace DB -{ - -class Context; - -/// Replies "Ok.\n" if all replicas on this server don't lag too much. Otherwise output lag information. -class HTTPReplicasStatusRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPReplicasStatusRequestHandler(Context & context_); - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - Context & context; -}; - - -} diff --git a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp deleted file mode 100644 index 032b51d5b7a..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.cpp +++ /dev/null @@ -1,45 +0,0 @@ -#include "HTTPRootRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void HTTPRootRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - response.setContentType("text/html; charset=UTF-8"); - - const std::string data = config.getString("http_server_default_response", "Ok.\n"); - response.sendBuffer(data.data(), data.size()); - } - catch (...) - { - tryLogCurrentException("HTTPRootRequestHandler"); - } -} - -HTTPHandlerMatcher createRootHandlerMatcher(IServer &, const String &) -{ - return [&](const Poco::Net::HTTPServerRequest & request) -> bool - { - return (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD) - && request.getURI() == "/"; - }; -} - -HTTPHandlerCreator createRootHandlerCreator(IServer & server, const String &) -{ - return [&]() { return new HTTPRootRequestHandler(server); }; -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h b/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h deleted file mode 100644 index 136f3292385..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPRootRequestHandler.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include "../IServer.h" -#include "../HTTPHandlerFactory.h" - -#include - - -namespace DB -{ - -/// Response with custom string. Can be used for browser. -class HTTPRootRequestHandler : public Poco::Net::HTTPRequestHandler -{ -public: - explicit HTTPRootRequestHandler(const IServer & server_) : server(server_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const IServer & server; -}; - -} diff --git a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp deleted file mode 100644 index 840044634aa..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.cpp +++ /dev/null @@ -1,105 +0,0 @@ -#include "HTTPSessionContextHolder.h" -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; -} - -static std::chrono::steady_clock::duration parseSessionTimeout( - const Poco::Util::AbstractConfiguration & config, - const HTMLForm & params) -{ - unsigned session_timeout = config.getInt("default_session_timeout", 60); - - if (params.has("session_timeout")) - { - unsigned max_session_timeout = config.getUInt("max_session_timeout", 3600); - std::string session_timeout_str = params.get("session_timeout"); - - ReadBufferFromString buf(session_timeout_str); - if (!tryReadIntText(session_timeout, buf) || !buf.eof()) - throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); - - if (session_timeout > max_session_timeout) - throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) - + ". Maximum session timeout could be modified in configuration file.", - ErrorCodes::INVALID_SESSION_TIMEOUT); - } - - return std::chrono::seconds(session_timeout); -} - -HTTPSessionContextHolder::~HTTPSessionContextHolder() -{ - if (session_context) - session_context->releaseSession(session_id, session_timeout); -} - -void HTTPSessionContextHolder::authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params) -{ - auto user = request.get("X-ClickHouse-User", ""); - auto password = request.get("X-ClickHouse-Key", ""); - auto quota_key = request.get("X-ClickHouse-Quota", ""); - - if (user.empty() && password.empty() && quota_key.empty()) - { - /// User name and password can be passed using query parameters - /// or using HTTP Basic auth (both methods are insecure). - if (request.hasCredentials()) - { - Poco::Net::HTTPBasicCredentials credentials(request); - - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else - { - user = params.get("user", "default"); - password = params.get("password", ""); - } - - quota_key = params.get("quota_key", ""); - } - else - { - /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() - || params.has("user") - || params.has("password") - || params.has("quota_key")) - { - throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); - } - } - - std::string query_id = params.get("query_id", ""); - query_context.setUser(user, password, request.clientAddress(), quota_key); - query_context.setCurrentQueryId(query_id); -} - -HTTPSessionContextHolder::HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params) - : query_context(query_context_) -{ - authentication(request, params); - - { - session_id = params.get("session_id", ""); - - if (!session_id.empty()) - { - session_timeout = parseSessionTimeout(query_context.getConfigRef(), params); - session_context = query_context.acquireSession(session_id, session_timeout, params.check("session_check", "1")); - - query_context = *session_context; - query_context.setSessionContext(*session_context); - } - } -} - -} diff --git a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h b/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h deleted file mode 100644 index deadd3910ad..00000000000 --- a/programs/server/HTTPRequestHandler/HTTPSessionContextHolder.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -/// Manage the lifetime of the session context. -struct HTTPSessionContextHolder -{ - ~HTTPSessionContextHolder(); - - void authentication(Poco::Net::HTTPServerRequest & request, HTMLForm & params); - - HTTPSessionContextHolder(Context & query_context_, Poco::Net::HTTPServerRequest & request, HTMLForm & params); - - String session_id; - Context & query_context; - std::shared_ptr session_context = nullptr; - std::chrono::steady_clock::duration session_timeout; -}; - -} diff --git a/programs/server/NotFoundHandler.cpp b/programs/server/NotFoundHandler.cpp index 26df1a1b7d4..f9b51719b0d 100644 --- a/programs/server/NotFoundHandler.cpp +++ b/programs/server/NotFoundHandler.cpp @@ -17,14 +17,16 @@ void NotFoundHandler::handleRequest( try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - - std::stringstream output_description; - output_description << "There is no handle " << request.getURI() << "\n\n"; - - if (!no_handler_description.empty()) - output_description << no_handler_description << "\n"; - - response.send() << output_description.str(); + response.send() << "There is no handle " << request.getURI() << "\n\n" + << "Use / or /ping for health checks.\n" + << "Or /replicas_status for more sophisticated health checks.\n\n" + << "Send queries from your program with POST method or GET /?query=...\n\n" + << "Use clickhouse-client:\n\n" + << "For interactive data analysis:\n" + << " clickhouse-client\n\n" + << "For batch query processing:\n" + << " clickhouse-client --query='SELECT 1' > result\n" + << " clickhouse-client < query > result\n"; } catch (...) { @@ -32,4 +34,4 @@ void NotFoundHandler::handleRequest( } } -} +} \ No newline at end of file diff --git a/programs/server/NotFoundHandler.h b/programs/server/NotFoundHandler.h index caf527726c4..0f76fe9c46e 100644 --- a/programs/server/NotFoundHandler.h +++ b/programs/server/NotFoundHandler.h @@ -10,12 +10,9 @@ namespace DB class NotFoundHandler : public Poco::Net::HTTPRequestHandler { public: - NotFoundHandler(const std::string & no_handler_description_) : no_handler_description(no_handler_description_) {} - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - const std::string no_handler_description; + void handleRequest( + Poco::Net::HTTPServerRequest & request, + Poco::Net::HTTPServerResponse & response) override; }; -} +} \ No newline at end of file diff --git a/programs/server/PingRequestHandler.cpp b/programs/server/PingRequestHandler.cpp deleted file mode 100644 index 141161ef45c..00000000000 --- a/programs/server/PingRequestHandler.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#include "PingRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void PingRequestHandler::handleRequest( - Poco::Net::HTTPServerRequest &, - Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - catch (...) - { - tryLogCurrentException("PingRequestHandler"); - } -} - -} diff --git a/programs/server/PingRequestHandler.h b/programs/server/PingRequestHandler.h deleted file mode 100644 index 3728fb40adb..00000000000 --- a/programs/server/PingRequestHandler.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include "IServer.h" - -#include - - -namespace DB -{ - -/// Response with "Ok.\n". Used for availability checks. -class PingRequestHandler : public Poco::Net::HTTPRequestHandler -{ -private: - IServer & server; - -public: - explicit PingRequestHandler(IServer & server_) : server(server_) - { - } - - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; -}; - -} diff --git a/programs/server/ReplicasStatusHandler.h b/programs/server/ReplicasStatusHandler.h index 479b013cfe8..a32f1ba905f 100644 --- a/programs/server/ReplicasStatusHandler.h +++ b/programs/server/ReplicasStatusHandler.h @@ -17,7 +17,7 @@ private: Context & context; public: - explicit ReplicasStatusHandler(IServer & server); + explicit ReplicasStatusHandler(IServer & server_); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; }; diff --git a/programs/server/RootRequestHandler.cpp b/programs/server/RootRequestHandler.cpp deleted file mode 100644 index 86b57b63208..00000000000 --- a/programs/server/RootRequestHandler.cpp +++ /dev/null @@ -1,33 +0,0 @@ -#include "RootRequestHandler.h" - -#include - -#include - -#include -#include - -namespace DB -{ - -void RootRequestHandler::handleRequest( - Poco::Net::HTTPServerRequest &, - Poco::Net::HTTPServerResponse & response) -{ - try - { - const auto & config = server.config(); - setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10)); - - response.setContentType("text/html; charset=UTF-8"); - - const std::string data = config.getString("http_server_default_response", "Ok.\n"); - response.sendBuffer(data.data(), data.size()); - } - catch (...) - { - tryLogCurrentException("RootRequestHandler"); - } -} - -} diff --git a/programs/server/RootRequestHandler.h b/programs/server/RootRequestHandler.h deleted file mode 100644 index 1b6d53eeeda..00000000000 --- a/programs/server/RootRequestHandler.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include "IServer.h" - -#include - - -namespace DB -{ - -/// Response with custom string. Can be used for browser. -class RootRequestHandler : public Poco::Net::HTTPRequestHandler -{ -private: - IServer & server; - -public: - explicit RootRequestHandler(IServer & server_) : server(server_) - { - } - - void handleRequest( - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response) override; -}; - -} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3490ff6a445..96a14dd3d87 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -767,9 +767,9 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = createDefaultHandlerFatory(*this, "HTTPHandler-factory"); - if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) - handler_factory->addHandler(async_metrics); + auto handler_factory = createHandlerFactory(*this, "HTTPHandler-factory"); +// if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) +// handler_factory->addHandler(async_metrics); servers.emplace_back(std::make_unique( handler_factory, @@ -789,7 +789,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "HTTPSHandler-factory"), + createHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, http_params)); @@ -847,7 +847,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "InterserverIOHTTPHandler-factory"), + createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); @@ -863,7 +863,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createDefaultHandlerFatory(*this, "InterserverIOHTTPHandler-factory"), + createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); @@ -892,22 +892,22 @@ int Server::main(const std::vector & /*args*/) }); /// Prometheus (if defined and not setup yet with http_port) - create_server("prometheus.port", [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socket_bind_listen(socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory"); - handler_factory->addHandler(async_metrics); - servers.emplace_back(std::make_unique( - handler_factory, - server_pool, - socket, - http_params)); - - LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); - }); +// create_server("prometheus.port", [&](UInt16 port) +// { +// Poco::Net::ServerSocket socket; +// auto address = socket_bind_listen(socket, listen_host, port); +// socket.setReceiveTimeout(settings.http_receive_timeout); +// socket.setSendTimeout(settings.http_send_timeout); +// auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory"); +// handler_factory->addHandler(async_metrics); +// servers.emplace_back(std::make_unique( +// handler_factory, +// server_pool, +// socket, +// http_params)); +// +// LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); +// }); } if (servers.empty()) diff --git a/programs/server/StaticRequestHandler.cpp b/programs/server/StaticRequestHandler.cpp new file mode 100644 index 00000000000..730060dea2e --- /dev/null +++ b/programs/server/StaticRequestHandler.cpp @@ -0,0 +1,86 @@ +#include "StaticRequestHandler.h" + +#include "HTTPHandlerFactory.h" +#include "HTTPHandlerRequestFilter.h" + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_FILE_NAME; + extern const int INVALID_CONFIG_PARAMETER; +} + +void StaticRequestHandler::handleRequest(Poco::Net::HTTPServerRequest &, Poco::Net::HTTPServerResponse & response) +{ + try + { + setResponseDefaultHeaders(response, server.config().getUInt("keep_alive_timeout", 10)); + + response.setContentType(content_type); + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTPStatus(status)); + + response.sendBuffer(response_content.data(), response_content.size()); + } + catch (...) + { + tryLogCurrentException("StaticRequestHandler"); + } +} + +StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_) + : server(server_), status(status_), content_type(content_type_) +{ + static const String file_prefix = "file://"; + static const String config_prefix = "config://"; + + if (startsWith(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()); + + if (!Poco::File(file_path).exists()) + throw Exception("Invalid file name for static HTTPHandler." + file_path, ErrorCodes::INCORRECT_FILE_NAME); + + WriteBufferFromOwnString out; + ReadBufferFromFile in(file_path); + copyData(in, out); + response_content = out.str(); + } + else if (startsWith(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", + ErrorCodes::INVALID_CONFIG_PARAMETER); + + response_content = server.config().getString(expression.substr(config_prefix.size(), expression.size() - config_prefix.size()), "Ok.\n"); + } + else + response_content = expression; +} + +Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix) +{ + const auto & status = server.config().getInt(config_prefix + ".handler.status", 200); + const auto & response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); + const auto & response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); + + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( + server, response_content, status, response_content_type), server.config(), config_prefix); +} + +} diff --git a/programs/server/StaticRequestHandler.h b/programs/server/StaticRequestHandler.h new file mode 100644 index 00000000000..bfe7f7a3e7b --- /dev/null +++ b/programs/server/StaticRequestHandler.h @@ -0,0 +1,28 @@ +#pragma once + +#include "IServer.h" + +#include +#include + + +namespace DB +{ + +/// Response with custom string. Can be used for browser. +class StaticRequestHandler : public Poco::Net::HTTPRequestHandler +{ +private: + IServer & server; + + int status; + String content_type; + String response_content; + +public: + StaticRequestHandler(IServer & server, const String & expression, int status_ = 200, const String & content_type_ = "text/html; charset=UTF-8"); + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; +}; + +} diff --git a/src/Common/HTMLForm.h b/src/Common/HTMLForm.h index fa6f31bf5d2..2490d613160 100644 --- a/src/Common/HTMLForm.h +++ b/src/Common/HTMLForm.h @@ -26,12 +26,6 @@ struct HTMLForm : public Poco::Net::HTMLForm readUrl(istr); } - template - bool check(const std::string & key, T check_value) - { - const auto & value = getParsed(key, T()); - return value == check_value; - } template T getParsed(const std::string & key, T default_value) diff --git a/dbms/Interpreters/QueryParameterVisitor.h b/src/Interpreters/QueryParameterVisitor.h similarity index 59% rename from dbms/Interpreters/QueryParameterVisitor.h rename to src/Interpreters/QueryParameterVisitor.h index d765aa00bea..d3e618058c0 100644 --- a/dbms/Interpreters/QueryParameterVisitor.h +++ b/src/Interpreters/QueryParameterVisitor.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -32,4 +34,16 @@ private: } }; +NameSet analyzeReceiveQueryParams(const std::string & query) +{ + NameSet query_params; + const char * query_begin = query.data(); + const char * query_end = query.data() + query.size(); + + ParserQuery parser(query_end, false); + ASTPtr extract_query_ast = parseQuery(parser, query_begin, query_end, "analyzeReceiveQueryParams", 0, 0); + QueryParameterVisitor(query_params).visit(extract_query_ast); + return query_params; +} + } From 318ab3b51eaf3efe702dc8b579304b7dd4c3dfb2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 20 Apr 2020 12:49:17 +0800 Subject: [PATCH 16/23] ISSUES-5436 try fix build failure & pvs & style --- programs/server/HTTPHandlerFactory.cpp | 55 +++++++++++++++----------- programs/server/HTTPHandlerFactory.h | 3 +- programs/server/Server.cpp | 23 ++--------- 3 files changed, 36 insertions(+), 45 deletions(-) diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 8dee745d0e3..55aaa7dc6a2 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -23,13 +23,12 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COMPILE_REGEXP; - extern const int NO_ELEMENTS_IN_CONFIG; + extern const int LOGICAL_ERROR; extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_) - : server(server_), log(&Logger::get(name_)), name(name_) +HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) + : log(&Logger::get(name_)), name(name_) { } @@ -77,29 +76,37 @@ HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler static inline auto createHandlersFactoryFromConfig(IServer & server, const std::string & name, const String & prefix) { - auto main_handler_factory = new HTTPRequestHandlerFactoryMain(server, name); + auto main_handler_factory = new HTTPRequestHandlerFactoryMain(name); - Poco::Util::AbstractConfiguration::Keys keys; - server.config().keys(prefix, keys); - - for (const auto & key : keys) + try { - if (!startsWith(key, "routing_rule")) - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + Poco::Util::AbstractConfiguration::Keys keys; + server.config().keys(prefix, keys); - const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); + for (const auto & key : keys) + { + if (!startsWith(key, "routing_rule")) + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - if (handler_type == "static") - main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); - else if (handler_type == "dynamic_query_handler") - main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); - else if (handler_type == "predefine_query_handler") - main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); - else - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); + + if (handler_type == "static") + main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); + else if (handler_type == "dynamic_query_handler") + main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); + else if (handler_type == "predefine_query_handler") + main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); + else + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + return main_handler_factory; + } + catch (...) + { + delete main_handler_factory; + throw; } - - return main_handler_factory; } static const auto ping_response_expression = "Ok.\n"; @@ -111,7 +118,7 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS return createHandlersFactoryFromConfig(server, name, "routing_rules"); else { - return (new HTTPRequestHandlerFactoryMain(server, name)) + return (new HTTPRequestHandlerFactoryMain(name)) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) @@ -127,7 +134,7 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandlerFactory(IServer & server, const std::string & name) { - return (new HTTPRequestHandlerFactoryMain(server, name)) + return (new HTTPRequestHandlerFactoryMain(name)) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index 15f15db9768..c3686d9f479 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -16,7 +16,6 @@ class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactor private: using TThis = HTTPRequestHandlerFactoryMain; - IServer & server; Logger * log; std::string name; @@ -25,7 +24,7 @@ public: ~HTTPRequestHandlerFactoryMain(); - HTTPRequestHandlerFactoryMain(IServer & server_, const std::string & name_); + HTTPRequestHandlerFactoryMain(const std::string & name_); TThis * addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 96a14dd3d87..5ee7a39d6a0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -767,15 +767,9 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, port); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - auto handler_factory = createHandlerFactory(*this, "HTTPHandler-factory"); -// if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0) -// handler_factory->addHandler(async_metrics); servers.emplace_back(std::make_unique( - handler_factory, - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "HTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for http://" + address.toString()); }); @@ -789,10 +783,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "HTTPSHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for https://" + address.toString()); #else @@ -847,10 +838,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString()); }); @@ -863,10 +851,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params)); + createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString()); #else From 0070f75218ae05feb38bdf41dc994a3da97c3d02 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 21 Apr 2020 19:30:45 +0800 Subject: [PATCH 17/23] ISSUES-5436 fix integration test failure & add test --- .../test_http_handlers_config/test.py | 97 ----------------- .../test_insert_and_select_dynamic/config.xml | 23 ---- .../config.xml | 26 ----- programs/server/HTTPHandler.cpp | 91 ++++++++++++++-- programs/server/HTTPHandler.h | 11 +- programs/server/HTTPHandlerFactory.cpp | 46 ++++---- programs/server/HTTPHandlerFactory.h | 8 +- programs/server/HTTPHandlerRequestFilter.h | 101 +++++++++++------- programs/server/PrometheusRequestHandler.h | 31 +----- programs/server/Server.cpp | 35 +++--- programs/server/StaticRequestHandler.cpp | 8 +- programs/server/config.xml | 59 ++++++---- tests/integration/helpers/cluster.py | 2 +- .../test_http_handlers_config/__init__.py | 0 .../common_configs/common_config.xml | 0 .../common_configs/common_users.xml | 0 .../other_tests_configs/config.xml | 0 .../other_tests_configs/users.xml | 0 .../test_http_handlers_config/test.py | 50 +++++++++ .../test_dynamic_handler/config.xml | 15 +++ .../test_dynamic_handler}/users.xml | 0 .../config.xml | 0 .../users.xml | 0 .../config.xml | 0 .../users.xml | 0 .../test_predefined_handler/config.xml | 15 +++ .../test_predefined_handler}/users.xml | 0 27 files changed, 326 insertions(+), 292 deletions(-) delete mode 100644 dbms/tests/integration/test_http_handlers_config/test.py delete mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml delete mode 100644 dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml rename {dbms/tests => tests}/integration/test_http_handlers_config/__init__.py (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/common_configs/common_config.xml (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/common_configs/common_users.xml (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/other_tests_configs/config.xml (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/other_tests_configs/users.xml (100%) create mode 100644 tests/integration/test_http_handlers_config/test.py create mode 100644 tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml rename {dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic => tests/integration/test_http_handlers_config/test_dynamic_handler}/users.xml (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml (100%) rename {dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined => tests/integration/test_http_handlers_config/test_param_and_settings_dynamic}/users.xml (100%) rename {dbms/tests => tests}/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml (100%) rename {dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic => tests/integration/test_http_handlers_config/test_param_and_settings_predefined}/users.xml (100%) create mode 100644 tests/integration/test_http_handlers_config/test_predefined_handler/config.xml rename {dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined => tests/integration/test_http_handlers_config/test_predefined_handler}/users.xml (100%) diff --git a/dbms/tests/integration/test_http_handlers_config/test.py b/dbms/tests/integration/test_http_handlers_config/test.py deleted file mode 100644 index a996fbdc908..00000000000 --- a/dbms/tests/integration/test_http_handlers_config/test.py +++ /dev/null @@ -1,97 +0,0 @@ -import os -import urllib -import contextlib - -from helpers.cluster import ClickHouseCluster - - -class SimpleCluster: - def close(self): - self.cluster.shutdown() - - def __init__(self, cluster, name, config_dir): - self.cluster = cluster - self.instance = self.add_instance(name, config_dir) - cluster.start() - - def add_instance(self, name, config_dir): - script_path = os.path.dirname(os.path.realpath(__file__)) - return self.cluster.add_instance(name, config_dir=os.path.join(script_path, config_dir), - main_configs=[os.path.join(script_path, 'common_configs', 'common_config.xml')], - user_configs=[os.path.join(script_path, 'common_configs', 'common_users.xml')]) - - -def test_dynamic_query_handler_with_insert_and_select(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_insert_and_select", "test_insert_and_select_dynamic")) as cluster: - insert_data_query = urllib.quote_plus('INSERT INTO test.test VALUES') - select_data_query = urllib.quote_plus('SELECT * FROM test.test ORDER BY id') - create_database_query = urllib.quote_plus('CREATE DATABASE test') - create_test_table_query = 'CREATE TABLE test.test (id UInt8) Engine = Memory' - assert cluster.instance.http_request('create_test_table?max_threads=1&test_create_query_param=' + create_database_query, method='PUT') == '' - assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT', data=create_test_table_query) == '' - assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query + '(1)', method='POST') == '' - assert cluster.instance.http_request('insert_data_to_test?max_threads=1&test_insert_query_param=' + insert_data_query, method='POST', data='(2)') == '' - assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='INSERT INTO test.test VALUES(3)(4)') == '' - assert cluster.instance.http_request('query_data_from_test?max_threads=1&test_select_query_param=' + select_data_query, method='GET') == '1\n2\n3\n4\n' - - -def test_predefined_query_handler_with_insert_and_select(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_insert_and_select", "test_insert_and_select_predefined")) as cluster: - assert cluster.instance.http_request('create_test_table?max_threads=1', method='PUT') == '' - assert cluster.instance.http_request('insert_data_to_test?max_threads=1', method='POST', data='(1)(2)(3)(4)') == '' - assert cluster.instance.http_request('query_data_from_test?max_threads=1', method='GET') == '1\n2\n3\n4\n' - - -def test_dynamic_query_handler_with_params_and_settings(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_params_and_settings", "test_param_and_settings_dynamic")) as cluster: - settings = 'max_threads=1&max_alter_threads=2' - query_param = 'param_name_1=max_threads¶m_name_2=max_alter_threads' - test_query = 'SELECT value FROM system.settings where name = {name_1:String} OR name = {name_2:String}' - quoted_test_query = urllib.quote_plus(test_query) - assert cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + quoted_test_query + '&' + query_param + '&' + settings, method='POST') == '1\n2\n' - assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST', data=test_query) == '1\n2\n' - assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + settings, method='POST', data=query_param) - assert 'Syntax error' in cluster.instance.http_request('post_query_params_and_settings?post_query_param=' + test_query + '&' + query_param, method='POST', data=settings) - - assert cluster.instance.http_request('get_query_params_and_settings?get_query_param=' + quoted_test_query + '&' + query_param + '&' + settings) == '1\n2\n' - assert cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads') == '1\n2\n' - assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?query_param=' + quoted_test_query + '&' + settings) == '1\n2\n' - assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?query_param=' + quoted_test_query + '&' + settings) - assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?query_param=' + quoted_test_query + '&' + query_param + '&' + settings) - - assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\n2\n' - assert cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings + '¶m_name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\n2\n' - assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) - assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?query_param=' + quoted_test_query + '&' + settings) - - -def test_predefined_query_handler_with_params_and_settings(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_params_and_settings", "test_param_and_settings_predefined")) as cluster: - settings = 'max_threads=1&max_alter_threads=2' - query_param = 'name_1=max_threads&name_2=max_alter_threads' - assert cluster.instance.http_request('get_query_params_and_settings?' + query_param + '&' + settings, method='GET') == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('query_param_with_url/123/max_threads/max_alter_threads?' + settings) == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings + '&name_2=max_alter_threads') == '1\nmax_alter_threads\t2\n' - assert '`name_2` is not set' in cluster.instance.http_request('query_param_with_url/123/max_threads?' + settings) - assert 'Duplicate name' in cluster.instance.http_request('query_param_with_url/123/max_threads_dump/max_alter_threads_dump?' + query_param + '&' + settings) - - assert cluster.instance.http_request('post_query_params_and_settings?' + query_param, method='POST', data=settings) == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('post_query_params_and_settings?' + settings, method='POST', data=query_param) == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('post_query_params_and_settings?' + query_param + '&' + settings, method='POST') == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('post_query_params_and_settings', method='POST', data=query_param + '&' + settings) == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads/max_alter_threads'}) == '1\nmax_alter_threads\t2\n' - assert cluster.instance.http_request('test_match_headers?' + settings + '&name_2=max_alter_threads', headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) == '1\nmax_alter_threads\t2\n' - assert '`name_2` is not set' in cluster.instance.http_request('test_match_headers?' + settings, headers={'XXX': 'TEST_HEADER_VALUE', 'PARAMS_XXX': 'max_threads'}) - assert 'There is no handle /test_match_headers' in cluster.instance.http_request('test_match_headers?' + settings) - - -def test_other_configs(): - with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "test_other_configs", "other_tests_configs")) as cluster: - assert cluster.instance.http_request('', method='GET') == 'Ok.\n' - assert cluster.instance.http_request('ping_test', method='GET') == 'Ok.\n' - assert cluster.instance.http_request('404/NOT_FOUND', method='GET') == 'There is no handle /404/NOT_FOUND\n\ntest not found\n' - - cluster.instance.query('CREATE DATABASE test') - cluster.instance.query('CREATE TABLE test.test (id UInt8) Engine = Memory') - assert cluster.instance.http_request('test_one_handler_with_insert_and_select?id=1', method='POST', data='(1)(2)') == '2\n' - assert 'Cannot parse input' in cluster.instance.http_request('test_one_handler_with_insert_and_select', method='POST', data='id=1') diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml deleted file mode 100644 index a4ec94fe62e..00000000000 --- a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/config.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - - - PUT - /create_test_table - test_create_query_param - - - - POST - /insert_data_to_test - test_insert_query_param - - - - GET - /query_data_from_test - test_select_query_param - - - diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml b/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml deleted file mode 100644 index 6319bbf327f..00000000000 --- a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/config.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - PUT - /create_test_table - - CREATE DATABASE test - CREATE TABLE test.test (id UInt8) Engine = Memory - - - - - POST - /insert_data_to_test - INSERT INTO test.test VALUES - - - - GET - /query_data_from_test - SELECT * FROM test.test ORDER BY id - - - diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 98093e94c79..c14f5f4434b 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -569,7 +569,7 @@ void HTTPHandler::processQuery( }); } - customizeContext(context); + customizeContext(request, context); executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) @@ -754,8 +754,11 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request return full_query; } -PredefineQueryHandler::PredefineQueryHandler(IServer & server, const NameSet & receive_params_, const std::string & predefine_query_) +PredefineQueryHandler::PredefineQueryHandler( + IServer & server, const NameSet & receive_params_, const std::string & predefine_query_ + , const std::unordered_map & header_name_with_capture_regex_) : HTTPHandler(server, "PredefineQueryHandler"), receive_params(receive_params_), predefine_query(predefine_query_) + , header_name_with_capture_regex(header_name_with_capture_regex_) { } @@ -770,6 +773,50 @@ bool PredefineQueryHandler::customizeQueryParam(Context & context, const std::st return false; } +void PredefineQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & request, DB::Context & context) +{ + /// If in the configuration file, the handler's header is regex and contains named capture group + /// We will extract regex named capture groups as query parameters + + const auto & set_query_params = [&](const char * begin, const char * end, const std::string & regex) + { + auto compiled_regex = std::make_shared(regex); + + if (!compiled_regex->ok()) + throw Exception("cannot compile re2: " + regex + " for routing_rule, error: " + compiled_regex->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece input(begin, end - begin); + if (compiled_regex->Match(input, 0, end - begin, re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures)) + { + for (const auto & [capturing_name, capturing_index] : compiled_regex->NamedCapturingGroups()) + { + const auto & capturing_value = matches[capturing_index]; + + if (capturing_value.data()) + context.setQueryParameter(capturing_name, String(capturing_value.data(), capturing_value.size())); + } + } + }; + + 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); + } + } +} + std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) { if (unlikely(startsWith(request.getContentType(), "multipart/form-data"))) @@ -784,8 +831,8 @@ std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & reque Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) { - const auto & query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); - return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory(server, query_param_name), server.config(), config_prefix); + std::string query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); } @@ -794,9 +841,41 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & s if (!server.config().has(config_prefix + ".handler.query")) throw Exception("There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - const auto & predefine_query = server.config().getString(config_prefix + ".handler.query"); + std::string predefine_query = server.config().getString(config_prefix + ".handler.query"); + NameSet analyze_receive_params = analyzeReceiveQueryParams(predefine_query); + std::unordered_map type_and_regex; + Poco::Util::AbstractConfiguration::Keys filters_type; + server.config().keys(config_prefix, filters_type); + + for (const auto & filter_type : filters_type) + { + auto expression = server.config().getString(config_prefix + "." + filter_type); + + if (startsWith(expression, "regex:")) + { + expression = expression.substr(6); + auto compiled_regex = std::make_shared(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 & 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)); + } + } return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( - server, analyzeReceiveQueryParams(predefine_query), predefine_query), server.config(), config_prefix); + server, std::move(analyze_receive_params), std::move(predefine_query), std::move(type_and_regex)), server.config(), config_prefix); } } \ No newline at end of file diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index f5139b6a9cc..8415bffe5dd 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -29,7 +29,7 @@ public: void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; /// This method is called right before the query execution. - virtual void customizeContext(Context & /* context */) {} + virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & /* context */) {} virtual bool customizeQueryParam(Context & context, const std::string & key, const std::string & value) = 0; @@ -101,12 +101,17 @@ class PredefineQueryHandler : public HTTPHandler private: NameSet receive_params; std::string predefine_query; + std::unordered_map header_name_with_capture_regex; public: - explicit PredefineQueryHandler(IServer & server, const NameSet & receive_params, const std::string & predefine_query_); + explicit PredefineQueryHandler( + IServer & server, const NameSet & receive_params_, const std::string & predefine_query_ + , const std::unordered_map & header_name_with_capture_regex_); + + virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & context) override; std::string getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) override; - bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; + bool customizeQueryParam(Context & context, const std::string & key, const std::string & value) override; }; } diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 55aaa7dc6a2..3e6d21f7455 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -10,6 +10,7 @@ #include "StaticRequestHandler.h" #include "ReplicasStatusHandler.h" #include "InterserverIOHTTPHandler.h" +#include "PrometheusRequestHandler.h" #if USE_RE2_ST #include @@ -35,17 +36,14 @@ HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler(const Poco::Net::HTTPServerRequest & request) // override { LOG_TRACE(log, "HTTP Request for " << name << ". " - << "Method: " - << request.getMethod() - << ", Address: " - << request.clientAddress().toString() - << ", User-Agent: " - << (request.has("User-Agent") ? request.get("User-Agent") : "none") + << "Method: " << request.getMethod() + << ", Address: " << request.clientAddress().toString() + << ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none") << (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")) << ", Content Type: " << request.getContentType() << ", Transfer Encoding: " << request.getTransferEncoding()); - for (auto & handler_factory : child_handler_factories) + for (auto & handler_factory : child_factories) { auto handler = handler_factory->createRequestHandler(request); if (handler != nullptr) @@ -64,13 +62,13 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand HTTPRequestHandlerFactoryMain::~HTTPRequestHandlerFactoryMain() { - while (!child_handler_factories.empty()) - delete child_handler_factories.back(), child_handler_factories.pop_back(); + while (!child_factories.empty()) + delete child_factories.back(), child_factories.pop_back(); } HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory) { - child_handler_factories.emplace_back(child_factory); + child_factories.emplace_back(child_factory); return this; } @@ -91,11 +89,11 @@ static inline auto createHandlersFactoryFromConfig(IServer & server, const std:: const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); if (handler_type == "static") - main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix)); + main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix + "." + key)); else if (handler_type == "dynamic_query_handler") - main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix)); + main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix + "." + key)); else if (handler_type == "predefine_query_handler") - main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix)); + main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix + "." + key)); else throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } @@ -112,13 +110,13 @@ static inline auto createHandlersFactoryFromConfig(IServer & server, const std:: static const auto ping_response_expression = "Ok.\n"; static const auto root_response_expression = "config://http_server_default_response"; -static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IServer & server, const std::string & name) +static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IServer & server, const std::string & name, AsynchronousMetrics & async_metrics) { if (server.config().has("routing_rules")) return createHandlersFactoryFromConfig(server, name, "routing_rules"); else { - return (new HTTPRequestHandlerFactoryMain(name)) + auto factory = (new HTTPRequestHandlerFactoryMain(name)) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) @@ -126,9 +124,13 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) ->addHandler((new RoutingRuleHTTPHandlerFactory(server, "query"))->allowPostAndGetParamsRequest()); - /// TODO: -// if (configuration.has("prometheus") && configuration.getInt("prometheus.port", 0) == 0) -// handler_factory->addHandler(async_metrics); + + if (server.config().has("prometheus") && server.config().getInt("prometheus.port", 0) == 0) + factory->addHandler((new RoutingRuleHTTPHandlerFactory( + server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics))) + ->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"))->allowGetAndHeadRequest()); + + return factory; } } @@ -144,12 +146,16 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandle ->addHandler((new RoutingRuleHTTPHandlerFactory(server))->allowPostAndGetParamsRequest()); } -Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name) +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name) { if (name == "HTTPHandler-factory" || name == "HTTPSHandler-factory") - return createHTTPHandlerFactory(server, name); + return createHTTPHandlerFactory(server, name, async_metrics); else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); + else if (name == "PrometheusHandler-factory") + return (new HTTPRequestHandlerFactoryMain(name))->addHandler((new RoutingRuleHTTPHandlerFactory( + server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics))) + ->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"))->allowGetAndHeadRequest()); throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); } diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index c3686d9f479..a3582c48618 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -2,10 +2,12 @@ #include "IServer.h" #include +#include #include #include #include #include +#include namespace DB { @@ -19,7 +21,7 @@ private: Logger * log; std::string name; - std::vector child_handler_factories; + std::vector child_factories; public: ~HTTPRequestHandlerFactoryMain(); @@ -101,12 +103,12 @@ private: std::function creator; }; -Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, const std::string & name); - Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix); Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix); Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix); +Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name); + } diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h index 5f794884239..1b3d908c6d5 100644 --- a/programs/server/HTTPHandlerRequestFilter.h +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -20,58 +20,82 @@ namespace DB namespace ErrorCodes { + extern const int NOT_IMPLEMENTED; extern const int CANNOT_COMPILE_REGEXP; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -static inline std::string uriPathGetter(const Poco::Net::HTTPServerRequest & request) +static inline bool checkRegexExpression(const StringRef & match_str, const StringRef & expression) { - const auto & uri = request.getURI(); - const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); + re2_st::StringPiece regex(expression.data, expression.size); - return std::string(uri.data(), end - uri.data()); + auto compiled_regex = std::make_shared(regex); + + if (!compiled_regex->ok()) + throw Exception("cannot compile re2: " + expression.toString() + " for routing_rule, error: " + compiled_regex->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; + + re2_st::StringPiece matches[num_captures]; + re2_st::StringPiece match_input(match_str.data, match_str.size); + return compiled_regex->Match(match_input, 0, match_str.size, re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); } -static inline std::function headerGetter(const std::string & header_name) +static inline bool checkExpression(const StringRef & match_str, const std::string & expression) { - return [header_name](const Poco::Net::HTTPServerRequest & request) { return request.get(header_name, ""); }; + if (startsWith(expression, "regex:")) + return checkRegexExpression(match_str, expression.substr(6)); + + return match_str == expression; } -static inline auto methodsExpressionFilter(const std::string &methods_expression) +static inline auto methodsFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) { - Poco::StringTokenizer tokenizer(Poco::toUpper(Poco::trim(methods_expression)), ","); - return [methods = std::vector(tokenizer.begin(), tokenizer.end())](const Poco::Net::HTTPServerRequest & request) + std::vector methods; + Poco::StringTokenizer tokenizer(config.getString(config_path), ","); + + for (auto iterator = tokenizer.begin(); iterator != tokenizer.end(); ++iterator) + methods.emplace_back(Poco::toUpper(Poco::trim(*iterator))); + + return [methods](const Poco::Net::HTTPServerRequest & request) { return std::count(methods.begin(), methods.end(), request.getMethod()); }; +} + +static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) +{ + return [expression = config.getString(config_path)](const Poco::Net::HTTPServerRequest & request) { - return std::count(methods.begin(), methods.end(), request.getMethod()); + const auto & uri = request.getURI(); + const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); + + return checkExpression(StringRef(uri.data(), end - uri.data()), expression); }; } -template -static inline auto regularExpressionFilter(const std::string & regular_expression, const GetFunction & get) +static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, const std::string & prefix) { - auto compiled_regex = std::make_shared(regular_expression); + std::unordered_map headers_expression; + Poco::Util::AbstractConfiguration::Keys headers_name; + config.keys(prefix, headers_name); - if (!compiled_regex->ok()) - throw Exception("cannot compile re2: " + regular_expression + " for routing_rule, error: " + compiled_regex->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); - - return std::make_pair(compiled_regex, [get = std::move(get), compiled_regex](const Poco::Net::HTTPServerRequest & request) + for (const auto & header_name : headers_name) { - const auto & test_content = get(request); - int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; + const auto & expression = config.getString(prefix + "." + header_name); + checkExpression("", expression); /// Check expression syntax is correct + headers_expression.emplace(std::make_pair(header_name, expression)); + } - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(test_content.data(), test_content.size()); - return compiled_regex->Match(input, 0, test_content.size(), re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); - }); -} + return [headers_expression](const Poco::Net::HTTPServerRequest & request) + { + for (const auto & [header_name, header_expression] : headers_expression) + { + const auto & header_value = request.get(header_name); + if (!checkExpression(StringRef(header_value.data(), header_value.size()), header_expression)) + return false; + } -template -static inline std::function expressionFilter(const std::string & expression, const GetFunction & get) -{ - if (startsWith(expression, "regex:")) - return regularExpressionFilter(expression, get).second; - - return [expression, get = std::move(get)](const Poco::Net::HTTPServerRequest & request) { return get(request) == expression; }; + return true; + }; } template @@ -84,12 +108,15 @@ static inline Poco::Net::HTTPRequestHandlerFactory * addFiltersFromConfig( for (const auto & filter_type : filters_type) { if (filter_type == "handler") - continue; /// Skip handler config - else if (filter_type == "method") - factory->addFilter(methodsExpressionFilter(config.getString(prefix + "." + filter_type))); + continue; + else if (filter_type == "url") + factory->addFilter(urlFilter(config, prefix + ".url")); + else if (filter_type == "headers") + factory->addFilter(headersFilter(config, prefix + ".headers")); + else if (filter_type == "methods") + factory->addFilter(methodsFilter(config, prefix + ".methods")); else - factory->addFilter(expressionFilter(config.getString(prefix + "." + filter_type), filter_type == "url" - ? uriPathGetter : headerGetter(filter_type))); + throw Exception("Unknown element in config: " + prefix + "." + filter_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } return factory; diff --git a/programs/server/PrometheusRequestHandler.h b/programs/server/PrometheusRequestHandler.h index d3d1dee88b1..47c8adf4774 100644 --- a/programs/server/PrometheusRequestHandler.h +++ b/programs/server/PrometheusRequestHandler.h @@ -18,7 +18,7 @@ private: const PrometheusMetricsWriter & metrics_writer; public: - explicit PrometheusRequestHandler(IServer & server_, PrometheusMetricsWriter & metrics_writer_) + explicit PrometheusRequestHandler(IServer & server_, const PrometheusMetricsWriter & metrics_writer_) : server(server_) , metrics_writer(metrics_writer_) { @@ -29,33 +29,4 @@ public: Poco::Net::HTTPServerResponse & response) override; }; - -template -class PrometheusRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -private: - IServer & server; - std::string endpoint_path; - PrometheusMetricsWriter metrics_writer; - -public: - PrometheusRequestHandlerFactory(IServer & server_, const AsynchronousMetrics & async_metrics_) - : server(server_) - , endpoint_path(server_.config().getString("prometheus.endpoint", "/metrics")) - , metrics_writer(server_.config(), "prometheus", async_metrics_) - { - } - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override - { - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET - && request.getURI() == endpoint_path) - return new HandlerType(server, metrics_writer); - - return nullptr; - } -}; - -using PrometheusHandlerFactory = PrometheusRequestHandlerFactory; - } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5ee7a39d6a0..ec0d8d2fe1d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -769,7 +769,7 @@ int Server::main(const std::vector & /*args*/) socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "HTTPHandler-factory"), server_pool, socket, http_params)); + createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for http://" + address.toString()); }); @@ -783,7 +783,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, http_params)); + createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for https://" + address.toString()); #else @@ -838,7 +838,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); + createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString()); }); @@ -851,7 +851,7 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); servers.emplace_back(std::make_unique( - createHandlerFactory(*this, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); + createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params)); LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString()); #else @@ -877,22 +877,17 @@ int Server::main(const std::vector & /*args*/) }); /// Prometheus (if defined and not setup yet with http_port) -// create_server("prometheus.port", [&](UInt16 port) -// { -// Poco::Net::ServerSocket socket; -// auto address = socket_bind_listen(socket, listen_host, port); -// socket.setReceiveTimeout(settings.http_receive_timeout); -// socket.setSendTimeout(settings.http_send_timeout); -// auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory"); -// handler_factory->addHandler(async_metrics); -// servers.emplace_back(std::make_unique( -// handler_factory, -// server_pool, -// socket, -// http_params)); -// -// LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); -// }); + create_server("prometheus.port", [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socket_bind_listen(socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + servers.emplace_back(std::make_unique( + createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + + LOG_INFO(log, "Listening for Prometheus: http://" + address.toString()); + }); } if (servers.empty()) diff --git a/programs/server/StaticRequestHandler.cpp b/programs/server/StaticRequestHandler.cpp index 730060dea2e..1ced1768fc6 100644 --- a/programs/server/StaticRequestHandler.cpp +++ b/programs/server/StaticRequestHandler.cpp @@ -75,12 +75,12 @@ StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & exp Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & server, const std::string & config_prefix) { - const auto & status = server.config().getInt(config_prefix + ".handler.status", 200); - const auto & response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); - const auto & response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); + int status = server.config().getInt(config_prefix + ".handler.status", 200); + std::string response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); + std::string response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( - server, response_content, status, response_content_type), server.config(), config_prefix); + server, std::move(response_content), status, std::move(response_content_type)), server.config(), config_prefix); } } diff --git a/programs/server/config.xml b/programs/server/config.xml index 74557a6ab7d..da462a77e3e 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,30 +525,45 @@ --> - - - /ping - /replicas_status - query + - + --> + diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 76d9e9238be..6a3aec0a5d2 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -701,7 +701,7 @@ class ClickHouseInstance: # Connects to the instance via HTTP interface, sends a query and returns the answer def http_request(self, url, method='GET', params=None, data=None, headers=None): url = "http://" + self.ip_address + ":8123/"+url - return requests.request(method=method, url=url, params=params, data=data, headers=headers).content + return requests.request(method=method, url=url, params=params, data=data, headers=headers) # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): diff --git a/dbms/tests/integration/test_http_handlers_config/__init__.py b/tests/integration/test_http_handlers_config/__init__.py similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/__init__.py rename to tests/integration/test_http_handlers_config/__init__.py diff --git a/dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml b/tests/integration/test_http_handlers_config/common_configs/common_config.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/common_configs/common_config.xml rename to tests/integration/test_http_handlers_config/common_configs/common_config.xml diff --git a/dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml b/tests/integration/test_http_handlers_config/common_configs/common_users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/common_configs/common_users.xml rename to tests/integration/test_http_handlers_config/common_configs/common_users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/tests/integration/test_http_handlers_config/other_tests_configs/config.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/other_tests_configs/config.xml rename to tests/integration/test_http_handlers_config/other_tests_configs/config.xml diff --git a/dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml b/tests/integration/test_http_handlers_config/other_tests_configs/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/other_tests_configs/users.xml rename to tests/integration/test_http_handlers_config/other_tests_configs/users.xml diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py new file mode 100644 index 00000000000..007a93e5fd4 --- /dev/null +++ b/tests/integration/test_http_handlers_config/test.py @@ -0,0 +1,50 @@ +import os +import urllib +import contextlib + +from helpers.cluster import ClickHouseCluster + + +class SimpleCluster: + def close(self): + self.cluster.shutdown() + + def __init__(self, cluster, name, config_dir): + self.cluster = cluster + self.instance = self.add_instance(name, config_dir) + cluster.start() + + def add_instance(self, name, config_dir): + script_path = os.path.dirname(os.path.realpath(__file__)) + return self.cluster.add_instance(name, config_dir=os.path.join(script_path, config_dir), + main_configs=[os.path.join(script_path, 'common_configs', 'common_config.xml')], + user_configs=[os.path.join(script_path, 'common_configs', 'common_users.xml')]) + + +def test_dynamic_query_handler(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "dynamic_handler", "test_dynamic_handler")) as cluster: + test_query = urllib.quote_plus('SELECT * FROM system.settings WHERE name = \'max_threads\'') + + assert 404 == cluster.instance.http_request('?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code + + assert 404 == cluster.instance.http_request('test_dynamic_handler_get?max_threads=1', method='POST', headers={'XXX': 'xxx'}).status_code + + assert 404 == cluster.instance.http_request('test_dynamic_handler_get?max_threads=1', method='GET', headers={'XXX': 'bad'}).status_code + + assert 400 == cluster.instance.http_request('test_dynamic_handler_get?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code + + assert 200 == cluster.instance.http_request('test_dynamic_handler_get?max_threads=1&get_dynamic_handler_query=' + test_query, + method='GET', headers={'XXX': 'xxx'}).status_code + + +def test_predefine_query_handler(): + with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_handler", "test_predefined_handler")) as cluster: + assert 404 == cluster.instance.http_request('?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code + + assert 404 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='GET', headers={'XXX': 'bad'}).status_code + + 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 '1\n' == cluster.instance.http_request('test_predefine_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content \ No newline at end of file diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml new file mode 100644 index 00000000000..7fe152d576e --- /dev/null +++ b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml @@ -0,0 +1,15 @@ + + + + + + xxx + GET + /test_dynamic_handler_get + + dynamic_query_handler + get_dynamic_handler_query + + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_dynamic/users.xml rename to tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml rename to tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_insert_and_select_predefined/users.xml rename to tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml rename to tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml rename to tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml new file mode 100644 index 00000000000..b9673f94fca --- /dev/null +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -0,0 +1,15 @@ + + + + + + GET + xxx + /test_predefine_handler_get + + predefine_query_handler + SELECT * FROM system.settings WHERE name = {setting_name:String} + + + + diff --git a/dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/users.xml similarity index 100% rename from dbms/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml rename to tests/integration/test_http_handlers_config/test_predefined_handler/users.xml From 81230944a4231d013a9f59a7891b57e6a568fc98 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 26 Apr 2020 19:06:14 +0800 Subject: [PATCH 18/23] ISSUES-5436 add integration test --- programs/server/HTTPHandler.cpp | 100 ++++++++------- programs/server/HTTPHandler.h | 3 +- programs/server/HTTPHandlerRequestFilter.h | 3 +- programs/server/StaticRequestHandler.cpp | 118 +++++++++++++++--- programs/server/StaticRequestHandler.h | 4 +- programs/server/config.xml | 42 ++++--- .../other_tests_configs/config.xml | 18 --- .../test_http_handlers_config/test.py | 71 ++++++++++- .../config.xml | 31 ----- .../test_param_and_settings_dynamic/users.xml | 3 - .../config.xml | 43 ------- .../users.xml | 3 - .../test_predefined_handler/config.xml | 12 +- .../test_static_handler/config.xml | 52 ++++++++ .../users.xml | 0 .../test_https_replication/configs/config.xml | 9 -- .../0_stateless/00408_http_keep_alive.sh | 3 + 17 files changed, 322 insertions(+), 193 deletions(-) delete mode 100644 tests/integration/test_http_handlers_config/other_tests_configs/config.xml delete mode 100644 tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml delete mode 100644 tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml delete mode 100644 tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml delete mode 100644 tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml create mode 100644 tests/integration/test_http_handlers_config/test_static_handler/config.xml rename tests/integration/test_http_handlers_config/{other_tests_configs => test_static_handler}/users.xml (100%) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index c14f5f4434b..e1d82dc7735 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -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 & header_name_with_capture_regex_) + , const std::optional & url_regex_, const std::unordered_map & 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(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(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 type_and_regex; - Poco::Util::AbstractConfiguration::Keys filters_type; - server.config().keys(config_prefix, filters_type); + std::unordered_map 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(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( + server, std::move(analyze_receive_params), std::move(predefine_query), std::optional(url_expression), + std::move(headers_name_with_regex)), configuration, config_prefix); + } + return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( - 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{} ,std::move(headers_name_with_regex)), + configuration, config_prefix); +} + } -} \ No newline at end of file diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 8415bffe5dd..39167747358 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -101,11 +101,12 @@ class PredefineQueryHandler : public HTTPHandler private: NameSet receive_params; std::string predefine_query; + std::optional url_regex; std::unordered_map header_name_with_capture_regex; public: explicit PredefineQueryHandler( IServer & server, const NameSet & receive_params_, const std::string & predefine_query_ - , const std::unordered_map & header_name_with_capture_regex_); + , const std::optional & url_regex_, const std::unordered_map & header_name_with_regex_); virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & context) override; diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h index 1b3d908c6d5..2a3b2f4bed6 100644 --- a/programs/server/HTTPHandlerRequestFilter.h +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -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; } diff --git a/programs/server/StaticRequestHandler.cpp b/programs/server/StaticRequestHandler.cpp index 1ced1768fc6..ff57a4764e0 100644 --- a/programs/server/StaticRequestHandler.cpp +++ b/programs/server/StaticRequestHandler.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include @@ -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( + 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(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::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( - 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); } } diff --git a/programs/server/StaticRequestHandler.h b/programs/server/StaticRequestHandler.h index bfe7f7a3e7b..bdbc17f5e00 100644 --- a/programs/server/StaticRequestHandler.h +++ b/programs/server/StaticRequestHandler.h @@ -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; }; diff --git a/programs/server/config.xml b/programs/server/config.xml index da462a77e3e..a53b0f4ff48 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -526,18 +526,23 @@ --> - diff --git a/tests/integration/test_http_handlers_config/other_tests_configs/config.xml b/tests/integration/test_http_handlers_config/other_tests_configs/config.xml deleted file mode 100644 index 0c908a4f877..00000000000 --- a/tests/integration/test_http_handlers_config/other_tests_configs/config.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - - - /ping_test - - POST - /test_one_handler_with_insert_and_select - - INSERT INTO test.test VALUES(1) - SELECT COUNT() FROM test.test WHERE id = {id:UInt8} - - - - test not found - - diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 007a93e5fd4..d152c7e329a 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -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 "Absolute Path File" > /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 'Absolute Path File\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 "Relative Path File" > /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 'Relative Path File\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 \ No newline at end of file diff --git a/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml deleted file mode 100644 index f9c576e5546..00000000000 --- a/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/config.xml +++ /dev/null @@ -1,31 +0,0 @@ - - - - - - POST - /post_query_params_and_settings - post_query_param - - - - GET - /get_query_params_and_settings - get_query_param - - - - GET - [^/]+)(/(?P[^/]+))?]]> - query_param - - - - - TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> - - query_param - - - diff --git a/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml deleted file mode 100644 index 9aba4ac0914..00000000000 --- a/tests/integration/test_http_handlers_config/test_param_and_settings_dynamic/users.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - diff --git a/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml deleted file mode 100644 index 1ad1fecb2e5..00000000000 --- a/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/config.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - - - POST - /post_query_params_and_settings - - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} - - - - - GET - /get_query_params_and_settings - - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} - - - - - GET - [^/]+)(/(?P[^/]+))?]]> - - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} - - - - - - TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> - - - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} - - - - diff --git a/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml b/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml deleted file mode 100644 index 9aba4ac0914..00000000000 --- a/tests/integration/test_http_handlers_config/test_param_and_settings_predefined/users.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index b9673f94fca..8ac4b2950e6 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -8,7 +8,17 @@ /test_predefine_handler_get predefine_query_handler - SELECT * FROM system.settings WHERE name = {setting_name:String} + SELECT name, value FROM system.settings WHERE name = {setting_name:String} + + + + [^/]+)]]> + + .+)]]> + + + predefine_query_handler + SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String} diff --git a/tests/integration/test_http_handlers_config/test_static_handler/config.xml b/tests/integration/test_http_handlers_config/test_static_handler/config.xml new file mode 100644 index 00000000000..b1609cd055d --- /dev/null +++ b/tests/integration/test_http_handlers_config/test_static_handler/config.xml @@ -0,0 +1,52 @@ + + + + Test get static handler and config content + + + + GET + xxx + /test_get_fixed_static_handler + + static + 402 + text/html; charset=UTF-8 + Test get static handler and fix content + + + + + GET + xxx + /test_get_config_static_handler + + static + config://test_get_config_static_handler + + + + + GET + xxx + /test_get_absolute_path_static_handler + + static + text/html; charset=UTF-8 + file:///absolute_path_file.html + + + + + GET + xxx + /test_get_relative_path_static_handler + + static + text/html; charset=UTF-8 + file://./relative_path_file.html + + + + + diff --git a/tests/integration/test_http_handlers_config/other_tests_configs/users.xml b/tests/integration/test_http_handlers_config/test_static_handler/users.xml similarity index 100% rename from tests/integration/test_http_handlers_config/other_tests_configs/users.xml rename to tests/integration/test_http_handlers_config/test_static_handler/users.xml diff --git a/tests/integration/test_https_replication/configs/config.xml b/tests/integration/test_https_replication/configs/config.xml index be81ccfc145..35a43b2fc54 100644 --- a/tests/integration/test_https_replication/configs/config.xml +++ b/tests/integration/test_https_replication/configs/config.xml @@ -354,15 +354,6 @@ - - - /ping - /replicas_status - query - - Use / or /ping for health checks. Or /replicas_status for more sophisticated health checks. Send queries from your program with POST method or GET /?query=... Use clickhouse-client: For interactive data analysis: clickhouse-client For batch query processing: clickhouse-client --query='SELECT 1' > result clickhouse-client < query > result - - diff --git a/tests/queries/0_stateless/00408_http_keep_alive.sh b/tests/queries/0_stateless/00408_http_keep_alive.sh index 83ac8c55f24..79e39d83704 100755 --- a/tests/queries/0_stateless/00408_http_keep_alive.sh +++ b/tests/queries/0_stateless/00408_http_keep_alive.sh @@ -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/'; From d56002b81fab3506fc3d8f43dfd1a61a6d60f8dd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 27 Apr 2020 20:17:16 +0800 Subject: [PATCH 19/23] ISSUES-5436 fix ya make --- programs/server/ya.make | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/server/ya.make b/programs/server/ya.make index 7f0a3969fd2..2c74c01c7cb 100644 --- a/programs/server/ya.make +++ b/programs/server/ya.make @@ -18,11 +18,10 @@ SRCS( MySQLHandler.cpp MySQLHandlerFactory.cpp NotFoundHandler.cpp - PingRequestHandler.cpp PrometheusMetricsWriter.cpp PrometheusRequestHandler.cpp ReplicasStatusHandler.cpp - RootRequestHandler.cpp + StaticRequestHandler.cpp Server.cpp TCPHandler.cpp ) From e72a484bebf00dccfe299f44129b5fe4af246276 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 27 Apr 2020 23:51:39 +0300 Subject: [PATCH 20/23] minor fixes --- programs/server/HTTPHandler.cpp | 34 +++++++------- programs/server/HTTPHandler.h | 10 ++-- programs/server/HTTPHandlerFactory.cpp | 46 +++++++++++-------- programs/server/HTTPHandlerFactory.h | 10 ++-- programs/server/HTTPHandlerRequestFilter.h | 4 +- programs/server/StaticRequestHandler.cpp | 4 +- programs/server/config.xml | 28 +++++------ src/IO/WriteBufferFromHTTPServerResponse.h | 5 -- .../test_http_handlers_config/test.py | 10 ++-- .../test_dynamic_handler/config.xml | 8 ++-- .../test_predefined_handler/config.xml | 18 ++++---- .../test_static_handler/config.xml | 20 ++++---- 12 files changed, 100 insertions(+), 97 deletions(-) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index e1d82dc7735..6175e3e57c2 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -755,15 +755,15 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request return full_query; } -PredefineQueryHandler::PredefineQueryHandler( - IServer & server, const NameSet & receive_params_, const std::string & predefine_query_ +PredefinedQueryHandler::PredefinedQueryHandler( + IServer & server, const NameSet & receive_params_, const std::string & predefined_query_ , const std::optional & url_regex_, const std::unordered_map & header_name_with_regex_) - : HTTPHandler(server, "PredefineQueryHandler"), receive_params(receive_params_), predefine_query(predefine_query_) + : HTTPHandler(server, "PredefinedQueryHandler"), receive_params(receive_params_), predefined_query(predefined_query_) , url_regex(url_regex_), header_name_with_capture_regex(header_name_with_regex_) { } -bool PredefineQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) +bool PredefinedQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) { if (receive_params.count(key)) { @@ -774,7 +774,7 @@ bool PredefineQueryHandler::customizeQueryParam(Context & context, const std::st return false; } -void PredefineQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & request, DB::Context & context) +void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & request, DB::Context & context) { /// If in the configuration file, the handler's header is regex and contains named capture group /// We will extract regex named capture groups as query parameters @@ -784,7 +784,7 @@ void PredefineQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & requ auto compiled_regex = std::make_shared(regex); if (!compiled_regex->ok()) - throw Exception("cannot compile re2: " + regex + " for routing_rule, error: " + compiled_regex->error() + + throw Exception("cannot compile re2: " + regex + " for http handling rule, error: " + compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; @@ -816,7 +816,7 @@ void PredefineQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & requ } } -std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) +std::string PredefinedQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) { if (unlikely(startsWith(request.getContentType(), "multipart/form-data"))) { @@ -825,13 +825,13 @@ std::string PredefineQueryHandler::getQuery(Poco::Net::HTTPServerRequest & reque params.load(request, request.stream(), handler); } - return predefine_query; + return predefined_query; } Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) { std::string query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); - return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); } static inline bool capturingNamedQueryParam(NameSet receive_params, const std::string & expression) @@ -839,7 +839,7 @@ static inline bool capturingNamedQueryParam(NameSet receive_params, const std::s auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) - throw Exception("Cannot compile re2: " + expression + " for routing_rule, error: " + + throw Exception("Cannot compile re2: " + expression + " for http handling 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(); @@ -850,15 +850,15 @@ static inline bool capturingNamedQueryParam(NameSet receive_params, const std::s }); } -Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix) +Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix) { 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 = configuration.getString(config_prefix + ".handler.query"); - NameSet analyze_receive_params = analyzeReceiveQueryParams(predefine_query); + std::string predefined_query = configuration.getString(config_prefix + ".handler.query"); + NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query); std::unordered_map headers_name_with_regex; Poco::Util::AbstractConfiguration::Keys headers_name; @@ -884,13 +884,13 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & s url_expression = url_expression.substr(6); if (capturingNamedQueryParam(analyze_receive_params, url_expression)) - return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefine_query), std::optional(url_expression), + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), std::optional(url_expression), std::move(headers_name_with_regex)), configuration, config_prefix); } - return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefine_query), std::optional{} ,std::move(headers_name_with_regex)), + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), std::optional{} ,std::move(headers_name_with_regex)), configuration, config_prefix); } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 39167747358..0049ac30ccb 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -29,7 +29,7 @@ public: void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; /// This method is called right before the query execution. - virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & /* context */) {} + virtual void customizeContext(Poco::Net::HTTPServerRequest & /*request*/, Context & /* context */) {} virtual bool customizeQueryParam(Context & context, const std::string & key, const std::string & value) = 0; @@ -96,16 +96,16 @@ public: bool customizeQueryParam(Context &context, const std::string &key, const std::string &value) override; }; -class PredefineQueryHandler : public HTTPHandler +class PredefinedQueryHandler : public HTTPHandler { private: NameSet receive_params; - std::string predefine_query; + std::string predefined_query; std::optional url_regex; std::unordered_map header_name_with_capture_regex; public: - explicit PredefineQueryHandler( - IServer & server, const NameSet & receive_params_, const std::string & predefine_query_ + explicit PredefinedQueryHandler( + IServer & server, const NameSet & receive_params_, const std::string & predefined_query_ , const std::optional & url_regex_, const std::unordered_map & header_name_with_regex_); virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & context) override; diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index 3e6d21f7455..a576c4a781c 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int INVALID_CONFIG_PARAMETER; } HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) @@ -63,7 +64,10 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand HTTPRequestHandlerFactoryMain::~HTTPRequestHandlerFactoryMain() { while (!child_factories.empty()) - delete child_factories.back(), child_factories.pop_back(); + { + delete child_factories.back(); + child_factories.pop_back(); + } } HTTPRequestHandlerFactoryMain::TThis * HTTPRequestHandlerFactoryMain::addHandler(Poco::Net::HTTPRequestHandlerFactory * child_factory) @@ -83,8 +87,8 @@ static inline auto createHandlersFactoryFromConfig(IServer & server, const std:: for (const auto & key : keys) { - if (!startsWith(key, "routing_rule")) - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + if (!startsWith(key, "rule")) + throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); const auto & handler_type = server.config().getString(prefix + "." + key + ".handler.type", ""); @@ -92,10 +96,14 @@ static inline auto createHandlersFactoryFromConfig(IServer & server, const std:: main_handler_factory->addHandler(createStaticHandlerFactory(server, prefix + "." + key)); else if (handler_type == "dynamic_query_handler") main_handler_factory->addHandler(createDynamicHandlerFactory(server, prefix + "." + key)); - else if (handler_type == "predefine_query_handler") - main_handler_factory->addHandler(createPredefineHandlerFactory(server, prefix + "." + key)); + else if (handler_type == "predefined_query_handler") + main_handler_factory->addHandler(createPredefinedHandlerFactory(server, prefix + "." + key)); + else if (handler_type.empty()) + throw Exception("Handler type in config is not specified here: " + + prefix + "." + key + ".handler.type", ErrorCodes::INVALID_CONFIG_PARAMETER); else - throw Exception("Unknown element in config: " + prefix + "." + key + ", must be 'routing_rule'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown handler type '" + handler_type +"' in config here: " + + prefix + "." + key + ".handler.type",ErrorCodes::INVALID_CONFIG_PARAMETER); } return main_handler_factory; @@ -112,21 +120,21 @@ static const auto root_response_expression = "config://http_server_default_respo static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IServer & server, const std::string & name, AsynchronousMetrics & async_metrics) { - if (server.config().has("routing_rules")) - return createHandlersFactoryFromConfig(server, name, "routing_rules"); + if (server.config().has("http_handlers")) + return createHandlersFactoryFromConfig(server, name, "http_handlers"); else { auto factory = (new HTTPRequestHandlerFactoryMain(name)) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server, ping_response_expression)) ->attachStrictPath("/ping")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server)) ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server, "query"))->allowPostAndGetParamsRequest()); + ->addHandler((new HandlingRuleHTTPHandlerFactory(server, "query"))->allowPostAndGetParamsRequest()); if (server.config().has("prometheus") && server.config().getInt("prometheus.port", 0) == 0) - factory->addHandler((new RoutingRuleHTTPHandlerFactory( + factory->addHandler((new HandlingRuleHTTPHandlerFactory( server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics))) ->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"))->allowGetAndHeadRequest()); @@ -137,13 +145,13 @@ static inline Poco::Net::HTTPRequestHandlerFactory * createHTTPHandlerFactory(IS static inline Poco::Net::HTTPRequestHandlerFactory * createInterserverHTTPHandlerFactory(IServer & server, const std::string & name) { return (new HTTPRequestHandlerFactoryMain(name)) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server, root_response_expression)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server, root_response_expression)) ->attachStrictPath("/")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server, ping_response_expression)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server, ping_response_expression)) ->attachStrictPath("/ping")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server)) + ->addHandler((new HandlingRuleHTTPHandlerFactory(server)) ->attachNonStrictPath("/replicas_status")->allowGetAndHeadRequest()) - ->addHandler((new RoutingRuleHTTPHandlerFactory(server))->allowPostAndGetParamsRequest()); + ->addHandler((new HandlingRuleHTTPHandlerFactory(server))->allowPostAndGetParamsRequest()); } Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name) @@ -153,11 +161,11 @@ Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, As else if (name == "InterserverIOHTTPHandler-factory" || name == "InterserverIOHTTPSHandler-factory") return createInterserverHTTPHandlerFactory(server, name); else if (name == "PrometheusHandler-factory") - return (new HTTPRequestHandlerFactoryMain(name))->addHandler((new RoutingRuleHTTPHandlerFactory( + return (new HTTPRequestHandlerFactoryMain(name))->addHandler((new HandlingRuleHTTPHandlerFactory( server, PrometheusMetricsWriter(server.config(), "prometheus", async_metrics))) ->attachStrictPath(server.config().getString("prometheus.endpoint", "/metrics"))->allowGetAndHeadRequest()); throw Exception("LOGICAL ERROR: Unknown HTTP handler factory name.", ErrorCodes::LOGICAL_ERROR); } -} \ No newline at end of file +} diff --git a/programs/server/HTTPHandlerFactory.h b/programs/server/HTTPHandlerFactory.h index a3582c48618..cf061e5b626 100644 --- a/programs/server/HTTPHandlerFactory.h +++ b/programs/server/HTTPHandlerFactory.h @@ -13,7 +13,7 @@ namespace DB { /// Handle request using child handlers -class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory +class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactory, boost::noncopyable { private: using TThis = HTTPRequestHandlerFactoryMain; @@ -34,14 +34,14 @@ public: }; template -class RoutingRuleHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory +class HandlingRuleHTTPHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { public: - using TThis = RoutingRuleHTTPHandlerFactory; + using TThis = HandlingRuleHTTPHandlerFactory; using Filter = std::function; template - RoutingRuleHTTPHandlerFactory(TArgs &&... args) + HandlingRuleHTTPHandlerFactory(TArgs &&... args) { creator = [args = std::tuple(std::forward(args) ...)]() { @@ -107,7 +107,7 @@ Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & serv Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix); -Poco::Net::HTTPRequestHandlerFactory * createPredefineHandlerFactory(IServer & server, const std::string & config_prefix); +Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix); Poco::Net::HTTPRequestHandlerFactory * createHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name); diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h index 2a3b2f4bed6..13d2e17dd80 100644 --- a/programs/server/HTTPHandlerRequestFilter.h +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -31,7 +31,7 @@ static inline bool checkRegexExpression(const StringRef & match_str, const Strin auto compiled_regex = std::make_shared(regex); if (!compiled_regex->ok()) - throw Exception("cannot compile re2: " + expression.toString() + " for routing_rule, error: " + compiled_regex->error() + + throw Exception("cannot compile re2: " + expression.toString() + " for http handling rule, error: " + compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; @@ -99,7 +99,7 @@ static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, con template static inline Poco::Net::HTTPRequestHandlerFactory * addFiltersFromConfig( - RoutingRuleHTTPHandlerFactory * factory, Poco::Util::AbstractConfiguration & config, const std::string & prefix) + HandlingRuleHTTPHandlerFactory * factory, Poco::Util::AbstractConfiguration & config, const std::string & prefix) { Poco::Util::AbstractConfiguration::Keys filters_type; config.keys(prefix, filters_type); diff --git a/programs/server/StaticRequestHandler.cpp b/programs/server/StaticRequestHandler.cpp index ff57a4764e0..22f32e6a0e7 100644 --- a/programs/server/StaticRequestHandler.cpp +++ b/programs/server/StaticRequestHandler.cpp @@ -140,7 +140,7 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) else if (startsWith(response_expression, config_prefix)) { if (response_expression.size() <= config_prefix.size()) - throw Exception( "Static routing rule handler must contain a complete configuration path, for example: config://config_key", + throw Exception( "Static handling rule handler must contain a complete configuration path, for example: config://config_key", ErrorCodes::INVALID_CONFIG_PARAMETER); const auto & config_path = response_expression.substr(config_prefix.size(), response_expression.size() - config_prefix.size()); @@ -161,7 +161,7 @@ Poco::Net::HTTPRequestHandlerFactory * createStaticHandlerFactory(IServer & serv std::string response_content = server.config().getRawString(config_prefix + ".handler.response_content", "Ok.\n"); std::string response_content_type = server.config().getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8"); - return addFiltersFromConfig(new RoutingRuleHTTPHandlerFactory( + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( server, std::move(response_content), std::move(status), std::move(response_content_type)), server.config(), config_prefix); } diff --git a/programs/server/config.xml b/programs/server/config.xml index a53b0f4ff48..7ed2ead3703 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -525,21 +525,21 @@ --> - diff --git a/src/IO/WriteBufferFromHTTPServerResponse.h b/src/IO/WriteBufferFromHTTPServerResponse.h index c10288fff9f..ffa36c11c5b 100644 --- a/src/IO/WriteBufferFromHTTPServerResponse.h +++ b/src/IO/WriteBufferFromHTTPServerResponse.h @@ -113,11 +113,6 @@ public: compress = enable_compression; } - bool getCompression() - { - return compress; - } - /// Set compression level if the compression is turned on. /// The setting has any effect only if HTTP headers haven't been sent yet. void setCompressionLevel(int level) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index d152c7e329a..407733e9f5d 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -37,17 +37,17 @@ def test_dynamic_query_handler(): method='GET', headers={'XXX': 'xxx'}).status_code -def test_predefine_query_handler(): +def test_predefined_query_handler(): with contextlib.closing(SimpleCluster(ClickHouseCluster(__file__), "predefined_handler", "test_predefined_handler")) as cluster: assert 404 == cluster.instance.http_request('?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code - assert 404 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='GET', headers={'XXX': 'bad'}).status_code + assert 404 == cluster.instance.http_request('test_predefined_handler_get?max_threads=1', method='GET', headers={'XXX': 'bad'}).status_code - assert 404 == cluster.instance.http_request('test_predefine_handler_get?max_threads=1', method='POST', headers={'XXX': 'xxx'}).status_code + assert 404 == cluster.instance.http_request('test_predefined_handler_get?max_threads=1', method='POST', 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 500 == cluster.instance.http_request('test_predefined_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\n' == cluster.instance.http_request('test_predefined_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 diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml index 7fe152d576e..9e0af81ab99 100644 --- a/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_dynamic_handler/config.xml @@ -1,8 +1,8 @@ - - + + xxx GET /test_dynamic_handler_get @@ -10,6 +10,6 @@ dynamic_query_handler get_dynamic_handler_query - - + + diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml index 8ac4b2950e6..7ed6d29c613 100644 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_predefined_handler/config.xml @@ -1,25 +1,25 @@ - - + + GET xxx - /test_predefine_handler_get + /test_predefined_handler_get - predefine_query_handler + predefined_query_handler SELECT name, value FROM system.settings WHERE name = {setting_name:String} - - + + [^/]+)]]> .+)]]> - predefine_query_handler + predefined_query_handler SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String} - - + + diff --git a/tests/integration/test_http_handlers_config/test_static_handler/config.xml b/tests/integration/test_http_handlers_config/test_static_handler/config.xml index b1609cd055d..7938a9fd8f5 100644 --- a/tests/integration/test_http_handlers_config/test_static_handler/config.xml +++ b/tests/integration/test_http_handlers_config/test_static_handler/config.xml @@ -3,8 +3,8 @@ Test get static handler and config content - - + + GET xxx /test_get_fixed_static_handler @@ -14,9 +14,9 @@ text/html; charset=UTF-8 Test get static handler and fix content - + - + GET xxx /test_get_config_static_handler @@ -24,9 +24,9 @@ static config://test_get_config_static_handler - + - + GET xxx /test_get_absolute_path_static_handler @@ -35,9 +35,9 @@ text/html; charset=UTF-8 file:///absolute_path_file.html - + - + GET xxx /test_get_relative_path_static_handler @@ -46,7 +46,7 @@ text/html; charset=UTF-8 file://./relative_path_file.html - + - + From 9d43c2f6c2a572b80426ddc33019526785d991ac Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Apr 2020 01:19:24 +0300 Subject: [PATCH 21/23] save compiled regex --- programs/server/HTTPHandler.cpp | 45 +++++++++++++++++---------------- programs/server/HTTPHandler.h | 13 +++++++--- 2 files changed, 33 insertions(+), 25 deletions(-) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 6175e3e57c2..4e52dbbc64e 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -757,7 +757,7 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request PredefinedQueryHandler::PredefinedQueryHandler( IServer & server, const NameSet & receive_params_, const std::string & predefined_query_ - , const std::optional & url_regex_, const std::unordered_map & header_name_with_regex_) + , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_) : HTTPHandler(server, "PredefinedQueryHandler"), receive_params(receive_params_), predefined_query(predefined_query_) , url_regex(url_regex_), header_name_with_capture_regex(header_name_with_regex_) { @@ -779,14 +779,8 @@ void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & req /// If in the configuration file, the handler's header is regex and contains named capture group /// We will extract regex named capture groups as query parameters - const auto & set_query_params = [&](const char * begin, const char * end, const std::string & regex) + const auto & set_query_params = [&](const char * begin, const char * end, const CompiledRegexPtr & compiled_regex) { - auto compiled_regex = std::make_shared(regex); - - if (!compiled_regex->ok()) - throw Exception("cannot compile re2: " + regex + " for http handling rule, error: " + compiled_regex->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); - int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; re2_st::StringPiece matches[num_captures]; @@ -806,7 +800,7 @@ void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & req if (url_regex) { const auto & uri = request.getURI(); - set_query_params(uri.data(), find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()), *url_regex); + 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) @@ -834,14 +828,8 @@ Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & ser return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); } -static inline bool capturingNamedQueryParam(NameSet receive_params, const std::string & expression) +static inline bool capturingNamedQueryParam(NameSet receive_params, const CompiledRegexPtr & compiled_regex) { - auto compiled_regex = std::make_shared(expression); - - if (!compiled_regex->ok()) - throw Exception("Cannot compile re2: " + expression + " for http handling 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) { @@ -850,6 +838,17 @@ static inline bool capturingNamedQueryParam(NameSet receive_params, const std::s }); } +static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) +{ + auto compiled_regex = std::make_shared(expression); + + if (!compiled_regex->ok()) + throw Exception("Cannot compile re2: " + expression + " for http handling rule, error: " + + compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + + return compiled_regex; +} + Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & server, const std::string & config_prefix) { Poco::Util::AbstractConfiguration & configuration = server.config(); @@ -860,7 +859,7 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & std::string predefined_query = configuration.getString(config_prefix + ".handler.query"); NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query); - std::unordered_map headers_name_with_regex; + std::unordered_map headers_name_with_regex; Poco::Util::AbstractConfiguration::Keys headers_name; configuration.keys(config_prefix + ".headers", headers_name); @@ -872,8 +871,9 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & continue; expression = expression.substr(6); - if (capturingNamedQueryParam(analyze_receive_params, expression)) - headers_name_with_regex.emplace(std::make_pair(header_name, expression)); + auto regex = getCompiledRegex(expression); + if (capturingNamedQueryParam(analyze_receive_params, regex)) + headers_name_with_regex.emplace(std::make_pair(header_name, regex)); } if (configuration.has(config_prefix + ".url")) @@ -883,14 +883,15 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & if (startsWith(url_expression, "regex:")) url_expression = url_expression.substr(6); - if (capturingNamedQueryParam(analyze_receive_params, url_expression)) + auto regex = getCompiledRegex(url_expression); + if (capturingNamedQueryParam(analyze_receive_params, regex)) return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefined_query), std::optional(url_expression), + server, std::move(analyze_receive_params), std::move(predefined_query), std::move(regex), std::move(headers_name_with_regex)), configuration, config_prefix); } return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefined_query), std::optional{} ,std::move(headers_name_with_regex)), + server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{} ,std::move(headers_name_with_regex)), configuration, config_prefix); } diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index 0049ac30ccb..a504b88248f 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -7,6 +7,12 @@ #include #include +#include +#if USE_RE2_ST +#include +#else +#define re2_st re2 +#endif namespace CurrentMetrics { @@ -20,6 +26,7 @@ namespace DB class WriteBufferFromHTTPServerResponse; +typedef std::shared_ptr CompiledRegexPtr; class HTTPHandler : public Poco::Net::HTTPRequestHandler { @@ -101,12 +108,12 @@ class PredefinedQueryHandler : public HTTPHandler private: NameSet receive_params; std::string predefined_query; - std::optional url_regex; - std::unordered_map header_name_with_capture_regex; + CompiledRegexPtr url_regex; + std::unordered_map header_name_with_capture_regex; public: explicit PredefinedQueryHandler( IServer & server, const NameSet & receive_params_, const std::string & predefined_query_ - , const std::optional & url_regex_, const std::unordered_map & header_name_with_regex_); + , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_); virtual void customizeContext(Poco::Net::HTTPServerRequest & request, Context & context) override; From 5a1d22a363b814be05189cc4913abd43e68af590 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Apr 2020 02:02:58 +0300 Subject: [PATCH 22/23] remove redundant configs from test --- .../common_configs/common_config.xml | 415 ------------------ .../common_configs/common_users.xml | 138 ------ .../test_http_handlers_config/test.py | 4 +- .../test_dynamic_handler/users.xml | 3 - .../test_predefined_handler/users.xml | 3 - .../test_static_handler/users.xml | 3 - 6 files changed, 1 insertion(+), 565 deletions(-) delete mode 100644 tests/integration/test_http_handlers_config/common_configs/common_config.xml delete mode 100644 tests/integration/test_http_handlers_config/common_configs/common_users.xml delete mode 100644 tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml delete mode 100644 tests/integration/test_http_handlers_config/test_predefined_handler/users.xml delete mode 100644 tests/integration/test_http_handlers_config/test_static_handler/users.xml diff --git a/tests/integration/test_http_handlers_config/common_configs/common_config.xml b/tests/integration/test_http_handlers_config/common_configs/common_config.xml deleted file mode 100644 index 154ebf6c35e..00000000000 --- a/tests/integration/test_http_handlers_config/common_configs/common_config.xml +++ /dev/null @@ -1,415 +0,0 @@ - - - - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - - - 8123 - 9000 - - - - - - - - - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key - - /etc/clickhouse-server/dhparam.pem - none - true - true - sslv2,sslv3 - true - - - - true - true - sslv2,sslv3 - true - - - - RejectCertificateHandler - - - - - - - - - 9009 - - - - - - - - - - - - - - - - - - - - 4096 - 3 - - - 100 - - - - - - 8589934592 - - - 5368709120 - - - - /var/lib/clickhouse/ - - - /var/lib/clickhouse/tmp/ - - - /var/lib/clickhouse/user_files/ - - - users.xml - - - default - - - - - - default - - - - - - - - - false - - - - - - - - localhost - 9000 - - - - - - - localhost - 9000 - - - - - localhost - 9000 - - - - - - - localhost - 9440 - 1 - - - - - - - localhost - 9000 - - - - - localhost - 1 - - - - - - - - - - - - - - - - - 3600 - - - - 3600 - - - 60 - - - - - - - - - - system - query_log
- - toYYYYMM(event_date) - - 7500 -
- - - - system - query_thread_log
- toYYYYMM(event_date) - 7500 -
- - - - - - - - - - - - - - - *_dictionary.xml - - - - - - - - - - /clickhouse/task_queue/ddl - - - - - - - - - - - - - - - - click_cost - any - - 0 - 3600 - - - 86400 - 60 - - - - max - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - - - /var/lib/clickhouse/format_schemas/ - - - -
diff --git a/tests/integration/test_http_handlers_config/common_configs/common_users.xml b/tests/integration/test_http_handlers_config/common_configs/common_users.xml deleted file mode 100644 index 9755c29d480..00000000000 --- a/tests/integration/test_http_handlers_config/common_configs/common_users.xml +++ /dev/null @@ -1,138 +0,0 @@ - - - - - - - - 10000000000 - - - 0 - - - random - - - - - 1 - - - - - - - - - - - - - ::/0 - - - - default - - - default - - - - - - - a = 1 - - - - - a + b < 1 or c - d > 5 - - - - - c = 1 - - - - - - - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 407733e9f5d..31d40bd8a1d 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -16,9 +16,7 @@ class SimpleCluster: def add_instance(self, name, config_dir): script_path = os.path.dirname(os.path.realpath(__file__)) - return self.cluster.add_instance(name, config_dir=os.path.join(script_path, config_dir), - main_configs=[os.path.join(script_path, 'common_configs', 'common_config.xml')], - user_configs=[os.path.join(script_path, 'common_configs', 'common_users.xml')]) + return self.cluster.add_instance(name, main_configs=[os.path.join(script_path, config_dir, 'config.xml')]) def test_dynamic_query_handler(): diff --git a/tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml b/tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml deleted file mode 100644 index 9aba4ac0914..00000000000 --- a/tests/integration/test_http_handlers_config/test_dynamic_handler/users.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - diff --git a/tests/integration/test_http_handlers_config/test_predefined_handler/users.xml b/tests/integration/test_http_handlers_config/test_predefined_handler/users.xml deleted file mode 100644 index 9aba4ac0914..00000000000 --- a/tests/integration/test_http_handlers_config/test_predefined_handler/users.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - diff --git a/tests/integration/test_http_handlers_config/test_static_handler/users.xml b/tests/integration/test_http_handlers_config/test_static_handler/users.xml deleted file mode 100644 index 9aba4ac0914..00000000000 --- a/tests/integration/test_http_handlers_config/test_static_handler/users.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - From aeac8cb6212ce3658a5ffb99116ddedf3910ac65 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Apr 2020 10:19:37 +0300 Subject: [PATCH 23/23] fix --- programs/server/HTTPHandler.cpp | 8 ++-- programs/server/HTTPHandler.h | 7 +-- programs/server/HTTPHandlerFactory.cpp | 6 --- programs/server/HTTPHandlerRequestFilter.h | 51 ++++++++++++---------- 4 files changed, 32 insertions(+), 40 deletions(-) diff --git a/programs/server/HTTPHandler.cpp b/programs/server/HTTPHandler.cpp index 4e52dbbc64e..1a579e2bff3 100644 --- a/programs/server/HTTPHandler.cpp +++ b/programs/server/HTTPHandler.cpp @@ -783,9 +783,9 @@ void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & req { int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece input(begin, end - begin); - if (compiled_regex->Match(input, 0, end - begin, re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures)) + re2::StringPiece matches[num_captures]; + re2::StringPiece input(begin, end - begin); + if (compiled_regex->Match(input, 0, end - begin, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures)) { for (const auto & [capturing_name, capturing_index] : compiled_regex->NamedCapturingGroups()) { @@ -840,7 +840,7 @@ static inline bool capturingNamedQueryParam(NameSet receive_params, const Compil static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) { - auto compiled_regex = std::make_shared(expression); + auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) throw Exception("Cannot compile re2: " + expression + " for http handling rule, error: " + diff --git a/programs/server/HTTPHandler.h b/programs/server/HTTPHandler.h index a504b88248f..9af5344f70b 100644 --- a/programs/server/HTTPHandler.h +++ b/programs/server/HTTPHandler.h @@ -8,11 +8,6 @@ #include #include -#if USE_RE2_ST -#include -#else -#define re2_st re2 -#endif namespace CurrentMetrics { @@ -26,7 +21,7 @@ namespace DB class WriteBufferFromHTTPServerResponse; -typedef std::shared_ptr CompiledRegexPtr; +typedef std::shared_ptr CompiledRegexPtr; class HTTPHandler : public Poco::Net::HTTPRequestHandler { diff --git a/programs/server/HTTPHandlerFactory.cpp b/programs/server/HTTPHandlerFactory.cpp index a576c4a781c..91cf9ddf25b 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/programs/server/HTTPHandlerFactory.cpp @@ -12,12 +12,6 @@ #include "InterserverIOHTTPHandler.h" #include "PrometheusRequestHandler.h" -#if USE_RE2_ST -#include -#else -#define re2_st re2 -#endif - namespace DB { diff --git a/programs/server/HTTPHandlerRequestFilter.h b/programs/server/HTTPHandlerRequestFilter.h index 13d2e17dd80..b0b748506e5 100644 --- a/programs/server/HTTPHandlerRequestFilter.h +++ b/programs/server/HTTPHandlerRequestFilter.h @@ -9,11 +9,6 @@ #include -#if USE_RE2_ST -#include -#else -#define re2_st re2 -#endif namespace DB { @@ -24,29 +19,24 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -static inline bool checkRegexExpression(const StringRef & match_str, const StringRef & expression) + +typedef std::shared_ptr CompiledRegexPtr; + +static inline bool checkRegexExpression(const StringRef & match_str, const CompiledRegexPtr & compiled_regex) { - re2_st::StringPiece regex(expression.data, expression.size); - - auto compiled_regex = std::make_shared(regex); - - if (!compiled_regex->ok()) - throw Exception("cannot compile re2: " + expression.toString() + " for http handling rule, error: " + compiled_regex->error() + - ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); - int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; - re2_st::StringPiece matches[num_captures]; - re2_st::StringPiece match_input(match_str.data, match_str.size); - return compiled_regex->Match(match_input, 0, match_str.size, re2_st::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); + re2::StringPiece matches[num_captures]; + re2::StringPiece match_input(match_str.data, match_str.size); + return compiled_regex->Match(match_input, 0, match_str.size, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures); } -static inline bool checkExpression(const StringRef & match_str, const std::string & expression) +static inline bool checkExpression(const StringRef & match_str, const std::pair & expression) { - if (startsWith(expression, "regex:")) - return checkRegexExpression(match_str, expression.substr(6)); + if (expression.second) + return checkRegexExpression(match_str, expression.second); - return match_str == expression; + return match_str == expression.first; } static inline auto methodsFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) @@ -60,9 +50,22 @@ static inline auto methodsFilter(Poco::Util::AbstractConfiguration & config, con return [methods](const Poco::Net::HTTPServerRequest & request) { return std::count(methods.begin(), methods.end(), request.getMethod()); }; } +static inline auto getExpression(const std::string & expression) +{ + if (!startsWith(expression, "regex:")) + return std::make_pair(expression, CompiledRegexPtr{}); + + auto compiled_regex = std::make_shared(expression.substr(6)); + + if (!compiled_regex->ok()) + throw Exception("cannot compile re2: " + expression + " for http handling rule, error: " + compiled_regex->error() + + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); + return std::make_pair(expression, compiled_regex); +} + static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const std::string & config_path) { - return [expression = config.getString(config_path)](const Poco::Net::HTTPServerRequest & request) + return [expression = getExpression(config.getString(config_path))](const Poco::Net::HTTPServerRequest & request) { const auto & uri = request.getURI(); const auto & end = find_first_symbols<'?'>(uri.data(), uri.data() + uri.size()); @@ -73,13 +76,13 @@ static inline auto urlFilter(Poco::Util::AbstractConfiguration & config, const s static inline auto headersFilter(Poco::Util::AbstractConfiguration & config, const std::string & prefix) { - std::unordered_map headers_expression; + std::unordered_map> headers_expression; Poco::Util::AbstractConfiguration::Keys headers_name; config.keys(prefix, headers_name); for (const auto & header_name : headers_name) { - const auto & expression = config.getString(prefix + "." + header_name); + const auto & expression = getExpression(config.getString(prefix + "." + header_name)); checkExpression("", expression); /// Check expression syntax is correct headers_expression.emplace(std::make_pair(header_name, expression)); }