diff --git a/dbms/programs/server/HTTPHandlerFactory.h b/dbms/programs/server/HTTPHandlerFactory.h new file mode 100644 index 0000000000000000000000000000000000000000..73e0a2b733993684cd628b8d71b95b0fb42a0dab --- /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 d3d0bfc5aeb95abfe595b12f66b8c2fa1aea3183..3e984deb8aadfded2215e3fab357da3108a633df 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 0000000000000000000000000000000000000000..154ebf6c35e1d441a879eee3d5266beda4c2cf98 --- /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 0000000000000000000000000000000000000000..9755c29d480b6d4f04f60571e8806c3a4fed0aa7 --- /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 eb56cd6c18c215d65b3b81dd6c6f8117f0dc58f1..0000000000000000000000000000000000000000 --- 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 0000000000000000000000000000000000000000..6fab88bcf809653aca3d2502a4b45d516fc38213 --- /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 0000000000000000000000000000000000000000..9aba4ac091429b942158eb1af40251a92e1f9929 --- /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 805c616383903daec4d4cc08e9541cca1b2f04de..08f8d37586cfb18cdffbac2b9a0800391f3d938b 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 1c04dee6c3a6e488bdd462da027389b2413bbc7e..706149b66b16b7de2d7796c34a85fe47a1048aa1 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 a9fd572a8b0c9444074718cd9c77955938079b78..efe47da9f46712f83e2df75752669d11c4d5ec2c 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 79e39d837042e8c5d222140e77ed2ccbc30b2e03..83ac8c55f24056f7f9391108cb2a62ce1f70b142 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/';