Merge pull request #7092 from PerformanceVision/acl_file_storage

Let the choice to write credential for HTTP source on external dictionaries
This commit is contained in:
alexey-milovidov 2019-09-30 13:20:33 +03:00 committed by GitHub
commit 42c9ea9aa3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 97 additions and 7 deletions

View File

@ -34,6 +34,29 @@ HTTPDictionarySource::HTTPDictionarySource(
, context(context_)
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
{
const auto & credentials_prefix = config_prefix + ".credentials";
if (config.has(credentials_prefix))
{
credentials.setUsername(config.getString(credentials_prefix + ".user", ""));
credentials.setPassword(config.getString(credentials_prefix + ".password", ""));
}
const auto & headers_prefix = config_prefix + ".headers";
if (config.has(headers_prefix))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(headers_prefix, config_keys);
header_entries.reserve(config_keys.size());
for (const auto & key : config_keys)
{
const auto header_key = config.getString(headers_prefix + "." + key + ".name", "");
const auto header_value = config.getString(headers_prefix + "." + key + ".value", "");
header_entries.emplace_back(std::make_tuple(header_key, header_value));
}
}
}
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
@ -41,12 +64,15 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, url{other.url}
, header_entries{other.header_entries}
, update_field{other.update_field}
, format{other.format}
, sample_block{other.sample_block}
, context(other.context)
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
{
credentials.setUsername(other.credentials.getUsername());
credentials.setPassword(other.credentials.getPassword());
}
void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri)
@ -74,7 +100,8 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll()
LOG_TRACE(log, "loadAll " + toString());
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
@ -85,7 +112,8 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
getUpdateFieldAndDate(uri);
LOG_TRACE(log, "loadUpdatedAll " + uri.toString());
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
@ -102,7 +130,9 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & id
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts,
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
@ -119,7 +149,9 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns,
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts,
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}

View File

@ -1,6 +1,8 @@
#pragma once
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/URI.h>
#include <common/LocalDateTime.h>
#include "DictionaryStructure.h"
@ -56,6 +58,8 @@ private:
std::chrono::time_point<std::chrono::system_clock> update_time;
const DictionaryStructure dict_struct;
const std::string url;
Poco::Net::HTTPBasicCredentials credentials;
ReadWriteBufferFromHTTP::HTTPHeaderEntries header_entries;
std::string update_field;
const std::string format;
Block sample_block;

View File

@ -86,6 +86,10 @@ namespace detail
template <typename UpdatableSessionPtr>
class ReadWriteBufferFromHTTPBase : public ReadBuffer
{
public:
using HTTPHeaderEntry = std::tuple<std::string, std::string>;
using HTTPHeaderEntries = std::vector<HTTPHeaderEntry>;
protected:
Poco::URI uri;
std::string method;
@ -96,6 +100,7 @@ namespace detail
std::function<void(std::ostream &)> out_stream_callback;
const Poco::Net::HTTPBasicCredentials & credentials;
std::vector<Poco::Net::HTTPCookie> cookies;
HTTPHeaderEntries http_header_entries;
std::istream * call(const Poco::URI uri_, Poco::Net::HTTPResponse & response)
{
@ -109,6 +114,11 @@ namespace detail
if (out_stream_callback)
request.setChunkedTransferEncoding(true);
for (auto & http_header_entry: http_header_entries)
{
request.set(std::get<0>(http_header_entry), std::get<1>(http_header_entry));
}
if (!credentials.getUsername().empty())
credentials.authenticate(request);
@ -146,13 +156,15 @@ namespace detail
const std::string & method_ = {},
OutStreamCallback out_stream_callback_ = {},
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
HTTPHeaderEntries http_header_entries_ = {})
: ReadBuffer(nullptr, 0)
, uri {uri_}
, method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
, session {session_}
, out_stream_callback {out_stream_callback_}
, credentials {credentials_}
, http_header_entries {http_header_entries_}
{
Poco::Net::HTTPResponse response;
@ -230,8 +242,9 @@ public:
const ConnectionTimeouts & timeouts = {},
const DB::SettingUInt64 max_redirects = 0,
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects), uri_, method_, out_stream_callback_, credentials_, buffer_size_)
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const HTTPHeaderEntries & http_header_entries_ = {})
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects), uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_)
{
}
};

View File

@ -333,6 +333,16 @@ class SourceHTTPBase(ExternalSource):
<http>
<url>{url}</url>
<format>TabSeparated</format>
<credentials>
<user>foo</user>
<password>bar</password>
</credentials>
<headers>
<header>
<name>api-key</name>
<value>secret</value>
</header>
</headers>
</http>
'''.format(url=url)

View File

@ -6,12 +6,26 @@ import ssl
import csv
# Decorator used to see if authentification works for external dictionary who use a HTTP source.
def check_auth(fn):
def wrapper(req):
auth_header = req.headers.get('authorization', None)
api_key = req.headers.get('api-key', None)
if not auth_header or auth_header != 'Basic Zm9vOmJhcg==' or not api_key or api_key != 'secret':
req.send_response(401)
else:
fn(req)
return wrapper
def start_server(server_address, data_path, schema, cert_path, address_family):
class TSVHTTPHandler(BaseHTTPRequestHandler):
@check_auth
def do_GET(self):
self.__send_headers()
self.__send_data()
@check_auth
def do_POST(self):
ids = self.__read_and_decode_post_ids()
print "ids=", ids

View File

@ -85,6 +85,16 @@ Example of settings:
<http>
<url>http://[::1]/os.tsv</url>
<format>TabSeparated</format>
<credentials>
<user>user</user>
<password>password</password>
</credentials>
<headers>
<header>
<name>API-KEY</name>
<value>key</value>
</header>
</headers>
</http>
</source>
```
@ -95,6 +105,13 @@ Setting fields:
- `url` The source URL.
- `format` The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported.
- `credentials` Basic HTTP authentification. Optional parameter.
- `user` Username required for the authentification.
- `password` Password required for the authentification.
- `headers` All custom HTTP headers entries used for the HTTP request. Optional parameter.
- `header` Single HTTP header entry.
- `name` Identifiant name used for the header send on the request.
- `value` Value set for a specific identifiant name.
## ODBC {#dicts-external_dicts_dict_sources-odbc}