Work with any number of replicas simultaneously, support max_parallel_replicas

This commit is contained in:
Pavel Kruglov 2021-01-27 12:33:11 +03:00
parent 2aa29e1bc7
commit b3b832cde7
8 changed files with 511 additions and 405 deletions

View File

@ -91,6 +91,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
void Connection::disconnect()
{
// LOG_DEBUG(log_wrapper.get(), "disconnect");
in = nullptr;
last_input_packet_type.reset();
out = nullptr; // can write to socket
@ -102,6 +103,8 @@ void Connection::disconnect()
void Connection::prepare(const ConnectionTimeouts & timeouts)
{
// LOG_DEBUG(log_wrapper.get(), "Connect");
LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}",
default_database.empty() ? "(not specified)" : default_database,
user,
@ -154,6 +157,8 @@ void Connection::prepare(const ConnectionTimeouts & timeouts)
void Connection::sendHello()
{
// LOG_DEBUG(log_wrapper.get(), "sendHello");
/** Disallow control characters in user controlled parameters
* to mitigate the possibility of SSRF.
* The user may do server side requests with 'remote' table function.
@ -210,6 +215,8 @@ void Connection::sendHello()
void Connection::receiveHello()
{
// LOG_DEBUG(log_wrapper.get(), "receiveHello");
/// Receive hello packet.
UInt64 packet_type = 0;
@ -313,6 +320,8 @@ const String & Connection::getServerDisplayName(const ConnectionTimeouts & timeo
void Connection::forceConnected(const ConnectionTimeouts & timeouts)
{
// LOG_DEBUG(log_wrapper.get(), "forceConnected");
if (!connected)
{
connect(timeouts);
@ -339,6 +348,8 @@ void Connection::sendClusterNameAndSalt()
bool Connection::ping()
{
// LOG_DEBUG(log_wrapper.get(), "ping");
TimeoutSetter timeout_setter(*socket, sync_request_timeout, true);
try
{
@ -390,6 +401,8 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time
void Connection::sendTablesStatusRequest(const TablesStatusRequest & request)
{
// LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest");
writeVarUInt(Protocol::Client::TablesStatusRequest, *out);
request.write(*out, server_revision);
out->next();
@ -397,6 +410,8 @@ void Connection::sendTablesStatusRequest(const TablesStatusRequest & request)
TablesStatusResponse Connection::receiveTablesStatusResponse()
{
// LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse");
UInt64 response_type = 0;
readVarUInt(response_type, *in);
@ -422,6 +437,8 @@ void Connection::sendQuery(
if (!connected)
connect(timeouts);
// LOG_DEBUG(log_wrapper.get(), "sendQuery");
TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true);
if (settings)
@ -520,6 +537,8 @@ void Connection::sendCancel()
if (!out)
return;
// LOG_DEBUG(log_wrapper.get(), "sendCancel");
writeVarUInt(Protocol::Client::Cancel, *out);
out->next();
}
@ -527,6 +546,8 @@ void Connection::sendCancel()
void Connection::sendData(const Block & block, const String & name, bool scalar)
{
// LOG_DEBUG(log_wrapper.get(), "sendData");
if (!block_out)
{
if (compression == Protocol::Compression::Enable)
@ -557,6 +578,7 @@ void Connection::sendData(const Block & block, const String & name, bool scalar)
void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name)
{
/// NOTE 'Throttler' is not used in this method (could use, but it's not important right now).
// LOG_DEBUG(log_wrapper.get(), "sendPreparedData");
writeVarUInt(Protocol::Client::Data, *out);
writeStringBinary(name, *out);
@ -574,6 +596,8 @@ void Connection::sendScalarsData(Scalars & data)
if (data.empty())
return;
// LOG_DEBUG(log_wrapper.get(), "sendScalarsData");
Stopwatch watch;
size_t out_bytes = out ? out->count() : 0;
size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0;
@ -659,6 +683,8 @@ void Connection::sendExternalTablesData(ExternalTablesData & data)
return;
}
// LOG_DEBUG(log_wrapper.get(), "sendExternalTablesData");
Stopwatch watch;
size_t out_bytes = out ? out->count() : 0;
size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0;
@ -757,6 +783,8 @@ std::optional<UInt64> Connection::checkPacket(size_t timeout_microseconds)
Packet Connection::receivePacket(AsyncCallback async_callback)
{
// LOG_DEBUG(log_wrapper.get(), "receivePacket");
in->setAsyncCallback(std::move(async_callback));
SCOPE_EXIT(in->setAsyncCallback({}));

View File

@ -353,7 +353,7 @@ void TryGetConnection::reset()
{
resetResult();
stage = Stage::CONNECT;
epoll = nullptr;
action_before_disconnect = nullptr;
socket_fd = -1;
fail_message.clear();
}
@ -369,8 +369,8 @@ void TryGetConnection::resetResult()
void TryGetConnection::processFail(bool add_description)
{
if (epoll)
epoll->remove(socket_fd);
if (action_before_disconnect)
action_before_disconnect(socket_fd);
fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false);
if (add_description)

View File

@ -62,9 +62,7 @@ public:
/// Reset class to initial stage.
void reset();
/// If connection is failed and epoll is set, before disconnecting
/// socket will be removed from epoll.
void setEpoll(Epoll * epoll_) { epoll = epoll_; }
void setActionBeforeDisconnect(std::function<void(int)> action) { action_before_disconnect = action; }
/// Process fail connection.
void processFail(bool add_description = false);
@ -78,7 +76,7 @@ public:
TryResult result;
Stage stage;
int socket_fd;
Epoll * epoll = nullptr;
std::function<void(int)> action_before_disconnect;
};
class ConnectionPoolWithFailover : public IConnectionPool, private PoolWithFailoverBase<IConnectionPool>

View File

@ -7,6 +7,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int ALL_CONNECTION_TRIES_FAILED;
}
@ -26,6 +27,9 @@ GetHedgedConnections::GetHedgedConnections(
= (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 : false;
entries_count = 0;
usable_count = 0;
failed_pools_count = 0;
}
GetHedgedConnections::~GetHedgedConnections()
@ -33,173 +37,175 @@ GetHedgedConnections::~GetHedgedConnections()
pool->updateSharedError(shuffled_pools);
}
GetHedgedConnections::Replicas GetHedgedConnections::getConnections()
std::vector<GetHedgedConnections::ReplicaStatePtr> GetHedgedConnections::getManyConnections(PoolMode pool_mode)
{
entries_count = 0;
usable_count = 0;
failed_pools_count = 0;
size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1;
ReplicaStatePtr replica = &first_replica;
int index = 0;
size_t max_entries;
if (pool_mode == PoolMode::GET_ALL)
{
min_entries = shuffled_pools.size();
max_entries = shuffled_pools.size();
}
else if (pool_mode == PoolMode::GET_ONE)
max_entries = 1;
else if (pool_mode == PoolMode::GET_MANY)
max_entries = settings ? size_t(settings->max_parallel_replicas) : 1;
else
throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR);
std::vector<ReplicaStatePtr> replicas;
replicas.reserve(max_entries);
for (size_t i = 0; i != max_entries; ++i)
{
auto replica = getNextConnection(false);
if (replica->isCannotChoose())
{
if (replicas.size() >= min_entries)
break;
/// Determine the reason of not enough replicas.
if (!fallback_to_stale_replicas && usable_count >= min_entries)
throw DB::Exception(
"Could not find enough connections to up-to-date replicas. Got: " + std::to_string(replicas.size())
+ ", needed: " + std::to_string(min_entries),
DB::ErrorCodes::ALL_REPLICAS_ARE_STALE);
throw DB::NetException(
"Could not connect to " + std::to_string(min_entries) + " replicas. Log: \n\n" + fail_messages + "\n",
DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
}
replicas.push_back(replica);
}
return replicas;
}
GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bool non_blocking)
{
// LOG_DEBUG(log, "getNextConnection");
ReplicaStatePtr replica = createNewReplica();
int index;
/// Check if it's the first time.
if (epoll.size() == 0 && ready_indexes.size() == 0)
{
index = 0;
last_used_index = 0;
}
else
index = getNextIndex();
bool is_first = true;
while (index != -1 || epoll.size() != 0)
{
if (index == -1 && !is_first && non_blocking)
{
replica->state = State::NOT_READY;
return replica;
}
if (is_first)
is_first = false;
if (index != -1)
{
Action action = startTryGetConnection(index, replica);
if (action == Action::TRY_NEXT_REPLICA)
{
index = getNextIndex(index);
continue;
}
if (action == Action::FINISH)
{
swapReplicasIfNeeded();
return {&first_replica, &second_replica};
}
}
/// Process epoll events
replica = processEpollEvents();
if (replica->isReady())
{
swapReplicasIfNeeded();
return {&first_replica, &second_replica};
}
index = getNextIndex(index);
}
/// We reach this point only if there was no up to date replica
if (usable_count == 0)
{
if (settings && settings->skip_unavailable_shards)
{
first_replica.state = State::CANNOT_CHOOSE;
second_replica.state = State::CANNOT_CHOOSE;
return {&first_replica, &second_replica};
}
throw NetException("All connection tries failed. Log: \n\n" + fail_messages + "\n", ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
}
if (!fallback_to_stale_replicas)
throw DB::Exception("Could not find connection to up-to-date replica.", DB::ErrorCodes::ALL_REPLICAS_ARE_STALE);
setBestUsableReplica(first_replica);
return {&first_replica, &second_replica};
}
void GetHedgedConnections::chooseSecondReplica()
{
LOG_DEBUG(log, "choose second replica");
if (second_replica.isCannotChoose() || second_replica.isReady())
return;
int index;
if (second_replica.isNotReady())
index = second_replica.index;
else
index = first_replica.index;
while (true)
{
if (second_replica.isEmpty())
{
index = getNextIndex(index);
if (index == -1)
break;
Action action = startTryGetConnection(index, &second_replica);
return replica;
if (action == Action::TRY_NEXT_REPLICA)
{
index = getNextIndex();
continue;
}
/// Second replica is ready or we are waiting for response from it
return;
if (action == Action::PROCESS_EPOLL_EVENTS && non_blocking)
return replica;
}
if (!second_replica.isNotReady())
throw Exception("Second replica state must be 'NOT_READY' before process epoll events", ErrorCodes::LOGICAL_ERROR);
replica = processEpollEvents(non_blocking);
if (replica->isReady() || (replica->isNotReady() && non_blocking))
return replica;
ReplicaStatePtr replica = processEpollEvents( true);
if (replica->isNotReady())
throw Exception("Not ready replica after processing epoll events.", ErrorCodes::LOGICAL_ERROR);
if (replica != &second_replica)
throw Exception("Epoll could return only second replica here", ErrorCodes::LOGICAL_ERROR);
/// If replica is not empty than it is ready or we are waiting for a response from it
if (!second_replica.isEmpty())
return;
index = getNextIndex();
}
/// There is no up to date replica
/// We reach this point only if there was no free up to date replica.
LOG_DEBUG(log, "there is no up to date replica for second replica");
/// Check if there is no even a free usable replica
if (!canGetNewConnection())
{
replica->state = State::CANNOT_CHOOSE;
return replica;
}
if (!fallback_to_stale_replicas || usable_count <= 1)
second_replica.state = State::CANNOT_CHOOSE;
else
setBestUsableReplica(second_replica, first_replica.index);
if (!fallback_to_stale_replicas)
{
replica->state = State::CANNOT_CHOOSE;
return replica;
}
setBestUsableReplica(replica);
return replica;
}
void GetHedgedConnections::stopChoosingSecondReplica()
void GetHedgedConnections::stopChoosingReplicas()
{
LOG_DEBUG(log, "stop choosing second replica");
// LOG_DEBUG(log, "stopChoosingReplicas");
for (auto & [fd, replica] : fd_to_replica)
{
removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica);
epoll.remove(fd);
try_get_connections[replica->index].reset();
replica->reset();
}
if (!second_replica.isNotReady())
throw Exception("Can't stop choosing second replica, because it's not in process of choosing", ErrorCodes::LOGICAL_ERROR);
removeTimeoutsFromReplica(&second_replica, epoll);
epoll.remove(second_replica.fd);
try_get_connections[second_replica.index].reset();
second_replica.reset();
fd_to_replica.clear();
}
int GetHedgedConnections::getNextIndex(int cur_index)
int GetHedgedConnections::getNextIndex()
{
/// Check if there is no more available replicas
if (cur_index == -1 || entries_count + failed_pools_count >= shuffled_pools.size())
if (entries_count + failed_pools_count >= shuffled_pools.size())
return -1;
/// We can work with two replicas simultaneously and they must have different indexes
int skip_index = -1;
if (!first_replica.isEmpty())
skip_index = first_replica.index;
else if (!second_replica.isEmpty())
skip_index = second_replica.index;
bool finish = false;
int next_index = cur_index;
int next_index = last_used_index;
while (!finish)
{
next_index = (next_index + 1) % shuffled_pools.size();
/// Check if we can try this replica
if (next_index != skip_index && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries)
if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries)
&& try_get_connections[next_index].stage != TryGetConnection::Stage::FINISHED)
finish = true;
/// If we made a complete round, there is no replica to connect
else if (next_index == cur_index)
else if (next_index == last_used_index)
return -1;
}
LOG_DEBUG(log, "get next index: {}", next_index);
// LOG_DEBUG(log, "get next index: {}", next_index);
last_used_index = next_index;
return next_index;
}
GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr replica)
GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr & replica)
{
LOG_DEBUG(log, "start try get connection with {} replica", index);
// LOG_DEBUG(log, "start try get connection with {} replica", index);
TryGetConnection & try_get_connection = try_get_connections[index];
replica->state = State::NOT_READY;
replica->index = index;
indexes_in_process.insert(index);
try_get_connection.reset();
try_get_connection.run();
@ -215,7 +221,13 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind
if (action == Action::PROCESS_EPOLL_EVENTS)
{
epoll.add(try_get_connection.socket_fd);
try_get_connection.setEpoll(&epoll);
fd_to_replica[try_get_connection.socket_fd] = replica;
try_get_connection.setActionBeforeDisconnect(
[&](int fd)
{
epoll.remove(fd);
fd_to_replica.erase(fd);
});
addTimeouts(replica);
}
@ -223,51 +235,58 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind
}
GetHedgedConnections::Action
GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll)
GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll)
{
LOG_DEBUG(log, "process get connection stage for {} replica", replica->index);
// LOG_DEBUG(log, "process get connection stage for {} replica", replica->index);
TryGetConnection & try_get_connection = try_get_connections[replica->index];
if (try_get_connection.stage == TryGetConnection::Stage::FINISHED)
{
LOG_DEBUG(log, "stage: FINISHED");
indexes_in_process.erase(replica->index);
// LOG_DEBUG(log, "stage: FINISHED");
++entries_count;
if (remove_from_epoll)
{
epoll.remove(try_get_connection.socket_fd);
fd_to_replica.erase(try_get_connection.socket_fd);
}
if (try_get_connection.result.is_usable)
{
LOG_DEBUG(log, "replica is usable");
// LOG_DEBUG(log, "replica is usable");
++usable_count;
if (try_get_connection.result.is_up_to_date)
{
LOG_DEBUG(log, "replica is up to date, finish get hedged connections");
// LOG_DEBUG(log, "replica is up to date, finish get hedged connections");
replica->state = State::READY;
ready_indexes.insert(replica->index);
return Action::FINISH;
}
/// This replica is not up to date, we will try to find up to date
replica->reset();
return Action::TRY_NEXT_REPLICA;
}
/// This replica is not up to date, we will try to find up to date
fd_to_replica.erase(replica->fd);
replica->reset();
return Action::TRY_NEXT_REPLICA;
}
else if (try_get_connection.stage == TryGetConnection::Stage::FAILED)
{
LOG_DEBUG(log, "stage: FAILED");
// LOG_DEBUG(log, "stage: FAILED");
processFailedConnection(replica);
return Action::TRY_NEXT_REPLICA;
}
LOG_DEBUG(log, "middle stage, process epoll events");
// LOG_DEBUG(log, "middle stage, process epoll events");
/// Get connection process is not finished
return Action::PROCESS_EPOLL_EVENTS;
}
void GetHedgedConnections::processFailedConnection(ReplicaStatePtr replica)
void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica)
{
LOG_DEBUG(log, "failed connection with {} replica", replica->index);
// LOG_DEBUG(log, "failed connection with {} replica", replica->index);
ShuffledPool & shuffled_pool = shuffled_pools[replica->index];
LOG_WARNING(
@ -286,105 +305,65 @@ void GetHedgedConnections::processFailedConnection(ReplicaStatePtr replica)
if (!fail_message.empty())
fail_messages += fail_message + "\n";
indexes_in_process.erase(replica->index);
replica->reset();
}
void GetHedgedConnections::addTimeouts(ReplicaState * replica)
void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica)
{
LOG_DEBUG(log, "add timeouts for {} replica", replica->index);
// LOG_DEBUG(log, "add timeouts for {} replica", replica->index);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeouts);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts);
/// If we haven't connected to second replica yet, set special timeout for it
if (second_replica.isEmpty())
{
auto stage = try_get_connections[replica->index].stage;
if (stage == TryGetConnection::Stage::RECEIVE_HELLO)
addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeouts);
else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS)
addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeouts);
}
}
void GetHedgedConnections::swapReplicasIfNeeded()
{
if ((!first_replica.isReady() && second_replica.isReady()))
{
LOG_DEBUG(log, "swap replicas");
swapReplicas();
}
auto stage = try_get_connections[replica->index].stage;
if (stage == TryGetConnection::Stage::RECEIVE_HELLO)
addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts);
else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS)
addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts);
}
GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking)
{
LOG_DEBUG(log, "process epoll events");
// LOG_DEBUG(log, "process epoll events");
int event_fd;
ReplicaStatePtr replica = nullptr;
bool finish = false;
while (!finish)
{
event_fd = getReadyFileDescriptor(epoll);
event_fd = getReadyFileDescriptor();
if ((replica = isEventReplica(event_fd)))
finish = processReplicaEvent(replica, non_blocking);
else if (auto * timeout_descriptor = isEventTimeout(event_fd, replica))
if (fd_to_replica.find(event_fd) != fd_to_replica.end())
{
processTimeoutEvent(replica, timeout_descriptor);
finish = true;
replica = fd_to_replica[event_fd];
finish = processReplicaEvent(replica, non_blocking);
}
else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end())
{
replica = timeout_fd_to_replica[event_fd];
finish = processTimeoutEvent(replica, replica->active_timeouts[event_fd].get(), non_blocking);
}
else
throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR);
}
LOG_DEBUG(log, "cancel process epoll events");
// LOG_DEBUG(log, "cancel process epoll events");
return replica;
}
GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::isEventReplica(int event_fd)
int GetHedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback)
{
if (event_fd == first_replica.fd)
return &first_replica;
for (auto & [fd, replica] : fd_to_replica)
if (replica->connection->hasReadPendingData())
return replica->fd;
if (event_fd == second_replica.fd)
return &second_replica;
return nullptr;
return epoll.getReady(std::move(async_callback)).data.fd;
}
TimerDescriptorPtr GetHedgedConnections::isEventTimeout(int event_fd, ReplicaStatePtr & replica_out)
bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking)
{
if (first_replica.active_timeouts.find(event_fd) != first_replica.active_timeouts.end())
{
replica_out = &first_replica;
return first_replica.active_timeouts[event_fd].get();
}
if (second_replica.active_timeouts.find(event_fd) != second_replica.active_timeouts.end())
{
replica_out = &second_replica;
return second_replica.active_timeouts[event_fd].get();
}
return nullptr;
}
int GetHedgedConnections::getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback)
{
if (first_replica.connection && first_replica.connection->hasReadPendingData())
return first_replica.fd;
if (second_replica.connection && second_replica.connection->hasReadPendingData())
return second_replica.fd;
return epoll_.getReady(std::move(async_callback)).data.fd;
}
bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr replica, bool non_blocking)
{
LOG_DEBUG(log, "epoll event is {} replica", replica->index);
removeTimeoutsFromReplica(replica, epoll);
// LOG_DEBUG(log, "epoll event is {} replica", replica->index);
removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica);
try_get_connections[replica->index].run();
Action action = processTryGetConnectionStage(replica, true);
if (action == Action::PROCESS_EPOLL_EVENTS)
@ -396,70 +375,84 @@ bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr replica, bool non
return true;
}
void GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor)
bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking)
{
LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index);
// LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index);
epoll.remove(timeout_descriptor->getDescriptor());
replica->active_timeouts.erase(timeout_descriptor->getDescriptor());
timeout_fd_to_replica[timeout_descriptor->getDescriptor()];
if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT)
{
LOG_DEBUG(log, "process receive timeout for {} replica", replica->index);
removeTimeoutsFromReplica(replica, epoll);
// LOG_DEBUG(log, "process receive timeout for {} replica", replica->index);
removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica);
epoll.remove(replica->fd);
fd_to_replica.erase(replica->fd);
TryGetConnection & try_get_connection = try_get_connections[replica->index];
try_get_connection.fail_message = "Receive timeout expired (" + try_get_connection.result.entry->getDescription() + ")";
try_get_connection.resetResult();
try_get_connection.stage = TryGetConnection::Stage::FAILED;
processFailedConnection(replica);
return true;
}
else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT
else if ((timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT
|| timeout_descriptor->getType() == TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT)
&& entries_count + ready_indexes.size() + failed_pools_count < shuffled_pools.size())
{
if (replica->index == second_replica.index || !second_replica.isEmpty())
throw Exception(
"Received timeout to connect with second replica, but current replica is second or second replica is not empty",
ErrorCodes::LOGICAL_ERROR);
replica = &second_replica;
replica = createNewReplica();
return true;
}
return non_blocking;
}
void GetHedgedConnections::setBestUsableReplica(ReplicaState & replica, int skip_index)
void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica)
{
LOG_DEBUG(log, "set best usable replica");
// LOG_DEBUG(log, "set best usable replica");
std::vector<int> indexes(try_get_connections.size());
for (size_t i = 0; i != indexes.size(); ++i)
indexes[i] = i;
/// Remove unusable and failed replicas, skip the replica with skip_index index
/// Remove unusable and failed replicas, skip ready replicas
indexes.erase(
std::remove_if(
indexes.begin(),
indexes.end(),
[&](int i) {
return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || i == skip_index;
return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable ||
indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end();
}),
indexes.end());
if (indexes.empty())
throw Exception("There is no usable replica to choose", ErrorCodes::LOGICAL_ERROR);
{
replica->state = State::CANNOT_CHOOSE;
return;
}
/// Sort replicas by staleness
std::stable_sort(indexes.begin(), indexes.end(), [&](size_t lhs, size_t rhs) {
return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness;
});
replica.index = indexes[0];
replica.connection = &*try_get_connections[indexes[0]].result.entry;
replica.state = State::READY;
replica.fd = replica.connection->getSocket()->impl()->sockfd();
replica->index = indexes[0];
replica->connection = &*try_get_connections[indexes[0]].result.entry;
replica->state = State::READY;
replica->fd = replica->connection->getSocket()->impl()->sockfd();
ready_indexes.insert(replica->index);
}
void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts)
void addTimeoutToReplica(
int type,
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica,
const ConnectionTimeouts & timeouts)
{
Poco::Timespan timeout;
switch (type)
@ -484,17 +477,28 @@ void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica
timeout_descriptor->setType(type);
timeout_descriptor->setRelative(timeout);
epoll.add(timeout_descriptor->getDescriptor());
timeout_fd_to_replica[timeout_descriptor->getDescriptor()] = replica;
replica->active_timeouts[timeout_descriptor->getDescriptor()] = std::move(timeout_descriptor);
}
void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll)
void removeTimeoutsFromReplica(
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica)
{
for (auto & [fd, _] : replica->active_timeouts)
{
epoll.remove(fd);
timeout_fd_to_replica.erase(fd);
}
replica->active_timeouts.clear();
}
void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll)
void removeTimeoutFromReplica(
int type,
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica)
{
auto it = std::find_if(
replica->active_timeouts.begin(),
@ -505,6 +509,7 @@ void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr re
if (it != replica->active_timeouts.end())
{
epoll.remove(it->first);
timeout_fd_to_replica.erase(it->first);
replica->active_timeouts.erase(it);
}
}

View File

@ -31,6 +31,7 @@ public:
State state = State::EMPTY;
int index = -1;
int fd = -1;
size_t parallel_replica_offset = 0;
std::unordered_map<int, std::unique_ptr<TimerDescriptor>> active_timeouts;
void reset()
@ -39,6 +40,7 @@ public:
state = State::EMPTY;
index = -1;
fd = -1;
parallel_replica_offset = 0;
active_timeouts.clear();
}
@ -48,7 +50,8 @@ public:
bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; };
};
using ReplicaStatePtr = ReplicaState *;
using ReplicaStatePtr = std::shared_ptr<ReplicaState>;
struct Replicas
{
@ -61,32 +64,15 @@ public:
const ConnectionTimeouts & timeouts_,
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr);
/// Establish connection with replicas. Return replicas as soon as connection with one of them is finished.
/// The first replica is always has state FINISHED and ready for sending query, the second replica
/// may have any state. To continue working with second replica call chooseSecondReplica().
Replicas getConnections();
std::vector<ReplicaStatePtr> getManyConnections(PoolMode pool_mode);
/// Continue choosing second replica, this function is not blocking. Second replica will be ready
/// for sending query when it has state FINISHED.
void chooseSecondReplica();
ReplicaStatePtr getNextConnection(bool non_blocking);
void stopChoosingSecondReplica();
bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); }
void swapReplicas() { std::swap(first_replica, second_replica); }
void stopChoosingReplicas();
/// Move ready replica to the first place.
void swapReplicasIfNeeded();
/// Check if the file descriptor is belong to one of replicas. If yes, return this replica, if no, return nullptr.
ReplicaStatePtr isEventReplica(int event_fd);
/// Check if the file descriptor is belong to timeout to any replica.
/// If yes, return corresponding TimerDescriptor and set timeout owner to replica,
/// if no, return nullptr.
TimerDescriptorPtr isEventTimeout(int event_fd, ReplicaStatePtr & replica);
/// Get file rescriptor that ready for reading.
int getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback = {});
bool hasEventsInProcess() const { return epoll.size() > 0; }
int getFileDescriptor() const { return epoll.getFileDescriptor(); }
@ -103,25 +89,29 @@ private:
TRY_NEXT_REPLICA = 2,
};
Action startTryGetConnection(int index, ReplicaStatePtr replica);
Action startTryGetConnection(int index, ReplicaStatePtr & replica);
Action processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll = false);
Action processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll = false);
int getNextIndex(int cur_index = -1);
int getNextIndex();
void addTimeouts(ReplicaStatePtr replica);
int getReadyFileDescriptor(AsyncCallback async_callback = {});
void processFailedConnection(ReplicaStatePtr replica);
void addTimeouts(ReplicaStatePtr & replica);
void processReceiveTimeout(ReplicaStatePtr replica);
void processFailedConnection(ReplicaStatePtr & replica);
bool processReplicaEvent(ReplicaStatePtr replica, bool non_blocking);
void processReceiveTimeout(ReplicaStatePtr & replica);
void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor);
bool processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking);
bool processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking);
ReplicaStatePtr processEpollEvents(bool non_blocking = false);
void setBestUsableReplica(ReplicaState & replica, int skip_index = -1);
void setBestUsableReplica(ReplicaStatePtr & replica);
ReplicaStatePtr createNewReplica() { return std::make_shared<ReplicaState>(); }
const ConnectionPoolWithFailoverPtr pool;
const Settings * settings;
@ -129,8 +119,14 @@ private:
std::shared_ptr<QualifiedTableName> table_to_check;
std::vector<TryGetConnection> try_get_connections;
std::vector<ShuffledPool> shuffled_pools;
ReplicaState first_replica;
ReplicaState second_replica;
std::unordered_map<int, ReplicaStatePtr> fd_to_replica;
std::unordered_map<int, ReplicaStatePtr> timeout_fd_to_replica;
// std::vector<std::unique_ptr<ReplicaState>> replicas;
// std::unordered_map<ReplicaStatePtr, std::unique_ptr<ReplicaState>> replicas_store;
// ReplicaState first_replica;
// ReplicaState second_replica;
bool fallback_to_stale_replicas;
Epoll epoll;
Poco::Logger * log;
@ -139,16 +135,30 @@ private:
size_t usable_count;
size_t failed_pools_count;
size_t max_tries;
int last_used_index;
std::unordered_set<int> indexes_in_process;
std::unordered_set<int> ready_indexes;
};
/// Add timeout with particular type to replica and add it to epoll.
void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts);
void addTimeoutToReplica(
int type,
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica,
const ConnectionTimeouts & timeouts);
/// Remove timeout with particular type from replica and epoll.
void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll);
void removeTimeoutFromReplica(
int type,
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica);
/// Remove all timeouts from replica and epoll.
void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll);
void removeTimeoutsFromReplica(
GetHedgedConnections::ReplicaStatePtr & replica,
Epoll & epoll,
std::unordered_map<int, GetHedgedConnections::ReplicaStatePtr> & timeout_fd_to_replica);
}

View File

@ -15,91 +15,83 @@ HedgedConnections::HedgedConnections(
const Settings & settings_,
const ConnectionTimeouts & timeouts_,
const ThrottlerPtr & throttler_,
PoolMode pool_mode,
std::shared_ptr<QualifiedTableName> table_to_check_)
: get_hedged_connections(pool_, &settings_, timeouts_, table_to_check_), settings(settings_), throttler(throttler_), log(&Poco::Logger::get("HedgedConnections"))
{
replicas = get_hedged_connections.getConnections();
std::vector<ReplicaStatePtr> replicas_states = get_hedged_connections.getManyConnections(pool_mode);
/// First replica may have state CANNOT_CHOOSE if setting skip_unavailable_shards is enabled
if (replicas.first_replica->isReady())
replicas.first_replica->connection->setThrottler(throttler);
if (!replicas.second_replica->isCannotChoose())
for (size_t i = 0; i != replicas_states.size(); ++i)
{
if (replicas.second_replica->isNotReady())
epoll.add(get_hedged_connections.getFileDescriptor());
auto set_throttler = [throttler_](ReplicaStatePtr replica)
{
replica->connection->setThrottler(throttler_);
};
second_replica_pipeline.add(std::function<void(ReplicaStatePtr)>(set_throttler));
replicas_states[i]->parallel_replica_offset = i;
replicas_states[i]->connection->setThrottler(throttler_);
epoll.add(replicas_states[i]->fd);
fd_to_replica[replicas_states[i]->fd] = replicas_states[i];
replicas.push_back({std::move(replicas_states[i])});
active_connections_count_by_offset[i] = 1;
}
pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr & replica_){ replica_->connection->setThrottler(throttler_); });
}
void HedgedConnections::Pipeline::add(std::function<void(ReplicaStatePtr replica)> send_function)
void HedgedConnections::Pipeline::add(std::function<void(ReplicaStatePtr & replica)> send_function)
{
pipeline.push_back(send_function);
}
void HedgedConnections::Pipeline::run(ReplicaStatePtr replica)
void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica)
{
for (auto & send_func : pipeline)
send_func(replica);
pipeline.clear();
}
size_t HedgedConnections::size() const
{
if (replicas.first_replica->isReady() || replicas.second_replica->isReady())
return 1;
if (replicas.empty())
return 0;
return 0;
}
bool HedgedConnections::hasActiveConnections() const
{
return replicas.first_replica->isReady() || replicas.second_replica->isReady();
return 1;
}
void HedgedConnections::sendScalarsData(Scalars & data)
{
std::lock_guard lock(cancel_mutex);
// LOG_DEBUG(log, "sendScalarsData");
if (!sent_query)
throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
auto send_scalars_data = [&data](ReplicaStatePtr replica) { replica->connection->sendScalarsData(data); };
auto send_scalars_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendScalarsData(data); };
if (replicas.first_replica->isReady())
send_scalars_data(replicas.first_replica);
for (auto & replicas_with_same_offset : replicas)
for (auto & replica : replicas_with_same_offset)
if (replica->isReady())
send_scalars_data(replica);
if (replicas.second_replica->isReady())
send_scalars_data(replicas.second_replica);
else if (!replicas.second_replica->isCannotChoose())
second_replica_pipeline.add(std::function<void(ReplicaStatePtr)>(send_scalars_data));
pipeline_for_new_replicas.add(send_scalars_data);
}
void HedgedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
{
std::lock_guard lock(cancel_mutex);
// LOG_DEBUG(log, "sendExternalTablesData");
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](ReplicaStatePtr replica) { replica->connection->sendExternalTablesData(data[0]); };
auto send_external_tables_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendExternalTablesData(data[0]); };
if (replicas.first_replica->isReady())
send_external_tables_data(replicas.first_replica);
for (auto & replicas_with_same_offset : replicas)
for (auto & replica : replicas_with_same_offset)
if (replica->isReady())
send_external_tables_data(replica);
if (replicas.second_replica->isReady())
send_external_tables_data(replicas.second_replica);
else if (!replicas.second_replica->isCannotChoose())
second_replica_pipeline.add(send_external_tables_data);
pipeline_for_new_replicas.add(send_external_tables_data);
}
void HedgedConnections::sendQuery(
@ -112,35 +104,52 @@ void HedgedConnections::sendQuery(
{
std::lock_guard lock(cancel_mutex);
// LOG_DEBUG(log, "sendQuery");
if (sent_query)
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr replica)
for (auto & replicas_with_same_offset : replicas)
{
Settings modified_settings = settings;
if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
for (auto & replica : replicas_with_same_offset)
{
if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
{
has_two_level_aggregation_incompatibility = true;
break;
}
}
if (has_two_level_aggregation_incompatibility)
break;
}
auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica)
{
Settings modified_settings = this->settings;
if (this->has_two_level_aggregation_incompatibility)
{
/// 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 (this->replicas.size() > 1)
{
modified_settings.parallel_replicas_count = this->replicas.size();
modified_settings.parallel_replica_offset = replica->parallel_replica_offset;
}
replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data);
this->epoll.add(replica->fd);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, timeouts);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts);
addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts);
};
if (replicas.first_replica->isReady())
{
send_query(replicas.first_replica);
if (replicas.second_replica->isEmpty())
addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replicas.first_replica, epoll, timeouts);
}
if (replicas.second_replica->isReady())
send_query(replicas.second_replica);
else if (!replicas.second_replica->isCannotChoose())
second_replica_pipeline.add(send_query);
for (auto & replicas_with_same_offset : replicas)
for (auto & replica : replicas_with_same_offset)
send_query(replica);
pipeline_for_new_replicas.add(send_query);
sent_query = true;
}
@ -148,32 +157,41 @@ void HedgedConnections::disconnect()
{
std::lock_guard lock(cancel_mutex);
if (replicas.first_replica->isReady())
{
replicas.first_replica->connection->disconnect();
replicas.first_replica->reset();
}
// LOG_DEBUG(log, "disconnect");
if (replicas.second_replica->isReady())
for (auto & replicas_with_same_offset : replicas)
for (auto & replica : replicas_with_same_offset)
if (replica->isReady())
finishProcessReplica(replica, true);
if (get_hedged_connections.hasEventsInProcess())
{
replicas.second_replica->connection->disconnect();
replicas.second_replica->reset();
get_hedged_connections.stopChoosingReplicas();
if (next_replica_in_process)
epoll.remove(get_hedged_connections.getFileDescriptor());
}
else if (replicas.second_replica->isNotReady())
get_hedged_connections.stopChoosingSecondReplica();
}
std::string HedgedConnections::dumpAddresses() const
{
std::lock_guard lock(cancel_mutex);
// LOG_DEBUG(log, "dumpAddresses");
std::string addresses = "";
bool is_first = true;
if (replicas.first_replica->isReady())
addresses += replicas.first_replica->connection->getDescription();
if (replicas.second_replica->isReady())
addresses += "; " + replicas.second_replica->connection->getDescription();
for (auto & replicas_with_same_offset : replicas)
{
for (auto & replica : replicas_with_same_offset)
{
if (replica->isReady())
{
addresses += (is_first ? "" : "; ") + replica->connection->getDescription();
is_first = false;
}
}
}
return addresses;
}
@ -182,14 +200,15 @@ void HedgedConnections::sendCancel()
{
std::lock_guard lock(cancel_mutex);
// LOG_DEBUG(log, "sendCancel");
if (!sent_query || cancelled)
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
if (replicas.first_replica->isReady())
replicas.first_replica->connection->sendCancel();
if (replicas.second_replica->isReady())
replicas.second_replica->connection->sendCancel();
for (auto & replicas_with_same_offset : replicas)
for (auto & replica : replicas_with_same_offset)
if (replica->isReady())
replica->connection->sendCancel();
cancelled = true;
}
@ -202,6 +221,8 @@ Packet HedgedConnections::drain()
if (!cancelled)
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
// LOG_DEBUG(log, "drain");
Packet res;
res.type = Protocol::Server::EndOfStream;
@ -250,23 +271,31 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback)
Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback)
{
// LOG_DEBUG(log, "sreceivePacketImpl");
int event_fd;
ReplicaStatePtr replica;
ReplicaStatePtr replica = nullptr;
Packet packet;
bool finish = false;
while (!finish)
{
event_fd = get_hedged_connections.getReadyFileDescriptor(epoll, async_callback);
event_fd = getReadyFileDescriptor(async_callback);
if (auto timeout_descriptor = get_hedged_connections.isEventTimeout(event_fd, replica))
processTimeoutEvent(replica, timeout_descriptor);
else if ((replica = get_hedged_connections.isEventReplica(event_fd)))
if (fd_to_replica.find(event_fd) != fd_to_replica.end())
{
// LOG_DEBUG(log, "event is replica");
replica = fd_to_replica[event_fd];
packet = receivePacketFromReplica(replica, async_callback);
finish = true;
}
else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end())
{
// LOG_DEBUG(log, "event is timeout");
replica = timeout_fd_to_replica[event_fd];
processTimeoutEvent(replica, replica->active_timeouts[event_fd].get());
}
else if (event_fd == get_hedged_connections.getFileDescriptor())
processGetHedgedConnectionsEvent();
tryGetNewReplica();
else
throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR);
}
@ -274,23 +303,33 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback)
return packet;
};
Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback)
int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback)
{
for (auto & [fd, replica] : fd_to_replica)
if (replica->connection->hasReadPendingData())
return replica->fd;
return epoll.getReady(std::move(async_callback)).data.fd;
}
Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback)
{
// LOG_DEBUG(log, "sreceivePacketFromReplica");
Packet packet = replica->connection->receivePacket(std::move(async_callback));
switch (packet.type)
{
case Protocol::Server::Data:
removeTimeoutsFromReplica(replica, epoll);
removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica);
processReceiveData(replica);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts());
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts());
break;
case Protocol::Server::Progress:
case Protocol::Server::ProfileInfo:
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::Log:
removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts());
removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica);
addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts());
break;
case Protocol::Server::EndOfStream:
@ -306,26 +345,29 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, Asyn
return packet;
}
void HedgedConnections::processReceiveData(ReplicaStatePtr replica)
void HedgedConnections::processReceiveData(ReplicaStatePtr & replica)
{
/// When we receive first packet of data from any replica, we continue working with this replica
/// and stop working with another replica (if there is another replica). If current replica is
/// second, move it to the first place.
if (replica == replicas.second_replica)
get_hedged_connections.swapReplicas();
/// and stop working with other replicas (if there are other replicas).
if (replicas.second_replica->isCannotChoose() || replicas.second_replica->isEmpty())
return;
// LOG_DEBUG(log, "processReceiveData");
if (replicas.second_replica->isNotReady())
offsets_with_received_data.insert(replica->parallel_replica_offset);
for (auto & other_replica : replicas[replica->parallel_replica_offset])
{
get_hedged_connections.stopChoosingSecondReplica();
epoll.remove(get_hedged_connections.getFileDescriptor());
if (other_replica->isReady() && other_replica != replica)
{
other_replica->connection->sendCancel();
finishProcessReplica(other_replica, true);
}
}
else if (replicas.second_replica->isReady())
if (get_hedged_connections.hasEventsInProcess() && offsets_with_received_data.size() == replicas.size())
{
replicas.second_replica->connection->sendCancel();
finishProcessReplica(replicas.second_replica, true);
get_hedged_connections.stopChoosingReplicas();
if (next_replica_in_process)
epoll.remove(get_hedged_connections.getFileDescriptor());
}
}
@ -333,57 +375,73 @@ void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDesc
{
epoll.remove(timeout_descriptor->getDescriptor());
replica->active_timeouts.erase(timeout_descriptor->getDescriptor());
timeout_fd_to_replica.erase(timeout_descriptor->getDescriptor());
if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT)
{
size_t offset = replica->parallel_replica_offset;
finishProcessReplica(replica, true);
if (!replicas.first_replica->isReady() && !replicas.second_replica->isNotReady())
/// Check if there is no active connection with same offset.
if (active_connections_count_by_offset[offset] == 0)
throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT);
}
else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT)
{
if (!replicas.second_replica->isEmpty())
throw Exception("Cannot start choosing second replica, it's not empty", ErrorCodes::LOGICAL_ERROR);
get_hedged_connections.chooseSecondReplica();
if (replicas.second_replica->isReady())
processChosenSecondReplica();
else if (replicas.second_replica->isNotReady())
epoll.add(get_hedged_connections.getFileDescriptor());
offsets_queue.push(replica->parallel_replica_offset);
tryGetNewReplica();
}
}
void HedgedConnections::processGetHedgedConnectionsEvent()
void HedgedConnections::tryGetNewReplica()
{
get_hedged_connections.chooseSecondReplica();
if (replicas.second_replica->isReady())
processChosenSecondReplica();
// LOG_DEBUG(log, "tryGetNewReplica");
if (!replicas.second_replica->isNotReady())
ReplicaStatePtr new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true);
/// Skip replicas with old server version if we didn't disable two-level aggregation in sendQuery.
while (new_replica->isReady() && !has_two_level_aggregation_incompatibility
&& new_replica->connection->getServerRevision(get_hedged_connections.getConnectionTimeouts()) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD)
new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true);
if (new_replica->isReady())
{
// LOG_DEBUG(log, "processNewReadyReplica");
new_replica->parallel_replica_offset = offsets_queue.front();
offsets_queue.pop();
replicas[new_replica->parallel_replica_offset].push_back(new_replica);
epoll.add(new_replica->fd);
fd_to_replica[new_replica->fd] = new_replica;
++active_connections_count_by_offset[new_replica->parallel_replica_offset];
pipeline_for_new_replicas.run(new_replica);
}
else if (new_replica->isNotReady() && !next_replica_in_process)
{
epoll.add(get_hedged_connections.getFileDescriptor());
next_replica_in_process = true;
}
if (next_replica_in_process && (new_replica->isCannotChoose() || offsets_queue.empty()))
{
epoll.remove(get_hedged_connections.getFileDescriptor());
next_replica_in_process = false;
}
}
void HedgedConnections::processChosenSecondReplica()
void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect)
{
second_replica_pipeline.run(replicas.second_replica);
// LOG_DEBUG(log, "finishProcessReplica");
/// In case when the first replica get receive timeout before the second is chosen,
/// we need to move the second replica to the first place
get_hedged_connections.swapReplicasIfNeeded();
}
void HedgedConnections::finishProcessReplica(ReplicaStatePtr replica, bool disconnect)
{
removeTimeoutsFromReplica(replica, epoll);
removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica);
epoll.remove(replica->fd);
fd_to_replica.erase(replica->fd);
--active_connections_count_by_offset[replica->parallel_replica_offset];
if (active_connections_count_by_offset[replica->parallel_replica_offset] == 0)
active_connections_count_by_offset.erase(replica->parallel_replica_offset);
if (disconnect)
replica->connection->disconnect();
replica->reset();
/// Move active connection to the first replica if it exists
get_hedged_connections.swapReplicasIfNeeded();
}
}

View File

@ -3,6 +3,7 @@
#include <Client/GetHedgedConnections.h>
#include <Client/IConnections.h>
#include <functional>
#include <queue>
namespace DB
{
@ -17,6 +18,7 @@ public:
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;
@ -45,47 +47,52 @@ public:
size_t size() const override;
bool hasActiveConnections() const override;
bool hasActiveConnections() const override { return !active_connections_count_by_offset.empty(); }
private:
class Pipeline
{
public:
void add(std::function<void(ReplicaStatePtr)> send_function);
void add(std::function<void(ReplicaStatePtr &)> send_function);
void run(ReplicaStatePtr replica);
void run(ReplicaStatePtr & replica);
bool empty() const { return pipeline.empty(); }
private:
std::vector<std::function<void(ReplicaStatePtr)>> pipeline;
std::vector<std::function<void(ReplicaStatePtr &)>> pipeline;
};
void processChosenSecondReplica();
Packet receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback = {});
Packet receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback = {});
Packet receivePacketImpl(AsyncCallback async_callback = {});
void processReceiveData(ReplicaStatePtr replica);
void processReceiveData(ReplicaStatePtr & replica);
void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor);
void processGetHedgedConnectionsEvent();
void tryGetNewReplica();
void removeReceiveTimeout(ReplicaStatePtr replica);
void finishProcessReplica(ReplicaStatePtr & replica, bool disconnect);
void finishProcessReplica(ReplicaStatePtr replica, bool disconnect);
int getReadyFileDescriptor(AsyncCallback async_callback = {});
GetHedgedConnections get_hedged_connections;
Replicas replicas;
std::vector<std::vector<ReplicaStatePtr>> replicas;
std::unordered_map<int, ReplicaStatePtr> fd_to_replica;
std::unordered_map<int, ReplicaStatePtr> timeout_fd_to_replica;
std::queue<int> offsets_queue;
Epoll epoll;
const Settings & settings;
ThrottlerPtr throttler;
Poco::Logger * log;
Pipeline second_replica_pipeline;
Pipeline pipeline_for_new_replicas;
bool sent_query = false;
bool cancelled = false;
std::unordered_map<size_t, size_t> active_connections_count_by_offset;
bool next_replica_in_process = false;
bool has_two_level_aggregation_incompatibility = false;
std::unordered_set<size_t> offsets_with_received_data;
mutable std::mutex cancel_mutex;
};

View File

@ -63,13 +63,13 @@ RemoteQueryExecutor::RemoteQueryExecutor(
const Settings & current_settings = context.getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
if (current_settings.use_hedged_requests && current_settings.max_parallel_replicas <= 1)
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, table_to_check);
return std::make_unique<HedgedConnections>(pool, current_settings, timeouts, throttler, pool_mode, table_to_check);
}
else
{