ClickHouse/src/Client/HedgedConnections.h

155 lines
5.1 KiB
C++
Raw Normal View History

2021-01-19 19:21:06 +00:00
#pragma once
2021-01-29 15:46:28 +00:00
#if defined(OS_LINUX)
2021-01-19 19:21:06 +00:00
#include <functional>
#include <queue>
2021-02-06 00:54:27 +00:00
#include <Client/HedgedConnectionsFactory.h>
#include <Client/IConnections.h>
2021-01-19 19:21:06 +00:00
namespace DB
{
2021-02-06 00:54:27 +00:00
/** 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 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.
*/
2021-01-19 19:21:06 +00:00
class HedgedConnections : public IConnections
{
public:
2021-02-06 00:54:27 +00:00
struct ReplicaState
{
Connection * connection = nullptr;
std::unordered_map<int, ConnectionTimeoutDescriptorPtr> active_timeouts;
};
struct ReplicaLocation
{
size_t offset;
size_t index;
};
struct OffsetState
{
std::vector<ReplicaState> replicas;
size_t active_connection_count;
bool first_packet_of_data_received;
};
2021-01-19 19:21:06 +00:00
HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_,
const Settings & settings_,
const ConnectionTimeouts & timeouts_,
const ThrottlerPtr & throttler,
PoolMode pool_mode,
2021-01-19 19:21:06 +00:00
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;
2021-01-29 15:46:28 +00:00
Packet receivePacketUnlocked(AsyncCallback async_callback) override;
2021-01-19 19:21:06 +00:00
void disconnect() override;
void sendCancel() override;
Packet drain() override;
std::string dumpAddresses() const override;
2021-02-06 00:54:27 +00:00
size_t size() const override { return offset_states.size(); }
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
bool hasActiveConnections() const override { return active_connection_count > 0; }
2021-01-19 19:21:06 +00:00
private:
2021-02-02 12:14:31 +00:00
/// We will save actions with replicas in pipeline to perform them on the new replicas.
2021-01-19 19:21:06 +00:00
class Pipeline
{
public:
2021-02-06 00:54:27 +00:00
void add(std::function<void(ReplicaState &)> send_function);
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
void run(ReplicaState & replica);
2021-01-19 19:21:06 +00:00
private:
2021-02-06 00:54:27 +00:00
std::vector<std::function<void(ReplicaState &)>> pipeline;
2021-01-19 19:21:06 +00:00
};
2021-02-06 00:54:27 +00:00
Packet receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback = {});
2021-01-19 19:21:06 +00:00
Packet receivePacketImpl(AsyncCallback async_callback = {});
2021-02-06 00:54:27 +00:00
void processReceivedFirstDataPacket(ReplicaLocation & replica_location);
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
void processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor);
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
void tryGetNewReplica(bool start_new_connection);
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
void finishProcessReplica(ReplicaState & replica, bool disconnect);
2021-01-19 19:21:06 +00:00
int getReadyFileDescriptor(AsyncCallback async_callback = {});
2021-01-19 19:21:06 +00:00
2021-02-06 00:54:27 +00:00
void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica);
void removeTimeoutsFromReplica(ReplicaState & replica);
void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica);
HedgedConnectionsFactory hedged_connections_factory;
2021-02-02 12:14:31 +00:00
2021-02-06 00:54:27 +00:00
/// All replicas in offset_states[offset] is responsible for process query
2021-02-02 12:14:31 +00:00
/// with setting parallel_replica_offset = offset. In common situations
2021-02-06 00:54:27 +00:00
/// replica_states[offset].replicas.size() = 1 (like in MultiplexedConnections).
std::vector<OffsetState> offset_states;
2021-02-02 12:14:31 +00:00
2021-02-06 00:54:27 +00:00
/// 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 timeout file descriptor to replica location (it's offset and index in OffsetState.replicas).
std::unordered_map<int, ReplicaLocation> timeout_fd_to_replica_location;
2021-02-02 12:14:31 +00:00
/// 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;
2021-02-02 12:14:31 +00:00
2021-02-06 00:54:27 +00:00
/// The current number of valid connections to the replicas of this shard.
size_t active_connection_count;
2021-02-02 12:14:31 +00:00
2021-02-06 00:54:27 +00:00
/// We count offsets which received first packet of data,
/// it's needed to cancel choosing new replicas when all offsets
/// received their first packet of data.
size_t offsets_with_received_first_data_packet;
2021-02-02 12:14:31 +00:00
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;
2021-02-06 00:54:27 +00:00
/// This flag means we need to get connection with new replica, but no replica is ready.
/// When it's true, hedged_connections_factory.getFileDescriptor() is in epoll.
2021-02-02 12:14:31 +00:00
bool next_replica_in_process = false;
2021-01-19 19:21:06 +00:00
Epoll epoll;
const Settings & settings;
ThrottlerPtr throttler;
Poco::Logger * log;
bool sent_query = false;
bool cancelled = false;
mutable std::mutex cancel_mutex;
};
}
2021-01-29 15:46:28 +00:00
#endif