Small fixes and tests updates

This commit is contained in:
Pavel Kruglov 2021-02-26 18:53:40 +03:00
parent 24e9abddb8
commit 91f6f9227b
9 changed files with 196 additions and 117 deletions

View File

@ -118,11 +118,9 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync(
void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &)
{
if (connection_establisher_async.socket_fd != fd)
/// Check if it's the first time and we need to add socket fd to epoll.
if (connection_establisher_async.socket_fd == -1)
{
if (connection_establisher_async.socket_fd != -1)
connection_establisher_async.epoll.remove(connection_establisher_async.socket_fd);
connection_establisher_async.epoll.add(fd);
connection_establisher_async.socket_fd = fd;
}

View File

@ -146,11 +146,7 @@ void HedgedConnections::sendQuery(
if (!disable_two_level_aggregation)
{
/// Tell hedged_connections_factory to skip replicas that doesn't support two-level aggregation.
hedged_connections_factory.setSkipPredicate(
[timeouts](Connection * connection)
{
return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD;
});
hedged_connections_factory.skipReplicasWithTwoLevelAggregationIncompatibility();
}
auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica)
@ -295,6 +291,15 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback)
HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback)
{
/// Firstly, resume replica with the last received packet if needed.
if (replica_with_last_received_packet)
{
ReplicaLocation location = replica_with_last_received_packet.value();
replica_with_last_received_packet.reset();
if (resumePacketReceiver(location))
return location;
}
int event_fd;
while (true)
{
@ -306,22 +311,8 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As
else if (fd_to_replica_location.contains(event_fd))
{
ReplicaLocation location = fd_to_replica_location[event_fd];
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));
if (resumePacketReceiver(location))
return location;
}
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);
}
}
else if (timeout_fd_to_replica_location.contains(event_fd))
{
@ -336,6 +327,28 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As
}
};
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;
@ -359,6 +372,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli
case Protocol::Server::Data:
if (!offset_states[replica_location.offset].first_packet_of_data_received)
processReceivedFirstDataPacket(replica_location);
replica_with_last_received_packet = replica_location;
break;
case Protocol::Server::PartUUIDs:
case Protocol::Server::Progress:
@ -366,6 +380,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::Log:
replica_with_last_received_packet = replica_location;
break;
case Protocol::Server::EndOfStream:
@ -423,7 +438,7 @@ void HedgedConnections::startNewReplica()
void HedgedConnections::checkNewReplica()
{
Connection * connection = nullptr;
HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(/*blocking = */false, connection);
HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(connection);
processNewReplicaState(state, connection);

View File

@ -3,6 +3,8 @@
#include <functional>
#include <queue>
#include <optional>
#include <Client/HedgedConnectionsFactory.h>
#include <Client/IConnections.h>
#include <Client/PacketReceiver.h>
@ -118,6 +120,8 @@ private:
ReplicaLocation getReadyReplicaLocation(AsyncCallback async_callback = {});
bool resumePacketReceiver(const ReplicaLocation & replica_location);
void processReceivedFirstDataPacket(const ReplicaLocation & replica_location);
void startNewReplica();
@ -162,6 +166,11 @@ private:
/// 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;

View File

@ -40,17 +40,25 @@ std::vector<Connection *> HedgedConnectionsFactory::getManyConnections(PoolMode
size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1;
size_t max_entries;
if (pool_mode == PoolMode::GET_ALL)
switch (pool_mode)
{
min_entries = shuffled_pools.size();
max_entries = shuffled_pools.size();
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;
}
}
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<Connection *> connections;
connections.reserve(max_entries);
@ -74,7 +82,7 @@ std::vector<Connection *> HedgedConnectionsFactory::getManyConnections(PoolMode
while (connections.size() < max_entries)
{
/// Set blocking = true to avoid busy-waiting here.
auto state = waitForReadyConnections(/*blocking = */true, connection);
auto state = waitForReadyConnectionsImpl(/*blocking = */true, connection);
if (state == State::READY)
connections.push_back(connection);
else if (state == State::CANNOT_CHOOSE)
@ -111,7 +119,12 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Con
return state;
}
HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(bool blocking, Connection *& connection_out)
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)
@ -254,7 +267,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnect
if (result.is_up_to_date)
{
++up_to_date_count;
if (!skip_predicate || !skip_predicate(&*result.entry))
if (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry))
{
replicas[index].is_ready = true;
++ready_replicas_count;
@ -343,7 +356,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C
if (!result.entry.isNull()
&& result.is_usable
&& !replicas[i].is_ready
&& (!skip_predicate || !skip_predicate(&*result.entry)))
&& (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry)))
indexes.push_back(i);
}
@ -365,5 +378,10 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C
return State::READY;
}
bool HedgedConnectionsFactory::isTwoLevelAggregationIncompatible(Connection * connection)
{
return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD;
}
}
#endif

View File

@ -53,12 +53,11 @@ public:
/// Create and return active connections according to pool_mode.
std::vector<Connection *> getManyConnections(PoolMode pool_mode);
/// Try to get connection to the new replica. If start_new_connection is true, we start establishing connection
/// with the new replica. Process all current events in epoll (connections, timeouts),
/// if there is no events in epoll and blocking is false, return NOT_READY.
/// Returned state might be READY, NOT_READY and CANNOT_CHOOSE.
/// 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(bool blocking, Connection *& connection_out);
State waitForReadyConnections(Connection *& connection_out);
State startNewConnection(Connection *& connection_out);
@ -73,11 +72,14 @@ public:
int numberOfProcessingReplicas() const;
void setSkipPredicate(std::function<bool(Connection *)> pred) { skip_predicate = std::move(pred); }
/// 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);
@ -104,6 +106,8 @@ private:
State setBestUsableReplica(Connection *& connection_out);
bool isTwoLevelAggregationIncompatible(Connection * connection);
const ConnectionPoolWithFailoverPtr pool;
const Settings * settings;
const ConnectionTimeouts timeouts;
@ -117,7 +121,9 @@ private:
/// Map timeout for changing replica to replica index.
std::unordered_map<int, int> timeout_fd_to_replica_index;
std::function<bool(Connection *)> skip_predicate;
/// 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;
@ -125,14 +131,27 @@ private:
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;
size_t requested_connections_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;
};
}

View File

@ -33,19 +33,9 @@ public:
epoll.add(receive_timeout.getDescriptor());
epoll.add(connection->getSocket()->impl()->sockfd());
if (-1 == pipe2(pipe_fd, O_NONBLOCK))
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE);
epoll.add(pipe_fd[0]);
fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this});
}
~PacketReceiver()
{
close(pipe_fd[0]);
close(pipe_fd[1]);
}
/// Resume packet receiving.
std::variant<int, Packet, Poco::Timespan> resume()
{
@ -64,22 +54,6 @@ public:
if (is_read_in_process)
return epoll.getFileDescriptor();
/// Write something in pipe when buffer has pending data, because
/// in this case socket won't be ready in epoll but we need to tell
/// outside that there is more data in buffer.
if (connection->hasReadPendingData())
{
uint64_t buf = 0;
while (-1 == write(pipe_fd[1], &buf, sizeof(buf)))
{
if (errno == EAGAIN)
break;
if (errno != EINTR)
throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET);
}
}
/// Receiving packet was finished.
return std::move(packet);
}
@ -98,7 +72,6 @@ private:
bool checkReceiveTimeout()
{
bool is_socket_ready = false;
bool is_pipe_ready = false;
bool is_receive_timeout_expired = false;
epoll_event events[2];
@ -109,18 +82,10 @@ private:
{
if (events[i].data.fd == connection->getSocket()->impl()->sockfd())
is_socket_ready = true;
if (events[i].data.fd == pipe_fd[0])
is_pipe_ready = true;
if (events[i].data.fd == receive_timeout.getDescriptor())
is_receive_timeout_expired = true;
}
if (is_pipe_ready)
{
drainPipe();
return true;
}
if (is_receive_timeout_expired && !is_socket_ready)
{
receive_timeout.reset();
@ -130,23 +95,6 @@ private:
return true;
}
void drainPipe()
{
uint64_t buf;
while (true)
{
ssize_t res = read(pipe_fd[0], &buf, sizeof(buf));
if (res < 0)
{
if (errno == EAGAIN)
break;
if (errno != EINTR)
throwFromErrno("Cannot drain pipe_fd", ErrorCodes::CANNOT_READ_FROM_SOCKET);
}
}
}
struct Routine
{
PacketReceiver & receiver;
@ -209,12 +157,6 @@ private:
/// in epoll, so we can return epoll file descriptor outside for polling.
Epoll epoll;
/// Pipe is used when there is pending data in buffer
/// after receiving packet socket won't be ready in epoll in this case),
/// so we add pipe_fd in epoll and write something in it to tell
/// outside that we are ready to receive new packet.
int pipe_fd[2];
/// If and exception occurred in fiber resume, we save it and rethrow.
std::exception_ptr exception;

View File

@ -3,6 +3,8 @@
<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>

View File

@ -8,6 +8,7 @@ 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)}
@ -18,7 +19,6 @@ 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'])
@ -54,28 +54,43 @@ config = '''<yandex>
</yandex>'''
def check_query():
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()
NODES['node'].query("SELECT * FROM distributed");
result = NODES['node'].query("SELECT hostName(), id FROM distributed SETTINGS receive_timeout={}".format(receive_timeout));
query_time = time.time() - start
assert TSV(result) == TSV(expected_replica + "\t1")
print("Query time:", query_time)
assert query_time < 10
def test_stuck_replica(started_cluster):
cluster.pause_container("node_1")
check_query(expected_replica="node_2")
cluster.unpause_container("node_1")
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))
time.sleep(2)
check_query()
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))
time.sleep(2)
check_query(expected_replica="node_2")
def test_send_table_status_sleep2(started_cluster):
NODES['node_1'].replace_config(
@ -86,8 +101,12 @@ def test_send_table_status_sleep2(started_cluster):
'/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))
time.sleep(2)
check_query()
check_query(expected_replica="node_3")
def test_send_data(started_cluster):
@ -95,9 +114,17 @@ def test_send_data(started_cluster):
'/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))
time.sleep(2)
check_query()
check_query(expected_replica="node_2")
def test_send_data2(started_cluster):
@ -109,8 +136,12 @@ def test_send_data2(started_cluster):
'/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))
time.sleep(2)
check_query()
check_query(expected_replica="node_3")
def test_combination1(started_cluster):
@ -123,7 +154,7 @@ def test_combination1(started_cluster):
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
time.sleep(2)
check_query()
check_query(expected_replica="node_3")
def test_combination2(started_cluster):
@ -135,8 +166,12 @@ def test_combination2(started_cluster):
'/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))
time.sleep(2)
check_query()
check_query(expected_replica="node_3")
def test_combination3(started_cluster):
@ -153,7 +188,7 @@ def test_combination3(started_cluster):
config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time))
time.sleep(2)
check_query()
check_query(expected_replica="node_2")
def test_combination4(started_cluster):
@ -170,5 +205,44 @@ def test_combination4(started_cluster):
config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0))
time.sleep(2)
check_query()
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))
time.sleep(2)
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))
time.sleep(2)
check_query(expected_replica="node_2", receive_timeout=3)

View File

@ -4,6 +4,8 @@
<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>