提交 57c336f2 编写于 作者: P proller 提交者: alexey-milovidov

HTTPDictionarySource support HTTPS (#510)

* https support in dictionaries

* Style
上级 6f106c74
......@@ -11,7 +11,7 @@
# Crypto
# Data
# Net
# NetSSL_OpenSSL
# NetSSL
# OSP
#
# Usage:
......@@ -121,10 +121,16 @@ foreach( component ${components} )
else ()
set (component_in_data "")
endif ()
if (${component} STREQUAL "NetSSL")
set (component_alt "Net")
else ()
set (component_alt ${component})
endif ()
find_path(Poco_${component}_INCLUDE_DIR
NAMES
Poco/${component}.h # e.g. Foundation.h
Poco/${component}/${component}.h # e.g. OSP/OSP.h Util/Util.h
Poco/${component_alt}/${component}.h # e.g. Net/NetSSL.h
Poco/Data/${component_in_data}/${component_in_data}.h # e.g. Data/ODBC/ODBC.h
HINTS
${Poco_ROOT_DIR}
......
......@@ -2,14 +2,17 @@ if (NOT USE_INTERNAL_BOOST_LIBRARY)
set (Boost_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES})
set (BOOST_ROOT "/usr/local")
find_package (Boost 1.55 COMPONENTS program_options system filesystem regex thread)
if (NOT Boost_FOUND)
# Try to find manually.
set (BOOST_PATHS "")
find_library (Boost_PROGRAM_OPTIONS_LIBRARY boost_program_options PATHS ${BOOST_PATHS})
find_library (Boost_SYSTEM_LIBRARY boost_system PATHS ${BOOST_PATHS})
find_library (Boost_FILESYSTEM_LIBRARY boost_filesystem PATHS ${BOOST_PATHS})
# incomplete, no include search, who use it?
#if (NOT Boost_FOUND)
# # Try to find manually.
# set (BOOST_PATHS "")
# find_library (Boost_PROGRAM_OPTIONS_LIBRARY boost_program_options PATHS ${BOOST_PATHS})
# find_library (Boost_SYSTEM_LIBRARY boost_system PATHS ${BOOST_PATHS})
# find_library (Boost_FILESYSTEM_LIBRARY boost_filesystem PATHS ${BOOST_PATHS})
#endif ()
if (Boost_INCLUDE_DIRS)
include_directories (${Boost_INCLUDE_DIRS})
endif ()
include_directories (${Boost_INCLUDE_DIRS})
endif ()
if (NOT Boost_SYSTEM_LIBRARY)
......
if (ENABLE_LIBTCMALLOC)
#contrib/libtcmalloc doesnt build debug version, try find in system
if (DEBUG_LIBTCMALLOC OR NOT USE_INTERNAL_GPERFTOOLS_LIBRARY)
find_package (Gperftools REQUIRED)
include_directories (${GPERFTOOLS_INCLUDE_DIR})
find_package (Gperftools)
if (GPERFTOOLS_FOUND)
include_directories (${GPERFTOOLS_INCLUDE_DIR})
endif ()
endif ()
if (NOT (GPERFTOOLS_INCLUDE_DIR AND GPERFTOOLS_TCMALLOC_MINIMAL))
if (NOT (GPERFTOOLS_FOUND AND GPERFTOOLS_INCLUDE_DIR AND GPERFTOOLS_TCMALLOC_MINIMAL))
set (USE_INTERNAL_GPERFTOOLS_LIBRARY 1)
set (GPERFTOOLS_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libtcmalloc/include")
set (GPERFTOOLS_TCMALLOC_MINIMAL tcmalloc_minimal_internal)
......
......@@ -4,7 +4,7 @@ if (APPLE OR CMAKE_SYSTEM MATCHES "FreeBSD")
set (ICONV_PATHS "/usr/local/opt/libiconv/lib")
find_library (ICONV_LIBRARY iconv PATHS ${ICONV_PATHS})
message (STATUS "Using iconv: ${ICONV_INCLUDE_DIR} : ${ICONV_LIBRARY}")
include_directories(${ICONV_INCLUDE_DIR})
include_directories (${ICONV_INCLUDE_DIR})
else ()
set (ICONV_LIBRARY "")
endif ()
if (ENABLE_JEMALLOC)
find_package(JeMalloc)
include_directories (${JEMALLOC_INCLUDE_DIR})
if (JEMALLOC_INCLUDE_DIR AND JEMALLOC_LIBRARIES)
include_directories (${JEMALLOC_INCLUDE_DIR})
set (USE_JEMALLOC 1)
if (USE_TCMALLOC)
message(WARNING "Disabling tcmalloc")
message (WARNING "Disabling tcmalloc")
set (USE_TCMALLOC 0)
endif ()
endif ()
message(STATUS "Using jemalloc=${USE_JEMALLOC}: ${JEMALLOC_INCLUDE_DIR} : ${JEMALLOC_LIBRARIES}")
message (STATUS "Using jemalloc=${USE_JEMALLOC}: ${JEMALLOC_INCLUDE_DIR} : ${JEMALLOC_LIBRARIES}")
endif ()
if (NOT USE_INTERNAL_POCO_LIBRARY)
find_package (Poco COMPONENTS Net XML Data Crypto DataODBC MongoDB)
if (Poco_INCLUDE_DIRS)
include_directories (${Poco_INCLUDE_DIRS})
endif ()
find_package (Poco COMPONENTS Net NetSSL XML Data Crypto DataODBC MongoDB)
endif ()
if (NOT (Poco_INCLUDE_DIRS AND Poco_Foundation_LIBRARY AND Poco_MongoDB_LIBRARY AND Poco_DataODBC_LIBRARY))
if (Poco_INCLUDE_DIRS AND Poco_Foundation_LIBRARY AND Poco_MongoDB_LIBRARY AND Poco_DataODBC_LIBRARY AND Poco_NetSSL_LIBRARY)
include_directories (${Poco_INCLUDE_DIRS})
else ()
set (USE_INTERNAL_POCO_LIBRARY 1)
set (Poco_INCLUDE_DIRS
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Foundation/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Util/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Net/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/NetSSL_OpenSSL/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Data/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Data/ODBC/include/"
"${ClickHouse_SOURCE_DIR}/contrib/libpoco/Crypto/include/"
......@@ -23,15 +23,16 @@ if (NOT (Poco_INCLUDE_DIRS AND Poco_Foundation_LIBRARY AND Poco_MongoDB_LIBRARY
set (Poco_INCLUDE_DIRS ${Poco_INCLUDE_DIRS} "${ClickHouse_SOURCE_DIR}/contrib/libzlib-ng/" "${ClickHouse_BINARY_DIR}/contrib/libzlib-ng/")
endif ()
set (Poco_Net_LIBRARY PocoNet)
set (Poco_Foundation_LIBRARY PocoFoundation)
set (Poco_Util_LIBRARY PocoUtil)
set (Poco_Net_LIBRARY PocoNet)
set (Poco_NetSSL_LIBRARY PocoNetSSL)
set (Poco_XML_LIBRARY PocoXML)
set (Poco_Data_LIBRARY PocoData)
set (Poco_Crypto_LIBRARY PocoCrypto)
set (Poco_DataODBC_LIBRARY PocoDataODBC)
set (Poco_MongoDB_LIBRARY PocoMongoDB)
set (Poco_Foundation_LIBRARY PocoFoundation)
include_directories (BEFORE ${Poco_INCLUDE_DIRS})
endif ()
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Net_LIBRARY},${Poco_Util_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_MongoDB_LIBRARY},${Poco_Foundation_LIBRARY}")
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Foundation_LIBRARY},${Poco_Util_LIBRARY},${Poco_Net_LIBRARY},${Poco_NetSSL_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_MongoDB_LIBRARY}")
......@@ -150,6 +150,7 @@ target_link_libraries(dbms
${Poco_DataODBC_LIBRARY}
${Poco_MongoDB_LIBRARY}
${Poco_Foundation_LIBRARY}
${Poco_NetSSL_LIBRARY}
${ICONV_LIBRARY}
${PLATFORM_LIBS}
${CMAKE_DL_LIBS}
......
#pragma once
#include <mutex>
namespace Poco
{
......@@ -15,4 +16,7 @@ namespace DB
void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response);
extern std::once_flag client_ssl_init_once;
void clientSSLInit();
}
......@@ -27,7 +27,8 @@ private:
std::string method;
HTTPTimeouts timeouts;
Poco::Net::HTTPClientSession session;
bool is_ssl;
std::unique_ptr<Poco::Net::HTTPClientSession> session;
std::istream * istr; /// owned by session
std::unique_ptr<ReadBuffer> impl;
......
......@@ -12,7 +12,9 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/IO/HTTPCommon.h>
#include <memory>
#include <mutex>
#include <Poco/Data/ODBC/Connector.h>
......@@ -129,11 +131,15 @@ DictionarySourcePtr DictionarySourceFactory::create(
}
else if ("http" == source_type)
{
if (dict_struct.has_expressions)
throw Exception{
"Dictionary source of type `http` does not support attribute expressions",
ErrorCodes::LOGICAL_ERROR};
// Used for https queries
std::call_once(client_ssl_init_once, clientSSLInit);
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
}
......
#include <DB/IO/HTTPCommon.h>
#include <Poco/Util/Application.h>
#include <Poco/Net/AcceptCertificateHandler.h>
#include <Poco/Net/Context.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/InvalidCertificateHandler.h>
#include <Poco/Net/PrivateKeyPassphraseHandler.h>
#include <Poco/Net/RejectCertificateHandler.h>
#include <Poco/Net/SSLManager.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/Application.h>
#include <Poco/Version.h>
namespace DB
......@@ -17,4 +25,29 @@ void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response)
response.set("Keep-Alive", "timeout=" + std::to_string(keep_alive_timeout.totalSeconds()));
}
std::once_flag client_ssl_init_once;
void clientSSLInit()
{
// http://stackoverflow.com/questions/18315472/https-request-in-c-using-poco
Poco::Net::initializeSSL();
bool insecure = Poco::Util::Application::instance().config().getInt("https_client_insecure", false);
Poco::SharedPtr<Poco::Net::InvalidCertificateHandler> ptr_handler(insecure
? dynamic_cast<Poco::Net::InvalidCertificateHandler *>(new Poco::Net::AcceptCertificateHandler(true))
: dynamic_cast<Poco::Net::InvalidCertificateHandler *>(new Poco::Net::RejectCertificateHandler(true)));
Poco::Net::Context::Ptr ptr_context(new Poco::Net::Context(Poco::Net::Context::CLIENT_USE,
"",
"",
"",
insecure ? Poco::Net::Context::VERIFY_NONE : Poco::Net::Context::VERIFY_RELAXED,
9,
true));
ptr_context->enableSessionCache(true);
#if POCO_VERSION >= 0x01070000
ptr_context->disableProtocols(Poco::Net::Context::PROTO_SSLV2 | Poco::Net::Context::PROTO_SSLV3);
ptr_context->preferServerCiphers();
#endif
Poco::Net::SSLManager::instance().initializeClient(nullptr, ptr_handler, ptr_context);
}
}
......@@ -5,14 +5,11 @@
#include <Poco/Net/DNS.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Net/HTTPSClientSession.h>
#include <DB/IO/ReadBufferFromIStream.h>
#include <DB/Common/SimpleCache.h>
#include <common/logger_useful.h>
namespace DB
{
......@@ -45,15 +42,17 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
ReadBuffer(nullptr, 0),
uri{uri},
method{!method.empty() ? method : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET},
timeouts{timeouts}
timeouts{timeouts},
is_ssl{uri.getScheme() == "https"},
session{ std::unique_ptr<Poco::Net::HTTPClientSession>(is_ssl ? new Poco::Net::HTTPSClientSession : new Poco::Net::HTTPClientSession) }
{
session.setHost(resolveHost(uri.getHost()).toString()); /// Cache DNS forever (until server restart)
session.setPort(uri.getPort());
session->setHost(resolveHost(uri.getHost()).toString()); /// Cache DNS forever (until server restart)
session->setPort(uri.getPort());
#if POCO_CLICKHOUSE_PATCH || POCO_VERSION >= 0x02000000
session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
session->setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
#else
session.setTimeout(timeouts.connection_timeout);
session->setTimeout(timeouts.connection_timeout);
#endif
Poco::Net::HTTPRequest request(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
......@@ -64,12 +63,12 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to " << uri.toString());
auto & stream_out = session.sendRequest(request);
auto & stream_out = session->sendRequest(request);
if (out_stream_callback)
out_stream_callback(stream_out);
istr = &session.receiveResponse(response);
istr = &session->receiveResponse(response);
auto status = response.getStatus();
......
......@@ -12,6 +12,7 @@
<listen_host>localhost</listen_host>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<https_client_insecure>1</https_client_insecure>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
......
......@@ -43,7 +43,7 @@ dictionaries = []
def generate_structure(args):
global dictionaries
# [ name, key_type, has_parent ]
dictionaries.extend( [
dictionaries.extend([
# Simple key dictionaries
[ 'file_flat', 0, True ],
[ 'clickhouse_flat', 0, True ],
......@@ -80,8 +80,15 @@ def generate_structure(args):
[ 'http_complex_mixed_key_hashed', 2, False ],
])
if args.use_https:
dictionaries.extend([
[ 'https_flat', 0, True ],
[ 'https_hashed', 0, True ],
[ 'https_cache', 0, True ],
])
if not args.no_mysql:
dictionaries.extend( [
dictionaries.extend([
[ 'mysql_flat', 0, True ],
[ 'mysql_hashed', 0, True ],
[ 'mysql_cache', 0, True ],
......@@ -92,7 +99,7 @@ def generate_structure(args):
])
if not args.no_mongo:
dictionaries.extend( [
dictionaries.extend([
[ 'mongodb_flat', 0, True ],
[ 'mongodb_hashed', 0, True ],
[ 'mongodb_cache', 0, True ],
......@@ -103,7 +110,7 @@ def generate_structure(args):
])
if args.use_mongo_user:
dictionaries.extend( [
dictionaries.extend([
[ 'mongodb_user_flat', 0, True ],
])
......@@ -340,6 +347,13 @@ def generate_dictionaries(args):
</http>
'''.format(http_host=args.http_host, http_port=args.http_port, http_path=args.http_path)
source_https = '''
<http>
<url>https://{https_host}:{https_port}{https_path}%s</url>
<format>TabSeparated</format>
</http>
'''.format(https_host=args.https_host, https_port=args.https_port, https_path=args.https_path)
layout_flat = '<flat />'
layout_hashed = '<hashed />'
layout_cache = '<cache><size_in_cells>128</size_in_cells></cache>'
......@@ -426,8 +440,15 @@ def generate_dictionaries(args):
[ source_http % (files[2]), layout_complex_key_cache ],
]
if args.use_https:
sources_and_layouts.extend([
[ source_https % (files[0]), layout_flat ],
[ source_https % (files[0]), layout_hashed ],
[ source_https % (files[0]), layout_cache ],
])
if not args.no_mysql:
sources_and_layouts.extend( [
sources_and_layouts.extend([
[ source_mysql, layout_flat ],
[ source_mysql, layout_hashed ],
[ source_mysql, layout_cache ],
......@@ -438,7 +459,7 @@ def generate_dictionaries(args):
])
if not args.no_mongo:
sources_and_layouts.extend( [
sources_and_layouts.extend([
[ source_mongodb, layout_flat ],
[ source_mongodb, layout_hashed ],
[ source_mongodb, layout_cache ],
......@@ -463,11 +484,17 @@ def generate_dictionaries(args):
def run_tests(args):
if args.use_http:
http_server = subprocess.Popen(["python", "http_server.py", str(args.http_port)]);
http_server = subprocess.Popen(["python", "http_server.py", "--port", str(args.http_port), "--host", args.http_host]);
@atexit.register
def http_killer():
http_server.kill()
if args.use_https:
https_server = subprocess.Popen(["python", "http_server.py", "--port", str(args.https_port), "--host", args.https_host, '--https']);
@atexit.register
def https_killer():
https_server.kill()
keys = [ 'toUInt64(n)', '(n, n)', '(toString(n), n)' ]
dict_get_query_skeleton = "select dictGet{type}('{name}', '{type}_', {key}) from system.one array join range(8) as n;"
dict_has_query_skeleton = "select dictHas('{name}', {key}) from system.one array join range(8) as n;"
......@@ -641,6 +668,10 @@ if __name__ == '__main__':
parser.add_argument('--http_port', default = 58000, help = 'http server port')
parser.add_argument('--http_host', default = 'localhost', help = 'http server host')
parser.add_argument('--http_path', default = '/generated/', help = 'http server path')
parser.add_argument('--use_https', default = True, help = 'Use https dictionaries')
parser.add_argument('--https_port', default = 58443, help = 'https server port')
parser.add_argument('--https_host', default = 'localhost', help = 'https server host')
parser.add_argument('--https_path', default = '/generated/', help = 'https server path')
parser.add_argument('--no_break', action='store_true', help = 'Dont stop on errors')
args = parser.parse_args()
......
......@@ -4,10 +4,17 @@ import socket
import sys
import BaseHTTPServer
import SocketServer
import ssl
import argparse
PORT_NUMBER = 80
if len(sys.argv) > 1 and int(sys.argv[1]):
PORT_NUMBER = int(sys.argv[1])
parser = argparse.ArgumentParser(description = 'Simple http/https server')
parser.add_argument('--https', action='store_true', help = 'Use https')
parser.add_argument('--port', type = int, default = 80, help = 'server port')
parser.add_argument('--host', default = "localhost", help = 'server host')
args = parser.parse_args()
if args.https and args.port == 80:
args.port = 443
class myHTTPServer(SocketServer.ForkingMixIn, BaseHTTPServer.HTTPServer):
address_family = socket.AF_INET6
......@@ -35,8 +42,12 @@ class myHandler(BaseHTTPServer.BaseHTTPRequestHandler):
return
try:
server = myHTTPServer(('', PORT_NUMBER), myHandler)
print 'Started httpserver on port' , PORT_NUMBER
server = myHTTPServer(('', args.port), myHandler)
if args.https:
os.system('openssl req -subj "/CN={host}/O=My Company Name LTD./C=US" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout server.key -out server.crt'.format(host=args.host))
server.socket = ssl.wrap_socket(server.socket, keyfile="server.key", certfile='server.crt', server_side=True)
print 'Started http' + ( 's' if args.https else '' ) + ' server on port' , args.port
server.serve_forever()
except KeyboardInterrupt:
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册