ClickHouse/src/IO/ReadWriteBufferFromHTTP.h

325 lines
10 KiB
C++
Raw Normal View History

2016-11-19 00:07:58 +00:00
#pragma once
#include <functional>
2020-09-15 09:55:57 +00:00
#include <common/types.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/HTTPCommon.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromIStream.h>
#include <Poco/Any.h>
#include <Poco/Net/HTTPBasicCredentials.h>
2016-11-19 00:07:58 +00:00
#include <Poco/Net/HTTPClientSession.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
2016-11-24 01:01:11 +00:00
#include <Poco/URI.h>
#include <Poco/Version.h>
#include <Common/DNSResolver.h>
#include <Common/RemoteHostFilter.h>
#include <common/logger_useful.h>
2019-09-11 16:39:30 +00:00
#include <Poco/URIStreamFactory.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
2016-11-19 00:07:58 +00:00
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1
2017-02-07 06:18:16 +00:00
2016-11-19 00:07:58 +00:00
namespace DB
{
/** Perform HTTP POST request and provide response to read.
*/
namespace ErrorCodes
{
extern const int TOO_MANY_REDIRECTS;
}
template <typename SessionPtr>
class UpdatableSessionBase
{
protected:
SessionPtr session;
UInt64 redirects { 0 };
Poco::URI initial_uri;
const ConnectionTimeouts & timeouts;
UInt64 max_redirects;
public:
virtual void buildNewSession(const Poco::URI & uri) = 0;
explicit UpdatableSessionBase(const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
UInt64 max_redirects_)
: initial_uri { uri }
, timeouts { timeouts_ }
, max_redirects { max_redirects_ }
{
}
SessionPtr getSession()
{
return session;
}
void updateSession(const Poco::URI & uri)
{
2019-09-23 18:16:34 +00:00
++redirects;
if (redirects <= max_redirects)
{
buildNewSession(uri);
}
else
{
2020-11-07 00:14:53 +00:00
throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects while trying to access {}", initial_uri.toString());
}
}
2021-02-04 19:06:43 +00:00
virtual ~UpdatableSessionBase() = default;
};
2019-09-23 18:16:34 +00:00
namespace detail
2016-11-19 00:07:58 +00:00
{
template <typename UpdatableSessionPtr>
class ReadWriteBufferFromHTTPBase : public ReadBuffer
{
public:
2019-09-26 03:34:22 +00:00
using HTTPHeaderEntry = std::tuple<std::string, std::string>;
using HTTPHeaderEntries = std::vector<HTTPHeaderEntry>;
protected:
Poco::URI uri;
std::string method;
2016-11-19 00:07:58 +00:00
UpdatableSessionPtr session;
std::istream * istr; /// owned by session
std::unique_ptr<ReadBuffer> impl;
std::function<void(std::ostream &)> out_stream_callback;
const Poco::Net::HTTPBasicCredentials & credentials;
2019-09-06 12:18:56 +00:00
std::vector<Poco::Net::HTTPCookie> cookies;
2019-09-26 03:34:22 +00:00
HTTPHeaderEntries http_header_entries;
2019-10-23 11:58:35 +00:00
RemoteHostFilter remote_host_filter;
2020-10-27 13:00:40 +00:00
std::function<void(size_t)> next_callback;
2016-11-19 00:07:58 +00:00
std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response)
{
// With empty path poco will send "POST HTTP/1.1" its bug.
if (uri_.getPath().empty())
uri_.setPath("/");
Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
request.setHost(uri_.getHost()); // use original, not resolved host name in header
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);
LOG_TRACE((&Poco::Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri_.toString());
auto sess = session->getSession();
try
{
2019-09-20 12:36:09 +00:00
auto & stream_out = sess->sendRequest(request);
if (out_stream_callback)
out_stream_callback(stream_out);
istr = receiveResponse(*sess, request, response, true);
response.getCookies(cookies);
return istr;
2016-11-25 00:16:20 +00:00
}
catch (const Poco::Exception & e)
{
/// We use session data storage as storage for exception text
/// Depend on it we can deduce to reconnect session or reresolve session host
sess->attachSessionData(e.message());
throw;
}
}
public:
2020-10-27 13:00:40 +00:00
using NextCallback = std::function<void(size_t)>;
using OutStreamCallback = std::function<void(std::ostream &)>;
explicit ReadWriteBufferFromHTTPBase(
UpdatableSessionPtr session_,
Poco::URI uri_,
const std::string & method_ = {},
OutStreamCallback out_stream_callback_ = {},
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
2019-10-23 11:58:35 +00:00
HTTPHeaderEntries http_header_entries_ = {},
const RemoteHostFilter & remote_host_filter_ = {})
: ReadBuffer(nullptr, 0)
, uri {uri_}
2019-09-18 14:53:45 +00:00
, 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_}
2019-10-23 11:58:35 +00:00
, remote_host_filter {remote_host_filter_}
{
Poco::Net::HTTPResponse response;
istr = call(uri, response);
while (isRedirect(response.getStatus()))
{
Poco::URI uri_redirect(response.get("Location"));
2019-10-23 11:58:35 +00:00
remote_host_filter.checkURL(uri_redirect);
session->updateSession(uri_redirect);
2020-11-02 10:45:37 +00:00
istr = call(uri_redirect, response);
}
try
{
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
}
catch (const Poco::Exception & e)
{
/// We use session data storage as storage for exception text
/// Depend on it we can deduce to reconnect session or reresolve session host
auto sess = session->getSession();
sess->attachSessionData(e.message());
throw;
}
}
bool nextImpl() override
{
2020-10-27 13:00:40 +00:00
if (next_callback)
next_callback(count());
2021-02-04 19:06:43 +00:00
if (!working_buffer.empty())
impl->position() = position();
if (!impl->next())
return false;
internal_buffer = impl->buffer();
working_buffer = internal_buffer;
return true;
}
2019-09-06 12:18:56 +00:00
std::string getResponseCookie(const std::string & name, const std::string & def) const
{
for (const auto & cookie : cookies)
if (cookie.getName() == name)
return cookie.getValue();
return def;
}
2020-10-26 16:38:35 +00:00
2020-10-27 12:50:43 +00:00
/// Set function to call on each nextImpl, useful when you need to track
/// progress.
/// NOTE: parameter on each call is not incremental -- it's all bytes count
/// passed through the buffer
2020-10-27 13:00:40 +00:00
void setNextCallback(NextCallback next_callback_)
2020-10-26 16:38:35 +00:00
{
2020-10-27 13:00:40 +00:00
next_callback = next_callback_;
2020-10-27 12:50:43 +00:00
/// Some data maybe already read
2020-10-27 13:00:40 +00:00
next_callback(count());
2020-10-26 16:38:35 +00:00
}
};
}
class UpdatableSession : public UpdatableSessionBase<HTTPSessionPtr>
{
using Parent = UpdatableSessionBase<HTTPSessionPtr>;
public:
2020-11-02 10:45:37 +00:00
explicit UpdatableSession(
const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
const UInt64 max_redirects_)
: Parent(uri, timeouts_, max_redirects_)
{
session = makeHTTPSession(initial_uri, timeouts);
}
void buildNewSession(const Poco::URI & uri) override
{
session = makeHTTPSession(uri, timeouts);
}
};
class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>
{
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession>>;
public:
2020-11-02 10:45:37 +00:00
explicit ReadWriteBufferFromHTTP(
Poco::URI uri_,
const std::string & method_,
OutStreamCallback out_stream_callback_,
const ConnectionTimeouts & timeouts,
const UInt64 max_redirects = 0,
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
2019-10-23 11:58:35 +00:00
const HTTPHeaderEntries & http_header_entries_ = {},
const RemoteHostFilter & remote_host_filter_ = {})
2020-11-02 10:45:37 +00:00
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects),
uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_, remote_host_filter_)
{
}
};
class UpdatablePooledSession : public UpdatableSessionBase<PooledHTTPSessionPtr>
{
using Parent = UpdatableSessionBase<PooledHTTPSessionPtr>;
private:
size_t per_endpoint_pool_size;
public:
explicit UpdatablePooledSession(const Poco::URI uri,
const ConnectionTimeouts & timeouts_,
const UInt64 max_redirects_,
size_t per_endpoint_pool_size_)
: Parent(uri, timeouts_, max_redirects_)
, per_endpoint_pool_size { per_endpoint_pool_size_ }
{
session = makePooledHTTPSession(initial_uri, timeouts, per_endpoint_pool_size);
}
void buildNewSession(const Poco::URI & uri) override
{
session = makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size);
}
};
class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatablePooledSession>>
{
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatablePooledSession>>;
2016-11-19 00:07:58 +00:00
public:
explicit PooledReadWriteBufferFromHTTP(Poco::URI uri_,
const std::string & method_ = {},
OutStreamCallback out_stream_callback_ = {},
const ConnectionTimeouts & timeouts_ = {},
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const UInt64 max_redirects = 0,
2018-11-16 13:33:43 +00:00
size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT)
: Parent(std::make_shared<UpdatablePooledSession>(uri_, timeouts_, max_redirects, max_connections_per_endpoint),
uri_,
method_,
out_stream_callback_,
credentials_,
buffer_size_)
{
}
2016-11-19 00:07:58 +00:00
};
}