ClickHouse/src/IO/HTTPCommon.cpp

446 lines
15 KiB
C++
Raw Normal View History

#include <IO/HTTPCommon.h>
#include <Server/HTTP/HTTPServerResponse.h>
2023-07-03 21:11:32 +00:00
#include <Poco/Any.h>
2023-07-14 09:09:22 +00:00
#include <Common/Concepts.h>
#include <Common/DNSResolver.h>
#include <Common/Exception.h>
#include <Common/MemoryTrackerSwitcher.h>
2019-02-10 17:40:52 +00:00
#include <Common/PoolBase.h>
#include <Common/ProfileEvents.h>
#include <Common/SipHash.h>
#include "config.h"
#if USE_SSL
# include <Poco/Net/AcceptCertificateHandler.h>
# include <Poco/Net/Context.h>
# include <Poco/Net/HTTPSClientSession.h>
# include <Poco/Net/InvalidCertificateHandler.h>
# include <Poco/Net/PrivateKeyPassphraseHandler.h>
# include <Poco/Net/RejectCertificateHandler.h>
# include <Poco/Net/SSLManager.h>
# include <Poco/Net/SecureStreamSocket.h>
#endif
2018-11-18 22:15:42 +00:00
#include <Poco/Util/Application.h>
2017-01-30 05:13:58 +00:00
2023-07-14 09:09:22 +00:00
#include <sstream>
2019-02-10 17:40:52 +00:00
#include <tuple>
#include <unordered_map>
2018-11-18 22:15:42 +00:00
namespace ProfileEvents
{
extern const Event CreatedHTTPConnections;
}
namespace DB
{
namespace ErrorCodes
{
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
2018-06-16 05:54:06 +00:00
extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME;
extern const int UNSUPPORTED_URI_SCHEME;
2023-07-03 21:11:32 +00:00
extern const int LOGICAL_ERROR;
}
2018-06-16 05:54:06 +00:00
namespace
{
2019-05-31 15:50:21 +00:00
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts)
{
session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
2019-03-29 18:10:03 +00:00
session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout);
}
2023-07-14 09:09:22 +00:00
template <typename Session>
requires std::derived_from<Session, Poco::Net::HTTPClientSession>
class HTTPSessionAdapter : public Session
{
static_assert(std::has_virtual_destructor_v<Session>, "The base class must have a virtual destructor");
public:
HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{&Poco::Logger::get("HTTPSessionAdapter")} { }
~HTTPSessionAdapter() override = default;
protected:
void reconnect() override
{
// First of all will try to establish connection with last used addr.
if (!Session::getResolvedHost().empty())
{
try
{
Session::reconnect();
return;
}
catch (...)
{
Session::close();
LOG_TRACE(
log,
2023-07-20 14:56:30 +00:00
"Last ip ({}) is unreachable for {}:{}. Will try another resolved address.",
2023-07-14 09:09:22 +00:00
Session::getResolvedHost(),
Session::getHost(),
Session::getPort());
}
}
const auto endpoinds = DNSResolver::instance().resolveHostAll(Session::getHost());
for (auto it = endpoinds.begin();;)
{
try
{
Session::setResolvedHost(it->toString());
Session::reconnect();
LOG_TRACE(
log,
"Created HTTP(S) session with {}:{} ({}:{})",
Session::getHost(),
Session::getPort(),
it->toString(),
Session::getPort());
break;
}
catch (...)
{
Session::close();
if (++it == endpoinds.end())
{
Session::setResolvedHost("");
throw;
}
LOG_TRACE(
log,
"Failed to create connection with {}:{}, Will try another resolved address. {}",
Session::getResolvedHost(),
Session::getPort(),
getCurrentExceptionMessage(false));
}
}
}
Poco::Logger * log;
};
bool isHTTPS(const Poco::URI & uri)
{
if (uri.getScheme() == "https")
return true;
else if (uri.getScheme() == "http")
return false;
else
throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString());
}
Add global proxy setting (#51749) * initial impl * fix env ut * move ut directory * make sure no null proxy resolver is returned by ProxyConfigurationResolverProvider * minor adjustment * add a few tests, still incomplete * add proxy support for url table function * use proxy for select from url as well * remove optional from return type, just returns empty config * fix style * style * black * ohg boy * rm in progress file * god pls don't let me kill anyone * ... * add use_aws guards * remove hard coded s3 proxy resolver * add concurrency-mt-unsafe * aa * black * add logging back * revert change * imrpove code a bit * helper functions and separate tests * for some reason, this env test is not working.. * formatting * :) * clangtidy * lint * revert some stupid things * small test adjusmtments * simplify tests * rename test * remove extra line * freaking style change * simplify a bit * fix segfault & remove an extra call * tightly couple proxy provider with context.. * remove useless include * rename config prefix parameter * simplify provider a bit * organize provider a bit * add a few comments * comment out proxy env tests * fix nullptr in unit tests * make sure old storage proxy config is properly covered without global context instance * move a few functions from class to anonymous namespace * fix no fallback for specific storage conf * change API to accept http method instead of bool * implement http/https distinction in listresolver, any still not implemented * implement http/https distinction in remote resolver * progress on code, improve tests and add url function working test * use protcol instead of method for http and https * small fix * few more adjustments * fix style * black * move enum to proxyconfiguration * wip * fix build * fix ut * delete atomicroundrobin class * remove stale include * add some tests.. need to spend some more time on the design.. * change design a bit * progress * use existing context for tests * rename aux function and fix ut * .. * rename test * try to simplify tests a bit * simplify tests a bit more * attempt to fix tests, accept more than one remote resolver * use proper log id * try waiting for resolver * proper wait logic * black * empty * address a few comments * refactor tests * remove old tests * baclk * use RAII to set/unset env * black * clang tidy * fix env proxy not respecting any * use log trace * fix wrong logic in getRemoteREsolver * fix wrong logic in getRemoteREsolver * fix test * remove unwanted code * remove ClientConfigurationperRequest and auxilary classes * remove unwanted code * remove adapter test * few adjustments and add test for s3 storage conf with new proxy settings * black * use chassert for context * Add getenv comment
2023-08-24 13:07:26 +00:00
HTTPSessionPtr makeHTTPSessionImpl(
const std::string & host,
UInt16 port,
bool https,
bool keep_alive,
Poco::Net::HTTPClientSession::ProxyConfig proxy_config = {})
{
HTTPSessionPtr session;
if (https)
{
#if USE_SSL
2023-07-14 09:09:22 +00:00
session = std::make_shared<HTTPSessionAdapter<Poco::Net::HTTPSClientSession>>(host, port);
#else
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without HTTPS support");
#endif
}
else
{
2023-07-14 09:09:22 +00:00
session = std::make_shared<HTTPSessionAdapter<Poco::Net::HTTPClientSession>>(host, port);
}
ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections);
/// doesn't work properly without patch
session->setKeepAlive(keep_alive);
Add global proxy setting (#51749) * initial impl * fix env ut * move ut directory * make sure no null proxy resolver is returned by ProxyConfigurationResolverProvider * minor adjustment * add a few tests, still incomplete * add proxy support for url table function * use proxy for select from url as well * remove optional from return type, just returns empty config * fix style * style * black * ohg boy * rm in progress file * god pls don't let me kill anyone * ... * add use_aws guards * remove hard coded s3 proxy resolver * add concurrency-mt-unsafe * aa * black * add logging back * revert change * imrpove code a bit * helper functions and separate tests * for some reason, this env test is not working.. * formatting * :) * clangtidy * lint * revert some stupid things * small test adjusmtments * simplify tests * rename test * remove extra line * freaking style change * simplify a bit * fix segfault & remove an extra call * tightly couple proxy provider with context.. * remove useless include * rename config prefix parameter * simplify provider a bit * organize provider a bit * add a few comments * comment out proxy env tests * fix nullptr in unit tests * make sure old storage proxy config is properly covered without global context instance * move a few functions from class to anonymous namespace * fix no fallback for specific storage conf * change API to accept http method instead of bool * implement http/https distinction in listresolver, any still not implemented * implement http/https distinction in remote resolver * progress on code, improve tests and add url function working test * use protcol instead of method for http and https * small fix * few more adjustments * fix style * black * move enum to proxyconfiguration * wip * fix build * fix ut * delete atomicroundrobin class * remove stale include * add some tests.. need to spend some more time on the design.. * change design a bit * progress * use existing context for tests * rename aux function and fix ut * .. * rename test * try to simplify tests a bit * simplify tests a bit more * attempt to fix tests, accept more than one remote resolver * use proper log id * try waiting for resolver * proper wait logic * black * empty * address a few comments * refactor tests * remove old tests * baclk * use RAII to set/unset env * black * clang tidy * fix env proxy not respecting any * use log trace * fix wrong logic in getRemoteREsolver * fix wrong logic in getRemoteREsolver * fix test * remove unwanted code * remove ClientConfigurationperRequest and auxilary classes * remove unwanted code * remove adapter test * few adjustments and add test for s3 storage conf with new proxy settings * black * use chassert for context * Add getenv comment
2023-08-24 13:07:26 +00:00
session->setProxyConfig(proxy_config);
return session;
}
class SingleEndpointHTTPSessionPool : public PoolBase<Poco::Net::HTTPClientSession>
{
private:
const std::string host;
const UInt16 port;
2023-04-14 10:18:20 +00:00
const bool https;
const String proxy_host;
const UInt16 proxy_port;
2023-04-14 10:18:20 +00:00
const bool proxy_https;
using Base = PoolBase<Poco::Net::HTTPClientSession>;
2023-04-14 10:18:20 +00:00
ObjectPtr allocObject() override
{
/// Pool is global, we shouldn't attribute this memory to query/user.
MemoryTrackerSwitcher switcher{&total_memory_tracker};
2023-07-14 09:09:22 +00:00
auto session = makeHTTPSessionImpl(host, port, https, true);
if (!proxy_host.empty())
{
const String proxy_scheme = proxy_https ? "https" : "http";
session->setProxyHost(proxy_host);
session->setProxyPort(proxy_port);
session->setProxyProtocol(proxy_scheme);
/// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS.
session->setProxyTunnel(!proxy_https && https);
}
return session;
}
public:
SingleEndpointHTTPSessionPool(
2023-04-14 10:18:20 +00:00
const std::string & host_,
UInt16 port_,
bool https_,
const std::string & proxy_host_,
UInt16 proxy_port_,
bool proxy_https_,
size_t max_pool_size_,
2023-06-12 14:19:05 +00:00
bool wait_on_pool_size_limit)
: Base(
static_cast<unsigned>(max_pool_size_),
&Poco::Logger::get("HTTPSessionPool"),
wait_on_pool_size_limit ? BehaviourOnLimit::Wait : BehaviourOnLimit::AllocateNewBypassingPool)
, host(host_)
, port(port_)
, https(https_)
, proxy_host(proxy_host_)
, proxy_port(proxy_port_)
, proxy_https(proxy_https_)
{
}
};
class HTTPSessionPool : private boost::noncopyable
{
public:
struct Key
{
String target_host;
UInt16 target_port;
bool is_target_https;
String proxy_host;
UInt16 proxy_port;
bool is_proxy_https;
2023-06-12 14:19:05 +00:00
bool wait_on_pool_size_limit;
2020-12-21 07:48:26 +00:00
bool operator ==(const Key & rhs) const
{
2023-06-12 14:19:05 +00:00
return std::tie(target_host, target_port, is_target_https, proxy_host, proxy_port, is_proxy_https, wait_on_pool_size_limit)
== std::tie(rhs.target_host, rhs.target_port, rhs.is_target_https, rhs.proxy_host, rhs.proxy_port, rhs.is_proxy_https, rhs.wait_on_pool_size_limit);
2020-12-21 07:48:26 +00:00
}
};
private:
using PoolPtr = std::shared_ptr<SingleEndpointHTTPSessionPool>;
using Entry = SingleEndpointHTTPSessionPool::Entry;
struct Hasher
{
size_t operator()(const Key & k) const
{
SipHash s;
s.update(k.target_host);
s.update(k.target_port);
s.update(k.is_target_https);
s.update(k.proxy_host);
s.update(k.proxy_port);
s.update(k.is_proxy_https);
2023-06-12 14:19:05 +00:00
s.update(k.wait_on_pool_size_limit);
return s.get64();
}
};
std::mutex mutex;
std::unordered_map<Key, PoolPtr, Hasher> endpoints_pool;
protected:
HTTPSessionPool() = default;
public:
static auto & instance()
{
static HTTPSessionPool instance;
return instance;
}
2019-03-29 18:10:03 +00:00
Entry getSession(
const Poco::URI & uri,
const Poco::URI & proxy_uri,
2019-03-29 18:10:03 +00:00
const ConnectionTimeouts & timeouts,
size_t max_connections_per_endpoint,
2023-06-12 14:19:05 +00:00
bool wait_on_pool_size_limit)
{
2023-06-12 14:19:05 +00:00
std::unique_lock lock(mutex);
const std::string & host = uri.getHost();
UInt16 port = uri.getPort();
bool https = isHTTPS(uri);
String proxy_host;
UInt16 proxy_port = 0;
bool proxy_https = false;
if (!proxy_uri.empty())
{
proxy_host = proxy_uri.getHost();
proxy_port = proxy_uri.getPort();
proxy_https = isHTTPS(proxy_uri);
}
2023-06-12 14:19:05 +00:00
HTTPSessionPool::Key key{host, port, https, proxy_host, proxy_port, proxy_https, wait_on_pool_size_limit};
auto pool_ptr = endpoints_pool.find(key);
if (pool_ptr == endpoints_pool.end())
std::tie(pool_ptr, std::ignore) = endpoints_pool.emplace(
2023-06-12 14:19:05 +00:00
key,
std::make_shared<SingleEndpointHTTPSessionPool>(
host,
port,
https,
proxy_host,
proxy_port,
proxy_https,
max_connections_per_endpoint,
wait_on_pool_size_limit));
/// Some routines held session objects until the end of its lifetime. Also this routines may create another sessions in this time frame.
/// If some other session holds `lock` because it waits on another lock inside `pool_ptr->second->get` it isn't possible to create any
/// new session and thus finish routine, return session to the pool and unlock the thread waiting inside `pool_ptr->second->get`.
/// To avoid such a deadlock we unlock `lock` before entering `pool_ptr->second->get`.
lock.unlock();
auto retry_timeout = timeouts.connection_timeout.totalMicroseconds();
auto session = pool_ptr->second->get(retry_timeout);
setTimeouts(*session, timeouts);
2019-03-29 18:10:03 +00:00
return session;
}
};
}
void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout)
2016-12-31 02:05:37 +00:00
{
if (!response.getKeepAlive())
return;
Poco::Timespan timeout(keep_alive_timeout, 0);
if (timeout.totalSeconds())
response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds()));
}
Add global proxy setting (#51749) * initial impl * fix env ut * move ut directory * make sure no null proxy resolver is returned by ProxyConfigurationResolverProvider * minor adjustment * add a few tests, still incomplete * add proxy support for url table function * use proxy for select from url as well * remove optional from return type, just returns empty config * fix style * style * black * ohg boy * rm in progress file * god pls don't let me kill anyone * ... * add use_aws guards * remove hard coded s3 proxy resolver * add concurrency-mt-unsafe * aa * black * add logging back * revert change * imrpove code a bit * helper functions and separate tests * for some reason, this env test is not working.. * formatting * :) * clangtidy * lint * revert some stupid things * small test adjusmtments * simplify tests * rename test * remove extra line * freaking style change * simplify a bit * fix segfault & remove an extra call * tightly couple proxy provider with context.. * remove useless include * rename config prefix parameter * simplify provider a bit * organize provider a bit * add a few comments * comment out proxy env tests * fix nullptr in unit tests * make sure old storage proxy config is properly covered without global context instance * move a few functions from class to anonymous namespace * fix no fallback for specific storage conf * change API to accept http method instead of bool * implement http/https distinction in listresolver, any still not implemented * implement http/https distinction in remote resolver * progress on code, improve tests and add url function working test * use protcol instead of method for http and https * small fix * few more adjustments * fix style * black * move enum to proxyconfiguration * wip * fix build * fix ut * delete atomicroundrobin class * remove stale include * add some tests.. need to spend some more time on the design.. * change design a bit * progress * use existing context for tests * rename aux function and fix ut * .. * rename test * try to simplify tests a bit * simplify tests a bit more * attempt to fix tests, accept more than one remote resolver * use proper log id * try waiting for resolver * proper wait logic * black * empty * address a few comments * refactor tests * remove old tests * baclk * use RAII to set/unset env * black * clang tidy * fix env proxy not respecting any * use log trace * fix wrong logic in getRemoteREsolver * fix wrong logic in getRemoteREsolver * fix test * remove unwanted code * remove ClientConfigurationperRequest and auxilary classes * remove unwanted code * remove adapter test * few adjustments and add test for s3 storage conf with new proxy settings * black * use chassert for context * Add getenv comment
2023-08-24 13:07:26 +00:00
HTTPSessionPtr makeHTTPSession(
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
Poco::Net::HTTPClientSession::ProxyConfig proxy_config
)
{
const std::string & host = uri.getHost();
UInt16 port = uri.getPort();
bool https = isHTTPS(uri);
Add global proxy setting (#51749) * initial impl * fix env ut * move ut directory * make sure no null proxy resolver is returned by ProxyConfigurationResolverProvider * minor adjustment * add a few tests, still incomplete * add proxy support for url table function * use proxy for select from url as well * remove optional from return type, just returns empty config * fix style * style * black * ohg boy * rm in progress file * god pls don't let me kill anyone * ... * add use_aws guards * remove hard coded s3 proxy resolver * add concurrency-mt-unsafe * aa * black * add logging back * revert change * imrpove code a bit * helper functions and separate tests * for some reason, this env test is not working.. * formatting * :) * clangtidy * lint * revert some stupid things * small test adjusmtments * simplify tests * rename test * remove extra line * freaking style change * simplify a bit * fix segfault & remove an extra call * tightly couple proxy provider with context.. * remove useless include * rename config prefix parameter * simplify provider a bit * organize provider a bit * add a few comments * comment out proxy env tests * fix nullptr in unit tests * make sure old storage proxy config is properly covered without global context instance * move a few functions from class to anonymous namespace * fix no fallback for specific storage conf * change API to accept http method instead of bool * implement http/https distinction in listresolver, any still not implemented * implement http/https distinction in remote resolver * progress on code, improve tests and add url function working test * use protcol instead of method for http and https * small fix * few more adjustments * fix style * black * move enum to proxyconfiguration * wip * fix build * fix ut * delete atomicroundrobin class * remove stale include * add some tests.. need to spend some more time on the design.. * change design a bit * progress * use existing context for tests * rename aux function and fix ut * .. * rename test * try to simplify tests a bit * simplify tests a bit more * attempt to fix tests, accept more than one remote resolver * use proper log id * try waiting for resolver * proper wait logic * black * empty * address a few comments * refactor tests * remove old tests * baclk * use RAII to set/unset env * black * clang tidy * fix env proxy not respecting any * use log trace * fix wrong logic in getRemoteREsolver * fix wrong logic in getRemoteREsolver * fix test * remove unwanted code * remove ClientConfigurationperRequest and auxilary classes * remove unwanted code * remove adapter test * few adjustments and add test for s3 storage conf with new proxy settings * black * use chassert for context * Add getenv comment
2023-08-24 13:07:26 +00:00
auto session = makeHTTPSessionImpl(host, port, https, false, proxy_config);
setTimeouts(*session, timeouts);
return session;
}
2023-06-12 14:19:05 +00:00
PooledHTTPSessionPtr makePooledHTTPSession(
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
size_t per_endpoint_pool_size,
bool wait_on_pool_size_limit)
{
2023-07-14 09:09:22 +00:00
return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit);
}
2023-06-12 14:19:05 +00:00
PooledHTTPSessionPtr makePooledHTTPSession(
const Poco::URI & uri,
const Poco::URI & proxy_uri,
const ConnectionTimeouts & timeouts,
size_t per_endpoint_pool_size,
bool wait_on_pool_size_limit)
{
2023-07-14 09:09:22 +00:00
return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit);
}
bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; }
2018-06-16 05:54:06 +00:00
std::istream * receiveResponse(
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, const bool allow_redirects)
{
2019-06-17 17:32:57 +00:00
auto & istr = session.receiveResponse(response);
2019-09-23 08:53:09 +00:00
assertResponseIsOk(request, response, istr, allow_redirects);
2019-06-17 17:32:57 +00:00
return &istr;
2019-05-31 10:58:43 +00:00
}
2019-09-23 08:53:09 +00:00
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects)
2019-06-17 18:06:28 +00:00
{
auto status = response.getStatus();
2020-10-29 21:29:10 +00:00
if (!(status == Poco::Net::HTTPResponse::HTTP_OK
|| status == Poco::Net::HTTPResponse::HTTP_CREATED
|| status == Poco::Net::HTTPResponse::HTTP_ACCEPTED
2021-06-19 16:36:39 +00:00
|| status == Poco::Net::HTTPResponse::HTTP_PARTIAL_CONTENT /// Reading with Range header was successful.
2020-10-29 21:29:10 +00:00
|| (isRedirect(status) && allow_redirects)))
{
int code = status == Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS
? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS
: ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
std::stringstream body; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
body.exceptions(std::ios::failbit);
body << istr.rdbuf();
throw HTTPException(code, request.getURI(), status, response.getReason(), body.str());
}
}
2018-06-16 05:54:06 +00:00
2023-01-17 16:39:07 +00:00
Exception HTTPException::makeExceptionMessage(
int code,
const std::string & uri,
Poco::Net::HTTPResponse::HTTPStatus http_status,
const std::string & reason,
const std::string & body)
{
2023-01-17 16:39:07 +00:00
return Exception(code,
"Received error from remote server {}. "
"HTTP status code: {} {}, "
"body: {}",
2022-12-04 01:23:43 +00:00
uri, static_cast<int>(http_status), reason, body);
}
2023-07-03 21:11:32 +00:00
void markSessionForReuse(Poco::Net::HTTPSession & session)
{
const auto & session_data = session.sessionData();
if (!session_data.empty() && !Poco::AnyCast<HTTPSessionReuseTag>(&session_data))
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Data of an unexpected type ({}) is attached to the session", session_data.type().name());
session.attachSessionData(HTTPSessionReuseTag{});
}
void markSessionForReuse(HTTPSessionPtr session)
{
markSessionForReuse(*session);
}
void markSessionForReuse(PooledHTTPSessionPtr session)
{
markSessionForReuse(static_cast<Poco::Net::HTTPSession &>(*session));
}
}