http connections pools

This commit is contained in:
Sema Checherinda 2024-03-03 14:22:40 +01:00
parent c240c33037
commit 5c41727725
92 changed files with 3507 additions and 1771 deletions

View File

@ -45,6 +45,8 @@ namespace Net
~HTTPChunkedStreamBuf();
void close();
bool isComplete() const { return _chunk == std::char_traits<char>::eof(); }
protected:
int readFromDevice(char * buffer, std::streamsize length);
int writeToDevice(const char * buffer, std::streamsize length);
@ -68,6 +70,8 @@ namespace Net
~HTTPChunkedIOS();
HTTPChunkedStreamBuf * rdbuf();
bool isComplete() const { return _buf.isComplete(); }
protected:
HTTPChunkedStreamBuf _buf;
};

View File

@ -210,7 +210,7 @@ namespace Net
void setKeepAliveTimeout(const Poco::Timespan & timeout);
/// Sets the connection timeout for HTTP connections.
const Poco::Timespan & getKeepAliveTimeout() const;
Poco::Timespan getKeepAliveTimeout() const;
/// Returns the connection timeout for HTTP connections.
virtual std::ostream & sendRequest(HTTPRequest & request);
@ -275,7 +275,7 @@ namespace Net
/// This method should only be called if the request contains
/// a "Expect: 100-continue" header.
void flushRequest();
virtual void flushRequest();
/// Flushes the request stream.
///
/// Normally this method does not need to be called.
@ -283,7 +283,7 @@ namespace Net
/// fully sent if receiveResponse() is not called, e.g.,
/// because the underlying socket will be detached.
void reset();
virtual void reset();
/// Resets the session and closes the socket.
///
/// The next request will initiate a new connection,
@ -303,6 +303,9 @@ namespace Net
/// Returns true if the proxy should be bypassed
/// for the current host.
const Poco::Timestamp & getLastRequest() const;
/// Returns time when connection has been used last time
protected:
enum
{
@ -338,6 +341,10 @@ namespace Net
/// Calls proxyConnect() and attaches the resulting StreamSocket
/// to the HTTPClientSession.
void setLastRequest(Poco::Timestamp time);
void assign(HTTPClientSession & session);
HTTPSessionFactory _proxySessionFactory;
/// Factory to create HTTPClientSession to proxy.
private:
@ -433,11 +440,20 @@ namespace Net
}
inline const Poco::Timespan & HTTPClientSession::getKeepAliveTimeout() const
inline Poco::Timespan HTTPClientSession::getKeepAliveTimeout() const
{
return _keepAliveTimeout;
}
inline const Poco::Timestamp & HTTPClientSession::getLastRequest() const
{
return _lastRequest;
}
inline void HTTPClientSession::setLastRequest(Poco::Timestamp time)
{
_lastRequest = time;
}
}
} // namespace Poco::Net

View File

@ -48,6 +48,8 @@ namespace Net
HTTPFixedLengthStreamBuf(HTTPSession & session, ContentLength length, openmode mode);
~HTTPFixedLengthStreamBuf();
bool isComplete() const;
protected:
int readFromDevice(char * buffer, std::streamsize length);
int writeToDevice(const char * buffer, std::streamsize length);
@ -67,6 +69,8 @@ namespace Net
~HTTPFixedLengthIOS();
HTTPFixedLengthStreamBuf * rdbuf();
bool isComplete() const { return _buf.isComplete(); }
protected:
HTTPFixedLengthStreamBuf _buf;
};

View File

@ -64,6 +64,15 @@ namespace Net
Poco::Timespan getTimeout() const;
/// Returns the timeout for the HTTP session.
Poco::Timespan getConnectionTimeout() const;
/// Returns connection timeout for the HTTP session.
Poco::Timespan getSendTimeout() const;
/// Returns send timeout for the HTTP session.
Poco::Timespan getReceiveTimeout() const;
/// Returns receive timeout for the HTTP session.
bool connected() const;
/// Returns true if the underlying socket is connected.
@ -217,12 +226,25 @@ namespace Net
return _keepAlive;
}
inline Poco::Timespan HTTPSession::getTimeout() const
{
return _receiveTimeout;
}
inline Poco::Timespan HTTPSession::getConnectionTimeout() const
{
return _connectionTimeout;
}
inline Poco::Timespan HTTPSession::getSendTimeout() const
{
return _sendTimeout;
}
inline Poco::Timespan HTTPSession::getReceiveTimeout() const
{
return _receiveTimeout;
}
inline StreamSocket & HTTPSession::socket()
{

View File

@ -63,6 +63,8 @@ namespace Net
~HTTPIOS();
HTTPStreamBuf * rdbuf();
bool isComplete() const { return false; }
protected:
HTTPStreamBuf _buf;
};

View File

@ -49,10 +49,12 @@ HTTPChunkedStreamBuf::~HTTPChunkedStreamBuf()
void HTTPChunkedStreamBuf::close()
{
if (_mode & std::ios::out)
if (_mode & std::ios::out && _chunk != std::char_traits<char>::eof())
{
sync();
_session.write("0\r\n\r\n", 5);
_chunk = std::char_traits<char>::eof();
}
}

View File

@ -227,7 +227,7 @@ void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout)
std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request)
{
_pRequestStream = 0;
_pResponseStream = 0;
_pResponseStream = 0;
clearException();
_responseReceived = false;
@ -501,5 +501,26 @@ bool HTTPClientSession::bypassProxy() const
else return false;
}
void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session)
{
poco_assert (this != &session);
if (session.buffered())
throw Poco::LogicException("assign a session with not empty buffered data");
if (buffered())
throw Poco::LogicException("assign to a session with not empty buffered data");
attachSocket(session.detachSocket());
setLastRequest(session.getLastRequest());
setResolvedHost(session.getResolvedHost());
setKeepAlive(session.getKeepAlive());
setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout());
setKeepAliveTimeout(session.getKeepAliveTimeout());
setProxyConfig(session.getProxyConfig());
session.reset();
}
} } // namespace Poco::Net

View File

@ -43,6 +43,12 @@ HTTPFixedLengthStreamBuf::~HTTPFixedLengthStreamBuf()
}
bool HTTPFixedLengthStreamBuf::isComplete() const
{
return _count == _length;
}
int HTTPFixedLengthStreamBuf::readFromDevice(char* buffer, std::streamsize length)
{
int n = 0;

View File

@ -46,6 +46,7 @@
#include <Common/makeSocketAddress.h>
#include <Common/FailPoint.h>
#include <Common/CPUID.h>
#include <Common/HTTPConnectionPool.h>
#include <Server/waitServersToFinish.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Core/ServerUUID.h>
@ -1547,6 +1548,23 @@ try
FileCacheFactory::instance().updateSettingsFromConfig(*config);
HTTPConnectionPools::instance().setLimits(
HTTPConnectionPools::Limits{
new_server_settings.disk_connections_soft_limit,
new_server_settings.disk_connections_warn_limit,
new_server_settings.disk_connections_store_limit,
},
HTTPConnectionPools::Limits{
new_server_settings.storage_connections_soft_limit,
new_server_settings.storage_connections_warn_limit,
new_server_settings.storage_connections_store_limit,
},
HTTPConnectionPools::Limits{
new_server_settings.http_connections_soft_limit,
new_server_settings.http_connections_warn_limit,
new_server_settings.http_connections_store_limit,
});
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
/// Must be the last.

View File

@ -65,7 +65,7 @@ void processFile(const fs::path & file_path, const fs::path & dst_path, bool tes
/// test mode for integration tests.
if (test_mode)
dst_buf = std::make_shared<WriteBufferFromHTTP>(Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT);
dst_buf = std::make_shared<WriteBufferFromHTTP>(HTTPConnectionGroupType::HTTP, Poco::URI(dst_file_path), Poco::Net::HTTPRequest::HTTP_PUT);
else
dst_buf = std::make_shared<WriteBufferFromFile>(dst_file_path);
@ -88,7 +88,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_
{
dst_path /= "store";
auto files_root = dst_path / prefix;
root_meta = std::make_shared<WriteBufferFromHTTP>(Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
root_meta = std::make_shared<WriteBufferFromHTTP>(HTTPConnectionGroupType::HTTP, Poco::URI(files_root / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
}
else
{
@ -112,7 +112,7 @@ void processTableFiles(const fs::path & data_path, fs::path dst_path, bool test_
if (test_mode)
{
auto files_root = dst_path / prefix;
directory_meta = std::make_shared<WriteBufferFromHTTP>(Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
directory_meta = std::make_shared<WriteBufferFromHTTP>(HTTPConnectionGroupType::HTTP, Poco::URI(dst_path / directory_prefix / ".index"), Poco::Net::HTTPRequest::HTTP_PUT);
}
else
{

View File

@ -154,7 +154,8 @@ enum class AccessType
M(SET_DEFINER, "", USER_NAME, ALL) \
\
M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \
M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_MMAP_CACHE, "SYSTEM DROP MMAP, DROP MMAP CACHE, DROP MMAP", GLOBAL, SYSTEM_DROP_CACHE) \

View File

@ -36,7 +36,7 @@ public:
Result authenticateRequest(Poco::Net::HTTPRequest & request) const
{
auto session = makeHTTPSession(uri, timeouts);
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts);
Poco::Net::HTTPResponse response;
auto milliseconds_to_wait = retry_initial_backoff_ms;

View File

@ -58,8 +58,12 @@ bool CatBoostLibraryBridgeHelper::bridgeHandShake()
String result;
try
{
ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials);
readString(result, buf);
auto buf = BuilderRWBufferFromHTTP(getPingURI())
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withTimeouts(http_timeouts)
.create(credentials);
readString(result, *buf);
}
catch (...)
{
@ -79,29 +83,29 @@ ExternalModelInfos CatBoostLibraryBridgeHelper::listModels()
{
startBridgeSync();
ReadWriteBufferFromHTTP buf(
createRequestURI(CATBOOST_LIST_METHOD),
Poco::Net::HTTPRequest::HTTP_POST,
[](std::ostream &) {},
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIST_METHOD))
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.create(credentials);
ExternalModelInfos result;
UInt64 num_rows;
readIntBinary(num_rows, buf);
readIntBinary(num_rows, *buf);
for (UInt64 i = 0; i < num_rows; ++i)
{
ExternalModelInfo info;
readStringBinary(info.model_path, buf);
readStringBinary(info.model_type, buf);
readStringBinary(info.model_path, *buf);
readStringBinary(info.model_type, *buf);
UInt64 t;
readIntBinary(t, buf);
readIntBinary(t, *buf);
info.loading_start_time = std::chrono::system_clock::from_time_t(t);
readIntBinary(t, buf);
readIntBinary(t, *buf);
info.loading_duration = std::chrono::milliseconds(t);
result.push_back(info);
@ -116,17 +120,19 @@ void CatBoostLibraryBridgeHelper::removeModel()
assert(model_path);
ReadWriteBufferFromHTTP buf(
createRequestURI(CATBOOST_REMOVEMODEL_METHOD),
Poco::Net::HTTPRequest::HTTP_POST,
[this](std::ostream & os)
{
os << "model_path=" << escapeForFileName(*model_path);
},
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEMODEL_METHOD))
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.withOutCallback(
[this](std::ostream & os)
{
os << "model_path=" << escapeForFileName(*model_path);
})
.create(credentials);
String result;
readStringBinary(result, buf);
readStringBinary(result, *buf);
assert(result == "1");
}
@ -134,14 +140,14 @@ void CatBoostLibraryBridgeHelper::removeAllModels()
{
startBridgeSync();
ReadWriteBufferFromHTTP buf(
createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD),
Poco::Net::HTTPRequest::HTTP_POST,
[](std::ostream &){},
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_REMOVEALLMODELS_METHOD))
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.create(credentials);
String result;
readStringBinary(result, buf);
readStringBinary(result, *buf);
assert(result == "1");
}
@ -151,18 +157,20 @@ size_t CatBoostLibraryBridgeHelper::getTreeCount()
assert(model_path && library_path);
ReadWriteBufferFromHTTP buf(
createRequestURI(CATBOOST_GETTREECOUNT_METHOD),
Poco::Net::HTTPRequest::HTTP_POST,
[this](std::ostream & os)
{
os << "library_path=" << escapeForFileName(*library_path) << "&";
os << "model_path=" << escapeForFileName(*model_path);
},
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_GETTREECOUNT_METHOD))
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.withOutCallback(
[this](std::ostream & os)
{
os << "library_path=" << escapeForFileName(*library_path) << "&";
os << "model_path=" << escapeForFileName(*model_path);
})
.create(credentials);
size_t result;
readIntBinary(result, buf);
readIntBinary(result, *buf);
return result;
}
@ -177,17 +185,19 @@ ColumnPtr CatBoostLibraryBridgeHelper::evaluate(const ColumnsWithTypeAndName & c
assert(model_path);
ReadWriteBufferFromHTTP buf(
createRequestURI(CATBOOST_LIB_EVALUATE_METHOD),
Poco::Net::HTTPRequest::HTTP_POST,
[this, serialized = string_write_buf.str()](std::ostream & os)
{
os << "model_path=" << escapeForFileName(*model_path) << "&";
os << "data=" << escapeForFileName(serialized);
},
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(createRequestURI(CATBOOST_LIB_EVALUATE_METHOD))
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.withOutCallback(
[this, serialized = string_write_buf.str()](std::ostream & os)
{
os << "model_path=" << escapeForFileName(*model_path) << "&";
os << "data=" << escapeForFileName(serialized);
})
.create(credentials);
NativeReader deserializer(buf, /*server_revision*/ 0);
NativeReader deserializer(*buf, /*server_revision*/ 0);
Block block_read = deserializer.read();
return block_read.getColumns()[0];

View File

@ -71,8 +71,12 @@ bool ExternalDictionaryLibraryBridgeHelper::bridgeHandShake()
String result;
try
{
ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, http_timeouts, credentials);
readString(result, buf);
auto buf = BuilderRWBufferFromHTTP(getPingURI())
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withTimeouts(http_timeouts)
.create(credentials);
readString(result, *buf);
}
catch (...)
{
@ -247,30 +251,28 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadKeys(const Block & requ
bool ExternalDictionaryLibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) const
{
ReadWriteBufferFromHTTP buf(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
std::move(out_stream_callback),
http_timeouts, credentials);
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(http_timeouts)
.withOutCallback(std::move(out_stream_callback))
.create(credentials);
bool res;
readBoolText(res, buf);
readBoolText(res, *buf);
return res;
}
QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback)
{
auto read_buf_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
std::move(out_stream_callback),
http_timeouts,
credentials,
0,
DBMS_DEFAULT_BUFFER_SIZE,
getContext()->getReadSettings(),
HTTPHeaderEntries{});
auto read_buf_ptr = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withSettings(getContext()->getReadSettings())
.withTimeouts(http_timeouts)
.withOutCallback(std::move(out_stream_callback))
.create(credentials);
auto source = FormatFactory::instance().getInput(ExternalDictionaryLibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE);
source->addBuffer(std::move(read_buf_ptr));

View File

@ -97,8 +97,12 @@ protected:
{
try
{
ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, getHTTPTimeouts(), credentials);
return checkString(PING_OK_ANSWER, buf);
auto buf = BuilderRWBufferFromHTTP(getPingURI())
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withTimeouts(getHTTPTimeouts())
.create(credentials);
return checkString(PING_OK_ANSWER, *buf);
}
catch (...)
{
@ -198,10 +202,14 @@ protected:
uri.addQueryParameter("connection_string", getConnectionString());
uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling));
ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials);
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(getHTTPTimeouts())
.create(credentials);
bool res;
readBoolText(res, buf);
bool res = false;
readBoolText(res, *buf);
is_schema_allowed = res;
}
@ -220,10 +228,14 @@ protected:
uri.addQueryParameter("connection_string", getConnectionString());
uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling));
ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials);
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(getHTTPTimeouts())
.create(credentials);
std::string character;
readStringBinary(character, buf);
readStringBinary(character, *buf);
if (character.length() > 1)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Failed to parse quoting style from '{}' for service {}",
character, BridgeHelperMixin::serviceAlias());

View File

@ -153,6 +153,12 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
current_resolved_address = *it;
break;
}
catch (DB::NetException &)
{
if (++it == addresses.end())
throw;
continue;
}
catch (Poco::Net::NetException &)
{
if (++it == addresses.end())
@ -199,6 +205,16 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.",
server_name, server_version_major, server_version_minor, server_version_patch);
}
catch (DB::NetException & e)
{
disconnect();
/// Remove this possible stale entry from cache
DNSResolver::instance().removeHostFromCache(host);
/// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost.
throw NetException(ErrorCodes::NETWORK_ERROR, "{} ({})", e.displayText(), getDescription());
}
catch (Poco::Net::NetException & e)
{
disconnect();

View File

@ -275,6 +275,18 @@
M(DistrCacheReadRequests, "Number of executed Read requests to Distributed Cache") \
M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \
M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache")
\
M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \
M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \
\
M(DiskConnectionsStored, "Total count of sessions stored in the session pool for disks") \
M(DiskConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for disks") \
\
M(HTTPConnectionsStored, "Total count of sessions stored in the session pool for http hosts") \
M(HTTPConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for http hosts") \
\
M(AddressesActive, "Total count of addresses which are used for creation connections with connection pools") \
#ifdef APPLY_FOR_EXTERNAL_METRICS
#define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M)

View File

@ -1,6 +1,7 @@
#include "DNSResolver.h"
#include <Common/CacheBase.h>
#include <Common/Exception.h>
#include <Common/NetException.h>
#include <Common/ProfileEvents.h>
#include <Common/thread_local_rng.h>
#include <Common/logger_useful.h>
@ -108,7 +109,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host)
if (addresses.empty())
{
ProfileEvents::increment(ProfileEvents::DNSError);
throw Exception(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host);
throw DB::NetException(ErrorCodes::DNS_ERROR, "Not found address of host: {}", host);
}
return addresses;

View File

@ -0,0 +1,865 @@
#include <Common/HTTPConnectionPool.h>
#include <Common/HostResolvePool.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Common/logger_useful.h>
#include <Common/Exception.h>
#include <Common/ErrorCodes.h>
#include <Common/ProxyConfiguration.h>
#include <Common/MemoryTrackerSwitcher.h>
#include <Common/SipHash.h>
#include <Poco/Net/HTTPClientSession.h>
#include <Poco/Net/HTTPStream.h>
#include <Poco/Net/HTTPFixedLengthStream.h>
#include <Poco/Net/HTTPChunkedStream.h>
#include <Poco/Timespan.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Net/HTTPRequest.h>
#include "config.h"
#if USE_SSL
#include <Poco/Net/HTTPSClientSession.h>
#endif
namespace ProfileEvents
{
extern const Event StorageConnectionsCreated;
extern const Event StorageConnectionsReused;
extern const Event StorageConnectionsReset;
extern const Event StorageConnectionsPreserved;
extern const Event StorageConnectionsExpired;
extern const Event StorageConnectionsErrors;
extern const Event StorageConnectionsElapsedMicroseconds;
extern const Event DiskConnectionsCreated;
extern const Event DiskConnectionsReused;
extern const Event DiskConnectionsReset;
extern const Event DiskConnectionsPreserved;
extern const Event DiskConnectionsExpired;
extern const Event DiskConnectionsErrors;
extern const Event DiskConnectionsElapsedMicroseconds;
extern const Event HTTPConnectionsCreated;
extern const Event HTTPConnectionsReused;
extern const Event HTTPConnectionsReset;
extern const Event HTTPConnectionsPreserved;
extern const Event HTTPConnectionsExpired;
extern const Event HTTPConnectionsErrors;
extern const Event HTTPConnectionsElapsedMicroseconds;
}
namespace CurrentMetrics
{
extern const Metric StorageConnectionsStored;
extern const Metric StorageConnectionsTotal;
extern const Metric DiskConnectionsStored;
extern const Metric DiskConnectionsTotal;
extern const Metric HTTPConnectionsStored;
extern const Metric HTTPConnectionsTotal;
}
namespace
{
Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration)
{
Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config;
poco_proxy_config.host = proxy_configuration.host;
poco_proxy_config.port = proxy_configuration.port;
poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol);
poco_proxy_config.tunnel = proxy_configuration.tunneling;
poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol);
return poco_proxy_config;
}
size_t roundUp(size_t x, size_t rounding)
{
chassert(rounding > 0);
return (x + (rounding - 1)) / rounding * rounding;
}
Poco::Timespan divide(const Poco::Timespan span, int divisor)
{
return Poco::Timespan(Poco::Timestamp::TimeDiff(span.totalMicroseconds() / divisor));
}
}
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
extern const int UNSUPPORTED_URI_SCHEME;
}
IHTTPConnectionPoolForEndpoint::Metrics getMetricsForStorageConnectionPool()
{
return IHTTPConnectionPoolForEndpoint::Metrics{
.created = ProfileEvents::StorageConnectionsCreated,
.reused = ProfileEvents::StorageConnectionsReused,
.reset = ProfileEvents::StorageConnectionsReset,
.preserved = ProfileEvents::StorageConnectionsPreserved,
.expired = ProfileEvents::StorageConnectionsExpired,
.errors = ProfileEvents::StorageConnectionsErrors,
.elapsed_microseconds = ProfileEvents::StorageConnectionsElapsedMicroseconds,
.stored_count = CurrentMetrics::StorageConnectionsStored,
.active_count = CurrentMetrics::StorageConnectionsTotal,
};
}
IHTTPConnectionPoolForEndpoint::Metrics getMetricsForDiskConnectionPool()
{
return IHTTPConnectionPoolForEndpoint::Metrics{
.created = ProfileEvents::DiskConnectionsCreated,
.reused = ProfileEvents::DiskConnectionsReused,
.reset = ProfileEvents::DiskConnectionsReset,
.preserved = ProfileEvents::DiskConnectionsPreserved,
.expired = ProfileEvents::DiskConnectionsExpired,
.errors = ProfileEvents::DiskConnectionsErrors,
.elapsed_microseconds = ProfileEvents::DiskConnectionsElapsedMicroseconds,
.stored_count = CurrentMetrics::DiskConnectionsStored,
.active_count = CurrentMetrics::DiskConnectionsTotal,
};
}
IHTTPConnectionPoolForEndpoint::Metrics getMetricsForHTTPConnectionPool()
{
return IHTTPConnectionPoolForEndpoint::Metrics{
.created = ProfileEvents::HTTPConnectionsCreated,
.reused = ProfileEvents::HTTPConnectionsReused,
.reset = ProfileEvents::HTTPConnectionsReset,
.preserved = ProfileEvents::HTTPConnectionsPreserved,
.expired = ProfileEvents::HTTPConnectionsExpired,
.errors = ProfileEvents::HTTPConnectionsErrors,
.elapsed_microseconds = ProfileEvents::HTTPConnectionsElapsedMicroseconds,
.stored_count = CurrentMetrics::HTTPConnectionsStored,
.active_count = CurrentMetrics::HTTPConnectionsTotal,
};
}
IHTTPConnectionPoolForEndpoint::Metrics getConnectionPoolMetrics(HTTPConnectionGroupType type)
{
switch (type)
{
case HTTPConnectionGroupType::STORAGE:
return getMetricsForStorageConnectionPool();
case HTTPConnectionGroupType::DISK:
return getMetricsForDiskConnectionPool();
case HTTPConnectionGroupType::HTTP:
return getMetricsForHTTPConnectionPool();
}
}
class ConnectionGroup
{
public:
using Ptr = std::shared_ptr<ConnectionGroup>;
explicit ConnectionGroup(HTTPConnectionGroupType type_) : type(type_), metrics(getConnectionPoolMetrics(type_)) { }
void setLimits(HTTPConnectionPools::Limits limits_)
{
std::lock_guard lock(mutex);
limits = std::move(limits_);
mute_warning_until = 0;
}
bool isSoftLimitReached() const
{
std::lock_guard lock(mutex);
return total_connections_in_group >= limits.soft_limit;
}
bool isStoreLimitReached() const
{
std::lock_guard lock(mutex);
return total_connections_in_group >= limits.store_limit;
}
void atConnectionCreate()
{
std::lock_guard lock(mutex);
++total_connections_in_group;
if (total_connections_in_group >= limits.warning_limit && total_connections_in_group >= mute_warning_until)
{
LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}", type, total_connections_in_group, limits.warning_limit);
mute_warning_until = roundUp(total_connections_in_group, limits.warning_step);
}
}
void atConnectionDestroy()
{
std::lock_guard lock(mutex);
--total_connections_in_group;
const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 10 : 1;
if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit)
{
LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group);
mute_warning_until = 0;
}
}
HTTPConnectionGroupType getType() const { return type; }
const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const { return metrics; }
private:
const HTTPConnectionGroupType type;
const IHTTPConnectionPoolForEndpoint::Metrics metrics;
LoggerPtr log = getLogger("ConnectionGroup");
mutable std::mutex mutex;
HTTPConnectionPools::Limits limits TSA_GUARDED_BY(mutex) = HTTPConnectionPools::Limits();
size_t total_connections_in_group TSA_GUARDED_BY(mutex) = 0;
size_t mute_warning_until TSA_GUARDED_BY(mutex) = 0;
};
class IExtendedPool : public IHTTPConnectionPoolForEndpoint
{
public:
using Ptr = std::shared_ptr<IExtendedPool>;
virtual HTTPConnectionGroupType getGroupType() const = 0;
virtual size_t wipeExpired() = 0;
};
// EndpointConnectionPool manage connections to the endpoint
// Features:
// - it uses HostResolver for address selecting. See Common/HostResolver.h for more info.
// - it minimizes number of `Session::connect()`/`Session::reconnect()` calls
// - stores only connected and ready to use sessions
// - connection could be reused even when limits are reached
// - soft limit, warn limit, store limit
// - `Session::reconnect()` uses the pool as well
// - comprehensive sensors
// - session is reused according its inner state, automatically
template <class Session>
class EndpointConnectionPool : public std::enable_shared_from_this<EndpointConnectionPool<Session>>, public IExtendedPool
{
private:
friend class HTTPConnectionPools;
using WeakPtr = std::weak_ptr<EndpointConnectionPool<Session>>;
class PooledConnection : public Session
{
public:
using Ptr = std::shared_ptr<PooledConnection>;
void reconnect() override
{
ProfileEvents::increment(metrics.reset);
Session::close();
if (auto lock = pool.lock())
{
auto timeouts = getTimeouts(*this);
auto new_connection = lock->getConnection(timeouts);
Session::assign(*new_connection);
}
else
{
auto timer = CurrentThread::getProfileEvents().timer(metrics.elapsed_microseconds);
Session::reconnect();
ProfileEvents::increment(metrics.created);
}
}
String getTarget() const
{
if (!Session::getProxyConfig().host.empty())
return fmt::format("{} over proxy {}", Session::getHost(), Session::getProxyConfig().host);
return Session::getHost();
}
void flushRequest() override
{
if (bool(request_stream))
{
request_stream->flush();
if (auto * fixed_steam = dynamic_cast<Poco::Net::HTTPFixedLengthOutputStream *>(request_stream))
{
request_stream_completed = fixed_steam->isComplete();
}
else if (auto * chunked_steam = dynamic_cast<Poco::Net::HTTPChunkedOutputStream *>(request_stream))
{
chunked_steam->rdbuf()->close();
request_stream_completed = chunked_steam->isComplete();
}
else if (auto * http_stream = dynamic_cast<Poco::Net::HTTPOutputStream *>(request_stream))
{
request_stream_completed = http_stream->isComplete();
}
else
{
request_stream_completed = false;
}
}
request_stream = nullptr;
Session::flushRequest();
}
std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override
{
std::ostream & result = Session::sendRequest(request);
result.exceptions(std::ios::badbit);
request_stream = &result;
request_stream_completed = false;
response_stream = nullptr;
response_stream_completed = true;
return result;
}
std::istream & receiveResponse(Poco::Net::HTTPResponse & response) override
{
std::istream & result = Session::receiveResponse(response);
result.exceptions(std::ios::badbit);
response_stream = &result;
response_stream_completed = false;
return result;
}
void reset() override
{
request_stream = nullptr;
request_stream_completed = false;
response_stream = nullptr;
response_stream_completed = false;
Session::reset();
}
~PooledConnection() override
{
if (bool(response_stream))
{
if (auto * fixed_steam = dynamic_cast<Poco::Net::HTTPFixedLengthInputStream *>(response_stream))
{
response_stream_completed = fixed_steam->isComplete();
}
else if (auto * chunked_steam = dynamic_cast<Poco::Net::HTTPChunkedInputStream *>(response_stream))
{
response_stream_completed = chunked_steam->isComplete();
}
else if (auto * http_stream = dynamic_cast<Poco::Net::HTTPInputStream *>(response_stream))
{
response_stream_completed = http_stream->isComplete();
}
else
{
response_stream_completed = false;
}
}
response_stream = nullptr;
if (auto lock = pool.lock())
lock->atConnectionDestroy(*this);
else
ProfileEvents::increment(metrics.reset);
CurrentMetrics::sub(metrics.active_count);
}
private:
friend class EndpointConnectionPool;
template <class... Args>
explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args)
: Session(args...), pool(std::move(pool_)), metrics(std::move(metrics_))
{
CurrentMetrics::add(metrics.active_count);
}
template <class... Args>
static Ptr create(Args &&... args)
{
/// Pool is global, we shouldn't attribute this memory to query/user.
MemoryTrackerSwitcher switcher{&total_memory_tracker};
struct make_shared_enabler : public PooledConnection
{
explicit make_shared_enabler(Args &&... args) : PooledConnection(std::forward<Args>(args)...) { }
};
return std::make_shared<make_shared_enabler>(std::forward<Args>(args)...);
}
void doConnect()
{
Session::reconnect();
}
bool isCompleted() const
{
return request_stream_completed && response_stream_completed;
}
WeakPtr pool;
IHTTPConnectionPoolForEndpoint::Metrics metrics;
Poco::Logger * log = &Poco::Logger::get("PooledConnection");
std::ostream * request_stream = nullptr;
std::istream * response_stream = nullptr;
bool request_stream_completed = true;
bool response_stream_completed = true;
};
using Connection = PooledConnection;
using ConnectionPtr = PooledConnection::Ptr;
struct GreaterByLastRequest
{
static bool operator()(const ConnectionPtr & l, const ConnectionPtr & r)
{
return l->getLastRequest() + l->getKeepAliveTimeout() > r->getLastRequest() + r->getKeepAliveTimeout();
}
};
using ConnectionsMinHeap = std::priority_queue<ConnectionPtr, std::vector<ConnectionPtr>, GreaterByLastRequest>;
public:
template <class... Args>
static Ptr create(Args &&... args)
{
struct make_shared_enabler : public EndpointConnectionPool<Session>
{
explicit make_shared_enabler(Args &&... args) : EndpointConnectionPool<Session>(std::forward<Args>(args)...) { }
};
return std::make_shared<make_shared_enabler>(std::forward<Args>(args)...);
}
~EndpointConnectionPool() override
{
CurrentMetrics::sub(group->getMetrics().stored_count, stored_connections.size());
}
String getTarget() const
{
if (!proxy_configuration.isEmpty())
return fmt::format("{} over proxy {}", host, proxy_configuration.host);
return host;
}
IHTTPConnectionPoolForEndpoint::ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) override
{
Poco::Timestamp now;
std::vector<ConnectionPtr> expired_connections;
SCOPE_EXIT({
MemoryTrackerSwitcher switcher{&total_memory_tracker};
expired_connections.clear();
});
{
std::lock_guard lock(mutex);
wipeExpiredImpl(expired_connections, now);
if (!stored_connections.empty())
{
auto it = stored_connections.top();
stored_connections.pop();
setTimeouts(*it, timeouts);
ProfileEvents::increment(getMetrics().reused, 1);
CurrentMetrics::sub(getMetrics().stored_count, 1);
return it;
}
}
return prepareNewConnection(timeouts);
}
const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const override
{
return group->getMetrics();
}
HTTPConnectionGroupType getGroupType() const override
{
return group->getType();
}
size_t wipeExpired() override
{
Poco::Timestamp now;
std::vector<ConnectionPtr> expired_connections;
SCOPE_EXIT({
MemoryTrackerSwitcher switcher{&total_memory_tracker};
expired_connections.clear();
});
std::lock_guard lock(mutex);
return wipeExpiredImpl(expired_connections, now);
}
size_t wipeExpiredImpl(std::vector<ConnectionPtr> & expired_connections, Poco::Timestamp now) TSA_REQUIRES(mutex)
{
while (!stored_connections.empty())
{
auto connection = stored_connections.top();
if (!isExpired(now, connection))
return stored_connections.size();
stored_connections.pop();
expired_connections.push_back(connection);
}
CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size());
ProfileEvents::increment(getMetrics().expired, expired_connections.size());
return stored_connections.size();
}
private:
EndpointConnectionPool(ConnectionGroup::Ptr group_, String host_, UInt16 port_, bool https_, ProxyConfiguration proxy_configuration_)
: host(std::move(host_))
, port(port_)
, https(https_)
, proxy_configuration(std::move(proxy_configuration_))
, group(group_)
{
}
WeakPtr getWeakFromThis() { return EndpointConnectionPool::weak_from_this(); }
bool isExpired(Poco::Timestamp & now, ConnectionPtr connection)
{
if (group->isSoftLimitReached())
return now > (connection->getLastRequest() + divide(connection->getKeepAliveTimeout(), 10));
return now > connection->getLastRequest() + connection->getKeepAliveTimeout();
}
ConnectionPtr allocateNewConnection()
{
ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), getMetrics(), host, port);
connection->setKeepAlive(true);
if (!proxy_configuration.isEmpty())
{
connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration));
}
group->atConnectionCreate();
return connection;
}
ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts)
{
auto address = HostResolversPool::instance().getResolver(host)->resolve();
auto session = allocateNewConnection();
setTimeouts(*session, timeouts);
session->setResolvedHost(*address);
try
{
auto timer = CurrentThread::getProfileEvents().timer(getMetrics().elapsed_microseconds);
session->doConnect();
}
catch (...)
{
address.setFail();
ProfileEvents::increment(getMetrics().errors);
session->reset();
throw;
}
ProfileEvents::increment(getMetrics().created);
return session;
}
void atConnectionDestroy(PooledConnection & connection)
{
group->atConnectionDestroy();
if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered()
|| group->isStoreLimitReached())
{
ProfileEvents::increment(getMetrics().reset, 1);
return;
}
auto connection_to_store = allocateNewConnection();
connection_to_store->assign(connection);
CurrentMetrics::add(getMetrics().stored_count, 1);
ProfileEvents::increment(getMetrics().preserved, 1);
{
MemoryTrackerSwitcher switcher{&total_memory_tracker};
std::lock_guard lock(mutex);
stored_connections.push(connection_to_store);
}
}
const std::string host;
const UInt16 port;
const bool https;
const ProxyConfiguration proxy_configuration;
const ConnectionGroup::Ptr group;
std::mutex mutex;
ConnectionsMinHeap stored_connections TSA_GUARDED_BY(mutex);
};
struct EndpointPoolKey
{
HTTPConnectionGroupType connection_group;
String target_host;
UInt16 target_port;
bool is_target_https;
ProxyConfiguration proxy_config;
bool operator==(const EndpointPoolKey & rhs) const
{
return std::tie(
connection_group,
target_host,
target_port,
is_target_https,
proxy_config.host,
proxy_config.port,
proxy_config.protocol,
proxy_config.tunneling,
proxy_config.original_request_protocol)
== std::tie(
rhs.connection_group,
rhs.target_host,
rhs.target_port,
rhs.is_target_https,
rhs.proxy_config.host,
rhs.proxy_config.port,
rhs.proxy_config.protocol,
rhs.proxy_config.tunneling,
rhs.proxy_config.original_request_protocol);
}
};
struct Hasher
{
size_t operator()(const EndpointPoolKey & k) const
{
SipHash s;
s.update(k.connection_group);
s.update(k.target_host);
s.update(k.target_port);
s.update(k.is_target_https);
s.update(k.proxy_config.host);
s.update(k.proxy_config.port);
s.update(k.proxy_config.protocol);
s.update(k.proxy_config.tunneling);
s.update(k.proxy_config.original_request_protocol);
return s.get64();
}
};
IExtendedPool::Ptr
createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, bool secure, ProxyConfiguration proxy_configuration)
{
if (secure)
{
#if USE_SSL
return EndpointConnectionPool<Poco::Net::HTTPSClientSession>::create(
group, std::move(host), port, secure, std::move(proxy_configuration));
#else
throw Exception(
ErrorCodes::SUPPORT_IS_DISABLED, "Inter-server secret support is disabled, because ClickHouse was built without SSL library");
#endif
}
else
{
return EndpointConnectionPool<Poco::Net::HTTPClientSession>::create(
group, std::move(host), port, secure, std::move(proxy_configuration));
}
}
class HTTPConnectionPools::Impl
{
private:
const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 5 * 60;
const Poco::Timespan wipe_timeout = Poco::Timespan(DEFAULT_WIPE_TIMEOUT_SECONDS, 0);
ConnectionGroup::Ptr disk_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::DISK);
ConnectionGroup::Ptr storage_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::STORAGE);
ConnectionGroup::Ptr http_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::HTTP);
/// If multiple mutexes are held simultaneously,
/// they should be locked in this order:
/// HTTPConnectionPools::mutex, then EndpointConnectionPool::mutex, then ConnectionGroup::mutex.
std::mutex mutex;
std::unordered_map<EndpointPoolKey, IExtendedPool::Ptr, Hasher> endpoints_pool TSA_GUARDED_BY(mutex);
Poco::Timestamp last_wipe_time TSA_GUARDED_BY(mutex);
public:
IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration)
{
Poco::Timestamp now;
std::lock_guard lock(mutex);
if (now - last_wipe_time > wipe_timeout)
{
wipeExpired();
last_wipe_time = now;
}
return getPoolImpl(type, uri, proxy_configuration);
}
void setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http)
{
disk_group->setLimits(std::move(disk));
storage_group->setLimits(std::move(storage));
http_group->setLimits(std::move(http));
}
void dropCache()
{
std::lock_guard lock(mutex);
endpoints_pool.clear();
}
protected:
ConnectionGroup::Ptr & getGroup(HTTPConnectionGroupType type)
{
switch (type)
{
case HTTPConnectionGroupType::DISK:
return disk_group;
case HTTPConnectionGroupType::STORAGE:
return storage_group;
case HTTPConnectionGroupType::HTTP:
return http_group;
}
}
IExtendedPool::Ptr getPoolImpl(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration)
TSA_REQUIRES(mutex)
{
auto [host, port, secure] = getHostPortSecure(uri, proxy_configuration);
auto key = EndpointPoolKey{type, host, port, secure, proxy_configuration};
auto it = endpoints_pool.find(key);
if (it != endpoints_pool.end())
return it->second;
it = endpoints_pool.emplace(key, createConnectionPool(getGroup(type), std::move(host), port, secure, proxy_configuration)).first;
return it->second;
}
void wipeExpired() TSA_REQUIRES(mutex)
{
std::vector<EndpointPoolKey> keys_to_drop;
for (auto & [key, pool] : endpoints_pool)
{
auto left_connections = pool->wipeExpired();
if (left_connections == 0 && pool->getGroupType() != HTTPConnectionGroupType::DISK)
keys_to_drop.push_back(key);
}
for (const auto & key : keys_to_drop)
endpoints_pool.erase(key);
}
static bool useSecureConnection(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration)
{
if (uri.getScheme() == "http")
return false;
if (uri.getScheme() != "https")
throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString());
if (!proxy_configuration.isEmpty())
{
if (ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol && !proxy_configuration.tunneling)
{
// If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request.
// The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server.
// There is a weak link in the security, but that's what the user opted for.
return false;
}
}
return true;
}
static std::tuple<std::string, UInt16, bool> getHostPortSecure(const Poco::URI & uri, const ProxyConfiguration & proxy_configuration)
{
return std::make_tuple(uri.getHost(), uri.getPort(), useSecureConnection(uri, proxy_configuration));
}
};
HTTPConnectionPools::HTTPConnectionPools()
: impl(std::make_unique<HTTPConnectionPools::Impl>())
{
}
HTTPConnectionPools & HTTPConnectionPools::instance()
{
static HTTPConnectionPools instance;
return instance;
}
void HTTPConnectionPools::setLimits(HTTPConnectionPools::Limits disk, HTTPConnectionPools::Limits storage, HTTPConnectionPools::Limits http)
{
impl->setLimits(std::move(disk), std::move(storage), std::move(http));
}
void HTTPConnectionPools::dropCache()
{
impl->dropCache();
}
IHTTPConnectionPoolForEndpoint::Ptr
HTTPConnectionPools::getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration)
{
return impl->getPool(type, uri, proxy_configuration);
}
}

View File

@ -0,0 +1,91 @@
#pragma once
#include <IO/ConnectionTimeouts.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Common/ProxyConfiguration.h>
#include <Common/logger_useful.h>
#include <base/defines.h>
#include <Poco/Timespan.h>
#include <Poco/Net/HTTPClientSession.h>
#include <mutex>
#include <memory>
namespace DB
{
class IHTTPConnectionPoolForEndpoint
{
public:
struct Metrics
{
const ProfileEvents::Event created = ProfileEvents::end();
const ProfileEvents::Event reused = ProfileEvents::end();
const ProfileEvents::Event reset = ProfileEvents::end();
const ProfileEvents::Event preserved = ProfileEvents::end();
const ProfileEvents::Event expired = ProfileEvents::end();
const ProfileEvents::Event errors = ProfileEvents::end();
const ProfileEvents::Event elapsed_microseconds = ProfileEvents::end();
const CurrentMetrics::Metric stored_count = CurrentMetrics::end();
const CurrentMetrics::Metric active_count = CurrentMetrics::end();
};
using Ptr = std::shared_ptr<IHTTPConnectionPoolForEndpoint>;
using Connection = Poco::Net::HTTPClientSession;
using ConnectionPtr = std::shared_ptr<Poco::Net::HTTPClientSession>;
/// can throw Poco::Net::Exception, DB::NetException, DB::Exception
virtual ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) = 0;
virtual const Metrics & getMetrics() const = 0;
virtual ~IHTTPConnectionPoolForEndpoint() = default;
protected:
IHTTPConnectionPoolForEndpoint() = default;
IHTTPConnectionPoolForEndpoint(const IHTTPConnectionPoolForEndpoint &) = delete;
IHTTPConnectionPoolForEndpoint & operator=(const IHTTPConnectionPoolForEndpoint &) = delete;
};
enum class HTTPConnectionGroupType
{
DISK,
STORAGE,
HTTP,
};
class HTTPConnectionPools
{
public:
struct Limits
{
size_t soft_limit = 100;
size_t warning_limit = 1000;
size_t store_limit = 10000;
static constexpr size_t warning_step = 100;
};
private:
HTTPConnectionPools();
HTTPConnectionPools(const HTTPConnectionPools &) = delete;
HTTPConnectionPools & operator=(const HTTPConnectionPools &) = delete;
public:
static HTTPConnectionPools & instance();
void setLimits(Limits disk, Limits storage, Limits http);
void dropCache();
IHTTPConnectionPoolForEndpoint::Ptr getPool(HTTPConnectionGroupType type, const Poco::URI & uri, const ProxyConfiguration & proxy_configuration);
private:
class Impl;
std::unique_ptr<Impl> impl;
};
}

View File

@ -0,0 +1,293 @@
#include <Common/HostResolvePool.h>
#include <Common/DNSResolver.h>
#include <Common/Exception.h>
#include <Common/NetException.h>
#include <Common/ErrorCodes.h>
#include <Common/thread_local_rng.h>
#include <Common/MemoryTrackerSwitcher.h>
#include <mutex>
namespace ProfileEvents
{
extern const Event AddressesDiscovered;
extern const Event AddressesExpired;
extern const Event AddressesFailScored;
}
namespace CurrentMetrics
{
extern const Metric AddressesActive;
}
namespace DB
{
namespace ErrorCodes
{
extern const int DNS_ERROR;
}
HostResolverMetrics HostResolver::getMetrics()
{
return HostResolverMetrics{
.discovered = ProfileEvents::AddressesDiscovered,
.expired = ProfileEvents::AddressesExpired,
.failed = ProfileEvents::AddressesFailScored,
.active_count = CurrentMetrics::AddressesActive,
};
}
HostResolver::WeakPtr HostResolver::getWeakFromThis()
{
return weak_from_this();
}
HostResolver::HostResolver(String host_, Poco::Timespan history_)
: host(std::move(host_))
, history(history_)
, resolve_function([](const String & host_to_resolve) { return DNSResolver::instance().resolveHostAll(host_to_resolve); })
{
update();
}
HostResolver::HostResolver(
ResolveFunction && resolve_function_, String host_, Poco::Timespan history_)
: host(std::move(host_)), history(history_), resolve_function(std::move(resolve_function_))
{
update();
}
HostResolver::~HostResolver()
{
std::lock_guard lock(mutex);
CurrentMetrics::sub(metrics.active_count, records.size());
records.clear();
}
void HostResolver::Entry::setFail()
{
fail = true;
if (auto lock = pool.lock())
lock->setFail(address);
}
HostResolver::Entry::~Entry()
{
if (!fail)
{
if (auto lock = pool.lock())
lock->setSuccess(address);
}
}
void HostResolver::update()
{
MemoryTrackerSwitcher switcher{&total_memory_tracker};
auto next_gen = resolve_function(host);
if (next_gen.empty())
throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host);
std::sort(next_gen.begin(), next_gen.end());
Poco::Timestamp now;
std::lock_guard lock(mutex);
updateImpl(now, next_gen);
}
void HostResolver::reset()
{
std::lock_guard lock(mutex);
CurrentMetrics::sub(metrics.active_count, records.size());
records.clear();
}
void HostResolver::updateWeights()
{
updateWeightsImpl();
if (getTotalWeight() == 0 && !records.empty())
{
for (auto & rec : records)
rec.failed = false;
updateWeightsImpl();
}
chassert((getTotalWeight() > 0 && !records.empty()) || records.empty());
random_weight_picker = std::uniform_int_distribution<size_t>(0, getTotalWeight() - 1);
}
HostResolver::Entry HostResolver::resolve()
{
if (isUpdateNeeded())
update();
std::lock_guard lock(mutex);
return Entry(*this, selectBest());
}
void HostResolver::setSuccess(const Poco::Net::IPAddress & address)
{
std::lock_guard lock(mutex);
auto it = find(address);
if (it == records.end())
return;
auto old_weight = it->getWeight();
++it->usage;
auto new_weight = it->getWeight();
if (old_weight != new_weight)
updateWeights();
}
void HostResolver::setFail(const Poco::Net::IPAddress & address)
{
Poco::Timestamp now;
{
std::lock_guard lock(mutex);
auto it = find(address);
if (it == records.end())
return;
it->failed = true;
it->fail_time = now;
}
ProfileEvents::increment(metrics.failed);
update();
}
Poco::Net::IPAddress HostResolver::selectBest()
{
chassert(!records.empty());
size_t weight = random_weight_picker(thread_local_rng);
auto it = std::partition_point(records.begin(), records.end(), [&](const Record & rec) { return rec.weight_prefix_sum <= weight; });
chassert(it != records.end());
return it->address;
}
HostResolver::Records::iterator HostResolver::find(const Poco::Net::IPAddress & addr) TSA_REQUIRES(mutex)
{
return std::lower_bound(
records.begin(), records.end(), addr, [](const Record & rec, const Poco::Net::IPAddress & value) { return rec.address < value; });
}
bool HostResolver::isUpdateNeeded()
{
Poco::Timestamp now;
std::lock_guard lock(mutex);
return last_resolve_time + history < now || records.empty();
}
void HostResolver::updateImpl(Poco::Timestamp now, std::vector<Poco::Net::IPAddress> & next_gen)
TSA_REQUIRES(mutex)
{
const auto last_effective_resolve = now - history;
Records merged;
merged.reserve(records.size() + next_gen.size());
auto it_before = records.begin();
auto it_next = next_gen.begin();
while (it_before != records.end() || it_next != next_gen.end())
{
if (it_next == next_gen.end() || (it_before != records.end() && it_before->address < *it_next))
{
if (it_before->resolve_time >= last_effective_resolve)
merged.push_back(*it_before);
else
{
CurrentMetrics::sub(metrics.active_count, 1);
ProfileEvents::increment(metrics.expired, 1);
}
++it_before;
}
else if (it_before == records.end() || (it_next != next_gen.end() && *it_next < it_before->address))
{
CurrentMetrics::add(metrics.active_count, 1);
ProfileEvents::increment(metrics.discovered, 1);
merged.push_back(Record(*it_next, now));
++it_next;
}
else
{
merged.push_back(*it_before);
merged.back().resolve_time = now;
++it_before;
++it_next;
}
}
for (auto & rec : merged)
if (rec.failed && rec.fail_time < last_effective_resolve)
rec.failed = false;
chassert(std::is_sorted(merged.begin(), merged.end()));
last_resolve_time = now;
records.swap(merged);
if (records.empty())
throw NetException(ErrorCodes::DNS_ERROR, "no endpoints resolved for host {}", host);
updateWeights();
}
size_t HostResolver::getTotalWeight() const
{
if (records.empty())
return 0;
return records.back().weight_prefix_sum;
}
void HostResolver::updateWeightsImpl()
{
size_t total_weight_next = 0;
for (auto & rec: records)
{
total_weight_next += rec.getWeight();
rec.weight_prefix_sum = total_weight_next;
}
}
HostResolversPool & HostResolversPool::instance()
{
static HostResolversPool instance;
return instance;
}
void HostResolversPool::dropCache()
{
std::lock_guard lock(mutex);
host_pools.clear();
}
HostResolver::Ptr HostResolversPool::getResolver(const String & host)
{
std::lock_guard lock(mutex);
auto it = host_pools.find(host);
if (it != host_pools.end())
return it->second;
it = host_pools.emplace(host, HostResolver::create(host)).first;
return it->second;
}
}

View File

@ -0,0 +1,218 @@
#pragma once
#include <Common/CurrentMetrics.h>
#include <Common/HTTPConnectionPool.h>
#include <Common/ProfileEvents.h>
#include <Common/logger_useful.h>
#include <base/defines.h>
#include <Poco/Net/IPAddress.h>
#include <mutex>
#include <memory>
// That class resolves host into multiply addresses
// Features:
// - balance address usage.
// `selectBest()` chooses the address by random with weights.
// The more ip is used the lesser weight it has. When new address is happened, it takes more weight.
// But still not all requests are assigned to the new address.
// - join resolve results
// In case when host is resolved into different set of addresses, this class join all that addresses and use them.
// An address expires after `history_` time.
// - failed address pessimization
// If an address marked with `setFail()` it is marked as faulty. Such address won't be selected until either
// a) it still occurs in resolve set after `history_` time or b) all other addresses are pessimized as well.
// - resolve schedule
// Addresses are resolved through `DB::DNSResolver::instance()`.
// Usually it does not happen more often than once in `history_` time.
// But also new resolve performed each `setFail()` call.
namespace DB
{
struct HostResolverMetrics
{
const ProfileEvents::Event discovered = ProfileEvents::end();
const ProfileEvents::Event expired = ProfileEvents::end();
const ProfileEvents::Event failed = ProfileEvents::end();
const CurrentMetrics::Metric active_count = CurrentMetrics::end();
};
constexpr size_t DEFAULT_RESOLVE_TIME_HISTORY_SECONDS = 2*60;
class HostResolver : public std::enable_shared_from_this<HostResolver>
{
private:
using WeakPtr = std::weak_ptr<HostResolver>;
public:
using Ptr = std::shared_ptr<HostResolver>;
template<class... Args>
static Ptr create(Args&&... args)
{
struct make_shared_enabler : public HostResolver
{
explicit make_shared_enabler(Args&&... args) : HostResolver(std::forward<Args>(args)...) {}
};
return std::make_shared<make_shared_enabler>(std::forward<Args>(args)...);
}
virtual ~HostResolver();
class Entry
{
public:
explicit Entry(Entry && entry) = default;
explicit Entry(Entry & entry) = delete;
// no access as r-value
const String * operator->() && = delete;
const String * operator->() const && = delete;
const String & operator*() && = delete;
const String & operator*() const && = delete;
const String * operator->() & { return &resolved_host; }
const String * operator->() const & { return &resolved_host; }
const String & operator*() & { return resolved_host; }
const String & operator*() const & { return resolved_host; }
void setFail();
~Entry();
private:
friend class HostResolver;
Entry(HostResolver & pool_, Poco::Net::IPAddress address_)
: pool(pool_.getWeakFromThis())
, address(std::move(address_))
, resolved_host(address.toString())
{ }
HostResolver::WeakPtr pool;
const Poco::Net::IPAddress address;
const String resolved_host;
bool fail = false;
};
/// can throw NetException(ErrorCodes::DNS_ERROR, ...), Exception(ErrorCodes::BAD_ARGUMENTS, ...)
Entry resolve();
void update();
void reset();
static HostResolverMetrics getMetrics();
protected:
explicit HostResolver(
String host_,
Poco::Timespan history_ = Poco::Timespan(DEFAULT_RESOLVE_TIME_HISTORY_SECONDS, 0));
using ResolveFunction = std::function<std::vector<Poco::Net::IPAddress> (const String & host)>;
HostResolver(ResolveFunction && resolve_function_,
String host_,
Poco::Timespan history_);
friend class Entry;
WeakPtr getWeakFromThis();
void setSuccess(const Poco::Net::IPAddress & address);
void setFail(const Poco::Net::IPAddress & address);
struct Record
{
Record(Poco::Net::IPAddress address_, Poco::Timestamp resolve_time_)
: address(std::move(address_))
, resolve_time(resolve_time_)
{}
explicit Record(Record && rec) = default;
Record& operator=(Record && s) = default;
explicit Record(const Record & rec) = default;
Record& operator=(const Record & s) = default;
Poco::Net::IPAddress address;
Poco::Timestamp resolve_time;
size_t usage = 0;
bool failed = false;
Poco::Timestamp fail_time = 0;
size_t weight_prefix_sum;
bool operator <(const Record & r) const
{
return address < r.address;
}
size_t getWeight() const
{
if (failed)
return 0;
/// There is no goal to make usage's distribution ideally even
/// The goal is to chose more often new address, but still use old addresses as well
/// when all addresses have usage counter greater than 10000,
/// no more corrections are needed, just random choice is ok
if (usage > 10000)
return 1;
if (usage > 1000)
return 5;
if (usage > 100)
return 8;
return 10;
}
};
using Records = std::vector<Record>;
Poco::Net::IPAddress selectBest() TSA_REQUIRES(mutex);
Records::iterator find(const Poco::Net::IPAddress & address) TSA_REQUIRES(mutex);
bool isUpdateNeeded();
void updateImpl(Poco::Timestamp now, std::vector<Poco::Net::IPAddress> & next_gen) TSA_REQUIRES(mutex);
void updateWeights() TSA_REQUIRES(mutex);
void updateWeightsImpl() TSA_REQUIRES(mutex);
size_t getTotalWeight() const TSA_REQUIRES(mutex);
const String host;
const Poco::Timespan history;
const HostResolverMetrics metrics = getMetrics();
// for tests purpose
const ResolveFunction resolve_function;
std::mutex mutex;
Poco::Timestamp last_resolve_time TSA_GUARDED_BY(mutex);
Records records TSA_GUARDED_BY(mutex);
std::uniform_int_distribution<size_t> random_weight_picker TSA_GUARDED_BY(mutex);
Poco::Logger * log = &Poco::Logger::get("ConnectionPool");
};
class HostResolversPool
{
private:
HostResolversPool() = default;
HostResolversPool(const HostResolversPool &) = delete;
HostResolversPool & operator=(const HostResolversPool &) = delete;
public:
static HostResolversPool & instance();
void dropCache();
HostResolver::Ptr getResolver(const String & host);
private:
std::mutex mutex;
std::unordered_map<String, HostResolver::Ptr> host_pools TSA_GUARDED_BY(mutex);
};
}

View File

@ -696,6 +696,35 @@ The server successfully detected this situation and will download merged part fr
\
M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \
M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas") \
\
M(StorageConnectionsCreated, "Number of created connections for storages") \
M(StorageConnectionsReused, "Number of reused connections for storages") \
M(StorageConnectionsReset, "Number of reset connections for storages") \
M(StorageConnectionsPreserved, "Number of preserved connections for storages") \
M(StorageConnectionsExpired, "Number of expired connections for storages") \
M(StorageConnectionsErrors, "Number of cases when creation of a connection for storage is failed") \
M(StorageConnectionsElapsedMicroseconds, "Total time spend on creating connections for storages") \
\
M(DiskConnectionsCreated, "Number of created connections for disk") \
M(DiskConnectionsReused, "Number of reused connections for disk") \
M(DiskConnectionsReset, "Number of reset connections for disk") \
M(DiskConnectionsPreserved, "Number of preserved connections for disk") \
M(DiskConnectionsExpired, "Number of expired connections for disk") \
M(DiskConnectionsErrors, "Number of cases when creation of a connection for disk is failed") \
M(DiskConnectionsElapsedMicroseconds, "Total time spend on creating connections for disk") \
\
M(HTTPConnectionsCreated, "Number of created http connections") \
M(HTTPConnectionsReused, "Number of reused http connections") \
M(HTTPConnectionsReset, "Number of reset http connections") \
M(HTTPConnectionsPreserved, "Number of preserved http connections") \
M(HTTPConnectionsExpired, "Number of expired http connections") \
M(HTTPConnectionsErrors, "Number of cases when creation of a http connection failed") \
M(HTTPConnectionsElapsedMicroseconds, "Total time spend on creating http connections") \
\
M(AddressesDiscovered, "Total count of new addresses in dns resolve results for connection pools") \
M(AddressesExpired, "Total count of expired addresses which is no longer presented in dns resolve results for for connection pools") \
M(AddressesFailScored, "Total count of new addresses in dns resolve results for for connection pools") \
#ifdef APPLY_FOR_EXTERNAL_EVENTS
#define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M)

View File

@ -44,11 +44,13 @@ struct ProxyConfiguration
}
}
std::string host;
Protocol protocol;
uint16_t port;
bool tunneling;
Protocol original_request_protocol;
std::string host = std::string{};
Protocol protocol = Protocol::HTTP;
uint16_t port = 0;
bool tunneling = false;
Protocol original_request_protocol = Protocol::HTTP;
bool isEmpty() const { return host.size() == 0; }
};
}

View File

@ -26,8 +26,6 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve()
auto & proxy = proxies[index];
LOG_DEBUG(getLogger("ProxyListConfigurationResolver"), "Use proxy: {}", proxies[index].toString());
return ProxyConfiguration {
proxy.getHost(),
ProxyConfiguration::protocolFromString(proxy.getScheme()),

View File

@ -69,7 +69,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve()
{
auto resolved_endpoint = endpoint;
resolved_endpoint.setHost(resolved_hosts[i].toString());
session = makeHTTPSession(resolved_endpoint, timeouts);
session = makeHTTPSession(HTTPConnectionGroupType::HTTP, resolved_endpoint, timeouts);
try
{

View File

@ -0,0 +1,558 @@
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Common/HTTPConnectionPool.h>
#include <Poco/URI.h>
#include <Poco/Net/ServerSocket.h>
#include <Poco/Net/MessageHeader.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTTPServer.h>
#include <Poco/Net/HTTPServerParams.h>
#include <Poco/Net/HTTPRequestHandler.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
#include <thread>
#include <gtest/gtest.h>
namespace
{
size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits<size_t>::max())
{
const size_t buffer_size = 4096;
char buffer[buffer_size];
size_t total_read = 0;
while (count > buffer_size)
{
in.read(buffer, buffer_size);
size_t read = in.gcount();
out.write(buffer, read);
count -= read;
total_read += read;
if (read == 0)
return total_read;
}
in.read(buffer, count);
size_t read = in.gcount();
out.write(buffer, read);
total_read += read;
return total_read;
}
class MockRequestHandler : public Poco::Net::HTTPRequestHandler
{
public:
explicit MockRequestHandler(std::shared_ptr<std::atomic<size_t>> slowdown_)
: slowdown(std::move(slowdown_))
{
}
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override
{
response.setStatus(Poco::Net::HTTPResponse::HTTP_OK);
auto size = request.getContentLength();
if (size > 0)
response.setContentLength(size); // ContentLength is required for keep alive
else
response.setChunkedTransferEncoding(true); // or chunk encoding
sleepForSeconds(*slowdown);
stream_copy_n(request.stream(), response.send(), size);
}
std::shared_ptr<std::atomic<size_t>> slowdown;
};
class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory
{
public:
explicit HTTPRequestHandlerFactory(std::shared_ptr<std::atomic<size_t>> slowdown_)
: slowdown(std::move(slowdown_))
{
}
Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override
{
return new MockRequestHandler(slowdown);
}
std::shared_ptr<std::atomic<size_t>> slowdown;
};
}
using HTTPSession = Poco::Net::HTTPClientSession;
using HTTPSessionPtr = std::shared_ptr<Poco::Net::HTTPClientSession>;
class ConnectionPoolTest : public testing::Test {
protected:
ConnectionPoolTest()
{
startServer();
}
void SetUp() override {
timeouts = DB::ConnectionTimeouts();
DB::HTTPConnectionPools::Limits def_limits{};
DB::HTTPConnectionPools::instance().setLimits(def_limits, def_limits, def_limits);
setSlowDown(0);
DB::HTTPConnectionPools::instance().dropCache();
DB::CurrentThread::getProfileEvents().reset();
// Code here will be called immediately after the constructor (right
// before each test).
}
void TearDown() override {
// Code here will be called immediately after each test (right
// before the destructor).
}
DB::IHTTPConnectionPoolForEndpoint::Ptr getPool()
{
auto uri = Poco::URI(getServerUrl());
return DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{});
}
std::string getServerUrl() const
{
return "http://" + server_data.socket->address().toString();
}
void startServer()
{
server_data.reset();
server_data.params = new Poco::Net::HTTPServerParams();
server_data.socket = std::make_unique<Poco::Net::ServerSocket>(server_data.port);
server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive);
server_data.server = std::make_unique<Poco::Net::HTTPServer>(
server_data.handler_factory, *server_data.socket, server_data.params);
server_data.server->start();
}
Poco::Net::HTTPServer & getServer() const
{
return *server_data.server;
}
void setSlowDown(size_t seconds)
{
*slowdown_receive = seconds;
}
DB::ConnectionTimeouts timeouts;
std::shared_ptr<std::atomic<size_t>> slowdown_receive = std::make_shared<std::atomic<size_t>>(0);
struct ServerData
{
// just some port to avoid collisions with others tests
UInt16 port = 9871;
Poco::Net::HTTPServerParams::Ptr params;
std::unique_ptr<Poco::Net::ServerSocket> socket;
HTTPRequestHandlerFactory::Ptr handler_factory;
std::unique_ptr<Poco::Net::HTTPServer> server;
ServerData() = default;
ServerData(ServerData &&) = default;
ServerData & operator =(ServerData &&) = delete;
void reset()
{
if (server)
server->stop();
server = nullptr;
handler_factory = nullptr;
socket = nullptr;
params = nullptr;
}
~ServerData() {
reset();
}
};
ServerData server_data;
};
void wait_until(std::function<bool()> pred)
{
while (!pred())
sleepForMilliseconds(250);
}
void echoRequest(String data, HTTPSession & session)
{
{
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_PUT, "/", "HTTP/1.1"); // HTTP/1.1 is required for keep alive
request.setContentLength(data.size());
std::ostream & ostream = session.sendRequest(request);
ostream << data;
}
{
std::stringstream result;
Poco::Net::HTTPResponse response;
std::istream & istream = session.receiveResponse(response);
ASSERT_EQ(response.getStatus(), Poco::Net::HTTPResponse::HTTP_OK);
stream_copy_n(istream, result);
ASSERT_EQ(data, result.str());
}
}
TEST_F(ConnectionPoolTest, CanConnect)
{
auto pool = getPool();
auto connection = pool->getConnection(timeouts);
ASSERT_TRUE(connection->connected());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
connection->reset();
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
}
TEST_F(ConnectionPoolTest, CanRequest)
{
auto pool = getPool();
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, getServer().currentConnections());
connection->reset();
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
}
TEST_F(ConnectionPoolTest, CanPreserve)
{
auto pool = getPool();
{
auto connection = pool->getConnection(timeouts);
}
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count));
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, CanReuse)
{
auto pool = getPool();
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
}
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count));
{
auto connection = pool->getConnection(timeouts);
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
echoRequest("Hello", *connection);
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, getServer().currentConnections());
connection->reset();
}
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, CanReuse10)
{
auto pool = getPool();
for (int i = 0; i < 10; ++i)
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
}
{
auto connection = pool->getConnection(timeouts);
connection->reset(); // reset just not to wait its expiration here
}
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, CanReuse5)
{
timeouts.withHTTPKeepAliveTimeout(1);
auto pool = getPool();
std::vector<DB::HTTPSessionPtr> connections;
connections.reserve(5);
for (int i = 0; i < 5; ++i)
{
connections.push_back(pool->getConnection(timeouts));
}
connections.clear();
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count));
wait_until([&] () { return getServer().currentConnections() == 5; });
ASSERT_EQ(5, getServer().currentConnections());
ASSERT_EQ(5, getServer().totalConnections());
for (int i = 0; i < 5; ++i)
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
}
ASSERT_EQ(5, getServer().totalConnections());
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count));
}
TEST_F(ConnectionPoolTest, CanReconnectAndCreate)
{
auto pool = getPool();
std::vector<HTTPSessionPtr> in_use;
const size_t count = 2;
for (int i = 0; i < count; ++i)
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
in_use.push_back(connection);
}
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(count, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
auto connection = std::move(in_use.back());
in_use.pop_back();
echoRequest("Hello", *connection);
connection->abort(); // further usage requires reconnect, new connection
echoRequest("Hello", *connection);
connection->reset();
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(count+1, getServer().totalConnections());
ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, CanReconnectAndReuse)
{
auto pool = getPool();
std::vector<HTTPSessionPtr> in_use;
const size_t count = 2;
for (int i = 0; i < count; ++i)
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
in_use.push_back(std::move(connection));
}
auto connection = std::move(in_use.back());
in_use.pop_back();
in_use.clear(); // other connection will be reused
echoRequest("Hello", *connection);
connection->abort(); // further usage requires reconnect, reuse connection from pool
echoRequest("Hello", *connection);
connection->reset();
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(2, getServer().totalConnections());
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, ReceiveTimeout)
{
setSlowDown(2);
timeouts.withReceiveTimeout(1);
auto pool = getPool();
{
auto connection = pool->getConnection(timeouts);
ASSERT_ANY_THROW(
echoRequest("Hello", *connection);
);
}
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
{
timeouts.withReceiveTimeout(3);
auto connection = pool->getConnection(timeouts);
ASSERT_NO_THROW(
echoRequest("Hello", *connection);
);
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
{
/// timeouts have effect for reused session
timeouts.withReceiveTimeout(1);
auto connection = pool->getConnection(timeouts);
ASSERT_ANY_THROW(
echoRequest("Hello", *connection);
);
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
}
TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP)
{
std::string_view message = "Hello ReadWriteBufferFromHTTP";
auto uri = Poco::URI(getServerUrl());
auto metrics = DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{})->getMetrics();
Poco::Net::HTTPBasicCredentials empty_creds;
auto buf_from_http = DB::BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(DB::HTTPConnectionGroupType::HTTP)
.withOutCallback(
[&] (std::ostream & in)
{
in << message;
})
.withDelayInit(false)
.create(empty_creds);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
char buf[256];
std::fill(buf, buf + sizeof(buf), 0);
buf_from_http->readStrict(buf, message.size());
ASSERT_EQ(std::string_view(buf), message);
ASSERT_TRUE(buf_from_http->eof());
buf_from_http.reset();
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, HardLimit)
{
DB::HTTPConnectionPools::Limits zero_limits {0, 0, 0};
DB::HTTPConnectionPools::instance().setLimits(zero_limits, zero_limits, zero_limits);
auto pool = getPool();
{
auto connection = pool->getConnection(timeouts);
}
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
}

View File

@ -0,0 +1,278 @@
#include <IO/WriteBufferFromFile.h>
#include <Common/HostResolvePool.h>
#include <base/sleep.h>
#include <thread>
#include <gtest/gtest.h>
class ResolvePoolMock : public DB::HostResolver
{
public:
using ResolveFunction = DB::HostResolver::ResolveFunction;
ResolvePoolMock(String host_, Poco::Timespan history_, ResolveFunction && func)
: DB::HostResolver(std::move(func), std::move(host_), history_)
{
}
};
class ResolvePoolTest : public testing::Test
{
protected:
ResolvePoolTest()
{
DB::HostResolversPool::instance().dropCache();
}
void SetUp() override {
DB::CurrentThread::getProfileEvents().reset();
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
addresses = std::set<String>{"127.0.0.1", "127.0.0.2", "127.0.0.3"};
// Code here will be called immediately after the constructor (right
// before each test).
}
void TearDown() override {
// Code here will be called immediately after each test (right
// before the destructor).
}
DB::HostResolver::Ptr make_resolver(size_t history_ms = 200)
{
auto resolve_func = [&] (const String &)
{
std::vector<Poco::Net::IPAddress> result;
result.reserve(addresses.size());
for (const auto & item : addresses)
{
result.push_back(Poco::Net::IPAddress(item));
}
return result;
};
return std::make_shared<ResolvePoolMock>("some_host", Poco::Timespan(history_ms * 1000), std::move(resolve_func));
}
DB::HostResolverMetrics metrics = DB::HostResolver::getMetrics();
std::set<String> addresses;
};
TEST_F(ResolvePoolTest, CanResolve)
{
auto resolver = make_resolver();
auto address = resolver->resolve();
ASSERT_TRUE(addresses.contains(*address));
ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]);
ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count));
}
TEST_F(ResolvePoolTest, CanResolveAll)
{
auto resolver = make_resolver();
std::set<String> results;
while (results.size() != addresses.size())
{
auto next_addr = resolver->resolve();
results.insert(*next_addr);
}
ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]);
}
size_t getSum(std::map<String, size_t> container)
{
size_t sum = 0;
for (auto & [_, val] : container)
{
sum += val;
}
return sum;
}
size_t getMin(std::map<String, size_t> container)
{
if (container.empty())
return 0;
size_t min_val = container.begin()->second;
for (auto & [_, val] : container)
{
min_val = std::min(min_val, val);
}
return min_val;
}
double getMean(std::map<String, size_t> container)
{
return 1.0 * getSum(container) / container.size();
}
double getMaxDiff(std::map<String, size_t> container, double ref_val)
{
double diff = 0.0;
for (auto & [_, val] : container)
{
diff = std::max(std::fabs(val - ref_val), diff);
}
return diff;
}
TEST_F(ResolvePoolTest, CanResolveEvenly)
{
auto resolver = make_resolver();
std::map<String, size_t> results;
for (size_t i = 0; i < 50000; ++i)
{
auto next_addr = resolver->resolve();
if (results.contains(*next_addr))
{
results[*next_addr] += 1;
}
else
{
results[*next_addr] = 1;
}
}
auto mean = getMean(results);
auto diff = getMaxDiff(results, mean);
ASSERT_GT(0.3 * mean, diff);
}
TEST_F(ResolvePoolTest, CanMerge)
{
auto resolver = make_resolver(100000);
auto address = resolver->resolve();
ASSERT_TRUE(addresses.contains(*address));
ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]);
auto old_addresses = addresses;
addresses = std::set<String>{"127.0.0.4", "127.0.0.5"};
resolver->update();
ASSERT_EQ(addresses.size() + old_addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]);
ASSERT_EQ(addresses.size() + old_addresses.size(), CurrentMetrics::get(metrics.active_count));
std::set<String> results;
while (results.size() != addresses.size() + old_addresses.size())
{
auto next_addr = resolver->resolve();
results.insert(*next_addr);
}
}
TEST_F(ResolvePoolTest, CanGainEven)
{
auto resolver = make_resolver();
auto address = resolver->resolve();
std::map<String, size_t> results;
for (size_t i = 0; i < 40000; ++i)
{
auto next_addr = resolver->resolve();
if (results.contains(*next_addr))
{
results[*next_addr] += 1;
}
else
{
results[*next_addr] = 1;
}
}
ASSERT_GT(getMin(results), 10000);
addresses.insert("127.0.0.4");
addresses.insert("127.0.0.5");
resolver->update();
/// return mostly new addresses
for (size_t i = 0; i < 3000; ++i)
{
auto next_addr = resolver->resolve();
if (results.contains(*next_addr))
{
results[*next_addr] += 1;
}
else
{
results[*next_addr] = 1;
}
}
ASSERT_EQ(results.size(), 5);
ASSERT_GT(getMin(results), 1000);
}
TEST_F(ResolvePoolTest, CanFail)
{
auto resolver = make_resolver(10000);
auto failed_addr = resolver->resolve();
failed_addr.setFail();
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.failed]);
ASSERT_EQ(addresses.size(), CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(addresses.size(), DB::CurrentThread::getProfileEvents()[metrics.discovered]);
for (size_t i = 0; i < 1000; ++i)
{
auto next_addr = resolver->resolve();
ASSERT_TRUE(addresses.contains(*next_addr));
ASSERT_NE(*next_addr, *failed_addr);
}
}
TEST_F(ResolvePoolTest, CanFailAndHeal)
{
auto resolver = make_resolver();
auto failed_addr = resolver->resolve();
failed_addr.setFail();
while (true)
{
auto next_addr = resolver->resolve();
if (*failed_addr == *next_addr)
break;
}
}
TEST_F(ResolvePoolTest, CanExpire)
{
auto resolver = make_resolver();
auto expired_addr = resolver->resolve();
ASSERT_TRUE(addresses.contains(*expired_addr));
addresses.erase(*expired_addr);
sleepForSeconds(1);
for (size_t i = 0; i < 1000; ++i)
{
auto next_addr = resolver->resolve();
ASSERT_TRUE(addresses.contains(*next_addr));
ASSERT_NE(*next_addr, *expired_addr);
}
ASSERT_EQ(addresses.size() + 1, DB::CurrentThread::getProfileEvents()[metrics.discovered]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]);
}

View File

@ -128,8 +128,17 @@ namespace DB
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
M(UInt64, disk_connections_soft_limit, 1000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_store_limit, 12000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \
M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \
M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \
M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \
M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)

View File

@ -110,7 +110,7 @@ class IColumn;
M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \
M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \
M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \
M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. This only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \
M(UInt64, s3_connect_timeout_ms, 1000, "Connection timeout for host from s3 disks.", 0) \
M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \
M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \
M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \

View File

@ -85,7 +85,8 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.3", {{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},
{"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"},
{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},
{"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"},
{"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"},
{"page_cache_inject_eviction", false, false, "Added userspace page cache"},

View File

@ -88,20 +88,18 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri)
QueryPipeline HTTPDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll {}", toString());
Poco::URI uri(configuration.url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
timeouts,
credentials,
0,
DBMS_DEFAULT_BUFFER_SIZE,
context->getReadSettings(),
configuration.header_entries,
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
Poco::URI uri(configuration.url);
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withSettings(context->getReadSettings())
.withTimeouts(timeouts)
.withHeaders(configuration.header_entries)
.withDelayInit(false)
.create(credentials);
return createWrappedBuffer(std::move(buf));
}
QueryPipeline HTTPDictionarySource::loadUpdatedAll()
@ -109,19 +107,16 @@ QueryPipeline HTTPDictionarySource::loadUpdatedAll()
Poco::URI uri(configuration.url);
getUpdateFieldAndDate(uri);
LOG_TRACE(log, "loadUpdatedAll {}", uri.toString());
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
timeouts,
credentials,
0,
DBMS_DEFAULT_BUFFER_SIZE,
context->getReadSettings(),
configuration.header_entries,
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withSettings(context->getReadSettings())
.withTimeouts(timeouts)
.withHeaders(configuration.header_entries)
.withDelayInit(false)
.create(credentials);
return createWrappedBuffer(std::move(buf));
}
QueryPipeline HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
@ -139,19 +134,18 @@ QueryPipeline HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
};
Poco::URI uri(configuration.url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
out_stream_callback,
timeouts,
credentials,
0,
DBMS_DEFAULT_BUFFER_SIZE,
context->getReadSettings(),
configuration.header_entries,
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withSettings(context->getReadSettings())
.withTimeouts(timeouts)
.withHeaders(configuration.header_entries)
.withOutCallback(std::move(out_stream_callback))
.withDelayInit(false)
.create(credentials);
return createWrappedBuffer(std::move(buf));
}
QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
@ -169,19 +163,18 @@ QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const
};
Poco::URI uri(configuration.url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
out_stream_callback,
timeouts,
credentials,
0,
DBMS_DEFAULT_BUFFER_SIZE,
context->getReadSettings(),
configuration.header_entries,
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withSettings(context->getReadSettings())
.withTimeouts(timeouts)
.withHeaders(configuration.header_entries)
.withOutCallback(std::move(out_stream_callback))
.withDelayInit(false)
.create(credentials);
return createWrappedBuffer(std::move(buf));
}
bool HTTPDictionarySource::isModified() const

View File

@ -203,7 +203,7 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request)
}
QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const
QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const
{
bridge_helper->startBridgeSync();
@ -214,10 +214,15 @@ QueryPipeline XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const B
os << "query=" << escapeForFileName(query);
};
auto read_buf = std::make_unique<ReadWriteBufferFromHTTP>(
url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts, credentials);
auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, max_block_size);
format->addBuffer(std::move(read_buf));
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(timeouts)
.withOutCallback(std::move(write_body_callback))
.create(credentials);
auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *buf, required_sample_block, max_block_size);
format->addBuffer(std::move(buf));
return QueryPipeline(std::move(format));
}

View File

@ -74,7 +74,7 @@ private:
// execute invalidate_query. expects single cell in result
std::string doInvalidateQuery(const std::string & request) const;
QueryPipeline loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const;
QueryPipeline loadFromQuery(const Poco::URI & uri, const Block & required_sample_block, const std::string & query) const;
LoggerPtr log;

View File

@ -1,8 +1,6 @@
#include "ReadBufferFromWebServer.h"
#include <Common/logger_useful.h>
#include <base/sleep.h>
#include <Core/Types.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
@ -45,12 +43,6 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
{
if (read_until_position < offset)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1);
LOG_DEBUG(log, "Reading with range: {}-{}", offset, read_until_position);
}
else
{
LOG_DEBUG(log, "Reading from offset: {}", offset);
}
const auto & settings = context->getSettingsRef();
@ -60,19 +52,14 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
connection_timeouts.withConnectionTimeout(std::max<Poco::Timespan>(settings.http_connection_timeout, Poco::Timespan(20, 0)));
connection_timeouts.withReceiveTimeout(std::max<Poco::Timespan>(settings.http_receive_timeout, Poco::Timespan(20, 0)));
auto res = std::make_unique<ReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
connection_timeouts,
credentials,
0,
buf_size,
read_settings,
HTTPHeaderEntries{},
&context->getRemoteHostFilter(),
/* delay_initialization */true,
use_external_buffer);
auto res = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::DISK)
.withSettings(read_settings)
.withTimeouts(connection_timeouts)
.withBufSize(buf_size)
.withHostFilter(&context->getRemoteHostFilter())
.withExternalBuf(use_external_buffer)
.create(credentials);
if (read_until_position)
res->setReadUntilPosition(read_until_position);
@ -101,44 +88,44 @@ bool ReadBufferFromWebServer::nextImpl()
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1);
}
if (impl)
{
if (!use_external_buffer)
{
/**
* impl was initialized before, pass position() to it to make
* sure there is no pending data which was not read, because
* this branch means we read sequentially.
*/
impl->position() = position();
assert(!impl->hasPendingData());
}
}
else
if (!impl)
{
impl = initialize();
if (!use_external_buffer)
{
BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset());
}
}
if (use_external_buffer)
{
/**
* use_external_buffer -- means we read into the buffer which
* was passed to us from somewhere else. We do not check whether
* previously returned buffer was read or not, because this branch
* means we are prefetching data, each nextImpl() call we can fill
* a different buffer.
*/
impl->set(internal_buffer.begin(), internal_buffer.size());
assert(working_buffer.begin() != nullptr);
assert(!internal_buffer.empty());
}
else
{
impl->position() = position();
}
chassert(available() == 0);
chassert(pos >= working_buffer.begin());
chassert(pos <= working_buffer.end());
chassert(working_buffer.begin() != nullptr);
chassert(impl->buffer().begin() != nullptr);
chassert(working_buffer.begin() == impl->buffer().begin());
chassert(impl->available() == 0);
auto result = impl->next();
BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset());
chassert(working_buffer.begin() == impl->buffer().begin());
if (result)
{
BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset());
offset += working_buffer.size();
}
return result;
}

View File

@ -67,10 +67,6 @@ std::unique_ptr<S3::Client> getClient(
client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS);
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS);
client_configuration.endpointOverride = uri.endpoint;
client_configuration.http_keep_alive_timeout_ms = config.getUInt(
config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000);
client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000);
client_configuration.wait_on_pool_size_limit = false;
client_configuration.s3_use_adaptive_timeouts = config.getBool(
config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts);

View File

@ -44,34 +44,33 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock<std::sha
{
Poco::Net::HTTPBasicCredentials credentials{};
ReadWriteBufferFromHTTP metadata_buf(
Poco::URI(fs::path(full_url) / ".index"),
Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(),
ConnectionTimeouts::getHTTPTimeouts(
getContext()->getSettingsRef(),
getContext()->getServerSettings().keep_alive_timeout),
credentials,
/* max_redirects= */ 0,
/* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE,
getContext()->getReadSettings());
auto timeouts = ConnectionTimeouts::getHTTPTimeouts(
getContext()->getSettingsRef(),
getContext()->getServerSettings().keep_alive_timeout);
auto metadata_buf = BuilderRWBufferFromHTTP(Poco::URI(fs::path(full_url) / ".index"))
.withConnectionGroup(HTTPConnectionGroupType::DISK)
.withSettings(getContext()->getReadSettings())
.withTimeouts(timeouts)
.withHostFilter(&getContext()->getRemoteHostFilter())
.create(credentials);
String file_name;
while (!metadata_buf.eof())
while (!metadata_buf->eof())
{
readText(file_name, metadata_buf);
assertChar('\t', metadata_buf);
readText(file_name, *metadata_buf);
assertChar('\t', *metadata_buf);
bool is_directory;
readBoolText(is_directory, metadata_buf);
readBoolText(is_directory, *metadata_buf);
size_t size = 0;
if (!is_directory)
{
assertChar('\t', metadata_buf);
readIntText(size, metadata_buf);
assertChar('\t', *metadata_buf);
readIntText(size, *metadata_buf);
}
assertChar('\n', metadata_buf);
assertChar('\n', *metadata_buf);
FileDataPtr file_data = is_directory
? FileData::createDirectoryInfo(false)

View File

@ -122,7 +122,7 @@ public:
for (size_t i = 0; i < input_rows_count; ++i)
{
std::string_view sqid = col_non_const->getDataAt(i).toView();
std::vector<UInt64> integers = sqids.decode(sqid);
std::vector<UInt64> integers = sqids.decode(String(sqid));
res_nested_data.insert(integers.begin(), integers.end());
res_offsets_data.push_back(integers.size());
}

View File

@ -141,4 +141,19 @@ ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method
.withReceiveTimeout(saturate(recv, receive_timeout));
}
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts)
{
session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout);
}
ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session)
{
return ConnectionTimeouts()
.withConnectionTimeout(session.getConnectionTimeout())
.withSendTimeout(session.getSendTimeout())
.withReceiveTimeout(session.getReceiveTimeout())
.withHTTPKeepAliveTimeout(session.getKeepAliveTimeout());
}
}

View File

@ -4,6 +4,7 @@
#include <Core/ServerSettings.h>
#include <Interpreters/Context_fwd.h>
#include <Poco/Net/HTTPClientSession.h>
#include <Poco/Timespan.h>
namespace DB
@ -111,4 +112,7 @@ inline ConnectionTimeouts & ConnectionTimeouts::withConnectionTimeout(Poco::Time
return *this;
}
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts);
ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session);
}

View File

@ -2,13 +2,7 @@
#include <Server/HTTP/HTTPServerResponse.h>
#include <Poco/Any.h>
#include <Common/Concepts.h>
#include <Common/DNSResolver.h>
#include <Common/Exception.h>
#include <Common/MemoryTrackerSwitcher.h>
#include <Common/PoolBase.h>
#include <Common/ProfileEvents.h>
#include <Common/SipHash.h>
#include "config.h"
@ -25,338 +19,18 @@
#include <Poco/Util/Application.h>
#include <sstream>
#include <tuple>
#include <istream>
#include <unordered_map>
#include <Common/ProxyConfiguration.h>
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;
extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME;
extern const int UNSUPPORTED_URI_SCHEME;
extern const int LOGICAL_ERROR;
}
namespace
{
Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const ProxyConfiguration & proxy_configuration)
{
Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config;
poco_proxy_config.host = proxy_configuration.host;
poco_proxy_config.port = proxy_configuration.port;
poco_proxy_config.protocol = ProxyConfiguration::protocolToString(proxy_configuration.protocol);
poco_proxy_config.tunnel = proxy_configuration.tunneling;
poco_proxy_config.originalRequestProtocol = ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol);
return poco_proxy_config;
}
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{getLogger("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,
"Last ip ({}) is unreachable for {}:{}. Will try another resolved address.",
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));
}
}
}
LoggerPtr 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());
}
HTTPSessionPtr makeHTTPSessionImpl(
const std::string & host,
UInt16 port,
bool https,
bool keep_alive,
DB::ProxyConfiguration proxy_configuration = {})
{
HTTPSessionPtr session;
if (!proxy_configuration.host.empty())
{
bool is_proxy_http_and_is_tunneling_off = DB::ProxyConfiguration::Protocol::HTTP == proxy_configuration.protocol
&& !proxy_configuration.tunneling;
// If it is an HTTPS request, proxy server is HTTP and user opted for tunneling off, we must not create an HTTPS request.
// The desired flow is: HTTP request to the proxy server, then proxy server will initiate an HTTPS request to the target server.
// There is a weak link in the security, but that's what the user opted for.
if (https && is_proxy_http_and_is_tunneling_off)
{
https = false;
}
}
if (https)
{
#if USE_SSL
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
{
session = std::make_shared<HTTPSessionAdapter<Poco::Net::HTTPClientSession>>(host, port);
}
ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections);
/// doesn't work properly without patch
session->setKeepAlive(keep_alive);
if (!proxy_configuration.host.empty())
{
session->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration));
}
return session;
}
class SingleEndpointHTTPSessionPool : public PoolBase<Poco::Net::HTTPClientSession>
{
private:
const std::string host;
const UInt16 port;
const bool https;
ProxyConfiguration proxy_config;
using Base = PoolBase<Poco::Net::HTTPClientSession>;
ObjectPtr allocObject() override
{
/// Pool is global, we shouldn't attribute this memory to query/user.
MemoryTrackerSwitcher switcher{&total_memory_tracker};
auto session = makeHTTPSessionImpl(host, port, https, true, proxy_config);
return session;
}
public:
SingleEndpointHTTPSessionPool(
const std::string & host_,
UInt16 port_,
bool https_,
ProxyConfiguration proxy_config_,
size_t max_pool_size_,
bool wait_on_pool_size_limit)
: Base(
static_cast<unsigned>(max_pool_size_),
getLogger("HTTPSessionPool"),
wait_on_pool_size_limit ? BehaviourOnLimit::Wait : BehaviourOnLimit::AllocateNewBypassingPool)
, host(host_)
, port(port_)
, https(https_)
, proxy_config(proxy_config_)
{
}
};
class HTTPSessionPool : private boost::noncopyable
{
public:
struct Key
{
String target_host;
UInt16 target_port;
bool is_target_https;
ProxyConfiguration proxy_config;
bool wait_on_pool_size_limit;
bool operator ==(const Key & rhs) const
{
return std::tie(
target_host,
target_port,
is_target_https,
proxy_config.host,
proxy_config.port,
proxy_config.protocol,
proxy_config.tunneling,
proxy_config.original_request_protocol,
wait_on_pool_size_limit)
== std::tie(
rhs.target_host,
rhs.target_port,
rhs.is_target_https,
rhs.proxy_config.host,
rhs.proxy_config.port,
rhs.proxy_config.protocol,
rhs.proxy_config.tunneling,
rhs.proxy_config.original_request_protocol,
rhs.wait_on_pool_size_limit);
}
};
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_config.host);
s.update(k.proxy_config.port);
s.update(k.proxy_config.protocol);
s.update(k.proxy_config.tunneling);
s.update(k.proxy_config.original_request_protocol);
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;
}
Entry getSession(
const Poco::URI & uri,
const ProxyConfiguration & proxy_config,
const ConnectionTimeouts & timeouts,
size_t max_connections_per_endpoint,
bool wait_on_pool_size_limit)
{
std::unique_lock lock(mutex);
const std::string & host = uri.getHost();
UInt16 port = uri.getPort();
bool https = isHTTPS(uri);
HTTPSessionPool::Key key{host, port, https, proxy_config, 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(
key,
std::make_shared<SingleEndpointHTTPSessionPool>(
host,
port,
https,
proxy_config,
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.totalMilliseconds();
auto session = pool_ptr->second->get(retry_timeout);
const auto & session_data = session->sessionData();
if (session_data.empty() || !Poco::AnyCast<HTTPSessionReuseTag>(&session_data))
{
/// Reset session if it is not reusable. See comment for HTTPSessionReuseTag.
session->reset();
}
session->attachSessionData({});
setTimeouts(*session, timeouts);
return session;
}
};
}
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts)
{
session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout);
}
void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout)
@ -370,28 +44,13 @@ void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_
}
HTTPSessionPtr makeHTTPSession(
HTTPConnectionGroupType group,
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
ProxyConfiguration proxy_configuration
)
ProxyConfiguration proxy_configuration)
{
const std::string & host = uri.getHost();
UInt16 port = uri.getPort();
bool https = isHTTPS(uri);
auto session = makeHTTPSessionImpl(host, port, https, false, proxy_configuration);
setTimeouts(*session, timeouts);
return session;
}
PooledHTTPSessionPtr makePooledHTTPSession(
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
size_t per_endpoint_pool_size,
bool wait_on_pool_size_limit,
ProxyConfiguration proxy_config)
{
return HTTPSessionPool::instance().getSession(uri, proxy_config, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit);
auto connection_pool = HTTPConnectionPools::instance().getPool(group, uri, proxy_configuration);
return connection_pool->getConnection(timeouts);
}
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; }
@ -400,11 +59,11 @@ std::istream * receiveResponse(
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, const bool allow_redirects)
{
auto & istr = session.receiveResponse(response);
assertResponseIsOk(request, response, istr, allow_redirects);
assertResponseIsOk(request.getURI(), response, istr, allow_redirects);
return &istr;
}
void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects)
void assertResponseIsOk(const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects)
{
auto status = response.getStatus();
@ -422,7 +81,7 @@ void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPR
body.exceptions(std::ios::failbit);
body << istr.rdbuf();
throw HTTPException(code, request.getURI(), status, response.getReason(), body.str());
throw HTTPException(code, uri, status, response.getReason(), body.str());
}
}
@ -440,24 +99,4 @@ Exception HTTPException::makeExceptionMessage(
uri, static_cast<int>(http_status), reason, body);
}
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));
}
}

View File

@ -7,9 +7,9 @@
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/URI.h>
#include <Common/PoolBase.h>
#include <Common/ProxyConfiguration.h>
#include <Poco/URIStreamFactory.h>
#include <Common/HTTPConnectionPool.h>
#include <Common/ProxyConfiguration.h>
#include <IO/ConnectionTimeouts.h>
@ -36,7 +36,7 @@ public:
HTTPException * clone() const override { return new HTTPException(*this); }
void rethrow() const override { throw *this; }
int getHTTPStatus() const { return http_status; }
Poco::Net::HTTPResponse::HTTPStatus getHTTPStatus() const { return http_status; }
private:
Poco::Net::HTTPResponse::HTTPStatus http_status{};
@ -52,55 +52,18 @@ private:
const char * className() const noexcept override { return "DB::HTTPException"; }
};
using PooledHTTPSessionPtr = PoolBase<Poco::Net::HTTPClientSession>::Entry; // SingleEndpointHTTPSessionPool::Entry
using HTTPSessionPtr = std::shared_ptr<Poco::Net::HTTPClientSession>;
/// If a session have this tag attached, it will be reused without calling `reset()` on it.
/// All pooled sessions don't have this tag attached after being taken from a pool.
/// If the request and the response were fully written/read, the client code should add this tag
/// explicitly by calling `markSessionForReuse()`.
///
/// Note that HTTP response may contain extra bytes after the last byte of the payload. Specifically,
/// when chunked encoding is used, there's an empty chunk at the end. Those extra bytes must also be
/// read before the session can be reused. So we usually put an `istr->ignore(INT64_MAX)` call
/// before `markSessionForReuse()`.
struct HTTPSessionReuseTag
{
};
void markSessionForReuse(Poco::Net::HTTPSession & session);
void markSessionForReuse(HTTPSessionPtr session);
void markSessionForReuse(PooledHTTPSessionPtr session);
void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout);
/// Create session object to perform requests and set required parameters.
HTTPSessionPtr makeHTTPSession(
HTTPConnectionGroupType group,
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
ProxyConfiguration proxy_config = {}
);
/// As previous method creates session, but takes it from pool, without and with proxy uri.
///
/// The max_connections_per_endpoint parameter makes it look like the pool size can be different for
/// different requests (whatever that means), but actually we just assign the endpoint's connection
/// pool size when we see the endpoint for the first time, then we never change it.
/// We should probably change how this configuration works, and how this pooling works in general:
/// * Make the per_endpoint_pool_size be a global server setting instead of per-disk or per-query.
/// * Have boolean per-disk/per-query settings for enabling/disabling pooling.
/// * Add a limit on the number of endpoints and the total number of sessions across all endpoints.
/// * Enable pooling by default everywhere. In particular StorageURL and StorageS3.
/// (Enabling it for StorageURL is scary without the previous item - the user may query lots of
/// different endpoints. So currently pooling is mainly used for S3.)
PooledHTTPSessionPtr makePooledHTTPSession(
const Poco::URI & uri,
const ConnectionTimeouts & timeouts,
size_t per_endpoint_pool_size,
bool wait_on_pool_size_limit = true,
ProxyConfiguration proxy_config = {});
bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status);
/** Used to receive response (response headers and possibly body)
@ -112,7 +75,6 @@ std::istream * receiveResponse(
Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects);
void assertResponseIsOk(
const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false);
const String & uri, Poco::Net::HTTPResponse & response, std::istream & istr, bool allow_redirects = false);
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts);
}

View File

@ -1,5 +1,4 @@
#include <IO/LimitReadBuffer.h>
#include <Common/Exception.h>
@ -15,7 +14,7 @@ namespace ErrorCodes
bool LimitReadBuffer::nextImpl()
{
assert(position() >= in->position());
chassert(position() >= in->position());
/// Let underlying buffer calculate read bytes in `next()` call.
in->position() = position();
@ -39,20 +38,18 @@ bool LimitReadBuffer::nextImpl()
if (exact_limit && bytes != *exact_limit)
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, *exact_limit);
/// Clearing the buffer with existing data.
set(in->position(), 0);
BufferBase::set(in->position(), 0, 0);
return false;
}
working_buffer = in->buffer();
if (limit - bytes < working_buffer.size())
working_buffer.resize(limit - bytes);
BufferBase::set(in->position(), std::min(in->available(), limit - bytes), 0);
return true;
}
LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_,
LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_,
std::optional<size_t> exact_limit_, std::string exception_message_)
: ReadBuffer(in_ ? in_->position() : nullptr, 0)
, in(in_)
@ -62,24 +59,20 @@ LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, boo
, exact_limit(exact_limit_)
, exception_message(std::move(exception_message_))
{
assert(in);
chassert(in);
size_t remaining_bytes_in_buffer = in->buffer().end() - in->position();
if (remaining_bytes_in_buffer > limit)
remaining_bytes_in_buffer = limit;
working_buffer = Buffer(in->position(), in->position() + remaining_bytes_in_buffer);
BufferBase::set(in->position(), std::min(in->available(), limit), 0);
}
LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_,
LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_,
std::optional<size_t> exact_limit_, std::string exception_message_)
: LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_)
{
}
LimitReadBuffer::LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, UInt64 limit_, bool throw_exception_,
LimitReadBuffer::LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t limit_, bool throw_exception_,
std::optional<size_t> exact_limit_, std::string exception_message_)
: LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_)
{

View File

@ -13,22 +13,24 @@ namespace DB
class LimitReadBuffer : public ReadBuffer
{
public:
LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_,
LimitReadBuffer(ReadBuffer & in_, size_t limit_, bool throw_exception_,
std::optional<size_t> exact_limit_, std::string exception_message_ = {});
LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, UInt64 limit_, bool throw_exception_, std::optional<size_t> exact_limit_,
LimitReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t limit_, bool throw_exception_, std::optional<size_t> exact_limit_,
std::string exception_message_ = {});
~LimitReadBuffer() override;
private:
ReadBuffer * in;
bool owns_in;
const bool owns_in;
UInt64 limit;
bool throw_exception;
std::optional<size_t> exact_limit;
std::string exception_message;
const size_t limit;
const bool throw_exception;
const std::optional<size_t> exact_limit;
const std::string exception_message;
LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::optional<size_t> exact_limit_, std::string exception_message_);
LoggerPtr log;
LimitReadBuffer(ReadBuffer * in_, bool owns, size_t limit_, bool throw_exception_, std::optional<size_t> exact_limit_, std::string exception_message_);
bool nextImpl() override;
};

View File

@ -92,7 +92,7 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize()
return getSizeFromFileDescriptor(getFD(), getFileName());
}
size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &)
size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) const
{
if (offset >= mapped.getLength())
return 0;

View File

@ -40,7 +40,7 @@ public:
size_t getFileSize() override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) const override;
bool supportsReadAt() override { return true; }
};

View File

@ -63,21 +63,23 @@ public:
*/
bool next()
{
assert(!hasPendingData());
assert(position() <= working_buffer.end());
chassert(!hasPendingData());
chassert(position() <= working_buffer.end());
bytes += offset();
bool res = nextImpl();
if (!res)
{
working_buffer = Buffer(pos, pos);
}
else
{
pos = working_buffer.begin() + nextimpl_working_buffer_offset;
assert(position() != working_buffer.end());
pos = working_buffer.begin() + std::min(nextimpl_working_buffer_offset, working_buffer.size());
chassert(position() < working_buffer.end());
}
nextimpl_working_buffer_offset = 0;
assert(position() <= working_buffer.end());
chassert(position() <= working_buffer.end());
return res;
}

View File

@ -49,7 +49,7 @@ std::string ReadBufferFromFileDescriptor::getFileName() const
}
size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset)
size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const
{
chassert(min_bytes <= max_bytes);
@ -265,7 +265,7 @@ bool ReadBufferFromFileDescriptor::checkIfActuallySeekable()
return res == 0 && S_ISREG(stat.st_mode);
}
size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &)
size_t ReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) const
{
chassert(use_pread);
return readImpl(to, n, n, offset);

View File

@ -34,7 +34,7 @@ protected:
/// Doesn't seek (`offset` must match fd's position if !use_pread).
/// Stops after min_bytes or eof. Returns 0 if eof.
/// Thread safe.
size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset);
size_t readImpl(char * to, size_t min_bytes, size_t max_bytes, size_t offset) const;
public:
explicit ReadBufferFromFileDescriptor(
@ -73,7 +73,7 @@ public:
bool checkIfActuallySeekable() override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> &) const override;
bool supportsReadAt() override { return use_pread; }
};

View File

@ -5,52 +5,44 @@
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_ISTREAM;
}
bool ReadBufferFromIStream::nextImpl()
{
if (eof)
return false;
chassert(internal_buffer.begin() != nullptr);
chassert(!internal_buffer.empty());
size_t bytes_read = 0;
char * read_to = internal_buffer.begin();
/// It is necessary to read in a loop, since socket usually returns only data available at the moment.
while (bytes_read < internal_buffer.size())
{
try
const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read);
if (bytes_read_last_time <= 0)
{
const auto bytes_read_last_time = stream_buf.readFromDevice(read_to, internal_buffer.size() - bytes_read);
if (bytes_read_last_time <= 0)
{
eof = true;
break;
}
eof = true;
break;
}
bytes_read += bytes_read_last_time;
read_to += bytes_read_last_time;
}
catch (...)
{
throw Exception(
ErrorCodes::CANNOT_READ_FROM_ISTREAM,
"Cannot read from istream at offset {}: {}",
count(),
getCurrentExceptionMessage(/*with_stacktrace=*/true));
}
bytes_read += bytes_read_last_time;
read_to += bytes_read_last_time;
}
if (bytes_read)
{
working_buffer = internal_buffer;
working_buffer.resize(bytes_read);
}
return bytes_read;
}
ReadBufferFromIStream::ReadBufferFromIStream(std::istream & istr_, size_t size)
: BufferWithOwnMemory<ReadBuffer>(size), istr(istr_), stream_buf(dynamic_cast<Poco::Net::HTTPBasicStreamBuf &>(*istr.rdbuf()))
: BufferWithOwnMemory<ReadBuffer>(size)
, istr(istr_)
, stream_buf(dynamic_cast<Poco::Net::HTTPBasicStreamBuf &>(*istr.rdbuf()))
{
}

View File

@ -34,61 +34,6 @@ namespace ProfileEvents
extern const Event RemoteReadThrottlerSleepMicroseconds;
}
namespace
{
DB::PooledHTTPSessionPtr getSession(Aws::S3::Model::GetObjectResult & read_result)
{
if (auto * session_aware_stream = dynamic_cast<DB::S3::SessionAwareIOStream<DB::PooledHTTPSessionPtr> *>(&read_result.GetBody()))
return static_cast<DB::PooledHTTPSessionPtr &>(session_aware_stream->getSession());
if (dynamic_cast<DB::S3::SessionAwareIOStream<DB::HTTPSessionPtr> *>(&read_result.GetBody()))
return {};
/// accept result from S# mock in gtest_writebuffer_s3.cpp
if (dynamic_cast<Aws::Utils::Stream::DefaultUnderlyingStream *>(&read_result.GetBody()))
return {};
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session of unexpected type encountered");
}
void resetSession(Aws::S3::Model::GetObjectResult & read_result)
{
if (auto session = getSession(read_result); !session.isNull())
{
auto & http_session = static_cast<Poco::Net::HTTPClientSession &>(*session);
http_session.reset();
}
}
void resetSessionIfNeeded(bool read_all_range_successfully, std::optional<Aws::S3::Model::GetObjectResult> & read_result)
{
if (!read_result)
return;
if (!read_all_range_successfully)
{
/// When we abandon a session with an ongoing GetObject request and there is another one trying to delete the same object this delete
/// operation will hang until GetObject's session idle timeouts. So we have to call `reset()` on GetObject's session session immediately.
resetSession(*read_result);
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions);
}
else if (auto session = getSession(*read_result); !session.isNull())
{
if (!session->getProxyHost().empty())
{
/// Reset proxified sessions because proxy can change for every request. See ProxyConfigurationResolver.
resetSession(*read_result);
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions);
}
else
{
DB::markSessionForReuse(session);
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions);
}
}
}
}
namespace DB
{
namespace ErrorCodes
@ -228,7 +173,7 @@ bool ReadBufferFromS3::nextImpl()
}
size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & progress_callback)
size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & progress_callback) const
{
size_t initial_n = n;
size_t sleep_time_with_backoff_milliseconds = 100;
@ -240,29 +185,6 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::ReadBufferFromS3Microseconds);
std::optional<Aws::S3::Model::GetObjectResult> result;
/// Connection is reusable if we've read the full response.
bool session_is_reusable = false;
SCOPE_EXIT(
{
if (!result.has_value())
return;
if (session_is_reusable)
{
auto session = getSession(*result);
if (!session.isNull())
{
DB::markSessionForReuse(session);
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions);
}
else
session_is_reusable = false;
}
if (!session_is_reusable)
{
resetSession(*result);
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions);
}
});
try
{
@ -276,9 +198,8 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons
if (read_settings.remote_throttler)
read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds);
/// Read remaining bytes after the end of the payload, see HTTPSessionReuseTag.
/// Read remaining bytes after the end of the payload
istr.ignore(INT64_MAX);
session_is_reusable = true;
}
catch (Poco::Exception & e)
{
@ -451,21 +372,8 @@ bool ReadBufferFromS3::atEndOfRequestedRangeGuess()
return false;
}
ReadBufferFromS3::~ReadBufferFromS3()
{
try
{
resetSessionIfNeeded(readAllRangeSuccessfully(), read_result);
}
catch (...)
{
tryLogCurrentException(log);
}
}
std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize(size_t attempt)
{
resetSessionIfNeeded(readAllRangeSuccessfully(), read_result);
read_all_range_successfully = false;
/**
@ -534,10 +442,6 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si
}
}
bool ReadBufferFromS3::readAllRangeSuccessfully() const
{
return read_until_position ? offset == read_until_position : read_all_range_successfully;
}
}
#endif

View File

@ -55,7 +55,7 @@ public:
bool restricted_seek_ = false,
std::optional<size_t> file_size = std::nullopt);
~ReadBufferFromS3() override;
~ReadBufferFromS3() override = default;
bool nextImpl() override;
@ -74,7 +74,7 @@ public:
String getFileName() const override { return bucket + "/" + key; }
size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & progress_callback) override;
size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function<bool(size_t)> & progress_callback) const override;
bool supportsReadAt() override { return true; }
@ -90,8 +90,6 @@ private:
Aws::S3::Model::GetObjectResult sendRequest(size_t attempt, size_t range_begin, std::optional<size_t> range_end_incl) const;
bool readAllRangeSuccessfully() const;
ReadSettings read_settings;
bool use_external_buffer;

View File

@ -122,7 +122,7 @@ struct ReadSettings
// Resource to be used during reading
ResourceLink resource_link;
size_t http_max_tries = 1;
size_t http_max_tries = 10;
size_t http_retry_initial_backoff_ms = 100;
size_t http_retry_max_backoff_ms = 1600;
bool http_skip_not_found_url_for_globs = true;

File diff suppressed because it is too large Load Diff

View File

@ -20,7 +20,6 @@
#include <Poco/Net/HTTPResponse.h>
#include <Poco/URI.h>
#include <Poco/URIStreamFactory.h>
#include <Common/DNSResolver.h>
#include <Common/RemoteHostFilter.h>
#include "config.h"
#include <Common/config_version.h>
@ -30,44 +29,19 @@
namespace DB
{
template <typename TSessionFactory>
class UpdatableSession
class ReadWriteBufferFromHTTP : public SeekableReadBuffer, public WithFileName, public WithFileSize
{
public:
using SessionPtr = typename TSessionFactory::SessionType;
explicit UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr<TSessionFactory> session_factory_);
SessionPtr getSession();
void updateSession(const Poco::URI & uri);
/// Thread safe.
SessionPtr createDetachedSession(const Poco::URI & uri);
std::shared_ptr<UpdatableSession<TSessionFactory>> clone(const Poco::URI & uri);
/// Information from HTTP response header.
struct HTTPFileInfo
{
// nullopt if the server doesn't report it.
std::optional<size_t> file_size;
std::optional<time_t> last_modified;
bool seekable = false;
};
private:
SessionPtr session;
UInt64 redirects{0};
UInt64 max_redirects;
Poco::URI initial_uri;
std::shared_ptr<TSessionFactory> session_factory;
};
/// Information from HTTP response header.
struct HTTPFileInfo
{
// nullopt if the server doesn't report it.
std::optional<size_t> file_size;
std::optional<time_t> last_modified;
bool seekable = false;
};
namespace detail
{
/// Byte range, including right bound [begin, end].
struct HTTPRange
{
@ -75,218 +49,208 @@ namespace detail
std::optional<size_t> end;
};
template <typename UpdatableSessionPtr>
class ReadWriteBufferFromHTTPBase : public SeekableReadBuffer, public WithFileName, public WithFileSize
struct CallResult
{
protected:
Poco::URI uri;
std::string method;
std::string content_encoding;
HTTPSessionPtr session;
std::istream * response_stream = nullptr;
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;
std::vector<Poco::Net::HTTPCookie> cookies;
HTTPHeaderEntries http_header_entries;
const RemoteHostFilter * remote_host_filter = nullptr;
std::function<void(size_t)> next_callback;
CallResult(HTTPSessionPtr && session_, std::istream & response_stream_)
: session(session_)
, response_stream(&response_stream_)
{}
CallResult(CallResult &&) = default;
CallResult & operator= (CallResult &&) = default;
size_t buffer_size;
bool use_external_buffer;
size_t offset_from_begin_pos = 0;
HTTPRange read_range;
std::optional<HTTPFileInfo> file_info;
/// Delayed exception in case retries with partial content are not satisfiable.
std::exception_ptr exception;
bool retry_with_range_header = false;
/// In case of redirects, save result uri to use it if we retry the request.
std::optional<Poco::URI> saved_uri_redirect;
bool http_skip_not_found_url;
ReadSettings settings;
LoggerPtr log;
ProxyConfiguration proxy_config;
bool withPartialContent(const HTTPRange & range) const;
size_t getOffset() const;
void prepareRequest(Poco::Net::HTTPRequest & request, Poco::URI uri_, std::optional<HTTPRange> range) const;
std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_, bool for_object_info = false);
size_t getFileSize() override;
bool supportsReadAt() override;
bool checkIfActuallySeekable() override;
String getFileName() const override;
enum class InitializeError
{
RETRYABLE_ERROR,
/// If error is not retriable, `exception` variable must be set.
NON_RETRYABLE_ERROR,
/// Allows to skip not found urls for globs
SKIP_NOT_FOUND_URL,
NONE,
};
InitializeError initialization_error = InitializeError::NONE;
private:
void getHeadResponse(Poco::Net::HTTPResponse & response);
void setupExternalBuffer();
public:
using NextCallback = std::function<void(size_t)>;
using OutStreamCallback = std::function<void(std::ostream &)>;
explicit ReadWriteBufferFromHTTPBase(
UpdatableSessionPtr session_,
Poco::URI uri_,
const Poco::Net::HTTPBasicCredentials & credentials_,
const std::string & method_ = {},
OutStreamCallback out_stream_callback_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const ReadSettings & settings_ = {},
HTTPHeaderEntries http_header_entries_ = {},
const RemoteHostFilter * remote_host_filter_ = nullptr,
bool delay_initialization = false,
bool use_external_buffer_ = false,
bool http_skip_not_found_url_ = false,
std::optional<HTTPFileInfo> file_info_ = std::nullopt,
ProxyConfiguration proxy_config_ = {});
void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false);
void call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false, bool for_object_info = false);
/**
* Throws if error is retryable, otherwise sets initialization_error = NON_RETRYABLE_ERROR and
* saves exception into `exception` variable. In case url is not found and skip_not_found_url == true,
* sets initialization_error = SKIP_NOT_FOUND_URL, otherwise throws.
*/
void initialize();
bool nextImpl() override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> & progress_callback) override;
off_t getPosition() override;
off_t seek(off_t offset_, int whence) override;
void setReadUntilPosition(size_t until) override;
void setReadUntilEnd() override;
bool supportsRightBoundedReads() const override;
// If true, if we destroy impl now, no work was wasted. Just for metrics.
bool atEndOfRequestedRangeGuess();
std::string getResponseCookie(const std::string & name, const std::string & def) const;
/// 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
void setNextCallback(NextCallback next_callback_);
const std::string & getCompressionMethod() const;
std::optional<time_t> tryGetLastModificationTime();
HTTPFileInfo getFileInfo();
HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin);
std::unique_ptr<ReadBuffer> transformToReadBuffer(size_t buf_size) &&;
};
}
class SessionFactory
{
public:
explicit SessionFactory(const ConnectionTimeouts & timeouts_, ProxyConfiguration proxy_config_ = {});
const HTTPConnectionGroupType connection_group;
const Poco::URI initial_uri;
const std::string method;
const ProxyConfiguration proxy_config;
const ReadSettings read_settings;
const ConnectionTimeouts timeouts;
using SessionType = HTTPSessionPtr;
const Poco::Net::HTTPBasicCredentials & credentials;
const RemoteHostFilter * remote_host_filter;
SessionType buildNewSession(const Poco::URI & uri);
private:
ConnectionTimeouts timeouts;
ProxyConfiguration proxy_config;
};
const size_t buffer_size;
const size_t max_redirects;
class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession<SessionFactory>>>
{
using SessionType = UpdatableSession<SessionFactory>;
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<SessionType>>;
const bool use_external_buffer;
const bool http_skip_not_found_url;
std::function<void(std::ostream &)> out_stream_callback;
Poco::URI current_uri;
size_t redirects = 0;
std::string content_encoding;
std::unique_ptr<ReadBuffer> impl;
std::vector<Poco::Net::HTTPCookie> cookies;
HTTPHeaderEntries http_header_entries;
std::function<void(size_t)> next_callback;
size_t offset_from_begin_pos = 0;
HTTPRange read_range;
std::optional<HTTPFileInfo> file_info;
LoggerPtr log;
bool withPartialContent() const;
void prepareRequest(Poco::Net::HTTPRequest & request, std::optional<HTTPRange> range) const;
void doWithRetries(std::function<void()> && callable, std::function<void()> on_retry = nullptr, bool mute_logging = false) const;
CallResult callImpl(
Poco::Net::HTTPResponse & response,
const Poco::URI & uri_,
const std::string & method_,
const std::optional<HTTPRange> & range,
bool allow_redirects) const;
CallResult callWithRedirects(
Poco::Net::HTTPResponse & response,
const String & method_,
const std::optional<HTTPRange> & range);
std::unique_ptr<ReadBuffer> initialize();
size_t getFileSize() override;
bool supportsReadAt() override;
bool checkIfActuallySeekable() override;
String getFileName() const override;
void getHeadResponse(Poco::Net::HTTPResponse & response);
void setupExternalBuffer();
size_t getOffset() const;
// If true, if we destroy impl now, no work was wasted. Just for metrics.
bool atEndOfRequestedRangeGuess();
public:
using NextCallback = std::function<void(size_t)>;
using OutStreamCallback = std::function<void(std::ostream &)>;
ReadWriteBufferFromHTTP(
Poco::URI uri_,
const HTTPConnectionGroupType & connection_group_,
const Poco::URI & uri_,
const std::string & method_,
OutStreamCallback out_stream_callback_,
const ConnectionTimeouts & timeouts,
const Poco::Net::HTTPBasicCredentials & credentials_,
const UInt64 max_redirects = 0,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const ReadSettings & settings_ = {},
const HTTPHeaderEntries & http_header_entries_ = {},
const RemoteHostFilter * remote_host_filter_ = nullptr,
bool delay_initialization_ = true,
bool use_external_buffer_ = false,
bool skip_not_found_url_ = false,
std::optional<HTTPFileInfo> file_info_ = std::nullopt,
ProxyConfiguration proxy_config_ = {});
};
class PooledSessionFactory
{
public:
explicit PooledSessionFactory(
const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_);
using SessionType = PooledHTTPSessionPtr;
/// Thread safe.
SessionType buildNewSession(const Poco::URI & uri);
private:
ConnectionTimeouts timeouts;
size_t per_endpoint_pool_size;
};
using PooledSessionFactoryPtr = std::shared_ptr<PooledSessionFactory>;
class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession<PooledSessionFactory>>>
{
using SessionType = UpdatableSession<PooledSessionFactory>;
using Parent = detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<SessionType>>;
public:
explicit PooledReadWriteBufferFromHTTP(
Poco::URI uri_,
const std::string & method_,
OutStreamCallback out_stream_callback_,
ProxyConfiguration proxy_config_,
ReadSettings read_settings_,
ConnectionTimeouts timeouts_,
const Poco::Net::HTTPBasicCredentials & credentials_,
const RemoteHostFilter * remote_host_filter_,
size_t buffer_size_,
const UInt64 max_redirects,
PooledSessionFactoryPtr session_factory);
size_t max_redirects_,
OutStreamCallback out_stream_callback_,
bool use_external_buffer_,
bool http_skip_not_found_url_,
HTTPHeaderEntries http_header_entries_,
bool delay_initialization,
std::optional<HTTPFileInfo> file_info_);
bool nextImpl() override;
size_t readBigAt(char * to, size_t n, size_t offset, const std::function<bool(size_t)> & progress_callback) const override;
off_t seek(off_t offset_, int whence) override;
void setReadUntilPosition(size_t until) override;
void setReadUntilEnd() override;
bool supportsRightBoundedReads() const override;
off_t getPosition() override;
std::string getResponseCookie(const std::string & name, const std::string & def) const;
/// 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
void setNextCallback(NextCallback next_callback_);
const std::string & getCompressionMethod() const;
std::optional<time_t> tryGetLastModificationTime();
HTTPFileInfo getFileInfo();
static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin);
};
using ReadWriteBufferFromHTTPPtr = std::unique_ptr<ReadWriteBufferFromHTTP>;
extern template class UpdatableSession<SessionFactory>;
extern template class UpdatableSession<PooledSessionFactory>;
extern template class detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession<SessionFactory>>>;
extern template class detail::ReadWriteBufferFromHTTPBase<std::shared_ptr<UpdatableSession<PooledSessionFactory>>>;
class BuilderRWBufferFromHTTP
{
Poco::URI uri;
std::string method = Poco::Net::HTTPRequest::HTTP_GET;
HTTPConnectionGroupType connection_group = HTTPConnectionGroupType::HTTP;
ProxyConfiguration proxy_config{};
ReadSettings read_settings{};
ConnectionTimeouts timeouts{};
const RemoteHostFilter * remote_host_filter = nullptr;
size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
size_t max_redirects = 0;
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = nullptr;
bool use_external_buffer = false;
HTTPHeaderEntries http_header_entries{};
bool delay_initialization = true;
public:
BuilderRWBufferFromHTTP(Poco::URI uri_)
: uri(uri_)
{}
#define setterMember(name, member) \
BuilderRWBufferFromHTTP & name(decltype(BuilderRWBufferFromHTTP::member) arg_##member) \
{ \
member = std::move(arg_##member); \
return *this; \
}
setterMember(withConnectionGroup, connection_group)
setterMember(withMethod, method)
setterMember(withProxy, proxy_config)
setterMember(withSettings, read_settings)
setterMember(withTimeouts, timeouts)
setterMember(withHostFilter, remote_host_filter)
setterMember(withBufSize, buffer_size)
setterMember(withRedirects, max_redirects)
setterMember(withOutCallback, out_stream_callback)
setterMember(withHeaders, http_header_entries)
setterMember(withExternalBuf, use_external_buffer)
setterMember(withDelayInit, delay_initialization)
#undef setterMember
ReadWriteBufferFromHTTPPtr create(const Poco::Net::HTTPBasicCredentials & credentials_)
{
return std::make_unique<ReadWriteBufferFromHTTP>(
connection_group,
uri,
method,
proxy_config,
read_settings,
timeouts,
credentials_,
remote_host_filter,
buffer_size,
max_redirects,
out_stream_callback,
use_external_buffer,
/*http_skip_not_found_url=*/ false,
http_header_entries,
delay_initialization,
/*file_info_=*/ std::nullopt);
}
};
}

View File

@ -1,5 +1,4 @@
#include <Poco/Timespan.h>
#include "Common/DNSResolver.h"
#include "config.h"
#if USE_AWS_S3
@ -147,9 +146,7 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio
.withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withTCPKeepAliveTimeout(Poco::Timespan(
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))
.withHTTPKeepAliveTimeout(Poco::Timespan(
client_configuration.http_keep_alive_timeout_ms * 1000)); /// flag indicating whether keep-alive is enabled is set to each session upon creation
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0));
}
PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration)
@ -164,8 +161,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config
, get_request_throttler(client_configuration.get_request_throttler)
, put_request_throttler(client_configuration.put_request_throttler)
, extra_headers(client_configuration.extra_headers)
, http_connection_pool_size(client_configuration.http_connection_pool_size)
, wait_on_pool_size_limit(client_configuration.wait_on_pool_size_limit)
{
}
@ -308,12 +303,8 @@ void PocoHTTPClient::makeRequestInternal(
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const
{
/// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session.
const auto request_configuration = per_request_configuration();
if (http_connection_pool_size)
makeRequestInternalImpl<true>(request, request_configuration, response, readLimiter, writeLimiter);
else
makeRequestInternalImpl<false>(request, request_configuration, response, readLimiter, writeLimiter);
makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter);
}
String getMethod(const Aws::Http::HttpRequest & request)
@ -335,7 +326,6 @@ String getMethod(const Aws::Http::HttpRequest & request)
}
}
template <bool pooled>
void PocoHTTPClient::makeRequestInternalImpl(
Aws::Http::HttpRequest & request,
const DB::ProxyConfiguration & proxy_configuration,
@ -343,8 +333,6 @@ void PocoHTTPClient::makeRequestInternalImpl(
Aws::Utils::RateLimits::RateLimiterInterface *,
Aws::Utils::RateLimits::RateLimiterInterface *) const
{
using SessionPtr = std::conditional_t<pooled, PooledHTTPSessionPtr, HTTPSessionPtr>;
LoggerPtr log = getLogger("AWSClient");
auto uri = request.GetUri().GetURIString();
@ -396,40 +384,17 @@ void PocoHTTPClient::makeRequestInternalImpl(
for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt)
{
Poco::URI target_uri(uri);
SessionPtr session;
if (!proxy_configuration.host.empty())
{
if (enable_s3_requests_logging)
LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri);
/// Reverse proxy can replace host header with resolved ip address instead of host name.
/// This can lead to request signature difference on S3 side.
if constexpr (pooled)
session = makePooledHTTPSession(
target_uri,
getTimeouts(method, first_attempt, /*first_byte*/ true),
http_connection_pool_size,
wait_on_pool_size_limit,
proxy_configuration);
else
session = makeHTTPSession(
target_uri,
getTimeouts(method, first_attempt, /*first_byte*/ true),
proxy_configuration);
}
else
{
if constexpr (pooled)
session = makePooledHTTPSession(
target_uri,
getTimeouts(method, first_attempt, /*first_byte*/ true),
http_connection_pool_size,
wait_on_pool_size_limit);
else
session = makeHTTPSession(
target_uri,
getTimeouts(method, first_attempt, /*first_byte*/ true));
}
if (enable_s3_requests_logging && !proxy_configuration.isEmpty())
LOG_TEST(log, "Due to reverse proxy host name ({}) won't be resolved on ClickHouse side", uri);
auto group = for_disk_s3 ? HTTPConnectionGroupType::DISK : HTTPConnectionGroupType::STORAGE;
auto session = makeHTTPSession(
group,
target_uri,
getTimeouts(method, first_attempt, /*first_byte*/ true),
proxy_configuration);
/// In case of error this address will be written to logs
request.SetResolvedRemoteHost(session->getResolvedAddress());
@ -612,10 +577,6 @@ void PocoHTTPClient::makeRequestInternalImpl(
response->SetClientErrorMessage(getCurrentExceptionMessage(false));
addMetric(request, S3MetricType::Errors);
/// Probably this is socket timeout or something more or less related to DNS
/// Let's just remove this host from DNS cache to be more safe
DNSResolver::instance().removeHostFromCache(Poco::URI(uri).getHost());
}
}

View File

@ -13,7 +13,7 @@
#include <IO/ConnectionTimeouts.h>
#include <IO/HTTPCommon.h>
#include <IO/HTTPHeaderEntries.h>
#include <IO/S3/SessionAwareIOStream.h>
#include <IO/SessionAwareIOStream.h>
#include <aws/core/client/ClientConfiguration.h>
#include <aws/core/http/HttpClient.h>
@ -49,12 +49,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
ThrottlerPtr put_request_throttler;
HTTPHeaderEntries extra_headers;
/// Not a client parameter in terms of HTTP and we won't send it to the server. Used internally to determine when connection have to be re-established.
uint32_t http_keep_alive_timeout_ms = 0;
/// Zero means pooling will not be used.
size_t http_connection_pool_size = 0;
/// See PoolBase::BehaviourOnLimit
bool wait_on_pool_size_limit = true;
bool s3_use_adaptive_timeouts = true;
std::function<void(const DB::ProxyConfiguration &)> error_report;
@ -98,12 +93,6 @@ public:
);
}
void SetResponseBody(Aws::IStream & incoming_stream, PooledHTTPSessionPtr & session_) /// NOLINT
{
body_stream = Aws::Utils::Stream::ResponseStream(
Aws::New<SessionAwareIOStream<PooledHTTPSessionPtr>>("http result streambuf", session_, incoming_stream.rdbuf()));
}
void SetResponseBody(std::string & response_body) /// NOLINT
{
auto stream = Aws::New<std::stringstream>("http result buf", response_body); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
@ -163,7 +152,6 @@ private:
EnumSize,
};
template <bool pooled>
void makeRequestInternalImpl(
Aws::Http::HttpRequest & request,
const DB::ProxyConfiguration & proxy_configuration,
@ -196,9 +184,6 @@ protected:
ThrottlerPtr put_request_throttler;
const HTTPHeaderEntries extra_headers;
size_t http_connection_pool_size = 0;
bool wait_on_pool_size_limit = true;
};
}

View File

@ -82,7 +82,7 @@ public:
/// (e.g. next() or supportsReadAt()).
/// * Performance: there's no buffering. Each readBigAt() call typically translates into actual
/// IO operation (e.g. HTTP request). Don't use it for small adjacent reads.
virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function<bool(size_t m)> & /*progress_callback*/ = nullptr)
virtual size_t readBigAt(char * /*to*/, size_t /*n*/, size_t /*offset*/, const std::function<bool(size_t m)> & /*progress_callback*/ = nullptr) const
{ throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readBigAt() not implemented"); }
/// Checks if readBigAt() is allowed. May be slow, may throw (e.g. it may do an HTTP request or an fstat).

View File

@ -3,7 +3,7 @@
#include <iosfwd>
namespace DB::S3
namespace DB
{
/**
* Wrapper of IOStream to store response stream and corresponding HTTP session.

View File

@ -7,6 +7,7 @@ namespace DB
{
WriteBufferFromHTTP::WriteBufferFromHTTP(
const HTTPConnectionGroupType & connection_group,
const Poco::URI & uri,
const std::string & method,
const std::string & content_type,
@ -14,9 +15,10 @@ WriteBufferFromHTTP::WriteBufferFromHTTP(
const HTTPHeaderEntries & additional_headers,
const ConnectionTimeouts & timeouts,
size_t buffer_size_,
ProxyConfiguration proxy_configuration)
ProxyConfiguration proxy_configuration
)
: WriteBufferFromOStream(buffer_size_)
, session{makeHTTPSession(uri, timeouts, proxy_configuration)}
, session{makeHTTPSession(connection_group, uri, timeouts, proxy_configuration)}
, request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1}
{
request.setHost(uri.getHost());

View File

@ -19,7 +19,8 @@ namespace DB
class WriteBufferFromHTTP : public WriteBufferFromOStream
{
public:
explicit WriteBufferFromHTTP(const Poco::URI & uri,
explicit WriteBufferFromHTTP(const HTTPConnectionGroupType & connection_group,
const Poco::URI & uri,
const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only
const std::string & content_type = "",
const std::string & content_encoding = "",

View File

@ -35,7 +35,7 @@ void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t
}
if (check_bytes && bytes > 0)
throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF.");
throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF, left to copy {} bytes.", bytes);
}
void copyDataImpl(ReadBuffer & from, WriteBuffer & to, bool check_bytes, size_t bytes, std::function<void()> cancellation_hook, ThrottlerPtr throttler)

View File

@ -380,8 +380,6 @@ struct ContextSharedPart : boost::noncopyable
OrdinaryBackgroundExecutorPtr moves_executor TSA_GUARDED_BY(background_executors_mutex);
OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex);
OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex);
/// The global pool of HTTP sessions for background fetches.
PooledSessionFactoryPtr fetches_session_factory TSA_GUARDED_BY(background_executors_mutex);
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
HTTPHeaderFilter http_header_filter; /// Forbidden HTTP headers from config.xml
@ -5039,11 +5037,6 @@ void Context::initializeBackgroundExecutorsIfNeeded()
);
LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size);
auto timeouts = ConnectionTimeouts::getFetchPartHTTPTimeouts(getServerSettings(), getSettingsRef());
/// The number of background fetches is limited by the number of threads in the background thread pool.
/// It doesn't make any sense to limit the number of connections per host any further.
shared->fetches_session_factory = std::make_shared<PooledSessionFactory>(timeouts, background_fetches_pool_size);
shared->fetch_executor = std::make_shared<OrdinaryBackgroundExecutor>
(
"Fetch",
@ -5097,12 +5090,6 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const
return shared->common_executor;
}
PooledSessionFactoryPtr Context::getCommonFetchesSessionFactory() const
{
SharedLockGuard lock(shared->background_executors_mutex);
return shared->fetches_session_factory;
}
IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const
{
callOnce(shared->readers_initialized, [&] {

View File

@ -206,9 +206,6 @@ using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>;
class PreparedSetsCache;
using PreparedSetsCachePtr = std::shared_ptr<PreparedSetsCache>;
class PooledSessionFactory;
using PooledSessionFactoryPtr = std::shared_ptr<PooledSessionFactory>;
class SessionTracker;
struct ServerSettings;
@ -1226,7 +1223,6 @@ public:
OrdinaryBackgroundExecutorPtr getMovesExecutor() const;
OrdinaryBackgroundExecutorPtr getFetchesExecutor() const;
OrdinaryBackgroundExecutorPtr getCommonExecutor() const;
PooledSessionFactoryPtr getCommonFetchesSessionFactory() const;
IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const;
#if USE_LIBURING

View File

@ -44,6 +44,11 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const
{
return DB::isLocalAddress(DNSResolver::instance().resolveAddress(host_name, port), clickhouse_port);
}
catch (const DB::NetException &)
{
/// Avoid "Host not found" exceptions
return false;
}
catch (const Poco::Net::NetException &)
{
/// Avoid "Host not found" exceptions

View File

@ -11,6 +11,7 @@
#include <Common/CurrentMetrics.h>
#include <Common/FailPoint.h>
#include <Common/PageCache.h>
#include <Common/HostResolvePool.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Context.h>
@ -333,10 +334,17 @@ BlockIO InterpreterSystemQuery::execute()
{
getContext()->checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE);
DNSResolver::instance().dropCache();
HostResolversPool::instance().dropCache();
/// Reinitialize clusters to update their resolved_addresses
system_context->reloadClusterConfig();
break;
}
case Type::DROP_CONNECTIONS_CACHE:
{
getContext()->checkAccess(AccessType::SYSTEM_DROP_CONNECTIONS_CACHE);
HTTPConnectionPools::instance().dropCache();
break;
}
case Type::DROP_MARK_CACHE:
getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE);
system_context->clearMarkCache();
@ -1201,6 +1209,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
break;
}
case Type::DROP_DNS_CACHE:
case Type::DROP_CONNECTIONS_CACHE:
case Type::DROP_MARK_CACHE:
case Type::DROP_MMAP_CACHE:
case Type::DROP_QUERY_CACHE:

View File

@ -384,6 +384,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s
case Type::KILL:
case Type::SHUTDOWN:
case Type::DROP_DNS_CACHE:
case Type::DROP_CONNECTIONS_CACHE:
case Type::DROP_MMAP_CACHE:
case Type::DROP_QUERY_CACHE:
case Type::DROP_MARK_CACHE:

View File

@ -22,6 +22,7 @@ public:
KILL,
SUSPEND,
DROP_DNS_CACHE,
DROP_CONNECTIONS_CACHE,
DROP_MARK_CACHE,
DROP_UNCOMPRESSED_CACHE,
DROP_INDEX_MARK_CACHE,

View File

@ -1016,7 +1016,7 @@ private:
http_basic_credentials.authenticate(request);
}
auto session = makePooledHTTPSession(url, timeouts, 1);
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, url, timeouts);
session->sendRequest(request);
Poco::Net::HTTPResponse response;
@ -1025,8 +1025,6 @@ private:
Poco::JSON::Parser parser;
auto json_body = parser.parse(*response_body).extract<Poco::JSON::Object::Ptr>();
/// Response was fully read.
markSessionForReuse(session);
auto schema = json_body->getValue<std::string>("schema");
LOG_TRACE((getLogger("AvroConfluentRowInputFormat")), "Successfully fetched schema id = {}\n{}", id, schema);

View File

@ -526,14 +526,12 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> Fetcher::fetchSelected
creds.setPassword(password);
}
std::unique_ptr<PooledReadWriteBufferFromHTTP> in = std::make_unique<PooledReadWriteBufferFromHTTP>(
uri,
Poco::Net::HTTPRequest::HTTP_POST,
nullptr,
creds,
DBMS_DEFAULT_BUFFER_SIZE,
0, /* no redirects */
context->getCommonFetchesSessionFactory());
auto in = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::HTTP)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(timeouts)
.withDelayInit(false)
.create(creds);
int server_protocol_version = parse<int>(in->getResponseCookie("server_protocol_version", "0"));
String remote_fs_metadata = parse<String>(in->getResponseCookie("remote_fs_metadata", ""));
@ -557,11 +555,13 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> Fetcher::fetchSelected
if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE)
{
readBinary(sum_files_size, *in);
if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS)
{
IMergeTreeDataPart::TTLInfos ttl_infos;
String ttl_infos_string;
readBinary(ttl_infos_string, *in);
ReadBufferFromString ttl_infos_buffer(ttl_infos_string);
assertString("ttl format version: 1\n", ttl_infos_buffer);
ttl_infos.read(ttl_infos_buffer);
@ -609,6 +609,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> Fetcher::fetchSelected
}
UInt64 revision = parse<UInt64>(in->getResponseCookie("disk_revision", "0"));
if (revision)
disk->syncRevision(revision);
@ -743,7 +744,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
const UUID & part_uuid,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
size_t projections,
bool is_projection,
ThrottlerPtr throttler)
@ -799,7 +800,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
void Fetcher::downloadBaseOrProjectionPartToDisk(
const String & replica_path,
const MutableDataPartStoragePtr & data_part_storage,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler,
@ -807,6 +808,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk(
{
size_t files;
readBinary(files, in);
LOG_DEBUG(log, "Downloading files {}", files);
std::vector<std::unique_ptr<WriteBufferFromFileBase>> written_files;
@ -872,7 +875,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
const String & tmp_prefix,
DiskPtr disk,
bool to_remote_disk,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
size_t projections,
ThrottlerPtr throttler,

View File

@ -20,7 +20,7 @@ namespace DB
{
class StorageReplicatedMergeTree;
class PooledReadWriteBufferFromHTTP;
class ReadWriteBufferFromHTTP;
namespace DataPartsExchange
{
@ -94,7 +94,7 @@ private:
void downloadBaseOrProjectionPartToDisk(
const String & replica_path,
const MutableDataPartStoragePtr & data_part_storage,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler,
@ -107,7 +107,7 @@ private:
const String & tmp_prefix_,
DiskPtr disk,
bool to_remote_disk,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
size_t projections,
ThrottlerPtr throttler,
@ -120,7 +120,7 @@ private:
const UUID & part_uuid,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
size_t projections,
bool is_projection,
ThrottlerPtr throttler);
@ -131,7 +131,7 @@ private:
bool to_detached,
const String & tmp_prefix_,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
ReadWriteBufferFromHTTP & in,
size_t projections,
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler);

View File

@ -963,7 +963,7 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata
else
columns_to_send = metadata_snapshot->getSampleBlockNonMaterialized().getNames();
/// DistributedSink will not own cluster, but will own ConnectionPools of the cluster
/// DistributedSink will not own cluster
return std::make_shared<DistributedSink>(
local_context, *this, metadata_snapshot, cluster, insert_sync, timeout,
StorageID{remote_database, remote_table}, columns_to_send);

View File

@ -1420,8 +1420,10 @@ void StorageS3::Configuration::connect(const ContextPtr & context)
url.uri.getScheme());
client_configuration.endpointOverride = url.endpoint;
/// seems as we don't use it
client_configuration.maxConnections = static_cast<unsigned>(request_settings.max_connections);
client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size;
client_configuration.connectTimeoutMs = local_settings.s3_connect_timeout_ms;
auto headers = auth_settings.headers;
if (!headers_from_ast.empty())
headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end());

View File

@ -461,21 +461,23 @@ std::pair<Poco::URI, std::unique_ptr<ReadWriteBufferFromHTTP>> StorageURLSource:
try
{
auto res = std::make_unique<ReadWriteBufferFromHTTP>(
HTTPConnectionGroupType::STORAGE,
request_uri,
http_method,
callback,
proxy_config,
read_settings,
timeouts,
credentials,
settings.max_http_get_redirects,
settings.max_read_buffer_size,
read_settings,
headers,
&context_->getRemoteHostFilter(),
settings.max_read_buffer_size,
settings.max_http_get_redirects,
callback,
/*use_external_buffer*/ false,
skip_url_not_found_error,
headers,
delay_initialization,
/* use_external_buffer */ false,
/* skip_url_not_found_error */ skip_url_not_found_error,
/* file_info */ std::nullopt,
proxy_config);
/*file_info_*/ std::nullopt);
if (context_->getSettingsRef().engine_url_skip_empty_files && res->eof() && option != std::prev(end))
{
@ -547,7 +549,7 @@ StorageURLSink::StorageURLSink(
auto proxy_config = getProxyConfiguration(http_method);
auto write_buffer = std::make_unique<WriteBufferFromHTTP>(
Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config
HTTPConnectionGroupType::STORAGE, Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config
);
const auto & settings = context->getSettingsRef();
@ -1320,24 +1322,17 @@ std::optional<time_t> IStorageURLBase::tryGetLastModificationTime(
auto proxy_config = getProxyConfiguration(uri.getScheme());
ReadWriteBufferFromHTTP buf(
uri,
Poco::Net::HTTPRequest::HTTP_GET,
{},
getHTTPTimeouts(context),
credentials,
settings.max_http_get_redirects,
settings.max_read_buffer_size,
context->getReadSettings(),
headers,
&context->getRemoteHostFilter(),
true,
false,
false,
std::nullopt,
proxy_config);
auto buf = BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withSettings(context->getReadSettings())
.withTimeouts(getHTTPTimeouts(context))
.withHostFilter(&context->getRemoteHostFilter())
.withBufSize(settings.max_read_buffer_size)
.withRedirects(settings.max_http_get_redirects)
.withHeaders(headers)
.create(credentials);
return buf.tryGetLastModificationTime();
return buf->tryGetLastModificationTime();
}
StorageURL::StorageURL(

View File

@ -153,17 +153,16 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex
columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls));
Poco::Net::HTTPBasicCredentials credentials{};
ReadWriteBufferFromHTTP buf(
columns_info_uri,
Poco::Net::HTTPRequest::HTTP_POST,
{},
ConnectionTimeouts::getHTTPTimeouts(
context->getSettingsRef(),
context->getServerSettings().keep_alive_timeout),
credentials);
auto buf = BuilderRWBufferFromHTTP(columns_info_uri)
.withConnectionGroup(HTTPConnectionGroupType::STORAGE)
.withMethod(Poco::Net::HTTPRequest::HTTP_POST)
.withTimeouts(ConnectionTimeouts::getHTTPTimeouts(
context->getSettingsRef(),
context->getServerSettings().keep_alive_timeout))
.create(credentials);
std::string columns_info;
readStringBinary(columns_info, buf);
readStringBinary(columns_info, *buf);
NamesAndTypesList columns = NamesAndTypesList::parse(columns_info);
return ColumnsDescription{columns};

View File

@ -177,7 +177,7 @@ def cancel_restore(restore_id):
def test_cancel_backup():
# We use partitioning so backups would contain more files.
node.query(
"CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%5"
"CREATE TABLE tbl (x UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY x%20"
)
node.query(f"INSERT INTO tbl SELECT number FROM numbers(500)")

View File

@ -5,6 +5,7 @@
<default>
<s3_check_objects_after_upload>1</s3_check_objects_after_upload>
<enable_s3_requests_logging>1</enable_s3_requests_logging>
<s3_connect_timeout_ms>10000</s3_connect_timeout_ms>
</default>
</profiles>
<users>

View File

@ -19,6 +19,7 @@
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
<skip_access_check>1</skip_access_check>
<s3_connect_timeout_ms>10000</s3_connect_timeout_ms>
</broken_s3>
</disks>

View File

@ -278,7 +278,7 @@ def test_unavailable_server(cluster):
"Caught exception while loading metadata.*Connection refused"
)
assert node2.contains_in_log(
"HTTP request to \`http://nginx:8080/test1/.*\` failed at try 1/10 with bytes read: 0/unknown. Error: Connection refused."
"Failed to make request to 'http://nginx:8080/test1/.*'. Error: 'Connection refused'. Failed at try 10/10."
)
finally:
node2.exec_in_container(

View File

@ -46,6 +46,7 @@ def cluster_without_dns_cache_update():
except Exception as ex:
print(ex)
raise
finally:
cluster.shutdown()
@ -61,6 +62,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update):
node2.set_hosts([("2001:3984:3989::1:1111", "node1")])
# drop DNS cache
node2.query("SYSTEM DROP DNS CACHE")
node2.query("SYSTEM DROP CONNECTIONS CACHE")
# First we check, that normal replication works
node1.query(
@ -86,6 +88,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update):
# drop DNS cache
node2.query("SYSTEM DROP DNS CACHE")
node2.query("SYSTEM DROP CONNECTIONS CACHE")
# Data is downloaded
assert_eq_with_retry(node2, "SELECT count(*) from test_table_drop", "6")
@ -124,6 +127,7 @@ def cluster_with_dns_cache_update():
except Exception as ex:
print(ex)
raise
finally:
cluster.shutdown()
@ -267,6 +271,11 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node):
privileged=True,
user="root",
)
node.exec_in_container(
["bash", "-c", 'clickhouse client -q "SYSTEM DROP CONNECTIONS CACHE"'],
privileged=True,
user="root",
)
retry_count = 1
assert_eq_with_retry(
@ -296,7 +305,8 @@ def test_host_is_drop_from_cache_after_consecutive_failures(
# Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back)
# dns_update_short -> dns_max_consecutive_failures set to 6
assert node4.wait_for_log_line(
"Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist."
regexp="Code: 198. DB::NetException: Not found address of host: InvalidHostThatDoesNotExist.",
look_behind_lines=300,
)
assert node4.wait_for_log_line(
"Cached hosts not found:.*InvalidHostThatDoesNotExist**",

View File

@ -56,9 +56,10 @@ def dst_node_addrs(started_cluster, request):
yield
# Clear static DNS entries
# Clear static DNS entries and all keep alive connections
src_node.set_hosts([])
src_node.query("SYSTEM DROP DNS CACHE")
src_node.query("SYSTEM DROP CONNECTIONS CACHE")
@pytest.mark.parametrize(
@ -77,7 +78,8 @@ def dst_node_addrs(started_cluster, request):
def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation):
with expectation:
result = src_node.query(
"SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')"
"SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')",
settings={"http_max_tries": "3"},
)
assert result == "42\n"

View File

@ -150,7 +150,7 @@ def test_url_reconnect(started_cluster):
def select():
global result
result = node1.query(
"select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries = 10, http_retry_max_backoff_ms=1000"
"select sum(cityHash64(id)) from url('http://hdfs1:50075/webhdfs/v1/storage_big?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'id Int32') settings http_max_tries=10, http_retry_max_backoff_ms=1000, http_make_head_request=false"
)
assert int(result) == 6581218782194912115

View File

@ -80,6 +80,7 @@ def test_s3_table_functions_timeouts(started_cluster):
Test with timeout limit of 1200ms.
This should raise an Exception and pass.
"""
with PartitionManager() as pm:
pm.add_network_delay(node, 1200)

View File

@ -118,5 +118,5 @@ class RequestHandler(http.server.BaseHTTPRequestHandler):
self.wfile.write(b"OK")
httpd = http.server.HTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler)
httpd = http.server.ThreadingHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler)
httpd.serve_forever()

View File

@ -12,6 +12,7 @@ import urllib.request
import subprocess
from io import StringIO
from http.server import BaseHTTPRequestHandler, HTTPServer
from socketserver import ThreadingMixIn
def is_ipv6(host):
@ -145,11 +146,19 @@ class HTTPServerV6(HTTPServer):
address_family = socket.AF_INET6
class ThreadedHTTPServer(ThreadingMixIn, HTTPServer):
pass
class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6):
pass
def start_server():
if IS_IPV6:
httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer)
httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, CSVHTTPServer)
else:
httpd = HTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer)
httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer)
t = threading.Thread(target=httpd.serve_forever)
return t, httpd

View File

@ -105,6 +105,7 @@ NAMED COLLECTION ADMIN ['NAMED COLLECTION CONTROL'] NAMED_COLLECTION ALL
SET DEFINER [] USER_NAME ALL
SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM
SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE
SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE
SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE
SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE
SYSTEM DROP MMAP CACHE ['SYSTEM DROP MMAP','DROP MMAP CACHE','DROP MMAP'] GLOBAL SYSTEM DROP CACHE

View File

@ -3,6 +3,7 @@ connect_timeout Seconds 10
connect_timeout_with_failover_ms Milliseconds 2000
connect_timeout_with_failover_secure_ms Milliseconds 3000
external_storage_connect_timeout_sec UInt64 10
s3_connect_timeout_ms UInt64 1000
filesystem_prefetch_max_memory_usage UInt64 1073741824
max_untracked_memory UInt64 1048576
memory_profiler_step UInt64 1048576

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
from http.server import SimpleHTTPRequestHandler, HTTPServer
from socketserver import ThreadingMixIn
import socket
import sys
import threading
@ -116,11 +117,19 @@ class HTTPServerV6(HTTPServer):
address_family = socket.AF_INET6
class ThreadedHTTPServer(ThreadingMixIn, HTTPServer):
pass
class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6):
pass
def start_server(requests_amount):
if IS_IPV6:
httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor)
httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor)
else:
httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor)
httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor)
def real_func():
for i in range(requests_amount):

View File

@ -1,6 +1,7 @@
#!/usr/bin/env python3
from http.server import BaseHTTPRequestHandler, HTTPServer
from socketserver import ThreadingMixIn
import socket
import sys
import re
@ -206,13 +207,22 @@ class HTTPServerV6(HTTPServer):
address_family = socket.AF_INET6
class ThreadedHTTPServer(ThreadingMixIn, HTTPServer):
pass
class ThreadedHTTPServerV6(ThreadingMixIn, HTTPServerV6):
pass
def start_server():
if IS_IPV6:
httpd = HTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor)
httpd = ThreadedHTTPServerV6(HTTP_SERVER_ADDRESS, HttpProcessor)
else:
httpd = HTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor)
httpd = ThreadedHTTPServer(HTTP_SERVER_ADDRESS, HttpProcessor)
t = threading.Thread(target=httpd.serve_forever)
t.start()
return t, httpd
@ -235,8 +245,6 @@ def run_test(allow_range, settings, check_retries=False):
HttpProcessor.responses_to_get = ["500", "200", "206"]
retries_num = len(HttpProcessor.responses_to_get)
t, httpd = start_server()
t.start()
test_select(settings)
download_buffer_size = settings["max_download_buffer_size"]
@ -261,12 +269,12 @@ def run_test(allow_range, settings, check_retries=False):
if HttpProcessor.range_used:
raise Exception("HTTP Range used while not supported")
httpd.shutdown()
t.join()
print("PASSED")
def main():
t, httpd = start_server()
settings = {"max_download_buffer_size": 20}
# Test Accept-Ranges=False
@ -285,10 +293,15 @@ def main():
settings["max_download_threads"] = 2
run_test(allow_range=True, settings=settings, check_retries=True)
httpd.shutdown()
t.join()
if __name__ == "__main__":
try:
main()
sys.stdout.flush()
os._exit(0)
except Exception as ex:
exc_type, exc_value, exc_traceback = sys.exc_info()
traceback.print_tb(exc_traceback, file=sys.stderr)

View File

@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
echo "INSERT TO S3"
$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq "
INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/profile_events.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10, s3_truncate_on_insert = 1;
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | sort
" 2>&1 | grep -o -e '\ \[\ .*\ \]\ S3.*:\ .*\ ' | grep -v 'Microseconds' | grep -v 'S3DiskConnections' | grep -v 'S3DiskAddresses' | sort
echo "CHECK WITH query_log"
$CLICKHOUSE_CLIENT -nq "

View File

@ -19,8 +19,8 @@ query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1"
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} -nm --query "
WITH
ProfileEvents['ReadBufferFromS3ResetSessions'] AS reset,
ProfileEvents['ReadBufferFromS3PreservedSessions'] AS preserved
ProfileEvents['DiskConnectionsReset'] AS reset,
ProfileEvents['DiskConnectionsPreserved'] AS preserved
SELECT preserved > reset
FROM system.query_log
WHERE type = 'QueryFinish'
@ -51,7 +51,7 @@ select queryID() from(
" 2>&1)
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} -nm --query "
SELECT ProfileEvents['ReadWriteBufferFromHTTPPreservedSessions'] > 0
SELECT ProfileEvents['StorageConnectionsPreserved'] > 0
FROM system.query_log
WHERE type = 'QueryFinish'
AND current_database = currentDatabase()

View File

@ -8,5 +8,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=1"
# Grep 'test%2Fa.tsv' to ensure that path wasn't encoded/decoded
$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | grep -o "test%2Fa.tsv" -m1
$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | \
grep -o "test%2Fa.tsv" -m1 | head -n 1