mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into persistent_nukeeper_snapshot_storage
This commit is contained in:
commit
eb5e416fd5
2
contrib/boost
vendored
2
contrib/boost
vendored
@ -1 +1 @@
|
||||
Subproject commit 48f40ebb539220d328958f8823b094c0b07a4e79
|
||||
Subproject commit ee24fa55bc46e4d2ce7d0d052cc5a0d9b1be8c36
|
@ -358,6 +358,8 @@ mkdir analyze analyze/tmp ||:
|
||||
build_log_column_definitions
|
||||
|
||||
# Split the raw test output into files suitable for analysis.
|
||||
# To debug calculations only for a particular test, substitute a suitable
|
||||
# wildcard here, e.g. `for test_file in modulo-raw.tsv`.
|
||||
for test_file in *-raw.tsv
|
||||
do
|
||||
test_name=$(basename "$test_file" "-raw.tsv")
|
||||
@ -467,7 +469,13 @@ create view broken_queries as
|
||||
create table query_run_metrics_for_stats engine File(
|
||||
TSV, -- do not add header -- will parse with grep
|
||||
'analyze/query-run-metrics-for-stats.tsv')
|
||||
as select test, query_index, 0 run, version, metric_values
|
||||
as select test, query_index, 0 run, version,
|
||||
-- For debugging, add a filter for a particular metric like this:
|
||||
-- arrayFilter(m, n -> n = 'client_time', metric_values, metric_names)
|
||||
-- metric_values
|
||||
-- Note that further reporting may break, because the metric names are
|
||||
-- not filtered.
|
||||
metric_values
|
||||
from query_run_metric_arrays
|
||||
where (test, query_index) not in broken_queries
|
||||
order by test, query_index, run, version
|
||||
@ -585,8 +593,19 @@ create view query_metric_stats as
|
||||
-- Main statistics for queries -- query time as reported in query log.
|
||||
create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv')
|
||||
as select
|
||||
abs(diff) > report_threshold and abs(diff) > stat_threshold as changed_fail,
|
||||
abs(diff) > report_threshold - 0.05 and abs(diff) > stat_threshold as changed_show,
|
||||
-- It is important to have a non-strict inequality with stat_threshold
|
||||
-- here. The randomization distribution is actually discrete, and when
|
||||
-- the number of runs is small, the quantile we need (e.g. 0.99) turns
|
||||
-- out to be the maximum value of the distribution. We can also hit this
|
||||
-- maximum possible value with our test run, and this obviously means
|
||||
-- that we have observed the difference to the best precision possible
|
||||
-- for the given number of runs. If we use a strict equality here, we
|
||||
-- will miss such cases. This happened in the wild and lead to some
|
||||
-- uncaught regressions, because for the default 7 runs we do for PRs,
|
||||
-- the randomization distribution has only 16 values, so the max quantile
|
||||
-- is actually 0.9375.
|
||||
abs(diff) > report_threshold and abs(diff) >= stat_threshold as changed_fail,
|
||||
abs(diff) > report_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show,
|
||||
|
||||
not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail,
|
||||
not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show,
|
||||
|
@ -0,0 +1,7 @@
|
||||
<yandex>
|
||||
<!-- Directory with user provided files that are accessible by 'file' table function. -->
|
||||
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
|
||||
|
||||
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
|
||||
<users_config>users.xml</users_config>
|
||||
</yandex>
|
@ -19,4 +19,9 @@
|
||||
<max_threads>12</max_threads>
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<access_management>1</access_management>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
||||
|
@ -1,4 +1,6 @@
|
||||
-- input is table(test text, query text, run UInt32, version int, metrics Array(float))
|
||||
-- The input is table(test text, query text, run UInt32, version UInt8, metrics Array(float)).
|
||||
-- Run like this:
|
||||
-- clickhouse-local --queries-file eqmed.sql -S 'test text, query text, run UInt32, version UInt8, metrics Array(float)' --file analyze/tmp/modulo_0.tsv
|
||||
select
|
||||
arrayMap(x -> floor(x, 4), original_medians_array.medians_by_version[1] as l) l_rounded,
|
||||
arrayMap(x -> floor(x, 4), original_medians_array.medians_by_version[2] as r) r_rounded,
|
||||
@ -8,14 +10,19 @@ select
|
||||
from
|
||||
(
|
||||
-- quantiles of randomization distributions
|
||||
-- note that for small number of runs, the exact quantile might not make
|
||||
-- sense, because the last possible value of randomization distribution
|
||||
-- might take a larger percentage of distirbution (i.e. the distribution
|
||||
-- actually has discrete values, and the last step can be large).
|
||||
select quantileExactForEach(0.99)(
|
||||
arrayMap(x, y -> abs(x - y), metrics_by_label[1], metrics_by_label[2]) as d
|
||||
) threshold
|
||||
---- uncomment to see what the distribution is really like
|
||||
--, uniqExact(d.1) u
|
||||
---- Uncomment to see what the distribution is really like. This debug
|
||||
---- code only works for single (the first) metric.
|
||||
--, uniqExact(d[1]) u
|
||||
--, arraySort(x->x.1,
|
||||
-- arrayZip(
|
||||
-- (sumMap([d.1], [1]) as f).1,
|
||||
-- (sumMap([d[1]], [1]) as f).1,
|
||||
-- f.2)) full_histogram
|
||||
from
|
||||
(
|
||||
|
@ -109,6 +109,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
|
||||
}
|
||||
|
||||
in = std::make_shared<ReadBufferFromPocoSocket>(*socket);
|
||||
in->setAsyncCallback(std::move(async_callback));
|
||||
|
||||
out = std::make_shared<WriteBufferFromPocoSocket>(*socket);
|
||||
|
||||
connected = true;
|
||||
@ -753,15 +755,8 @@ std::optional<UInt64> Connection::checkPacket(size_t timeout_microseconds)
|
||||
}
|
||||
|
||||
|
||||
Packet Connection::receivePacket(std::function<void(Poco::Net::Socket &)> async_callback)
|
||||
Packet Connection::receivePacket()
|
||||
{
|
||||
in->setAsyncCallback(std::move(async_callback));
|
||||
SCOPE_EXIT({
|
||||
/// disconnect() will reset "in".
|
||||
if (in)
|
||||
in->setAsyncCallback({});
|
||||
});
|
||||
|
||||
try
|
||||
{
|
||||
Packet res;
|
||||
|
@ -27,7 +27,6 @@
|
||||
#include <atomic>
|
||||
#include <optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -175,8 +174,7 @@ public:
|
||||
std::optional<UInt64> checkPacket(size_t timeout_microseconds = 0);
|
||||
|
||||
/// Receive packet from server.
|
||||
/// Each time read blocks and async_callback is set, it will be called. You can poll socket inside it.
|
||||
Packet receivePacket(std::function<void(Poco::Net::Socket &)> async_callback = {});
|
||||
Packet receivePacket();
|
||||
|
||||
/// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception.
|
||||
void forceConnected(const ConnectionTimeouts & timeouts);
|
||||
@ -195,6 +193,16 @@ public:
|
||||
size_t outBytesCount() const { return out ? out->count() : 0; }
|
||||
size_t inBytesCount() const { return in ? in->count() : 0; }
|
||||
|
||||
Poco::Net::Socket * getSocket() { return socket.get(); }
|
||||
|
||||
/// Each time read from socket blocks and async_callback is set, it will be called. You can poll socket inside it.
|
||||
void setAsyncCallback(AsyncCallback async_callback_)
|
||||
{
|
||||
async_callback = std::move(async_callback_);
|
||||
if (in)
|
||||
in->setAsyncCallback(std::move(async_callback));
|
||||
}
|
||||
|
||||
private:
|
||||
String host;
|
||||
UInt16 port;
|
||||
@ -282,6 +290,8 @@ private:
|
||||
|
||||
LoggerWrapper log_wrapper;
|
||||
|
||||
AsyncCallback async_callback = {};
|
||||
|
||||
void connect(const ConnectionTimeouts & timeouts);
|
||||
void sendHello();
|
||||
void receiveHello();
|
||||
@ -307,4 +317,20 @@ private:
|
||||
[[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
|
||||
};
|
||||
|
||||
class AsyncCallbackSetter
|
||||
{
|
||||
public:
|
||||
AsyncCallbackSetter(Connection * connection_, AsyncCallback async_callback) : connection(connection_)
|
||||
{
|
||||
connection->setAsyncCallback(std::move(async_callback));
|
||||
}
|
||||
|
||||
~AsyncCallbackSetter()
|
||||
{
|
||||
connection->setAsyncCallback({});
|
||||
}
|
||||
private:
|
||||
Connection * connection;
|
||||
};
|
||||
|
||||
}
|
||||
|
239
src/Client/ConnectionEstablisher.cpp
Normal file
239
src/Client/ConnectionEstablisher.cpp
Normal file
@ -0,0 +1,239 @@
|
||||
#include <Client/ConnectionEstablisher.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event DistributedConnectionMissingTable;
|
||||
extern const Event DistributedConnectionStaleReplica;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
}
|
||||
|
||||
ConnectionEstablisher::ConnectionEstablisher(
|
||||
IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings * settings_,
|
||||
Poco::Logger * log_,
|
||||
const QualifiedTableName * table_to_check_)
|
||||
: pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_), is_finished(false)
|
||||
{
|
||||
}
|
||||
|
||||
void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::string & fail_message)
|
||||
{
|
||||
is_finished = false;
|
||||
SCOPE_EXIT(is_finished = true);
|
||||
try
|
||||
{
|
||||
result.entry = pool->get(*timeouts, settings, /* force_connected = */ false);
|
||||
AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback));
|
||||
|
||||
UInt64 server_revision = 0;
|
||||
if (table_to_check)
|
||||
server_revision = result.entry->getServerRevision(*timeouts);
|
||||
|
||||
if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS)
|
||||
{
|
||||
result.entry->forceConnected(*timeouts);
|
||||
result.is_usable = true;
|
||||
result.is_up_to_date = true;
|
||||
return;
|
||||
}
|
||||
|
||||
/// Only status of the remote table corresponding to the Distributed table is taken into account.
|
||||
/// TODO: request status for joined tables also.
|
||||
TablesStatusRequest status_request;
|
||||
status_request.tables.emplace(*table_to_check);
|
||||
|
||||
TablesStatusResponse status_response = result.entry->getTablesStatus(*timeouts, status_request);
|
||||
auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
|
||||
if (table_status_it == status_response.table_states_by_id.end())
|
||||
{
|
||||
const char * message_pattern = "There is no table {}.{} on server: {}";
|
||||
fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
|
||||
LOG_WARNING(log, fail_message);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
return;
|
||||
}
|
||||
|
||||
result.is_usable = true;
|
||||
|
||||
UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0;
|
||||
if (!max_allowed_delay)
|
||||
{
|
||||
result.is_up_to_date = true;
|
||||
return;
|
||||
}
|
||||
|
||||
UInt32 delay = table_status_it->second.absolute_delay;
|
||||
|
||||
if (delay < max_allowed_delay)
|
||||
result.is_up_to_date = true;
|
||||
else
|
||||
{
|
||||
result.is_up_to_date = false;
|
||||
result.staleness = delay;
|
||||
|
||||
LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT
|
||||
&& e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||
throw;
|
||||
|
||||
fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false);
|
||||
|
||||
if (!result.entry.isNull())
|
||||
{
|
||||
result.entry->disconnect();
|
||||
result.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
ConnectionEstablisherAsync::ConnectionEstablisherAsync(
|
||||
IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings * settings_,
|
||||
Poco::Logger * log_,
|
||||
const QualifiedTableName * table_to_check_)
|
||||
: connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_)
|
||||
{
|
||||
epoll.add(receive_timeout.getDescriptor());
|
||||
}
|
||||
|
||||
void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &)
|
||||
{
|
||||
/// Check if it's the first time and we need to add socket fd to epoll.
|
||||
if (connection_establisher_async.socket_fd == -1)
|
||||
{
|
||||
connection_establisher_async.epoll.add(fd);
|
||||
connection_establisher_async.socket_fd = fd;
|
||||
}
|
||||
|
||||
connection_establisher_async.receive_timeout.setRelative(timeout);
|
||||
fiber = std::move(fiber).resume();
|
||||
connection_establisher_async.receive_timeout.reset();
|
||||
}
|
||||
|
||||
Fiber ConnectionEstablisherAsync::Routine::operator()(Fiber && sink)
|
||||
{
|
||||
try
|
||||
{
|
||||
connection_establisher_async.connection_establisher.setAsyncCallback(ReadCallback{connection_establisher_async, sink});
|
||||
connection_establisher_async.connection_establisher.run(connection_establisher_async.result, connection_establisher_async.fail_message);
|
||||
}
|
||||
catch (const boost::context::detail::forced_unwind &)
|
||||
{
|
||||
/// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited
|
||||
/// It should not be caught or it will segfault.
|
||||
/// Other exceptions must be caught
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
connection_establisher_async.exception = std::current_exception();
|
||||
}
|
||||
|
||||
return std::move(sink);
|
||||
}
|
||||
|
||||
std::variant<int, ConnectionEstablisher::TryResult> ConnectionEstablisherAsync::resume()
|
||||
{
|
||||
if (!fiber_created)
|
||||
{
|
||||
reset();
|
||||
fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this});
|
||||
fiber_created = true;
|
||||
} else if (!checkReceiveTimeout())
|
||||
return result;
|
||||
|
||||
fiber = std::move(fiber).resume();
|
||||
|
||||
if (exception)
|
||||
std::rethrow_exception(std::move(exception));
|
||||
|
||||
if (connection_establisher.isFinished())
|
||||
{
|
||||
destroyFiber();
|
||||
return result;
|
||||
}
|
||||
|
||||
return epoll.getFileDescriptor();
|
||||
}
|
||||
|
||||
bool ConnectionEstablisherAsync::checkReceiveTimeout()
|
||||
{
|
||||
bool is_socket_ready = false;
|
||||
bool is_receive_timeout_alarmed = false;
|
||||
|
||||
epoll_event events[2];
|
||||
events[0].data.fd = events[1].data.fd = -1;
|
||||
size_t ready_count = epoll.getManyReady(2, events, false);
|
||||
for (size_t i = 0; i != ready_count; ++i)
|
||||
{
|
||||
if (events[i].data.fd == socket_fd)
|
||||
is_socket_ready = true;
|
||||
if (events[i].data.fd == receive_timeout.getDescriptor())
|
||||
is_receive_timeout_alarmed = true;
|
||||
}
|
||||
|
||||
if (is_receive_timeout_alarmed && !is_socket_ready)
|
||||
{
|
||||
destroyFiber();
|
||||
/// In not async case this exception would be thrown and caught in ConnectionEstablisher::run,
|
||||
/// but in async case we process timeout outside and cannot throw exception. So, we just save fail message.
|
||||
fail_message = "Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")";
|
||||
epoll.remove(socket_fd);
|
||||
resetResult();
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void ConnectionEstablisherAsync::cancel()
|
||||
{
|
||||
destroyFiber();
|
||||
reset();
|
||||
}
|
||||
|
||||
void ConnectionEstablisherAsync::reset()
|
||||
{
|
||||
resetResult();
|
||||
fail_message.clear();
|
||||
socket_fd = -1;
|
||||
}
|
||||
|
||||
void ConnectionEstablisherAsync::resetResult()
|
||||
{
|
||||
if (!result.entry.isNull())
|
||||
{
|
||||
result.entry->disconnect();
|
||||
result.reset();
|
||||
}
|
||||
}
|
||||
|
||||
void ConnectionEstablisherAsync::destroyFiber()
|
||||
{
|
||||
Fiber to_destroy = std::move(fiber);
|
||||
fiber_created = false;
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
}
|
131
src/Client/ConnectionEstablisher.h
Normal file
131
src/Client/ConnectionEstablisher.h
Normal file
@ -0,0 +1,131 @@
|
||||
#pragma once
|
||||
|
||||
#include <variant>
|
||||
|
||||
#include <Common/Epoll.h>
|
||||
#include <Common/Fiber.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/TimerDescriptor.h>
|
||||
#include <Common/PoolWithFailoverBase.h>
|
||||
#include <Client/ConnectionPool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Class for establishing connection to the replica. It supports setting up
|
||||
/// an async callback that will be called when reading from socket blocks.
|
||||
class ConnectionEstablisher
|
||||
{
|
||||
public:
|
||||
using TryResult = PoolWithFailoverBase<IConnectionPool>::TryResult;
|
||||
|
||||
ConnectionEstablisher(IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings * settings_,
|
||||
Poco::Logger * log,
|
||||
const QualifiedTableName * table_to_check = nullptr);
|
||||
|
||||
/// Establish connection and save it in result, write possible exception message in fail_message.
|
||||
void run(TryResult & result, std::string & fail_message);
|
||||
|
||||
/// Set async callback that will be called when reading from socket blocks.
|
||||
void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); }
|
||||
|
||||
bool isFinished() const { return is_finished; }
|
||||
|
||||
private:
|
||||
IConnectionPool * pool;
|
||||
const ConnectionTimeouts * timeouts;
|
||||
const Settings * settings;
|
||||
Poco::Logger * log;
|
||||
const QualifiedTableName * table_to_check;
|
||||
|
||||
bool is_finished;
|
||||
AsyncCallback async_callback = {};
|
||||
|
||||
};
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
/// Class for nonblocking establishing connection to the replica.
|
||||
/// It runs establishing connection process in fiber and sets special
|
||||
/// read callback which is called when reading from socket blocks.
|
||||
/// When read callback is called, socket and receive timeout are added in epoll
|
||||
/// and execution returns to the main program.
|
||||
/// So, you can poll this epoll file descriptor to determine when to resume.
|
||||
class ConnectionEstablisherAsync
|
||||
{
|
||||
public:
|
||||
using TryResult = PoolWithFailoverBase<IConnectionPool>::TryResult;
|
||||
|
||||
ConnectionEstablisherAsync(IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings * settings_,
|
||||
Poco::Logger * log_,
|
||||
const QualifiedTableName * table_to_check = nullptr);
|
||||
|
||||
/// Resume establishing connection. If the process was not finished,
|
||||
/// return file descriptor (you can add it in epoll and poll it,
|
||||
/// when this fd become ready, call resume again),
|
||||
/// if the process was failed or finished, return it's result,
|
||||
std::variant<int, TryResult> resume();
|
||||
|
||||
/// Cancel establishing connections. Fiber will be destroyed,
|
||||
/// class will be set in initial stage.
|
||||
void cancel();
|
||||
|
||||
TryResult getResult() const { return result; }
|
||||
|
||||
const std::string & getFailMessage() const { return fail_message; }
|
||||
|
||||
private:
|
||||
/// When epoll file descriptor is ready, check if it's an expired timeout.
|
||||
/// Return false if receive timeout expired and socket is not ready, return true otherwise.
|
||||
bool checkReceiveTimeout();
|
||||
|
||||
struct Routine
|
||||
{
|
||||
ConnectionEstablisherAsync & connection_establisher_async;
|
||||
|
||||
struct ReadCallback
|
||||
{
|
||||
ConnectionEstablisherAsync & connection_establisher_async;
|
||||
Fiber & fiber;
|
||||
|
||||
void operator()(int fd, const Poco::Timespan & timeout, const std::string &);
|
||||
};
|
||||
|
||||
Fiber operator()(Fiber && sink);
|
||||
};
|
||||
|
||||
void reset();
|
||||
|
||||
void resetResult();
|
||||
|
||||
void destroyFiber();
|
||||
|
||||
ConnectionEstablisher connection_establisher;
|
||||
TryResult result;
|
||||
std::string fail_message;
|
||||
|
||||
Fiber fiber;
|
||||
FiberStack fiber_stack;
|
||||
|
||||
/// We use timer descriptor for checking socket receive timeout.
|
||||
TimerDescriptor receive_timeout;
|
||||
|
||||
/// In read callback we add socket file descriptor and timer descriptor with receive timeout
|
||||
/// in epoll, so we can return epoll file descriptor outside for polling.
|
||||
Epoll epoll;
|
||||
int socket_fd = -1;
|
||||
std::string socket_description;
|
||||
|
||||
/// If and exception occurred in fiber resume, we save it and rethrow.
|
||||
std::exception_ptr exception;
|
||||
|
||||
bool fiber_created = false;
|
||||
};
|
||||
|
||||
#endif
|
||||
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <Client/ConnectionEstablisher.h>
|
||||
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <Poco/Net/DNS.h>
|
||||
@ -23,9 +24,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -172,6 +170,43 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
|
||||
return getManyImpl(settings, pool_mode, try_get_entry);
|
||||
}
|
||||
|
||||
ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings * settings)
|
||||
{
|
||||
size_t offset = 0;
|
||||
if (settings)
|
||||
offset = settings->load_balancing_first_offset % nested_pools.size();
|
||||
|
||||
GetPriorityFunc get_priority;
|
||||
switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing)
|
||||
{
|
||||
case LoadBalancing::NEAREST_HOSTNAME:
|
||||
get_priority = [&](size_t i) { return hostname_differences[i]; };
|
||||
break;
|
||||
case LoadBalancing::IN_ORDER:
|
||||
get_priority = [](size_t i) { return i; };
|
||||
break;
|
||||
case LoadBalancing::RANDOM:
|
||||
break;
|
||||
case LoadBalancing::FIRST_OR_RANDOM:
|
||||
get_priority = [offset](size_t i) -> size_t { return i != offset; };
|
||||
break;
|
||||
case LoadBalancing::ROUND_ROBIN:
|
||||
if (last_used >= nested_pools.size())
|
||||
last_used = 0;
|
||||
++last_used;
|
||||
/* Consider nested_pools.size() equals to 5
|
||||
* last_used = 1 -> get_priority: 0 1 2 3 4
|
||||
* last_used = 2 -> get_priority: 5 0 1 2 3
|
||||
* last_used = 3 -> get_priority: 5 4 0 1 2
|
||||
* ...
|
||||
* */
|
||||
get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; };
|
||||
break;
|
||||
}
|
||||
|
||||
return get_priority;
|
||||
}
|
||||
|
||||
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyImpl(
|
||||
const Settings * settings,
|
||||
PoolMode pool_mode,
|
||||
@ -194,36 +229,7 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
|
||||
else
|
||||
throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t offset = 0;
|
||||
if (settings)
|
||||
offset = settings->load_balancing_first_offset % nested_pools.size();
|
||||
GetPriorityFunc get_priority;
|
||||
switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing)
|
||||
{
|
||||
case LoadBalancing::NEAREST_HOSTNAME:
|
||||
get_priority = [&](size_t i) { return hostname_differences[i]; };
|
||||
break;
|
||||
case LoadBalancing::IN_ORDER:
|
||||
get_priority = [](size_t i) { return i; };
|
||||
break;
|
||||
case LoadBalancing::RANDOM:
|
||||
break;
|
||||
case LoadBalancing::FIRST_OR_RANDOM:
|
||||
get_priority = [offset](size_t i) -> size_t { return i != offset; };
|
||||
break;
|
||||
case LoadBalancing::ROUND_ROBIN:
|
||||
if (last_used >= nested_pools.size())
|
||||
last_used = 0;
|
||||
++last_used;
|
||||
/* Consider nested_pools.size() equals to 5
|
||||
* last_used = 1 -> get_priority: 0 1 2 3 4
|
||||
* last_used = 2 -> get_priority: 5 0 1 2 3
|
||||
* last_used = 3 -> get_priority: 5 4 0 1 2
|
||||
* ...
|
||||
* */
|
||||
get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; };
|
||||
break;
|
||||
}
|
||||
GetPriorityFunc get_priority = makeGetPriorityFunc(settings);
|
||||
|
||||
UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0;
|
||||
bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true;
|
||||
@ -241,77 +247,17 @@ ConnectionPoolWithFailover::tryGetEntry(
|
||||
const Settings * settings,
|
||||
const QualifiedTableName * table_to_check)
|
||||
{
|
||||
ConnectionEstablisher connection_establisher(&pool, &timeouts, settings, log, table_to_check);
|
||||
TryResult result;
|
||||
try
|
||||
{
|
||||
result.entry = pool.get(timeouts, settings, /* force_connected = */ false);
|
||||
|
||||
UInt64 server_revision = 0;
|
||||
if (table_to_check)
|
||||
server_revision = result.entry->getServerRevision(timeouts);
|
||||
|
||||
if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS)
|
||||
{
|
||||
result.entry->forceConnected(timeouts);
|
||||
result.is_usable = true;
|
||||
result.is_up_to_date = true;
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Only status of the remote table corresponding to the Distributed table is taken into account.
|
||||
/// TODO: request status for joined tables also.
|
||||
TablesStatusRequest status_request;
|
||||
status_request.tables.emplace(*table_to_check);
|
||||
|
||||
TablesStatusResponse status_response = result.entry->getTablesStatus(timeouts, status_request);
|
||||
auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
|
||||
if (table_status_it == status_response.table_states_by_id.end())
|
||||
{
|
||||
const char * message_pattern = "There is no table {}.{} on server: {}";
|
||||
fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
|
||||
LOG_WARNING(log, fail_message);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
result.is_usable = true;
|
||||
|
||||
UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0;
|
||||
if (!max_allowed_delay)
|
||||
{
|
||||
result.is_up_to_date = true;
|
||||
return result;
|
||||
}
|
||||
|
||||
UInt32 delay = table_status_it->second.absolute_delay;
|
||||
|
||||
if (delay < max_allowed_delay)
|
||||
result.is_up_to_date = true;
|
||||
else
|
||||
{
|
||||
result.is_up_to_date = false;
|
||||
result.staleness = delay;
|
||||
|
||||
LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT
|
||||
&& e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||
throw;
|
||||
|
||||
fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false);
|
||||
|
||||
if (!result.entry.isNull())
|
||||
{
|
||||
result.entry->disconnect();
|
||||
result.reset();
|
||||
}
|
||||
}
|
||||
connection_establisher.run(result, fail_message);
|
||||
return result;
|
||||
}
|
||||
|
||||
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> ConnectionPoolWithFailover::getShuffledPools(const Settings * settings)
|
||||
{
|
||||
GetPriorityFunc get_priority = makeGetPriorityFunc(settings);
|
||||
UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0;
|
||||
return Base::getShuffledPools(max_ignored_errors, get_priority);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -80,6 +80,15 @@ public:
|
||||
using Status = std::vector<NestedPoolStatus>;
|
||||
Status getStatus() const;
|
||||
|
||||
std::vector<Base::ShuffledPool> getShuffledPools(const Settings * settings);
|
||||
|
||||
size_t getMaxErrorCup() const { return Base::max_error_cap; }
|
||||
|
||||
void updateSharedError(std::vector<ShuffledPool> & shuffled_pools)
|
||||
{
|
||||
Base::updateSharedErrorCounts(shuffled_pools);
|
||||
}
|
||||
|
||||
private:
|
||||
/// Get the values of relevant settings and call Base::getMany()
|
||||
std::vector<TryResult> getManyImpl(
|
||||
@ -97,6 +106,8 @@ private:
|
||||
const Settings * settings,
|
||||
const QualifiedTableName * table_to_check = nullptr);
|
||||
|
||||
GetPriorityFunc makeGetPriorityFunc(const Settings * settings);
|
||||
|
||||
private:
|
||||
std::vector<size_t> hostname_differences; /// Distances from name of this host to the names of hosts of pools.
|
||||
size_t last_used = 0; /// Last used for round_robin policy.
|
||||
|
524
src/Client/HedgedConnections.cpp
Normal file
524
src/Client/HedgedConnections.cpp
Normal file
@ -0,0 +1,524 @@
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <Client/HedgedConnections.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MISMATCH_REPLICAS_DATA_SOURCES;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
extern const int ALL_CONNECTION_TRIES_FAILED;
|
||||
}
|
||||
|
||||
HedgedConnections::HedgedConnections(
|
||||
const ConnectionPoolWithFailoverPtr & pool_,
|
||||
const Settings & settings_,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const ThrottlerPtr & throttler_,
|
||||
PoolMode pool_mode,
|
||||
std::shared_ptr<QualifiedTableName> table_to_check_)
|
||||
: hedged_connections_factory(pool_, &settings_, timeouts_, table_to_check_)
|
||||
, settings(settings_)
|
||||
, throttler(throttler_)
|
||||
{
|
||||
std::vector<Connection *> connections = hedged_connections_factory.getManyConnections(pool_mode);
|
||||
|
||||
if (connections.empty())
|
||||
return;
|
||||
|
||||
offset_states.reserve(connections.size());
|
||||
for (size_t i = 0; i != connections.size(); ++i)
|
||||
{
|
||||
offset_states.emplace_back();
|
||||
offset_states[i].replicas.emplace_back(connections[i]);
|
||||
offset_states[i].active_connection_count = 1;
|
||||
|
||||
ReplicaState & replica = offset_states[i].replicas.back();
|
||||
replica.connection->setThrottler(throttler_);
|
||||
|
||||
epoll.add(replica.packet_receiver->getFileDescriptor());
|
||||
fd_to_replica_location[replica.packet_receiver->getFileDescriptor()] = ReplicaLocation{i, 0};
|
||||
|
||||
epoll.add(replica.change_replica_timeout.getDescriptor());
|
||||
timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{i, 0};
|
||||
}
|
||||
|
||||
active_connection_count = connections.size();
|
||||
offsets_with_disabled_changing_replica = 0;
|
||||
pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); });
|
||||
}
|
||||
|
||||
void HedgedConnections::Pipeline::add(std::function<void(ReplicaState & replica)> send_function)
|
||||
{
|
||||
pipeline.push_back(send_function);
|
||||
}
|
||||
|
||||
void HedgedConnections::Pipeline::run(ReplicaState & replica)
|
||||
{
|
||||
for (auto & send_func : pipeline)
|
||||
send_func(replica);
|
||||
}
|
||||
|
||||
void HedgedConnections::sendScalarsData(Scalars & data)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto send_scalars_data = [&data](ReplicaState & replica) { replica.connection->sendScalarsData(data); };
|
||||
|
||||
for (auto & offset_state : offset_states)
|
||||
for (auto & replica : offset_state.replicas)
|
||||
if (replica.connection)
|
||||
send_scalars_data(replica);
|
||||
|
||||
pipeline_for_new_replicas.add(send_scalars_data);
|
||||
}
|
||||
|
||||
void HedgedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data.size() != size())
|
||||
throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES);
|
||||
|
||||
auto send_external_tables_data = [&data](ReplicaState & replica) { replica.connection->sendExternalTablesData(data[0]); };
|
||||
|
||||
for (auto & offset_state : offset_states)
|
||||
for (auto & replica : offset_state.replicas)
|
||||
if (replica.connection)
|
||||
send_external_tables_data(replica);
|
||||
|
||||
pipeline_for_new_replicas.add(send_external_tables_data);
|
||||
}
|
||||
|
||||
void HedgedConnections::sendIgnoredPartUUIDs(const std::vector<UUID> & uuids)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (sent_query)
|
||||
throw Exception("Cannot send uuids after query is sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto send_ignored_part_uuids = [&uuids](ReplicaState & replica) { replica.connection->sendIgnoredPartUUIDs(uuids); };
|
||||
|
||||
for (auto & offset_state : offset_states)
|
||||
for (auto & replica : offset_state.replicas)
|
||||
if (replica.connection)
|
||||
send_ignored_part_uuids(replica);
|
||||
|
||||
pipeline_for_new_replicas.add(send_ignored_part_uuids);
|
||||
}
|
||||
|
||||
void HedgedConnections::sendQuery(
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & query,
|
||||
const String & query_id,
|
||||
UInt64 stage,
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (sent_query)
|
||||
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (auto & offset_state : offset_states)
|
||||
{
|
||||
for (auto & replica : offset_state.replicas)
|
||||
{
|
||||
if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
|
||||
{
|
||||
disable_two_level_aggregation = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (disable_two_level_aggregation)
|
||||
break;
|
||||
}
|
||||
|
||||
if (!disable_two_level_aggregation)
|
||||
{
|
||||
/// Tell hedged_connections_factory to skip replicas that doesn't support two-level aggregation.
|
||||
hedged_connections_factory.skipReplicasWithTwoLevelAggregationIncompatibility();
|
||||
}
|
||||
|
||||
auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica)
|
||||
{
|
||||
Settings modified_settings = settings;
|
||||
|
||||
if (disable_two_level_aggregation)
|
||||
{
|
||||
/// Disable two-level aggregation due to version incompatibility.
|
||||
modified_settings.group_by_two_level_threshold = 0;
|
||||
modified_settings.group_by_two_level_threshold_bytes = 0;
|
||||
}
|
||||
|
||||
if (offset_states.size() > 1)
|
||||
{
|
||||
modified_settings.parallel_replicas_count = offset_states.size();
|
||||
modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset;
|
||||
}
|
||||
|
||||
replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data);
|
||||
replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout);
|
||||
};
|
||||
|
||||
for (auto & offset_status : offset_states)
|
||||
for (auto & replica : offset_status.replicas)
|
||||
send_query(replica);
|
||||
|
||||
pipeline_for_new_replicas.add(send_query);
|
||||
sent_query = true;
|
||||
}
|
||||
|
||||
void HedgedConnections::disconnect()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
for (auto & offset_status : offset_states)
|
||||
for (auto & replica : offset_status.replicas)
|
||||
if (replica.connection)
|
||||
finishProcessReplica(replica, true);
|
||||
|
||||
if (hedged_connections_factory.hasEventsInProcess())
|
||||
{
|
||||
if (hedged_connections_factory.numberOfProcessingReplicas() > 0)
|
||||
epoll.remove(hedged_connections_factory.getFileDescriptor());
|
||||
|
||||
hedged_connections_factory.stopChoosingReplicas();
|
||||
}
|
||||
}
|
||||
|
||||
std::string HedgedConnections::dumpAddresses() const
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
std::string addresses;
|
||||
bool is_first = true;
|
||||
|
||||
for (const auto & offset_state : offset_states)
|
||||
{
|
||||
for (const auto & replica : offset_state.replicas)
|
||||
{
|
||||
if (replica.connection)
|
||||
{
|
||||
addresses += (is_first ? "" : "; ") + replica.connection->getDescription();
|
||||
is_first = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return addresses;
|
||||
}
|
||||
|
||||
void HedgedConnections::sendCancel()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query || cancelled)
|
||||
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (auto & offset_status : offset_states)
|
||||
for (auto & replica : offset_status.replicas)
|
||||
if (replica.connection)
|
||||
replica.connection->sendCancel();
|
||||
|
||||
cancelled = true;
|
||||
}
|
||||
|
||||
Packet HedgedConnections::drain()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!cancelled)
|
||||
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
Packet res;
|
||||
res.type = Protocol::Server::EndOfStream;
|
||||
|
||||
while (!epoll.empty())
|
||||
{
|
||||
ReplicaLocation location = getReadyReplicaLocation();
|
||||
Packet packet = receivePacketFromReplica(location);
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::PartUUIDs:
|
||||
case Protocol::Server::Data:
|
||||
case Protocol::Server::Progress:
|
||||
case Protocol::Server::ProfileInfo:
|
||||
case Protocol::Server::Totals:
|
||||
case Protocol::Server::Extremes:
|
||||
case Protocol::Server::EndOfStream:
|
||||
break;
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
default:
|
||||
/// If we receive an exception or an unknown packet, we save it.
|
||||
res = std::move(packet);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Packet HedgedConnections::receivePacket()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
return receivePacketUnlocked({});
|
||||
}
|
||||
|
||||
Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback)
|
||||
{
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
if (!hasActiveConnections())
|
||||
throw Exception("No more packets are available.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (epoll.empty())
|
||||
throw Exception("No pending events in epoll.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ReplicaLocation location = getReadyReplicaLocation(std::move(async_callback));
|
||||
return receivePacketFromReplica(location);
|
||||
}
|
||||
|
||||
HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback)
|
||||
{
|
||||
/// Firstly, resume replica with the last received packet if it has pending data.
|
||||
if (replica_with_last_received_packet)
|
||||
{
|
||||
ReplicaLocation location = replica_with_last_received_packet.value();
|
||||
replica_with_last_received_packet.reset();
|
||||
if (offset_states[location.offset].replicas[location.index].connection->hasReadPendingData() && resumePacketReceiver(location))
|
||||
return location;
|
||||
}
|
||||
|
||||
int event_fd;
|
||||
while (true)
|
||||
{
|
||||
/// Get ready file descriptor from epoll and process it.
|
||||
event_fd = getReadyFileDescriptor(async_callback);
|
||||
|
||||
if (event_fd == hedged_connections_factory.getFileDescriptor())
|
||||
checkNewReplica();
|
||||
else if (fd_to_replica_location.contains(event_fd))
|
||||
{
|
||||
ReplicaLocation location = fd_to_replica_location[event_fd];
|
||||
if (resumePacketReceiver(location))
|
||||
return location;
|
||||
}
|
||||
else if (timeout_fd_to_replica_location.contains(event_fd))
|
||||
{
|
||||
ReplicaLocation location = timeout_fd_to_replica_location[event_fd];
|
||||
offset_states[location.offset].replicas[location.index].change_replica_timeout.reset();
|
||||
offset_states[location.offset].replicas[location.index].is_change_replica_timeout_expired = true;
|
||||
offset_states[location.offset].next_replica_in_process = true;
|
||||
offsets_queue.push(location.offset);
|
||||
startNewReplica();
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLocation & location)
|
||||
{
|
||||
ReplicaState & replica_state = offset_states[location.offset].replicas[location.index];
|
||||
auto res = replica_state.packet_receiver->resume();
|
||||
|
||||
if (std::holds_alternative<Packet>(res))
|
||||
{
|
||||
last_received_packet = std::move(std::get<Packet>(res));
|
||||
return true;
|
||||
}
|
||||
else if (std::holds_alternative<Poco::Timespan>(res))
|
||||
{
|
||||
finishProcessReplica(replica_state, true);
|
||||
|
||||
/// Check if there is no more active connections with the same offset and there is no new replica in process.
|
||||
if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process)
|
||||
throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback)
|
||||
{
|
||||
epoll_event event;
|
||||
event.data.fd = -1;
|
||||
size_t events_count = 0;
|
||||
while (events_count == 0)
|
||||
{
|
||||
events_count = epoll.getManyReady(1, &event, false);
|
||||
if (!events_count && async_callback)
|
||||
async_callback(epoll.getFileDescriptor(), 0, epoll.getDescription());
|
||||
}
|
||||
return event.data.fd;
|
||||
}
|
||||
|
||||
Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location)
|
||||
{
|
||||
ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index];
|
||||
Packet packet = std::move(last_received_packet);
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::Data:
|
||||
/// If we received the first not empty data packet and still can change replica,
|
||||
/// disable changing replica with this offset.
|
||||
if (offset_states[replica_location.offset].can_change_replica && packet.block.rows() > 0)
|
||||
disableChangingReplica(replica_location);
|
||||
replica_with_last_received_packet = replica_location;
|
||||
break;
|
||||
case Protocol::Server::Progress:
|
||||
/// Check if we have made some progress and still can change replica.
|
||||
if (offset_states[replica_location.offset].can_change_replica && packet.progress.read_bytes > 0)
|
||||
{
|
||||
/// If we are allowed to change replica until the first data packet,
|
||||
/// just restart timeout (if it hasn't expired yet). Otherwise disable changing replica with this offset.
|
||||
if (settings.allow_changing_replica_until_first_data_packet && !replica.is_change_replica_timeout_expired)
|
||||
replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout);
|
||||
else
|
||||
disableChangingReplica(replica_location);
|
||||
}
|
||||
replica_with_last_received_packet = replica_location;
|
||||
break;
|
||||
case Protocol::Server::PartUUIDs:
|
||||
case Protocol::Server::ProfileInfo:
|
||||
case Protocol::Server::Totals:
|
||||
case Protocol::Server::Extremes:
|
||||
case Protocol::Server::Log:
|
||||
replica_with_last_received_packet = replica_location;
|
||||
break;
|
||||
|
||||
case Protocol::Server::EndOfStream:
|
||||
finishProcessReplica(replica, false);
|
||||
break;
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
default:
|
||||
finishProcessReplica(replica, true);
|
||||
break;
|
||||
}
|
||||
|
||||
return packet;
|
||||
}
|
||||
|
||||
void HedgedConnections::disableChangingReplica(const ReplicaLocation & replica_location)
|
||||
{
|
||||
/// Stop working with replicas, that are responsible for the same offset.
|
||||
OffsetState & offset_state = offset_states[replica_location.offset];
|
||||
offset_state.replicas[replica_location.index].change_replica_timeout.reset();
|
||||
++offsets_with_disabled_changing_replica;
|
||||
offset_state.can_change_replica = false;
|
||||
|
||||
for (size_t i = 0; i != offset_state.replicas.size(); ++i)
|
||||
{
|
||||
if (i != replica_location.index && offset_state.replicas[i].connection)
|
||||
{
|
||||
offset_state.replicas[i].connection->sendCancel();
|
||||
finishProcessReplica(offset_state.replicas[i], true);
|
||||
}
|
||||
}
|
||||
|
||||
/// If we disabled changing replica with all offsets, we need to stop choosing new replicas.
|
||||
if (hedged_connections_factory.hasEventsInProcess() && offsets_with_disabled_changing_replica == offset_states.size())
|
||||
{
|
||||
if (hedged_connections_factory.numberOfProcessingReplicas() > 0)
|
||||
epoll.remove(hedged_connections_factory.getFileDescriptor());
|
||||
hedged_connections_factory.stopChoosingReplicas();
|
||||
}
|
||||
}
|
||||
|
||||
void HedgedConnections::startNewReplica()
|
||||
{
|
||||
Connection * connection = nullptr;
|
||||
HedgedConnectionsFactory::State state = hedged_connections_factory.startNewConnection(connection);
|
||||
|
||||
/// Check if we need to add hedged_connections_factory file descriptor to epoll.
|
||||
if (state == HedgedConnectionsFactory::State::NOT_READY && hedged_connections_factory.numberOfProcessingReplicas() == 1)
|
||||
epoll.add(hedged_connections_factory.getFileDescriptor());
|
||||
|
||||
processNewReplicaState(state, connection);
|
||||
}
|
||||
|
||||
void HedgedConnections::checkNewReplica()
|
||||
{
|
||||
Connection * connection = nullptr;
|
||||
HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(connection);
|
||||
|
||||
processNewReplicaState(state, connection);
|
||||
|
||||
/// Check if we don't need to listen hedged_connections_factory file descriptor in epoll anymore.
|
||||
if (hedged_connections_factory.numberOfProcessingReplicas() == 0)
|
||||
epoll.remove(hedged_connections_factory.getFileDescriptor());
|
||||
}
|
||||
|
||||
void HedgedConnections::processNewReplicaState(HedgedConnectionsFactory::State state, Connection * connection)
|
||||
{
|
||||
switch (state)
|
||||
{
|
||||
case HedgedConnectionsFactory::State::READY:
|
||||
{
|
||||
size_t offset = offsets_queue.front();
|
||||
offsets_queue.pop();
|
||||
|
||||
offset_states[offset].replicas.emplace_back(connection);
|
||||
++offset_states[offset].active_connection_count;
|
||||
offset_states[offset].next_replica_in_process = false;
|
||||
++active_connection_count;
|
||||
|
||||
ReplicaState & replica = offset_states[offset].replicas.back();
|
||||
epoll.add(replica.packet_receiver->getFileDescriptor());
|
||||
fd_to_replica_location[replica.packet_receiver->getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1};
|
||||
epoll.add(replica.change_replica_timeout.getDescriptor());
|
||||
timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1};
|
||||
|
||||
pipeline_for_new_replicas.run(replica);
|
||||
break;
|
||||
}
|
||||
case HedgedConnectionsFactory::State::CANNOT_CHOOSE:
|
||||
{
|
||||
while (!offsets_queue.empty())
|
||||
{
|
||||
/// Check if there is no active replica with needed offsets.
|
||||
if (offset_states[offsets_queue.front()].active_connection_count == 0)
|
||||
throw Exception("Cannot find enough connections to replicas", ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||
offset_states[offsets_queue.front()].next_replica_in_process = false;
|
||||
offsets_queue.pop();
|
||||
}
|
||||
break;
|
||||
}
|
||||
case HedgedConnectionsFactory::State::NOT_READY:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect)
|
||||
{
|
||||
replica.packet_receiver->cancel();
|
||||
replica.change_replica_timeout.reset();
|
||||
|
||||
epoll.remove(replica.packet_receiver->getFileDescriptor());
|
||||
--offset_states[fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset].active_connection_count;
|
||||
fd_to_replica_location.erase(replica.packet_receiver->getFileDescriptor());
|
||||
|
||||
epoll.remove(replica.change_replica_timeout.getDescriptor());
|
||||
timeout_fd_to_replica_location.erase(replica.change_replica_timeout.getDescriptor());
|
||||
|
||||
--active_connection_count;
|
||||
|
||||
if (disconnect)
|
||||
replica.connection->disconnect();
|
||||
replica.connection = nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
189
src/Client/HedgedConnections.h
Normal file
189
src/Client/HedgedConnections.h
Normal file
@ -0,0 +1,189 @@
|
||||
#pragma once
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <functional>
|
||||
#include <queue>
|
||||
#include <optional>
|
||||
|
||||
#include <Client/HedgedConnectionsFactory.h>
|
||||
#include <Client/IConnections.h>
|
||||
#include <Client/PacketReceiver.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/Fiber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** To receive data from multiple replicas (connections) from one shard asynchronously.
|
||||
* The principe of Hedged Connections is used to reduce tail latency:
|
||||
* if we don't receive data from replica and there is no progress in query execution
|
||||
* for a long time, we try to get new replica and send query to it,
|
||||
* without cancelling working with previous replica. This class
|
||||
* supports all functionality that MultipleConnections has.
|
||||
*/
|
||||
class HedgedConnections : public IConnections
|
||||
{
|
||||
public:
|
||||
using PacketReceiverPtr = std::unique_ptr<PacketReceiver>;
|
||||
struct ReplicaState
|
||||
{
|
||||
explicit ReplicaState(Connection * connection_) : connection(connection_), packet_receiver(std::make_unique<PacketReceiver>(connection_))
|
||||
{
|
||||
}
|
||||
|
||||
Connection * connection = nullptr;
|
||||
PacketReceiverPtr packet_receiver;
|
||||
TimerDescriptor change_replica_timeout;
|
||||
bool is_change_replica_timeout_expired = false;
|
||||
};
|
||||
|
||||
struct OffsetState
|
||||
{
|
||||
/// Replicas with the same offset.
|
||||
std::vector<ReplicaState> replicas;
|
||||
/// An amount of active replicas. When can_change_replica is false,
|
||||
/// active_connection_count is always <= 1 (because we stopped working with
|
||||
/// other replicas with the same offset)
|
||||
size_t active_connection_count = 0;
|
||||
bool can_change_replica = true;
|
||||
|
||||
/// This flag is true when this offset is in queue for
|
||||
/// new replicas. It's needed to process receive timeout
|
||||
/// (throw an exception when receive timeout expired and there is no
|
||||
/// new replica in process)
|
||||
bool next_replica_in_process = false;
|
||||
};
|
||||
|
||||
/// We process events in epoll, so we need to determine replica by it's
|
||||
/// file descriptor. We store map fd -> replica location. To determine
|
||||
/// where replica is, we need a replica offset
|
||||
/// (the same as parallel_replica_offset), and index, which is needed because
|
||||
/// we can have many replicas with same offset (when receive_data_timeout has expired).
|
||||
struct ReplicaLocation
|
||||
{
|
||||
size_t offset;
|
||||
size_t index;
|
||||
};
|
||||
|
||||
HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_,
|
||||
const Settings & settings_,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const ThrottlerPtr & throttler,
|
||||
PoolMode pool_mode,
|
||||
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr);
|
||||
|
||||
void sendScalarsData(Scalars & data) override;
|
||||
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data) override;
|
||||
|
||||
void sendQuery(
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & query,
|
||||
const String & query_id,
|
||||
UInt64 stage,
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data) override;
|
||||
|
||||
Packet receivePacket() override;
|
||||
|
||||
Packet receivePacketUnlocked(AsyncCallback async_callback) override;
|
||||
|
||||
void disconnect() override;
|
||||
|
||||
void sendCancel() override;
|
||||
|
||||
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids) override;
|
||||
|
||||
Packet drain() override;
|
||||
|
||||
std::string dumpAddresses() const override;
|
||||
|
||||
size_t size() const override { return offset_states.size(); }
|
||||
|
||||
bool hasActiveConnections() const override { return active_connection_count > 0; }
|
||||
|
||||
private:
|
||||
/// If we don't receive data from replica and there is no progress in query
|
||||
/// execution for receive_data_timeout, we are trying to get new
|
||||
/// replica and send query to it. Beside sending query, there are some
|
||||
/// additional actions like sendScalarsData or sendExternalTablesData and we need
|
||||
/// to perform these actions in the same order on the new replica. So, we will
|
||||
/// save actions with replicas in pipeline to perform them on the new replicas.
|
||||
class Pipeline
|
||||
{
|
||||
public:
|
||||
void add(std::function<void(ReplicaState &)> send_function);
|
||||
|
||||
void run(ReplicaState & replica);
|
||||
private:
|
||||
std::vector<std::function<void(ReplicaState &)>> pipeline;
|
||||
};
|
||||
|
||||
Packet receivePacketFromReplica(const ReplicaLocation & replica_location);
|
||||
|
||||
ReplicaLocation getReadyReplicaLocation(AsyncCallback async_callback = {});
|
||||
|
||||
bool resumePacketReceiver(const ReplicaLocation & replica_location);
|
||||
|
||||
void disableChangingReplica(const ReplicaLocation & replica_location);
|
||||
|
||||
void startNewReplica();
|
||||
|
||||
void checkNewReplica();
|
||||
|
||||
void processNewReplicaState(HedgedConnectionsFactory::State state, Connection * connection);
|
||||
|
||||
void finishProcessReplica(ReplicaState & replica, bool disconnect);
|
||||
|
||||
int getReadyFileDescriptor(AsyncCallback async_callback = {});
|
||||
|
||||
HedgedConnectionsFactory hedged_connections_factory;
|
||||
|
||||
/// All replicas in offset_states[offset] is responsible for process query
|
||||
/// with setting parallel_replica_offset = offset. In common situations
|
||||
/// replica_states[offset].replicas.size() = 1 (like in MultiplexedConnections).
|
||||
std::vector<OffsetState> offset_states;
|
||||
|
||||
/// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas).
|
||||
std::unordered_map<int, ReplicaLocation> fd_to_replica_location;
|
||||
|
||||
/// Map receive data timeout file descriptor to replica location.
|
||||
std::unordered_map<int, ReplicaLocation> timeout_fd_to_replica_location;
|
||||
|
||||
/// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from
|
||||
/// the replica, we push it's offset to this queue and start trying to get
|
||||
/// new replica.
|
||||
std::queue<int> offsets_queue;
|
||||
|
||||
/// The current number of valid connections to the replicas of this shard.
|
||||
size_t active_connection_count;
|
||||
|
||||
/// We count offsets in which we can't change replica anymore,
|
||||
/// it's needed to cancel choosing new replicas when we
|
||||
/// disabled replica changing in all offsets.
|
||||
size_t offsets_with_disabled_changing_replica;
|
||||
|
||||
Pipeline pipeline_for_new_replicas;
|
||||
|
||||
/// New replica may not support two-level aggregation due to version incompatibility.
|
||||
/// If we didn't disabled it, we need to skip this replica.
|
||||
bool disable_two_level_aggregation = false;
|
||||
|
||||
/// We will save replica with last received packet
|
||||
/// (except cases when packet type is EndOfStream or Exception)
|
||||
/// to resume it's packet receiver when new packet is needed.
|
||||
std::optional<ReplicaLocation> replica_with_last_received_packet;
|
||||
|
||||
Packet last_received_packet;
|
||||
|
||||
Epoll epoll;
|
||||
const Settings & settings;
|
||||
ThrottlerPtr throttler;
|
||||
bool sent_query = false;
|
||||
bool cancelled = false;
|
||||
|
||||
mutable std::mutex cancel_mutex;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
387
src/Client/HedgedConnectionsFactory.cpp
Normal file
387
src/Client/HedgedConnectionsFactory.cpp
Normal file
@ -0,0 +1,387 @@
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <Client/HedgedConnectionsFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ALL_CONNECTION_TRIES_FAILED;
|
||||
extern const int ALL_REPLICAS_ARE_STALE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::HedgedConnectionsFactory(
|
||||
const ConnectionPoolWithFailoverPtr & pool_,
|
||||
const Settings * settings_,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
std::shared_ptr<QualifiedTableName> table_to_check_)
|
||||
: pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("HedgedConnectionsFactory"))
|
||||
{
|
||||
shuffled_pools = pool->getShuffledPools(settings);
|
||||
for (size_t i = 0; i != shuffled_pools.size(); ++i)
|
||||
replicas.emplace_back(ConnectionEstablisherAsync(shuffled_pools[i].pool, &timeouts, settings, log, table_to_check.get()));
|
||||
|
||||
max_tries
|
||||
= (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES});
|
||||
|
||||
fallback_to_stale_replicas = settings && settings->fallback_to_stale_replicas_for_distributed_queries;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::~HedgedConnectionsFactory()
|
||||
{
|
||||
pool->updateSharedError(shuffled_pools);
|
||||
}
|
||||
|
||||
std::vector<Connection *> HedgedConnectionsFactory::getManyConnections(PoolMode pool_mode)
|
||||
{
|
||||
size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1;
|
||||
|
||||
size_t max_entries;
|
||||
switch (pool_mode)
|
||||
{
|
||||
case PoolMode::GET_ALL:
|
||||
{
|
||||
min_entries = shuffled_pools.size();
|
||||
max_entries = shuffled_pools.size();
|
||||
break;
|
||||
}
|
||||
case PoolMode::GET_ONE:
|
||||
{
|
||||
max_entries = 1;
|
||||
break;
|
||||
}
|
||||
case PoolMode::GET_MANY:
|
||||
{
|
||||
max_entries = settings ? size_t(settings->max_parallel_replicas) : 1;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<Connection *> connections;
|
||||
connections.reserve(max_entries);
|
||||
Connection * connection = nullptr;
|
||||
|
||||
/// Try to start establishing connections with max_entries replicas.
|
||||
for (size_t i = 0; i != max_entries; ++i)
|
||||
{
|
||||
++requested_connections_count;
|
||||
State state = startNewConnectionImpl(connection);
|
||||
if (state == State::READY)
|
||||
connections.push_back(connection);
|
||||
if (state == State::CANNOT_CHOOSE)
|
||||
break;
|
||||
}
|
||||
|
||||
/// Process connections until we get enough READY connections
|
||||
/// (work asynchronously with all connections we started).
|
||||
/// TODO: when we get GET_ALL mode we can start reading packets from ready
|
||||
/// TODO: connection as soon as we got it, not even waiting for the others.
|
||||
while (connections.size() < max_entries)
|
||||
{
|
||||
/// Set blocking = true to avoid busy-waiting here.
|
||||
auto state = waitForReadyConnectionsImpl(/*blocking = */true, connection);
|
||||
if (state == State::READY)
|
||||
connections.push_back(connection);
|
||||
else if (state == State::CANNOT_CHOOSE)
|
||||
{
|
||||
if (connections.size() >= min_entries)
|
||||
break;
|
||||
|
||||
/// Determine the reason of not enough replicas.
|
||||
if (!fallback_to_stale_replicas && up_to_date_count < min_entries)
|
||||
throw Exception(
|
||||
"Could not find enough connections to up-to-date replicas. Got: " + std::to_string(connections.size())
|
||||
+ ", needed: " + std::to_string(min_entries),
|
||||
DB::ErrorCodes::ALL_REPLICAS_ARE_STALE);
|
||||
if (usable_count < min_entries)
|
||||
throw NetException(
|
||||
"All connection tries failed. Log: \n\n" + fail_messages + "\n",
|
||||
DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
|
||||
|
||||
throw Exception("Unknown reason of not enough replicas.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
return connections;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Connection *& connection_out)
|
||||
{
|
||||
++requested_connections_count;
|
||||
State state = startNewConnectionImpl(connection_out);
|
||||
/// If we cannot start new connection but there are connections in epoll, return NOT_READY.
|
||||
if (state == State::CANNOT_CHOOSE && !epoll.empty())
|
||||
state = State::NOT_READY;
|
||||
|
||||
return state;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(Connection *& connection_out)
|
||||
{
|
||||
return waitForReadyConnectionsImpl(false, connection_out);
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnectionsImpl(bool blocking, Connection *& connection_out)
|
||||
{
|
||||
State state = processEpollEvents(blocking, connection_out);
|
||||
if (state != State::CANNOT_CHOOSE)
|
||||
return state;
|
||||
|
||||
/// We reach this point only if there was no free up to date replica.
|
||||
/// We will try to use usable replica.
|
||||
|
||||
/// Check if we are not allowed to use usable replicas or there is no even a free usable replica.
|
||||
if (!fallback_to_stale_replicas)
|
||||
return State::CANNOT_CHOOSE;
|
||||
|
||||
return setBestUsableReplica(connection_out);
|
||||
}
|
||||
|
||||
int HedgedConnectionsFactory::getNextIndex()
|
||||
{
|
||||
/// Check if there is no free replica.
|
||||
if (entries_count + replicas_in_process_count + failed_pools_count >= shuffled_pools.size())
|
||||
return -1;
|
||||
|
||||
/// Check if it's the first time.
|
||||
if (last_used_index == -1)
|
||||
{
|
||||
last_used_index = 0;
|
||||
return 0;
|
||||
}
|
||||
|
||||
bool finish = false;
|
||||
int next_index = last_used_index;
|
||||
while (!finish)
|
||||
{
|
||||
next_index = (next_index + 1) % shuffled_pools.size();
|
||||
|
||||
/// Check if we can try this replica.
|
||||
if (replicas[next_index].connection_establisher.getResult().entry.isNull()
|
||||
&& (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries))
|
||||
finish = true;
|
||||
|
||||
/// If we made a complete round, there is no replica to connect.
|
||||
else if (next_index == last_used_index)
|
||||
return -1;
|
||||
}
|
||||
|
||||
last_used_index = next_index;
|
||||
return next_index;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnectionImpl(Connection *& connection_out)
|
||||
{
|
||||
int index;
|
||||
State state;
|
||||
do
|
||||
{
|
||||
index = getNextIndex();
|
||||
if (index == -1)
|
||||
return State::CANNOT_CHOOSE;
|
||||
|
||||
state = resumeConnectionEstablisher(index, connection_out);
|
||||
}
|
||||
while (state == State::CANNOT_CHOOSE);
|
||||
|
||||
return state;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out)
|
||||
{
|
||||
int event_fd;
|
||||
while (!epoll.empty())
|
||||
{
|
||||
event_fd = getReadyFileDescriptor(blocking);
|
||||
|
||||
if (event_fd == -1)
|
||||
return State::NOT_READY;
|
||||
|
||||
if (fd_to_replica_index.contains(event_fd))
|
||||
{
|
||||
int index = fd_to_replica_index[event_fd];
|
||||
State state = resumeConnectionEstablisher(index, connection_out);
|
||||
if (state == State::NOT_READY)
|
||||
continue;
|
||||
|
||||
/// Connection establishing not in process now, remove all
|
||||
/// information about it from epoll.
|
||||
removeReplicaFromEpoll(index, event_fd);
|
||||
|
||||
if (state == State::READY)
|
||||
return state;
|
||||
}
|
||||
else if (timeout_fd_to_replica_index.contains(event_fd))
|
||||
replicas[timeout_fd_to_replica_index[event_fd]].change_replica_timeout.reset();
|
||||
else
|
||||
throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// We reach this point only if we need to start new connection
|
||||
/// (Special timeout expired or one of the previous connections failed).
|
||||
/// Return only if replica is ready.
|
||||
if (startNewConnectionImpl(connection_out) == State::READY)
|
||||
return State::READY;
|
||||
}
|
||||
|
||||
return State::CANNOT_CHOOSE;
|
||||
}
|
||||
|
||||
int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking)
|
||||
{
|
||||
epoll_event event;
|
||||
event.data.fd = -1;
|
||||
epoll.getManyReady(1, &event, blocking);
|
||||
return event.data.fd;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::resumeConnectionEstablisher(int index, Connection *& connection_out)
|
||||
{
|
||||
auto res = replicas[index].connection_establisher.resume();
|
||||
|
||||
if (std::holds_alternative<TryResult>(res))
|
||||
return processFinishedConnection(index, std::get<TryResult>(res), connection_out);
|
||||
|
||||
int fd = std::get<int>(res);
|
||||
if (!fd_to_replica_index.contains(fd))
|
||||
addNewReplicaToEpoll(index, fd);
|
||||
|
||||
return State::NOT_READY;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnection(int index, TryResult result, Connection *& connection_out)
|
||||
{
|
||||
const std::string & fail_message = replicas[index].connection_establisher.getFailMessage();
|
||||
if (!fail_message.empty())
|
||||
fail_messages += fail_message + "\n";
|
||||
|
||||
if (!result.entry.isNull())
|
||||
{
|
||||
++entries_count;
|
||||
|
||||
if (result.is_usable)
|
||||
{
|
||||
++usable_count;
|
||||
if (result.is_up_to_date)
|
||||
{
|
||||
++up_to_date_count;
|
||||
if (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry))
|
||||
{
|
||||
replicas[index].is_ready = true;
|
||||
++ready_replicas_count;
|
||||
connection_out = &*result.entry;
|
||||
return State::READY;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
ShuffledPool & shuffled_pool = shuffled_pools[index];
|
||||
LOG_WARNING(
|
||||
log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
|
||||
|
||||
shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1);
|
||||
|
||||
if (shuffled_pool.error_count >= max_tries)
|
||||
{
|
||||
++failed_pools_count;
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll);
|
||||
}
|
||||
}
|
||||
|
||||
return State::CANNOT_CHOOSE;
|
||||
}
|
||||
|
||||
void HedgedConnectionsFactory::stopChoosingReplicas()
|
||||
{
|
||||
for (auto & [fd, index] : fd_to_replica_index)
|
||||
{
|
||||
--replicas_in_process_count;
|
||||
epoll.remove(fd);
|
||||
replicas[index].connection_establisher.cancel();
|
||||
}
|
||||
|
||||
for (auto & [timeout_fd, index] : timeout_fd_to_replica_index)
|
||||
{
|
||||
replicas[index].change_replica_timeout.reset();
|
||||
epoll.remove(timeout_fd);
|
||||
}
|
||||
|
||||
fd_to_replica_index.clear();
|
||||
timeout_fd_to_replica_index.clear();
|
||||
}
|
||||
|
||||
void HedgedConnectionsFactory::addNewReplicaToEpoll(int index, int fd)
|
||||
{
|
||||
++replicas_in_process_count;
|
||||
epoll.add(fd);
|
||||
fd_to_replica_index[fd] = index;
|
||||
|
||||
/// Add timeout for changing replica.
|
||||
replicas[index].change_replica_timeout.setRelative(timeouts.hedged_connection_timeout);
|
||||
epoll.add(replicas[index].change_replica_timeout.getDescriptor());
|
||||
timeout_fd_to_replica_index[replicas[index].change_replica_timeout.getDescriptor()] = index;
|
||||
}
|
||||
|
||||
void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd)
|
||||
{
|
||||
--replicas_in_process_count;
|
||||
epoll.remove(fd);
|
||||
fd_to_replica_index.erase(fd);
|
||||
|
||||
replicas[index].change_replica_timeout.reset();
|
||||
epoll.remove(replicas[index].change_replica_timeout.getDescriptor());
|
||||
timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor());
|
||||
}
|
||||
|
||||
int HedgedConnectionsFactory::numberOfProcessingReplicas() const
|
||||
{
|
||||
if (epoll.empty())
|
||||
return 0;
|
||||
|
||||
return requested_connections_count - ready_replicas_count;
|
||||
}
|
||||
|
||||
HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out)
|
||||
{
|
||||
std::vector<int> indexes;
|
||||
for (size_t i = 0; i != replicas.size(); ++i)
|
||||
{
|
||||
/// Don't add unusable, failed replicas and replicas that are ready or in process.
|
||||
TryResult result = replicas[i].connection_establisher.getResult();
|
||||
if (!result.entry.isNull()
|
||||
&& result.is_usable
|
||||
&& !replicas[i].is_ready
|
||||
&& (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry)))
|
||||
indexes.push_back(i);
|
||||
}
|
||||
|
||||
if (indexes.empty())
|
||||
return State::CANNOT_CHOOSE;
|
||||
|
||||
/// Sort replicas by staleness.
|
||||
std::stable_sort(
|
||||
indexes.begin(),
|
||||
indexes.end(),
|
||||
[&](size_t lhs, size_t rhs)
|
||||
{
|
||||
return replicas[lhs].connection_establisher.getResult().staleness < replicas[rhs].connection_establisher.getResult().staleness;
|
||||
});
|
||||
|
||||
replicas[indexes[0]].is_ready = true;
|
||||
TryResult result = replicas[indexes[0]].connection_establisher.getResult();
|
||||
connection_out = &*result.entry;
|
||||
return State::READY;
|
||||
}
|
||||
|
||||
bool HedgedConnectionsFactory::isTwoLevelAggregationIncompatible(Connection * connection)
|
||||
{
|
||||
return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD;
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
158
src/Client/HedgedConnectionsFactory.h
Normal file
158
src/Client/HedgedConnectionsFactory.h
Normal file
@ -0,0 +1,158 @@
|
||||
#pragma once
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <Common/TimerDescriptor.h>
|
||||
#include <Common/Epoll.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/Fiber.h>
|
||||
#include <Client/ConnectionEstablisher.h>
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <unordered_map>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Class for establishing hedged connections with replicas.
|
||||
* The process of establishing connection is divided on stages, on each stage if
|
||||
* replica doesn't respond for a long time, we start establishing connection with
|
||||
* the next replica, without cancelling working with previous one.
|
||||
* It works with multiple replicas simultaneously without blocking by using epoll.
|
||||
*/
|
||||
class HedgedConnectionsFactory
|
||||
{
|
||||
public:
|
||||
using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool;
|
||||
using TryResult = PoolWithFailoverBase<IConnectionPool>::TryResult;
|
||||
|
||||
enum class State
|
||||
{
|
||||
READY,
|
||||
NOT_READY,
|
||||
CANNOT_CHOOSE,
|
||||
};
|
||||
|
||||
struct ReplicaStatus
|
||||
{
|
||||
explicit ReplicaStatus(ConnectionEstablisherAsync connection_stablisher_) : connection_establisher(std::move(connection_stablisher_))
|
||||
{
|
||||
}
|
||||
|
||||
ConnectionEstablisherAsync connection_establisher;
|
||||
TimerDescriptor change_replica_timeout;
|
||||
bool is_ready = false;
|
||||
};
|
||||
|
||||
HedgedConnectionsFactory(const ConnectionPoolWithFailoverPtr & pool_,
|
||||
const Settings * settings_,
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr);
|
||||
|
||||
/// Create and return active connections according to pool_mode.
|
||||
std::vector<Connection *> getManyConnections(PoolMode pool_mode);
|
||||
|
||||
/// Try to get connection to the new replica without blocking. Process all current events in epoll (connections, timeouts),
|
||||
/// Returned state might be READY (connection established successfully),
|
||||
/// NOT_READY (there are no ready events now) and CANNOT_CHOOSE (cannot produce new connection anymore).
|
||||
/// If state is READY, replica connection will be written in connection_out.
|
||||
State waitForReadyConnections(Connection *& connection_out);
|
||||
|
||||
State startNewConnection(Connection *& connection_out);
|
||||
|
||||
/// Stop working with all replicas that are not READY.
|
||||
void stopChoosingReplicas();
|
||||
|
||||
bool hasEventsInProcess() const { return !epoll.empty(); }
|
||||
|
||||
int getFileDescriptor() const { return epoll.getFileDescriptor(); }
|
||||
|
||||
const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; }
|
||||
|
||||
int numberOfProcessingReplicas() const;
|
||||
|
||||
/// Tell Factory to not return connections with two level aggregation incompatibility.
|
||||
void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; }
|
||||
|
||||
~HedgedConnectionsFactory();
|
||||
|
||||
private:
|
||||
State waitForReadyConnectionsImpl(bool blocking, Connection *& connection_out);
|
||||
|
||||
/// Try to start establishing connection to the new replica. Return
|
||||
/// the index of the new replica or -1 if cannot start new connection.
|
||||
State startNewConnectionImpl(Connection *& connection_out);
|
||||
|
||||
/// Find an index of the next free replica to start connection.
|
||||
/// Return -1 if there is no free replica.
|
||||
int getNextIndex();
|
||||
|
||||
int getReadyFileDescriptor(bool blocking);
|
||||
|
||||
void processFailedConnection(int index, const std::string & fail_message);
|
||||
|
||||
State resumeConnectionEstablisher(int index, Connection *& connection_out);
|
||||
|
||||
State processFinishedConnection(int index, TryResult result, Connection *& connection_out);
|
||||
|
||||
void removeReplicaFromEpoll(int index, int fd);
|
||||
|
||||
void addNewReplicaToEpoll(int index, int fd);
|
||||
|
||||
/// Return NOT_READY state if there is no ready events, READY if replica is ready
|
||||
/// and CANNOT_CHOOSE if there is no more events in epoll.
|
||||
State processEpollEvents(bool blocking, Connection *& connection_out);
|
||||
|
||||
State setBestUsableReplica(Connection *& connection_out);
|
||||
|
||||
bool isTwoLevelAggregationIncompatible(Connection * connection);
|
||||
|
||||
const ConnectionPoolWithFailoverPtr pool;
|
||||
const Settings * settings;
|
||||
const ConnectionTimeouts timeouts;
|
||||
|
||||
std::vector<ShuffledPool> shuffled_pools;
|
||||
std::vector<ReplicaStatus> replicas;
|
||||
|
||||
/// Map socket file descriptor to replica index.
|
||||
std::unordered_map<int, int> fd_to_replica_index;
|
||||
|
||||
/// Map timeout for changing replica to replica index.
|
||||
std::unordered_map<int, int> timeout_fd_to_replica_index;
|
||||
|
||||
/// If this flag is true, don't return connections with
|
||||
/// two level aggregation incompatibility
|
||||
bool skip_replicas_with_two_level_aggregation_incompatibility = false;
|
||||
|
||||
std::shared_ptr<QualifiedTableName> table_to_check;
|
||||
int last_used_index = -1;
|
||||
bool fallback_to_stale_replicas;
|
||||
Epoll epoll;
|
||||
Poco::Logger * log;
|
||||
std::string fail_messages;
|
||||
|
||||
/// The maximum number of attempts to connect to replicas.
|
||||
size_t max_tries;
|
||||
/// Total number of established connections.
|
||||
size_t entries_count = 0;
|
||||
/// The number of established connections that are usable.
|
||||
size_t usable_count = 0;
|
||||
/// The number of established connections that are up to date.
|
||||
size_t up_to_date_count = 0;
|
||||
/// The number of failed connections (replica is considered failed after max_tries attempts to connect).
|
||||
size_t failed_pools_count= 0;
|
||||
|
||||
/// The number of replicas that are in process of connection.
|
||||
size_t replicas_in_process_count = 0;
|
||||
/// The number of ready replicas (replica is considered ready when it's
|
||||
/// connection returns outside).
|
||||
size_t ready_replicas_count = 0;
|
||||
|
||||
/// The number of requested in startNewConnection replicas (it's needed for
|
||||
/// checking the number of requested replicas that are still in process).
|
||||
size_t requested_connections_count = 0;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
60
src/Client/IConnections.h
Normal file
60
src/Client/IConnections.h
Normal file
@ -0,0 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <Client/Connection.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Base class for working with multiple replicas (connections)
|
||||
/// from one shard within a single thread
|
||||
class IConnections : boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// Send all scalars to replicas.
|
||||
virtual void sendScalarsData(Scalars & data) = 0;
|
||||
/// Send all content of external tables to replicas.
|
||||
virtual void sendExternalTablesData(std::vector<ExternalTablesData> & data) = 0;
|
||||
|
||||
/// Send request to replicas.
|
||||
virtual void sendQuery(
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & query,
|
||||
const String & query_id,
|
||||
UInt64 stage,
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data) = 0;
|
||||
|
||||
/// Get packet from any replica.
|
||||
virtual Packet receivePacket() = 0;
|
||||
|
||||
/// Version of `receivePacket` function without locking.
|
||||
virtual Packet receivePacketUnlocked(AsyncCallback async_callback) = 0;
|
||||
|
||||
/// Break all active connections.
|
||||
virtual void disconnect() = 0;
|
||||
|
||||
/// Send a request to replicas to cancel the request
|
||||
virtual void sendCancel() = 0;
|
||||
|
||||
/// Send parts' uuids to replicas to exclude them from query processing
|
||||
virtual void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids) = 0;
|
||||
|
||||
/** On each replica, read and skip all packets to EndOfStream or Exception.
|
||||
* Returns EndOfStream if no exception has been received. Otherwise
|
||||
* returns the last received packet of type Exception.
|
||||
*/
|
||||
virtual Packet drain() = 0;
|
||||
|
||||
/// Get the replica addresses as a string.
|
||||
virtual std::string dumpAddresses() const = 0;
|
||||
|
||||
/// Returns the number of replicas.
|
||||
virtual size_t size() const = 0;
|
||||
|
||||
/// Check if there are any valid replicas.
|
||||
virtual bool hasActiveConnections() const = 0;
|
||||
|
||||
virtual ~IConnections() = default;
|
||||
};
|
||||
|
||||
}
|
@ -158,7 +158,7 @@ void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector<UUID> & uuid
|
||||
Packet MultiplexedConnections::receivePacket()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
Packet packet = receivePacketUnlocked();
|
||||
Packet packet = receivePacketUnlocked({});
|
||||
return packet;
|
||||
}
|
||||
|
||||
@ -206,7 +206,7 @@ Packet MultiplexedConnections::drain()
|
||||
|
||||
while (hasActiveConnections())
|
||||
{
|
||||
Packet packet = receivePacketUnlocked();
|
||||
Packet packet = receivePacketUnlocked({});
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
@ -253,7 +253,7 @@ std::string MultiplexedConnections::dumpAddressesUnlocked() const
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
Packet MultiplexedConnections::receivePacketUnlocked(std::function<void(Poco::Net::Socket &)> async_callback)
|
||||
Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callback)
|
||||
{
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -265,7 +265,11 @@ Packet MultiplexedConnections::receivePacketUnlocked(std::function<void(Poco::Ne
|
||||
if (current_connection == nullptr)
|
||||
throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA);
|
||||
|
||||
Packet packet = current_connection->receivePacket(std::move(async_callback));
|
||||
Packet packet;
|
||||
{
|
||||
AsyncCallbackSetter async_setter(current_connection, std::move(async_callback));
|
||||
packet = current_connection->receivePacket();
|
||||
}
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Client/Connection.h>
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <Client/IConnections.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -16,7 +17,7 @@ namespace DB
|
||||
*
|
||||
* The interface is almost the same as Connection.
|
||||
*/
|
||||
class MultiplexedConnections final : private boost::noncopyable
|
||||
class MultiplexedConnections final : public IConnections
|
||||
{
|
||||
public:
|
||||
/// Accepts ready connection.
|
||||
@ -27,52 +28,38 @@ public:
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const Settings & settings_, const ThrottlerPtr & throttler_);
|
||||
|
||||
/// Send all scalars to replicas.
|
||||
void sendScalarsData(Scalars & data);
|
||||
/// Send all content of external tables to replicas.
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
|
||||
void sendScalarsData(Scalars & data) override;
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data) override;
|
||||
|
||||
/// Send request to replicas.
|
||||
void sendQuery(
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & query,
|
||||
const String & query_id,
|
||||
UInt64 stage,
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data);
|
||||
bool with_pending_data) override;
|
||||
|
||||
/// Get packet from any replica.
|
||||
Packet receivePacket();
|
||||
Packet receivePacket() override;
|
||||
|
||||
/// Break all active connections.
|
||||
void disconnect();
|
||||
void disconnect() override;
|
||||
|
||||
/// Send a request to the replica to cancel the request
|
||||
void sendCancel();
|
||||
void sendCancel() override;
|
||||
|
||||
/// Send parts' uuids to replicas to exclude them from query processing
|
||||
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids);
|
||||
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids) override;
|
||||
|
||||
/** On each replica, read and skip all packets to EndOfStream or Exception.
|
||||
* Returns EndOfStream if no exception has been received. Otherwise
|
||||
* returns the last received packet of type Exception.
|
||||
*/
|
||||
Packet drain();
|
||||
Packet drain() override;
|
||||
|
||||
/// Get the replica addresses as a string.
|
||||
std::string dumpAddresses() const;
|
||||
std::string dumpAddresses() const override;
|
||||
|
||||
/// Returns the number of replicas.
|
||||
/// Without locking, because sendCancel() does not change this number.
|
||||
size_t size() const { return replica_states.size(); }
|
||||
size_t size() const override { return replica_states.size(); }
|
||||
|
||||
/// Check if there are any valid replicas.
|
||||
/// Without locking, because sendCancel() does not change the state of the replicas.
|
||||
bool hasActiveConnections() const { return active_connection_count > 0; }
|
||||
bool hasActiveConnections() const override { return active_connection_count > 0; }
|
||||
|
||||
private:
|
||||
/// Internal version of `receivePacket` function without locking.
|
||||
Packet receivePacketUnlocked(std::function<void(Poco::Net::Socket &)> async_callback = {});
|
||||
Packet receivePacketUnlocked(AsyncCallback async_callback) override;
|
||||
|
||||
/// Internal version of `dumpAddresses` function without locking.
|
||||
std::string dumpAddressesUnlocked() const;
|
||||
|
161
src/Client/PacketReceiver.h
Normal file
161
src/Client/PacketReceiver.h
Normal file
@ -0,0 +1,161 @@
|
||||
#pragma once
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <variant>
|
||||
|
||||
#include <Client/IConnections.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/Fiber.h>
|
||||
#include <Common/Epoll.h>
|
||||
#include <Common/TimerDescriptor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Class for nonblocking packet receiving. It runs connection->receivePacket
|
||||
/// in fiber and sets special read callback which is called when
|
||||
/// reading from socket blocks. When read callback is called,
|
||||
/// socket and receive timeout are added in epoll and execution returns to the main program.
|
||||
/// So, you can poll this epoll file descriptor to determine when to resume
|
||||
/// packet receiving.
|
||||
class PacketReceiver
|
||||
{
|
||||
public:
|
||||
explicit PacketReceiver(Connection * connection_) : connection(connection_)
|
||||
{
|
||||
epoll.add(receive_timeout.getDescriptor());
|
||||
epoll.add(connection->getSocket()->impl()->sockfd());
|
||||
|
||||
fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this});
|
||||
}
|
||||
|
||||
/// Resume packet receiving.
|
||||
std::variant<int, Packet, Poco::Timespan> resume()
|
||||
{
|
||||
/// If there is no pending data, check receive timeout.
|
||||
if (!connection->hasReadPendingData() && !checkReceiveTimeout())
|
||||
{
|
||||
/// Receive timeout expired.
|
||||
return Poco::Timespan();
|
||||
}
|
||||
|
||||
/// Resume fiber.
|
||||
fiber = std::move(fiber).resume();
|
||||
if (exception)
|
||||
std::rethrow_exception(std::move(exception));
|
||||
|
||||
if (is_read_in_process)
|
||||
return epoll.getFileDescriptor();
|
||||
|
||||
/// Receiving packet was finished.
|
||||
return std::move(packet);
|
||||
}
|
||||
|
||||
void cancel()
|
||||
{
|
||||
Fiber to_destroy = std::move(fiber);
|
||||
connection = nullptr;
|
||||
}
|
||||
|
||||
int getFileDescriptor() const { return epoll.getFileDescriptor(); }
|
||||
|
||||
private:
|
||||
/// When epoll file descriptor is ready, check if it's an expired timeout.
|
||||
/// Return false if receive timeout expired and socket is not ready, return true otherwise.
|
||||
bool checkReceiveTimeout()
|
||||
{
|
||||
bool is_socket_ready = false;
|
||||
bool is_receive_timeout_expired = false;
|
||||
|
||||
epoll_event events[2];
|
||||
events[0].data.fd = events[1].data.fd = -1;
|
||||
size_t ready_count = epoll.getManyReady(2, events, true);
|
||||
|
||||
for (size_t i = 0; i != ready_count; ++i)
|
||||
{
|
||||
if (events[i].data.fd == connection->getSocket()->impl()->sockfd())
|
||||
is_socket_ready = true;
|
||||
if (events[i].data.fd == receive_timeout.getDescriptor())
|
||||
is_receive_timeout_expired = true;
|
||||
}
|
||||
|
||||
if (is_receive_timeout_expired && !is_socket_ready)
|
||||
{
|
||||
receive_timeout.reset();
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
struct Routine
|
||||
{
|
||||
PacketReceiver & receiver;
|
||||
|
||||
struct ReadCallback
|
||||
{
|
||||
PacketReceiver & receiver;
|
||||
Fiber & sink;
|
||||
|
||||
void operator()(int, const Poco::Timespan & timeout, const std::string &)
|
||||
{
|
||||
receiver.receive_timeout.setRelative(timeout);
|
||||
receiver.is_read_in_process = true;
|
||||
sink = std::move(sink).resume();
|
||||
receiver.is_read_in_process = false;
|
||||
receiver.receive_timeout.reset();
|
||||
}
|
||||
};
|
||||
|
||||
Fiber operator()(Fiber && sink)
|
||||
{
|
||||
try
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
{
|
||||
AsyncCallbackSetter async_setter(receiver.connection, ReadCallback{receiver, sink});
|
||||
receiver.packet = receiver.connection->receivePacket();
|
||||
}
|
||||
sink = std::move(sink).resume();
|
||||
}
|
||||
|
||||
}
|
||||
catch (const boost::context::detail::forced_unwind &)
|
||||
{
|
||||
/// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited
|
||||
/// It should not be caught or it will segfault.
|
||||
/// Other exceptions must be caught
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
receiver.exception = std::current_exception();
|
||||
}
|
||||
|
||||
return std::move(sink);
|
||||
}
|
||||
};
|
||||
|
||||
Connection * connection;
|
||||
Packet packet;
|
||||
|
||||
Fiber fiber;
|
||||
FiberStack fiber_stack;
|
||||
|
||||
/// We use timer descriptor for checking socket receive timeout.
|
||||
TimerDescriptor receive_timeout;
|
||||
|
||||
/// In read callback we add socket file descriptor and timer descriptor with receive timeout
|
||||
/// in epoll, so we can return epoll file descriptor outside for polling.
|
||||
Epoll epoll;
|
||||
|
||||
/// If and exception occurred in fiber resume, we save it and rethrow.
|
||||
std::exception_ptr exception;
|
||||
|
||||
bool is_read_in_process = false;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
@ -11,7 +11,10 @@ PEERDIR(
|
||||
|
||||
SRCS(
|
||||
Connection.cpp
|
||||
ConnectionEstablisher.cpp
|
||||
ConnectionPoolWithFailover.cpp
|
||||
HedgedConnections.cpp
|
||||
HedgedConnectionsFactory.cpp
|
||||
MultiplexedConnections.cpp
|
||||
TimeoutSetter.cpp
|
||||
|
||||
|
86
src/Common/Epoll.cpp
Normal file
86
src/Common/Epoll.cpp
Normal file
@ -0,0 +1,86 @@
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include "Epoll.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <unistd.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EPOLL_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
Epoll::Epoll() : events_count(0)
|
||||
{
|
||||
epoll_fd = epoll_create1(0);
|
||||
if (epoll_fd == -1)
|
||||
throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR);
|
||||
}
|
||||
|
||||
Epoll::Epoll(Epoll && other) : epoll_fd(other.epoll_fd), events_count(other.events_count.load())
|
||||
{
|
||||
other.epoll_fd = -1;
|
||||
}
|
||||
|
||||
Epoll & Epoll::operator=(Epoll && other)
|
||||
{
|
||||
epoll_fd = other.epoll_fd;
|
||||
other.epoll_fd = -1;
|
||||
events_count.store(other.events_count.load());
|
||||
return *this;
|
||||
}
|
||||
|
||||
void Epoll::add(int fd, void * ptr)
|
||||
{
|
||||
epoll_event event;
|
||||
event.events = EPOLLIN | EPOLLPRI;
|
||||
if (ptr)
|
||||
event.data.ptr = ptr;
|
||||
else
|
||||
event.data.fd = fd;
|
||||
|
||||
++events_count;
|
||||
|
||||
if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1)
|
||||
throwFromErrno("Cannot add new descriptor to epoll", DB::ErrorCodes::EPOLL_ERROR);
|
||||
}
|
||||
|
||||
void Epoll::remove(int fd)
|
||||
{
|
||||
--events_count;
|
||||
|
||||
if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1)
|
||||
throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR);
|
||||
}
|
||||
|
||||
size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking) const
|
||||
{
|
||||
if (events_count == 0)
|
||||
throw Exception("There is no events in epoll", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
int ready_size;
|
||||
int timeout = blocking ? -1 : 0;
|
||||
do
|
||||
{
|
||||
ready_size = epoll_wait(epoll_fd, events_out, max_events, timeout);
|
||||
|
||||
if (ready_size == -1 && errno != EINTR)
|
||||
throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR);
|
||||
}
|
||||
while (ready_size <= 0 && (ready_size != 0 || blocking));
|
||||
|
||||
return ready_size;
|
||||
}
|
||||
|
||||
Epoll::~Epoll()
|
||||
{
|
||||
if (epoll_fd != -1)
|
||||
close(epoll_fd);
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
54
src/Common/Epoll.h
Normal file
54
src/Common/Epoll.h
Normal file
@ -0,0 +1,54 @@
|
||||
#pragma once
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include <sys/epoll.h>
|
||||
#include <vector>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using AsyncCallback = std::function<void(int, const Poco::Timespan &, const std::string &)>;
|
||||
|
||||
class Epoll
|
||||
{
|
||||
public:
|
||||
Epoll();
|
||||
|
||||
Epoll(const Epoll &) = delete;
|
||||
Epoll & operator=(const Epoll &) = delete;
|
||||
|
||||
Epoll & operator=(Epoll && other);
|
||||
Epoll(Epoll && other);
|
||||
|
||||
/// Add new file descriptor to epoll. If ptr set to nullptr, epoll_event.data.fd = fd,
|
||||
/// otherwise epoll_event.data.ptr = ptr.
|
||||
void add(int fd, void * ptr = nullptr);
|
||||
|
||||
/// Remove file descriptor to epoll.
|
||||
void remove(int fd);
|
||||
|
||||
/// Get events from epoll. Events are written in events_out, this function returns an amount of ready events.
|
||||
/// If blocking is false and there are no ready events,
|
||||
/// return empty vector, otherwise wait for ready events.
|
||||
size_t getManyReady(int max_events, epoll_event * events_out, bool blocking) const;
|
||||
|
||||
int getFileDescriptor() const { return epoll_fd; }
|
||||
|
||||
int size() const { return events_count; }
|
||||
|
||||
bool empty() const { return events_count == 0; }
|
||||
|
||||
const std::string & getDescription() const { return fd_description; }
|
||||
|
||||
~Epoll();
|
||||
|
||||
private:
|
||||
int epoll_fd;
|
||||
std::atomic<int> events_count;
|
||||
const std::string fd_description = "epoll";
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
@ -539,7 +539,8 @@
|
||||
M(570, DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD) \
|
||||
M(571, DATABASE_REPLICATION_FAILED) \
|
||||
M(572, TOO_MANY_QUERY_PLAN_OPTIMIZATIONS) \
|
||||
M(573, UNKNOWN_SNAPSHOT) \
|
||||
M(573, EPOLL_ERROR) \
|
||||
M(574, UNKNOWN_SNAPSHOT) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -93,6 +93,18 @@ public:
|
||||
double staleness = 0.0; /// Helps choosing the "least stale" option when all replicas are stale.
|
||||
};
|
||||
|
||||
struct PoolState;
|
||||
|
||||
using PoolStates = std::vector<PoolState>;
|
||||
|
||||
struct ShuffledPool
|
||||
{
|
||||
NestedPool * pool{};
|
||||
const PoolState * state{};
|
||||
size_t index = 0;
|
||||
size_t error_count = 0;
|
||||
};
|
||||
|
||||
/// This functor must be provided by a client. It must perform a single try that takes a connection
|
||||
/// from the provided pool and checks that it is good.
|
||||
using TryGetEntryFunc = std::function<TryResult(NestedPool & pool, std::string & fail_message)>;
|
||||
@ -113,9 +125,6 @@ public:
|
||||
const GetPriorityFunc & get_priority = GetPriorityFunc());
|
||||
|
||||
protected:
|
||||
struct PoolState;
|
||||
|
||||
using PoolStates = std::vector<PoolState>;
|
||||
|
||||
/// Returns a single connection.
|
||||
Entry get(size_t max_ignored_errors, bool fallback_to_stale_replicas,
|
||||
@ -124,6 +133,10 @@ protected:
|
||||
/// This function returns a copy of pool states to avoid race conditions when modifying shared pool states.
|
||||
PoolStates updatePoolStates(size_t max_ignored_errors);
|
||||
|
||||
std::vector<ShuffledPool> getShuffledPools(size_t max_ignored_errors, const GetPriorityFunc & get_priority);
|
||||
|
||||
inline void updateSharedErrorCounts(std::vector<ShuffledPool> & shuffled_pools);
|
||||
|
||||
auto getPoolExtendedStates() const
|
||||
{
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
@ -143,6 +156,46 @@ protected:
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
|
||||
template <typename TNestedPool>
|
||||
std::vector<typename PoolWithFailoverBase<TNestedPool>::ShuffledPool>
|
||||
PoolWithFailoverBase<TNestedPool>::getShuffledPools(
|
||||
size_t max_ignored_errors, const PoolWithFailoverBase::GetPriorityFunc & get_priority)
|
||||
{
|
||||
/// Update random numbers and error counts.
|
||||
PoolStates pool_states = updatePoolStates(max_ignored_errors);
|
||||
if (get_priority)
|
||||
{
|
||||
for (size_t i = 0; i < pool_states.size(); ++i)
|
||||
pool_states[i].priority = get_priority(i);
|
||||
}
|
||||
|
||||
/// Sort the pools into order in which they will be tried (based on respective PoolStates).
|
||||
std::vector<ShuffledPool> shuffled_pools;
|
||||
shuffled_pools.reserve(nested_pools.size());
|
||||
for (size_t i = 0; i < nested_pools.size(); ++i)
|
||||
shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0});
|
||||
std::sort(
|
||||
shuffled_pools.begin(), shuffled_pools.end(),
|
||||
[](const ShuffledPool & lhs, const ShuffledPool & rhs)
|
||||
{
|
||||
return PoolState::compare(*lhs.state, *rhs.state);
|
||||
});
|
||||
|
||||
return shuffled_pools;
|
||||
}
|
||||
|
||||
template <typename TNestedPool>
|
||||
inline void PoolWithFailoverBase<TNestedPool>::updateSharedErrorCounts(std::vector<ShuffledPool> & shuffled_pools)
|
||||
{
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
for (const ShuffledPool & pool: shuffled_pools)
|
||||
{
|
||||
auto & pool_state = shared_pool_states[pool.index];
|
||||
pool_state.error_count = std::min<UInt64>(max_error_cap, pool_state.error_count + pool.error_count);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TNestedPool>
|
||||
typename TNestedPool::Entry
|
||||
PoolWithFailoverBase<TNestedPool>::get(size_t max_ignored_errors, bool fallback_to_stale_replicas,
|
||||
@ -168,33 +221,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
|
||||
const TryGetEntryFunc & try_get_entry,
|
||||
const GetPriorityFunc & get_priority)
|
||||
{
|
||||
/// Update random numbers and error counts.
|
||||
PoolStates pool_states = updatePoolStates(max_ignored_errors);
|
||||
if (get_priority)
|
||||
{
|
||||
for (size_t i = 0; i < pool_states.size(); ++i)
|
||||
pool_states[i].priority = get_priority(i);
|
||||
}
|
||||
|
||||
struct ShuffledPool
|
||||
{
|
||||
NestedPool * pool{};
|
||||
const PoolState * state{};
|
||||
size_t index = 0;
|
||||
size_t error_count = 0;
|
||||
};
|
||||
|
||||
/// Sort the pools into order in which they will be tried (based on respective PoolStates).
|
||||
std::vector<ShuffledPool> shuffled_pools;
|
||||
shuffled_pools.reserve(nested_pools.size());
|
||||
for (size_t i = 0; i < nested_pools.size(); ++i)
|
||||
shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0});
|
||||
std::sort(
|
||||
shuffled_pools.begin(), shuffled_pools.end(),
|
||||
[](const ShuffledPool & lhs, const ShuffledPool & rhs)
|
||||
{
|
||||
return PoolState::compare(*lhs.state, *rhs.state);
|
||||
});
|
||||
std::vector<ShuffledPool> shuffled_pools = getShuffledPools(max_ignored_errors, get_priority);
|
||||
|
||||
/// We will try to get a connection from each pool until a connection is produced or max_tries is reached.
|
||||
std::vector<TryResult> try_results(shuffled_pools.size());
|
||||
@ -206,12 +233,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
|
||||
/// At exit update shared error counts with error counts occurred during this call.
|
||||
SCOPE_EXIT(
|
||||
{
|
||||
std::lock_guard lock(pool_states_mutex);
|
||||
for (const ShuffledPool & pool: shuffled_pools)
|
||||
{
|
||||
auto & pool_state = shared_pool_states[pool.index];
|
||||
pool_state.error_count = std::min<UInt64>(max_error_cap, pool_state.error_count + pool.error_count);
|
||||
}
|
||||
updateSharedErrorCounts(shuffled_pools);
|
||||
});
|
||||
|
||||
std::string fail_messages;
|
||||
|
@ -27,10 +27,16 @@ TimerDescriptor::TimerDescriptor(int clockid, int flags)
|
||||
throwFromErrno("Cannot set O_NONBLOCK for timer_fd", ErrorCodes::CANNOT_FCNTL);
|
||||
}
|
||||
|
||||
TimerDescriptor::TimerDescriptor(TimerDescriptor && other) : timer_fd(other.timer_fd)
|
||||
{
|
||||
other.timer_fd = -1;
|
||||
}
|
||||
|
||||
TimerDescriptor::~TimerDescriptor()
|
||||
{
|
||||
/// Do not check for result cause cannot throw exception.
|
||||
close(timer_fd);
|
||||
if (timer_fd != -1)
|
||||
close(timer_fd);
|
||||
}
|
||||
|
||||
void TimerDescriptor::reset() const
|
||||
@ -74,7 +80,7 @@ void TimerDescriptor::setRelative(const Poco::Timespan & timespan) const
|
||||
spec.it_interval.tv_nsec = 0;
|
||||
spec.it_interval.tv_sec = 0;
|
||||
spec.it_value.tv_sec = timespan.totalSeconds();
|
||||
spec.it_value.tv_nsec = timespan.useconds();
|
||||
spec.it_value.tv_nsec = timespan.useconds() * 1000;
|
||||
|
||||
if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr))
|
||||
throwFromErrno("Cannot set time for timer_fd", ErrorCodes::CANNOT_SET_TIMER_PERIOD);
|
||||
|
@ -12,12 +12,12 @@ private:
|
||||
int timer_fd;
|
||||
|
||||
public:
|
||||
explicit TimerDescriptor(int clockid, int flags);
|
||||
explicit TimerDescriptor(int clockid = CLOCK_MONOTONIC, int flags = 0);
|
||||
~TimerDescriptor();
|
||||
|
||||
TimerDescriptor(const TimerDescriptor &) = delete;
|
||||
TimerDescriptor & operator=(const TimerDescriptor &) = delete;
|
||||
TimerDescriptor(TimerDescriptor &&) = default;
|
||||
TimerDescriptor(TimerDescriptor && other);
|
||||
TimerDescriptor & operator=(TimerDescriptor &&) = default;
|
||||
|
||||
int getDescriptor() const { return timer_fd; }
|
||||
|
@ -39,6 +39,7 @@ SRCS(
|
||||
DNSResolver.cpp
|
||||
Dwarf.cpp
|
||||
Elf.cpp
|
||||
Epoll.cpp
|
||||
ErrorCodes.cpp
|
||||
Exception.cpp
|
||||
ExternalLoaderStatus.cpp
|
||||
|
@ -11,6 +11,9 @@
|
||||
#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS 100
|
||||
#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300
|
||||
#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300
|
||||
/// Timeouts for hedged requests.
|
||||
#define DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS 100
|
||||
#define DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC 2
|
||||
/// Timeout for synchronous request-result protocol call (like Ping or TablesStatus).
|
||||
#define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5
|
||||
#define DBMS_DEFAULT_POLL_INTERVAL 10
|
||||
|
@ -55,6 +55,10 @@ class IColumn;
|
||||
M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \
|
||||
M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \
|
||||
M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \
|
||||
M(Milliseconds, hedged_connection_timeout, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \
|
||||
M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \
|
||||
M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \
|
||||
M(Bool, allow_changing_replica_until_first_data_packet, false, "Allow HedgedConnections to change replica until receiving first data packet", 0) \
|
||||
M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \
|
||||
M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \
|
||||
M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \
|
||||
@ -215,6 +219,10 @@ class IColumn;
|
||||
M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \
|
||||
M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \
|
||||
\
|
||||
/** Settings for testing hedged requests */ \
|
||||
M(Int64, sleep_in_send_tables_status, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \
|
||||
M(Int64, sleep_in_send_data, 0, "Time to sleep in sending data in TCPHandler", 0) \
|
||||
\
|
||||
M(Bool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \
|
||||
M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \
|
||||
M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \
|
||||
|
@ -13,6 +13,8 @@
|
||||
#include <Interpreters/InternalTextLogsQueue.h>
|
||||
#include <IO/ConnectionTimeoutsContext.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Client/MultiplexedConnections.h>
|
||||
#include <Client/HedgedConnections.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
|
||||
namespace DB
|
||||
@ -31,23 +33,23 @@ RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_multiplexed_connections = [this, &connection, throttler]()
|
||||
create_connections = [this, &connection, throttler]()
|
||||
{
|
||||
return std::make_unique<MultiplexedConnections>(connection, context.getSettingsRef(), throttler);
|
||||
};
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_multiplexed_connections = [this, connections, throttler]() mutable
|
||||
create_connections = [this, connections_, throttler]() mutable
|
||||
{
|
||||
return std::make_unique<MultiplexedConnections>(
|
||||
std::move(connections), context.getSettingsRef(), throttler);
|
||||
std::move(connections_), context.getSettingsRef(), throttler);
|
||||
};
|
||||
}
|
||||
|
||||
@ -58,23 +60,34 @@ RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
create_multiplexed_connections = [this, pool, throttler]()
|
||||
create_connections = [this, pool, throttler]()->std::unique_ptr<IConnections>
|
||||
{
|
||||
const Settings & current_settings = context.getSettingsRef();
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
||||
std::vector<IConnectionPool::Entry> connections;
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
if (current_settings.use_hedged_requests)
|
||||
{
|
||||
std::shared_ptr<QualifiedTableName> table_to_check = nullptr;
|
||||
if (main_table)
|
||||
table_to_check = std::make_shared<QualifiedTableName>(main_table.getQualifiedName());
|
||||
|
||||
return std::make_unique<HedgedConnections>(pool, current_settings, timeouts, throttler, pool_mode, table_to_check);
|
||||
}
|
||||
#endif
|
||||
|
||||
std::vector<IConnectionPool::Entry> connection_entries;
|
||||
if (main_table)
|
||||
{
|
||||
auto try_results = pool->getManyChecked(timeouts, ¤t_settings, pool_mode, main_table.getQualifiedName());
|
||||
connections.reserve(try_results.size());
|
||||
connection_entries.reserve(try_results.size());
|
||||
for (auto & try_result : try_results)
|
||||
connections.emplace_back(std::move(try_result.entry));
|
||||
connection_entries.emplace_back(std::move(try_result.entry));
|
||||
}
|
||||
else
|
||||
connections = pool->getMany(timeouts, ¤t_settings, pool_mode);
|
||||
connection_entries = pool->getMany(timeouts, ¤t_settings, pool_mode);
|
||||
|
||||
return std::make_unique<MultiplexedConnections>(
|
||||
std::move(connections), current_settings, throttler);
|
||||
return std::make_unique<MultiplexedConnections>(std::move(connection_entries), current_settings, throttler);
|
||||
};
|
||||
}
|
||||
|
||||
@ -85,7 +98,7 @@ RemoteQueryExecutor::~RemoteQueryExecutor()
|
||||
* these connections did not remain hanging in the out-of-sync state.
|
||||
*/
|
||||
if (established || isQueryPending())
|
||||
multiplexed_connections->disconnect();
|
||||
connections->disconnect();
|
||||
}
|
||||
|
||||
/** If we receive a block with slightly different column types, or with excessive columns,
|
||||
@ -142,10 +155,10 @@ void RemoteQueryExecutor::sendQuery()
|
||||
if (sent_query)
|
||||
return;
|
||||
|
||||
multiplexed_connections = create_multiplexed_connections();
|
||||
connections = create_connections();
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
if (settings.skip_unavailable_shards && 0 == multiplexed_connections->size())
|
||||
if (settings.skip_unavailable_shards && 0 == connections->size())
|
||||
return;
|
||||
|
||||
/// Query cannot be canceled in the middle of the send query,
|
||||
@ -173,10 +186,10 @@ void RemoteQueryExecutor::sendQuery()
|
||||
{
|
||||
std::lock_guard lock(duplicated_part_uuids_mutex);
|
||||
if (!duplicated_part_uuids.empty())
|
||||
multiplexed_connections->sendIgnoredPartUUIDs(duplicated_part_uuids);
|
||||
connections->sendIgnoredPartUUIDs(duplicated_part_uuids);
|
||||
}
|
||||
|
||||
multiplexed_connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true);
|
||||
connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true);
|
||||
|
||||
established = false;
|
||||
sent_query = true;
|
||||
@ -192,7 +205,7 @@ Block RemoteQueryExecutor::read()
|
||||
{
|
||||
sendQuery();
|
||||
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size()))
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
return {};
|
||||
}
|
||||
|
||||
@ -201,7 +214,7 @@ Block RemoteQueryExecutor::read()
|
||||
if (was_cancelled)
|
||||
return Block();
|
||||
|
||||
Packet packet = multiplexed_connections->receivePacket();
|
||||
Packet packet = connections->receivePacket();
|
||||
|
||||
if (auto block = processPacket(std::move(packet)))
|
||||
return *block;
|
||||
@ -218,7 +231,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
||||
{
|
||||
sendQuery();
|
||||
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size()))
|
||||
if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
||||
return Block();
|
||||
}
|
||||
|
||||
@ -228,7 +241,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
||||
if (was_cancelled)
|
||||
return Block();
|
||||
|
||||
read_context = std::make_unique<ReadContext>(*multiplexed_connections);
|
||||
read_context = std::make_unique<ReadContext>(*connections);
|
||||
}
|
||||
|
||||
do
|
||||
@ -239,7 +252,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
||||
if (read_context->is_read_in_progress.load(std::memory_order_relaxed))
|
||||
{
|
||||
read_context->setTimer();
|
||||
return read_context->epoll_fd;
|
||||
return read_context->epoll.getFileDescriptor();
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -260,7 +273,7 @@ std::variant<Block, int> RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs
|
||||
{
|
||||
/// Cancel previous query and disconnect before retry.
|
||||
cancel(read_context);
|
||||
multiplexed_connections->disconnect();
|
||||
connections->disconnect();
|
||||
|
||||
/// Only resend once, otherwise throw an exception
|
||||
if (!resent_query)
|
||||
@ -300,7 +313,7 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
|
||||
break;
|
||||
|
||||
case Protocol::Server::EndOfStream:
|
||||
if (!multiplexed_connections->hasActiveConnections())
|
||||
if (!connections->hasActiveConnections())
|
||||
{
|
||||
finished = true;
|
||||
return Block();
|
||||
@ -342,7 +355,7 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
|
||||
got_unknown_packet_from_replica = true;
|
||||
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}",
|
||||
toString(packet.type),
|
||||
multiplexed_connections->dumpAddresses());
|
||||
connections->dumpAddresses());
|
||||
}
|
||||
|
||||
return {};
|
||||
@ -382,7 +395,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context)
|
||||
tryCancel("Cancelling query because enough data has been read", read_context);
|
||||
|
||||
/// Get the remaining packets so that there is no out of sync in the connections to the replicas.
|
||||
Packet packet = multiplexed_connections->drain();
|
||||
Packet packet = connections->drain();
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::EndOfStream:
|
||||
@ -404,7 +417,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context)
|
||||
got_unknown_packet_from_replica = true;
|
||||
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}",
|
||||
toString(packet.type),
|
||||
multiplexed_connections->dumpAddresses());
|
||||
connections->dumpAddresses());
|
||||
}
|
||||
}
|
||||
|
||||
@ -427,14 +440,14 @@ void RemoteQueryExecutor::cancel(std::unique_ptr<ReadContext> * read_context)
|
||||
|
||||
void RemoteQueryExecutor::sendScalars()
|
||||
{
|
||||
multiplexed_connections->sendScalarsData(scalars);
|
||||
connections->sendScalarsData(scalars);
|
||||
}
|
||||
|
||||
void RemoteQueryExecutor::sendExternalTables()
|
||||
{
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
size_t count = multiplexed_connections->size();
|
||||
size_t count = connections->size();
|
||||
|
||||
{
|
||||
std::lock_guard lock(external_tables_mutex);
|
||||
@ -472,7 +485,7 @@ void RemoteQueryExecutor::sendExternalTables()
|
||||
}
|
||||
}
|
||||
|
||||
multiplexed_connections->sendExternalTablesData(external_tables_data);
|
||||
connections->sendExternalTablesData(external_tables_data);
|
||||
}
|
||||
|
||||
void RemoteQueryExecutor::tryCancel(const char * reason, std::unique_ptr<ReadContext> * read_context)
|
||||
@ -489,11 +502,11 @@ void RemoteQueryExecutor::tryCancel(const char * reason, std::unique_ptr<ReadCon
|
||||
if (read_context && *read_context)
|
||||
(*read_context)->cancel();
|
||||
|
||||
multiplexed_connections->sendCancel();
|
||||
connections->sendCancel();
|
||||
}
|
||||
|
||||
if (log)
|
||||
LOG_TRACE(log, "({}) {}", multiplexed_connections->dumpAddresses(), reason);
|
||||
LOG_TRACE(log, "({}) {}", connections->dumpAddresses(), reason);
|
||||
}
|
||||
|
||||
bool RemoteQueryExecutor::isQueryPending() const
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Client/ConnectionPool.h>
|
||||
#include <Client/MultiplexedConnections.h>
|
||||
#include <Client/IConnections.h>
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
@ -40,7 +41,7 @@ public:
|
||||
|
||||
/// Accepts several connections already taken from pool.
|
||||
RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_, const Block & header_, const Context & context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
@ -103,8 +104,8 @@ private:
|
||||
Block totals;
|
||||
Block extremes;
|
||||
|
||||
std::function<std::unique_ptr<MultiplexedConnections>()> create_multiplexed_connections;
|
||||
std::unique_ptr<MultiplexedConnections> multiplexed_connections;
|
||||
std::function<std::unique_ptr<IConnections>()> create_connections;
|
||||
std::unique_ptr<IConnections> connections;
|
||||
|
||||
const String query;
|
||||
String query_id = "";
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <DataStreams/RemoteQueryExecutorReadContext.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/NetException.h>
|
||||
#include <Client/MultiplexedConnections.h>
|
||||
#include <Client/IConnections.h>
|
||||
#include <sys/epoll.h>
|
||||
|
||||
namespace DB
|
||||
@ -11,7 +11,7 @@ namespace DB
|
||||
|
||||
struct RemoteQueryExecutorRoutine
|
||||
{
|
||||
MultiplexedConnections & connections;
|
||||
IConnections & connections;
|
||||
RemoteQueryExecutorReadContext & read_context;
|
||||
|
||||
struct ReadCallback
|
||||
@ -19,15 +19,15 @@ struct RemoteQueryExecutorRoutine
|
||||
RemoteQueryExecutorReadContext & read_context;
|
||||
Fiber & fiber;
|
||||
|
||||
void operator()(Poco::Net::Socket & socket)
|
||||
void operator()(int fd, const Poco::Timespan & timeout = 0, const std::string fd_description = "")
|
||||
{
|
||||
try
|
||||
{
|
||||
read_context.setSocket(socket);
|
||||
read_context.setConnectionFD(fd, timeout, fd_description);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
e.addMessage(" while reading from socket ({})", socket.peerAddress().toString());
|
||||
e.addMessage(" while reading from {}", fd_description);
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -70,60 +70,38 @@ namespace ErrorCodes
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
}
|
||||
|
||||
RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(MultiplexedConnections & connections_)
|
||||
RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(IConnections & connections_)
|
||||
: connections(connections_)
|
||||
{
|
||||
epoll_fd = epoll_create(2);
|
||||
if (-1 == epoll_fd)
|
||||
throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
if (-1 == pipe2(pipe_fd, O_NONBLOCK))
|
||||
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
{
|
||||
epoll_event socket_event;
|
||||
socket_event.events = EPOLLIN | EPOLLPRI;
|
||||
socket_event.data.fd = pipe_fd[0];
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event))
|
||||
throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
epoll.add(pipe_fd[0]);
|
||||
}
|
||||
|
||||
{
|
||||
epoll_event timer_event;
|
||||
timer_event.events = EPOLLIN | EPOLLPRI;
|
||||
timer_event.data.fd = timer.getDescriptor();
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, timer_event.data.fd, &timer_event))
|
||||
throwFromErrno("Cannot add timer descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
epoll.add(timer.getDescriptor());
|
||||
}
|
||||
|
||||
auto routine = RemoteQueryExecutorRoutine{connections, *this};
|
||||
fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine));
|
||||
}
|
||||
|
||||
void RemoteQueryExecutorReadContext::setSocket(Poco::Net::Socket & socket)
|
||||
void RemoteQueryExecutorReadContext::setConnectionFD(int fd, const Poco::Timespan & timeout, const std::string & fd_description)
|
||||
{
|
||||
int fd = socket.impl()->sockfd();
|
||||
if (fd == socket_fd)
|
||||
if (fd == connection_fd)
|
||||
return;
|
||||
|
||||
epoll_event socket_event;
|
||||
socket_event.events = EPOLLIN | EPOLLPRI;
|
||||
socket_event.data.fd = fd;
|
||||
if (connection_fd != -1)
|
||||
epoll.remove(connection_fd);
|
||||
|
||||
if (socket_fd != -1)
|
||||
{
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, socket_fd, &socket_event))
|
||||
throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
}
|
||||
connection_fd = fd;
|
||||
epoll.add(connection_fd);
|
||||
|
||||
socket_fd = fd;
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, socket_fd, &socket_event))
|
||||
throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
receive_timeout = socket.impl()->getReceiveTimeout();
|
||||
receive_timeout = timeout;
|
||||
connection_fd_description = fd_description;
|
||||
}
|
||||
|
||||
bool RemoteQueryExecutorReadContext::checkTimeout() const
|
||||
@ -142,17 +120,11 @@ bool RemoteQueryExecutorReadContext::checkTimeout() const
|
||||
|
||||
bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const
|
||||
{
|
||||
/// Wait for epoll will not block if it was polled externally.
|
||||
epoll_event events[3];
|
||||
events[0].data.fd = events[1].data.fd = events[2].data.fd = -1;
|
||||
|
||||
/// Wait for epoll_fd will not block if it was polled externally.
|
||||
int num_events = 0;
|
||||
while (num_events <= 0)
|
||||
{
|
||||
num_events = epoll_wait(epoll_fd, events, 3, -1);
|
||||
if (num_events == -1 && errno != EINTR)
|
||||
throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET);
|
||||
}
|
||||
int num_events = epoll.getManyReady(3, events,/* blocking = */ false);
|
||||
|
||||
bool is_socket_ready = false;
|
||||
bool is_pipe_alarmed = false;
|
||||
@ -160,7 +132,7 @@ bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const
|
||||
|
||||
for (int i = 0; i < num_events; ++i)
|
||||
{
|
||||
if (events[i].data.fd == socket_fd)
|
||||
if (events[i].data.fd == connection_fd)
|
||||
is_socket_ready = true;
|
||||
if (events[i].data.fd == timer.getDescriptor())
|
||||
has_timer_alarm = true;
|
||||
@ -229,9 +201,7 @@ void RemoteQueryExecutorReadContext::cancel()
|
||||
|
||||
RemoteQueryExecutorReadContext::~RemoteQueryExecutorReadContext()
|
||||
{
|
||||
/// socket_fd is closed by Poco::Net::Socket
|
||||
if (epoll_fd != -1)
|
||||
close(epoll_fd);
|
||||
/// connection_fd is closed by Poco::Net::Socket or Epoll
|
||||
if (pipe_fd[0] != -1)
|
||||
close(pipe_fd[0]);
|
||||
if (pipe_fd[1] != -1)
|
||||
|
@ -7,7 +7,9 @@
|
||||
#include <Common/Fiber.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/TimerDescriptor.h>
|
||||
#include <Common/Epoll.h>
|
||||
#include <Client/Connection.h>
|
||||
#include <Client/IConnections.h>
|
||||
#include <Poco/Timespan.h>
|
||||
|
||||
namespace Poco::Net
|
||||
@ -33,26 +35,29 @@ public:
|
||||
std::mutex fiber_lock;
|
||||
|
||||
Poco::Timespan receive_timeout;
|
||||
MultiplexedConnections & connections;
|
||||
IConnections & connections;
|
||||
Poco::Net::Socket * last_used_socket = nullptr;
|
||||
|
||||
/// Here we have three descriptors we are going to wait:
|
||||
/// * socket_fd is a descriptor of connection. It may be changed in case of reading from several replicas.
|
||||
/// * connection_fd is a descriptor of connection. It may be changed in case of reading from several replicas.
|
||||
/// * timer is a timerfd descriptor to manually check socket timeout
|
||||
/// * pipe_fd is a pipe we use to cancel query and socket polling by executor.
|
||||
/// We put those descriptors into our own epoll_fd which is used by external executor.
|
||||
/// We put those descriptors into our own epoll which is used by external executor.
|
||||
TimerDescriptor timer{CLOCK_MONOTONIC, 0};
|
||||
int socket_fd = -1;
|
||||
int epoll_fd = -1;
|
||||
int connection_fd = -1;
|
||||
int pipe_fd[2] = { -1, -1 };
|
||||
|
||||
explicit RemoteQueryExecutorReadContext(MultiplexedConnections & connections_);
|
||||
Epoll epoll;
|
||||
|
||||
std::string connection_fd_description;
|
||||
|
||||
explicit RemoteQueryExecutorReadContext(IConnections & connections_);
|
||||
~RemoteQueryExecutorReadContext();
|
||||
|
||||
bool checkTimeout() const;
|
||||
bool checkTimeoutImpl() const;
|
||||
|
||||
void setSocket(Poco::Net::Socket & socket);
|
||||
void setConnectionFD(int fd, const Poco::Timespan & timeout = 0, const std::string & fd_description = "");
|
||||
void setTimer() const;
|
||||
|
||||
bool resumeRoutine();
|
||||
|
@ -17,6 +17,10 @@ struct ConnectionTimeouts
|
||||
Poco::Timespan http_keep_alive_timeout;
|
||||
Poco::Timespan secure_connection_timeout;
|
||||
|
||||
/// Timeouts for HedgedConnections
|
||||
Poco::Timespan hedged_connection_timeout;
|
||||
Poco::Timespan receive_data_timeout;
|
||||
|
||||
ConnectionTimeouts() = default;
|
||||
|
||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
||||
@ -27,7 +31,9 @@ struct ConnectionTimeouts
|
||||
receive_timeout(receive_timeout_),
|
||||
tcp_keep_alive_timeout(0),
|
||||
http_keep_alive_timeout(0),
|
||||
secure_connection_timeout(connection_timeout)
|
||||
secure_connection_timeout(connection_timeout),
|
||||
hedged_connection_timeout(receive_timeout_),
|
||||
receive_data_timeout(receive_timeout_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -40,7 +46,9 @@ struct ConnectionTimeouts
|
||||
receive_timeout(receive_timeout_),
|
||||
tcp_keep_alive_timeout(tcp_keep_alive_timeout_),
|
||||
http_keep_alive_timeout(0),
|
||||
secure_connection_timeout(connection_timeout)
|
||||
secure_connection_timeout(connection_timeout),
|
||||
hedged_connection_timeout(receive_timeout_),
|
||||
receive_data_timeout(receive_timeout_)
|
||||
{
|
||||
}
|
||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
||||
@ -53,7 +61,9 @@ struct ConnectionTimeouts
|
||||
receive_timeout(receive_timeout_),
|
||||
tcp_keep_alive_timeout(tcp_keep_alive_timeout_),
|
||||
http_keep_alive_timeout(http_keep_alive_timeout_),
|
||||
secure_connection_timeout(connection_timeout)
|
||||
secure_connection_timeout(connection_timeout),
|
||||
hedged_connection_timeout(receive_timeout_),
|
||||
receive_data_timeout(receive_timeout_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -62,13 +72,17 @@ struct ConnectionTimeouts
|
||||
const Poco::Timespan & receive_timeout_,
|
||||
const Poco::Timespan & tcp_keep_alive_timeout_,
|
||||
const Poco::Timespan & http_keep_alive_timeout_,
|
||||
const Poco::Timespan & secure_connection_timeout_)
|
||||
: connection_timeout(connection_timeout_),
|
||||
send_timeout(send_timeout_),
|
||||
receive_timeout(receive_timeout_),
|
||||
tcp_keep_alive_timeout(tcp_keep_alive_timeout_),
|
||||
http_keep_alive_timeout(http_keep_alive_timeout_),
|
||||
secure_connection_timeout(secure_connection_timeout_)
|
||||
const Poco::Timespan & secure_connection_timeout_,
|
||||
const Poco::Timespan & receive_hello_timeout_,
|
||||
const Poco::Timespan & receive_data_timeout_)
|
||||
: connection_timeout(connection_timeout_),
|
||||
send_timeout(send_timeout_),
|
||||
receive_timeout(receive_timeout_),
|
||||
tcp_keep_alive_timeout(tcp_keep_alive_timeout_),
|
||||
http_keep_alive_timeout(http_keep_alive_timeout_),
|
||||
secure_connection_timeout(secure_connection_timeout_),
|
||||
hedged_connection_timeout(receive_hello_timeout_),
|
||||
receive_data_timeout(receive_data_timeout_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -87,7 +101,9 @@ struct ConnectionTimeouts
|
||||
saturate(receive_timeout, limit),
|
||||
saturate(tcp_keep_alive_timeout, limit),
|
||||
saturate(http_keep_alive_timeout, limit),
|
||||
saturate(secure_connection_timeout, limit));
|
||||
saturate(secure_connection_timeout, limit),
|
||||
saturate(hedged_connection_timeout, limit),
|
||||
saturate(receive_data_timeout, limit));
|
||||
}
|
||||
|
||||
/// Timeouts for the case when we have just single attempt to connect.
|
||||
|
@ -16,7 +16,15 @@ inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(cons
|
||||
/// Timeouts for the case when we will try many addresses in a loop.
|
||||
inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings & settings)
|
||||
{
|
||||
return ConnectionTimeouts(settings.connect_timeout_with_failover_ms, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout, 0, settings.connect_timeout_with_failover_secure_ms);
|
||||
return ConnectionTimeouts(
|
||||
settings.connect_timeout_with_failover_ms,
|
||||
settings.send_timeout,
|
||||
settings.receive_timeout,
|
||||
settings.tcp_keep_alive_timeout,
|
||||
0,
|
||||
settings.connect_timeout_with_failover_secure_ms,
|
||||
settings.hedged_connection_timeout,
|
||||
settings.receive_data_timeout);
|
||||
}
|
||||
|
||||
inline ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Context & context)
|
||||
|
@ -14,7 +14,6 @@ namespace ProfileEvents
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NETWORK_ERROR;
|
||||
@ -42,7 +41,7 @@ bool ReadBufferFromPocoSocket::nextImpl()
|
||||
/// Note that receive timeout is not checked here. External code should check it while polling.
|
||||
while (bytes_read < 0 && async_callback && errno == EAGAIN)
|
||||
{
|
||||
async_callback(socket);
|
||||
async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), socket_description);
|
||||
bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size(), flags);
|
||||
}
|
||||
}
|
||||
@ -74,7 +73,10 @@ bool ReadBufferFromPocoSocket::nextImpl()
|
||||
}
|
||||
|
||||
ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size)
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size), socket(socket_), peer_address(socket.peerAddress())
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size)
|
||||
, socket(socket_)
|
||||
, peer_address(socket.peerAddress())
|
||||
, socket_description("socket (" + peer_address.toString() + ")")
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using AsyncCallback = std::function<void(int, const Poco::Timespan &, const std::string &)>;
|
||||
|
||||
/// Works with the ready Poco::Net::Socket. Blocking operations.
|
||||
class ReadBufferFromPocoSocket : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
@ -27,10 +29,11 @@ public:
|
||||
|
||||
bool poll(size_t timeout_microseconds) const;
|
||||
|
||||
void setAsyncCallback(std::function<void(Poco::Net::Socket &)> async_callback_) { async_callback = std::move(async_callback_); }
|
||||
void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); }
|
||||
|
||||
private:
|
||||
std::function<void(Poco::Net::Socket &)> async_callback;
|
||||
AsyncCallback async_callback;
|
||||
std::string socket_description;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -23,24 +23,14 @@ namespace ErrorCodes
|
||||
|
||||
PollingQueue::PollingQueue()
|
||||
{
|
||||
epoll_fd = epoll_create(1);
|
||||
if (-1 == epoll_fd)
|
||||
throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
if (-1 == pipe2(pipe_fd, O_NONBLOCK))
|
||||
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
epoll_event socket_event;
|
||||
socket_event.events = EPOLLIN | EPOLLPRI;
|
||||
socket_event.data.ptr = pipe_fd;
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event))
|
||||
throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
epoll.add(pipe_fd[0], pipe_fd);
|
||||
}
|
||||
|
||||
PollingQueue::~PollingQueue()
|
||||
{
|
||||
close(epoll_fd);
|
||||
close(pipe_fd[0]);
|
||||
close(pipe_fd[1]);
|
||||
}
|
||||
@ -52,13 +42,7 @@ void PollingQueue::addTask(size_t thread_number, void * data, int fd)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Task {} was already added to task queue", key);
|
||||
|
||||
tasks[key] = TaskData{thread_number, data, fd};
|
||||
|
||||
epoll_event socket_event;
|
||||
socket_event.events = EPOLLIN | EPOLLPRI;
|
||||
socket_event.data.ptr = data;
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &socket_event))
|
||||
throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
epoll.add(fd, data);
|
||||
}
|
||||
|
||||
static std::string dumpTasks(const std::unordered_map<std::uintptr_t, PollingQueue::TaskData> & tasks)
|
||||
@ -86,15 +70,7 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock<std::mutex> & lock)
|
||||
|
||||
epoll_event event;
|
||||
event.data.ptr = nullptr;
|
||||
int num_events = 0;
|
||||
|
||||
while (num_events <= 0)
|
||||
{
|
||||
num_events = epoll_wait(epoll_fd, &event, 1, -1);
|
||||
|
||||
if (num_events == -1 && errno != EINTR)
|
||||
throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET);
|
||||
}
|
||||
epoll.getManyReady(1, &event, true);
|
||||
|
||||
lock.lock();
|
||||
|
||||
@ -112,9 +88,7 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock<std::mutex> & lock)
|
||||
|
||||
auto res = it->second;
|
||||
tasks.erase(it);
|
||||
|
||||
if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, res.fd, &event))
|
||||
throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
epoll.remove(res.fd);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
#include <unordered_map>
|
||||
#include <Common/Epoll.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -25,7 +26,7 @@ public:
|
||||
};
|
||||
|
||||
private:
|
||||
int epoll_fd;
|
||||
Epoll epoll;
|
||||
int pipe_fd[2];
|
||||
std::atomic_bool is_finished = false;
|
||||
std::unordered_map<std::uintptr_t, TaskData> tasks;
|
||||
|
@ -717,7 +717,18 @@ void TCPHandler::processTablesStatusRequest()
|
||||
response.table_states_by_id.emplace(table_name, std::move(status));
|
||||
}
|
||||
|
||||
|
||||
writeVarUInt(Protocol::Server::TablesStatusResponse, *out);
|
||||
|
||||
/// For testing hedged requests
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
if (settings.sleep_in_send_tables_status)
|
||||
{
|
||||
out->next();
|
||||
std::chrono::seconds sec(settings.sleep_in_send_tables_status);
|
||||
std::this_thread::sleep_for(sec);
|
||||
}
|
||||
|
||||
response.write(*out, client_tcp_protocol_version);
|
||||
}
|
||||
|
||||
@ -1402,6 +1413,15 @@ void TCPHandler::sendData(const Block & block)
|
||||
/// Send external table name (empty name is the main table)
|
||||
writeStringBinary("", *out);
|
||||
|
||||
/// For testing hedged requests
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
if (block.rows() > 0 && settings.sleep_in_send_data)
|
||||
{
|
||||
out->next();
|
||||
std::chrono::seconds sec(settings.sleep_in_send_data);
|
||||
std::this_thread::sleep_for(sec);
|
||||
}
|
||||
|
||||
state.block_out->write(block);
|
||||
state.maybe_compressed_out->next();
|
||||
out->next();
|
||||
|
@ -0,0 +1,8 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<use_hedged_requests>0</use_hedged_requests>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
@ -166,6 +166,7 @@ def test_load_balancing_priority_round_robin(dist_table):
|
||||
|
||||
def test_distributed_replica_max_ignored_errors():
|
||||
settings = {
|
||||
'use_hedged_requests' : 0,
|
||||
'load_balancing': 'in_order',
|
||||
'prefer_localhost_replica': 0,
|
||||
'connect_timeout': 2,
|
||||
|
@ -78,6 +78,7 @@ def _check_exception(exception, expected_tries=3):
|
||||
expected_lines = (
|
||||
'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_TIMEOUT,
|
||||
'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_CONNECT,
|
||||
EXCEPTION_TIMEOUT,
|
||||
)
|
||||
|
||||
assert any(line.startswith(expected) for expected in expected_lines), \
|
||||
|
0
tests/integration/test_hedged_requests/__init__.py
Normal file
0
tests/integration/test_hedged_requests/__init__.py
Normal file
@ -0,0 +1,22 @@
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>node_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_3</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
||||
|
10
tests/integration/test_hedged_requests/configs/users.xml
Normal file
10
tests/integration/test_hedged_requests/configs/users.xml
Normal file
@ -0,0 +1,10 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<load_balancing>in_order</load_balancing>
|
||||
<hedged_connection_timeout>100</hedged_connection_timeout>
|
||||
<receive_data_timeout>2</receive_data_timeout>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
@ -0,0 +1,7 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
302
tests/integration/test_hedged_requests/test.py
Normal file
302
tests/integration/test_hedged_requests/test.py
Normal file
@ -0,0 +1,302 @@
|
||||
import os
|
||||
import sys
|
||||
import time
|
||||
|
||||
import pytest
|
||||
|
||||
sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.network import PartitionManager
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
NODES = {'node_' + str(i): None for i in (1, 2, 3)}
|
||||
|
||||
NODES['node'] = None
|
||||
|
||||
sleep_time = 30
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
NODES['node'] = cluster.add_instance(
|
||||
'node', with_zookeeper=True, stay_alive=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml'])
|
||||
|
||||
for name in NODES:
|
||||
if name != 'node':
|
||||
NODES[name] = cluster.add_instance(name, with_zookeeper=True, user_configs=['configs/users1.xml'])
|
||||
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
for node_id, node in list(NODES.items()):
|
||||
node.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE =
|
||||
ReplicatedMergeTree('/clickhouse/tables/replicated', '{}') ORDER BY id PARTITION BY toYYYYMM(date)'''.format(node_id))
|
||||
|
||||
NODES['node'].query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE =
|
||||
Distributed('test_cluster', 'default', 'replicated')''')
|
||||
|
||||
NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);")
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
config = '''<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<sleep_in_send_tables_status>{sleep_in_send_tables_status}</sleep_in_send_tables_status>
|
||||
<sleep_in_send_data>{sleep_in_send_data}</sleep_in_send_data>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>'''
|
||||
|
||||
|
||||
def check_query(expected_replica, receive_timeout=300):
|
||||
NODES['node'].restart_clickhouse()
|
||||
|
||||
# Without hedged requests select query will last more than 30 seconds,
|
||||
# with hedged requests it will last just around 1-2 second
|
||||
|
||||
start = time.time()
|
||||
result = NODES['node'].query("SELECT hostName(), id FROM distributed ORDER BY id LIMIT 1 SETTINGS receive_timeout={}".format(receive_timeout));
|
||||
query_time = time.time() - start
|
||||
|
||||
assert TSV(result) == TSV(expected_replica + "\t0")
|
||||
|
||||
print("Query time:", query_time)
|
||||
assert query_time < 10
|
||||
|
||||
|
||||
def check_settings(node_name, sleep_in_send_tables_status, sleep_in_send_data):
|
||||
attempts = 0
|
||||
while attempts < 1000:
|
||||
setting1 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_tables_status'")
|
||||
setting2 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_data'")
|
||||
if int(setting1) == sleep_in_send_tables_status and int(setting2) == sleep_in_send_data:
|
||||
return
|
||||
time.sleep(0.1)
|
||||
attempts += 1
|
||||
|
||||
assert attempts < 1000
|
||||
|
||||
|
||||
def test_stuck_replica(started_cluster):
|
||||
cluster.pause_container("node_1")
|
||||
check_query(expected_replica="node_2")
|
||||
cluster.unpause_container("node_1")
|
||||
|
||||
|
||||
def test_long_query(started_cluster):
|
||||
result = NODES['node'].query("select hostName(), max(id + sleep(1.5)) from distributed settings max_block_size = 1, max_threads = 1;")
|
||||
assert TSV(result) == TSV("node_1\t99")
|
||||
|
||||
NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);")
|
||||
|
||||
|
||||
def test_send_table_status_sleep(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', sleep_time, 0)
|
||||
check_settings('node_2', 0, 0)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_2")
|
||||
|
||||
|
||||
def test_send_table_status_sleep2(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', sleep_time, 0)
|
||||
check_settings('node_2', sleep_time, 0)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_3")
|
||||
|
||||
|
||||
def test_send_data(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', 0, 0)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_2")
|
||||
|
||||
|
||||
def test_send_data2(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', 0, sleep_time)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_3")
|
||||
|
||||
|
||||
def test_combination1(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', sleep_time, 0)
|
||||
check_settings('node_2', 0, sleep_time)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_3")
|
||||
|
||||
|
||||
def test_combination2(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', sleep_time, 0)
|
||||
check_settings('node_3', 0, 0)
|
||||
|
||||
check_query(expected_replica="node_3")
|
||||
|
||||
|
||||
def test_combination3(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', 1, 0)
|
||||
check_settings('node_3', 0, sleep_time)
|
||||
|
||||
check_query(expected_replica="node_2")
|
||||
|
||||
|
||||
def test_combination4(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', 1, sleep_time)
|
||||
check_settings('node_2', 1, 0)
|
||||
check_settings('node_3', 2, 0)
|
||||
|
||||
check_query(expected_replica="node_2")
|
||||
|
||||
|
||||
def test_receive_timeout1(started_cluster):
|
||||
# Check the situation when first two replicas get receive timeout
|
||||
# in establishing connection, but the third replica is ok.
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=3, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=3, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=1))
|
||||
|
||||
check_settings('node_1', 3, 0)
|
||||
check_settings('node_2', 3, 0)
|
||||
check_settings('node_3', 0, 1)
|
||||
|
||||
check_query(expected_replica="node_3", receive_timeout=2)
|
||||
|
||||
|
||||
def test_receive_timeout2(started_cluster):
|
||||
# Check the situation when first replica get receive timeout
|
||||
# in packet receiving but there are replicas in process of
|
||||
# connection establishing.
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=4))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', 0, 4)
|
||||
check_settings('node_2', 2, 0)
|
||||
check_settings('node_3', 2, 0)
|
||||
|
||||
check_query(expected_replica="node_2", receive_timeout=3)
|
||||
|
@ -0,0 +1,26 @@
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>node_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_3</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_4</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
||||
|
@ -0,0 +1,11 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<load_balancing>in_order</load_balancing>
|
||||
<max_parallel_replicas>2</max_parallel_replicas>
|
||||
<hedged_connection_timeout>100</hedged_connection_timeout>
|
||||
<receive_data_timeout>2</receive_data_timeout>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
@ -0,0 +1,7 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
158
tests/integration/test_hedged_requests_parallel/test.py
Normal file
158
tests/integration/test_hedged_requests_parallel/test.py
Normal file
@ -0,0 +1,158 @@
|
||||
import os
|
||||
import sys
|
||||
import time
|
||||
|
||||
import pytest
|
||||
|
||||
sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
NODES = {'node_' + str(i): None for i in (1, 2, 3, 4)}
|
||||
NODES['node'] = None
|
||||
|
||||
sleep_time = 30
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
NODES['node'] = cluster.add_instance(
|
||||
'node', with_zookeeper=True, stay_alive=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml'])
|
||||
|
||||
for name in NODES:
|
||||
if name != 'node':
|
||||
NODES[name] = cluster.add_instance(name, with_zookeeper=True, user_configs=['configs/users1.xml'])
|
||||
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
for node_id, node in list(NODES.items()):
|
||||
node.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE =
|
||||
ReplicatedMergeTree('/clickhouse/tables/replicated', '{}') ORDER BY id PARTITION BY toYYYYMM(date)'''.format(node_id))
|
||||
|
||||
NODES['node'].query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE =
|
||||
Distributed('test_cluster', 'default', 'replicated')''')
|
||||
|
||||
NODES['node'].query("INSERT INTO distributed VALUES (1, '2020-01-01'), (2, '2020-01-02')")
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
config = '''<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<sleep_in_send_tables_status>{sleep_in_send_tables_status}</sleep_in_send_tables_status>
|
||||
<sleep_in_send_data>{sleep_in_send_data}</sleep_in_send_data>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>'''
|
||||
|
||||
|
||||
def check_query():
|
||||
NODES['node'].restart_clickhouse()
|
||||
|
||||
# Without hedged requests select query will last more than 30 seconds,
|
||||
# with hedged requests it will last just around 1-2 second
|
||||
|
||||
start = time.time()
|
||||
NODES['node'].query("SELECT * FROM distributed");
|
||||
query_time = time.time() - start
|
||||
print("Query time:", query_time)
|
||||
|
||||
assert query_time < 5
|
||||
|
||||
|
||||
def check_settings(node_name, sleep_in_send_tables_status, sleep_in_send_data):
|
||||
attempts = 0
|
||||
while attempts < 1000:
|
||||
setting1 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_tables_status'")
|
||||
setting2 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_data'")
|
||||
if int(setting1) == sleep_in_send_tables_status and int(setting2) == sleep_in_send_data:
|
||||
return
|
||||
time.sleep(0.1)
|
||||
attempts += 1
|
||||
|
||||
assert attempts < 1000
|
||||
|
||||
|
||||
def test_send_table_status_sleep(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0))
|
||||
|
||||
check_settings('node_1', sleep_time, 0)
|
||||
check_settings('node_2', sleep_time, 0)
|
||||
|
||||
check_query()
|
||||
|
||||
|
||||
def test_send_data(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', 0, sleep_time)
|
||||
|
||||
check_query()
|
||||
|
||||
|
||||
def test_combination1(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
check_settings('node_1', 1, 0)
|
||||
check_settings('node_2', 1, 0)
|
||||
check_settings('node_3', 0, sleep_time)
|
||||
|
||||
check_query()
|
||||
|
||||
|
||||
def test_combination2(started_cluster):
|
||||
NODES['node_1'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_2'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
NODES['node_3'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
|
||||
|
||||
NODES['node_4'].replace_config(
|
||||
'/etc/clickhouse-server/users.d/users1.xml',
|
||||
config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0))
|
||||
|
||||
|
||||
check_settings('node_1', 0, sleep_time)
|
||||
check_settings('node_2', 1, 0)
|
||||
check_settings('node_3', 0, sleep_time)
|
||||
check_settings('node_4', 1, 0)
|
||||
|
||||
check_query()
|
||||
|
@ -1,7 +1,4 @@
|
||||
<test>
|
||||
|
||||
|
||||
|
||||
<query>SELECT number % 128 FROM numbers(300000000) FORMAT Null</query>
|
||||
<query>SELECT number % 255 FROM numbers(300000000) FORMAT Null</query>
|
||||
<query>SELECT number % 256 FROM numbers(300000000) FORMAT Null</query>
|
||||
|
@ -207,3 +207,4 @@
|
||||
01702_bitmap_native_integers
|
||||
01686_event_time_microseconds_part_log
|
||||
01017_uniqCombined_memory_usage
|
||||
01747_join_view_filter_dictionary
|
||||
|
@ -33,6 +33,7 @@ v20.9.5.5-stable 2020-11-13
|
||||
v20.9.4.76-stable 2020-10-29
|
||||
v20.9.3.45-stable 2020-10-09
|
||||
v20.9.2.20-stable 2020-09-22
|
||||
v20.8.14.4-lts 2021-03-03
|
||||
v20.8.13.15-lts 2021-02-20
|
||||
v20.8.12.2-lts 2021-01-16
|
||||
v20.8.11.17-lts 2020-12-25
|
||||
|
|
Loading…
Reference in New Issue
Block a user