ClickHouse/src/Client/MultiplexedConnections.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

116 lines
3.8 KiB
C++
Raw Normal View History

#pragma once
#include <mutex>
#include <Common/Throttler.h>
#include <Client/Connection.h>
#include <Client/ConnectionPoolWithFailover.h>
#include <IO/ConnectionTimeouts.h>
2021-01-19 19:21:06 +00:00
#include <Client/IConnections.h>
namespace DB
{
/** To retrieve data directly from multiple replicas (connections) from one shard
2017-03-09 00:56:38 +00:00
* within a single thread. As a degenerate case, it can also work with one connection.
* It is assumed that all functions except sendCancel are always executed in one thread.
*
2017-03-09 00:56:38 +00:00
* The interface is almost the same as Connection.
*/
2021-01-19 19:21:06 +00:00
class MultiplexedConnections final : public IConnections
{
public:
2017-03-09 04:26:17 +00:00
/// Accepts ready connection.
MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler_);
/// Accepts ready connection and keep it alive before drain
MultiplexedConnections(std::shared_ptr<Connection> connection_, const Settings & settings_, const ThrottlerPtr & throttler_);
2018-11-28 14:33:40 +00:00
/// Accepts a vector of connections to replicas of one shard already taken from pool.
MultiplexedConnections(
2018-11-28 14:33:40 +00:00
std::vector<IConnectionPool::Entry> && connections,
const Settings & settings_, const ThrottlerPtr & throttler_);
2021-01-19 19:21:06 +00:00
void sendScalarsData(Scalars & data) override;
void sendExternalTablesData(std::vector<ExternalTablesData> & data) override;
void sendQuery(
const ConnectionTimeouts & timeouts,
const String & query,
2020-05-17 05:45:20 +00:00
const String & query_id,
UInt64 stage,
ClientInfo & client_info,
2021-01-19 19:21:06 +00:00
bool with_pending_data) override;
2021-04-13 10:59:02 +00:00
void sendReadTaskResponse(const String &) override;
2023-02-03 13:34:18 +00:00
void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override;
2021-01-19 19:21:06 +00:00
Packet receivePacket() override;
2021-01-19 19:21:06 +00:00
void disconnect() override;
2021-01-19 19:21:06 +00:00
void sendCancel() override;
/// Send parts' uuids to replicas to exclude them from query processing
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids) override;
2021-01-19 19:21:06 +00:00
Packet drain() override;
2021-01-19 19:21:06 +00:00
std::string dumpAddresses() const override;
2017-03-09 04:26:17 +00:00
/// Without locking, because sendCancel() does not change this number.
2021-01-19 19:21:06 +00:00
size_t size() const override { return replica_states.size(); }
2017-03-09 04:26:17 +00:00
/// Without locking, because sendCancel() does not change the state of the replicas.
2021-01-19 19:21:06 +00:00
bool hasActiveConnections() const override { return active_connection_count > 0; }
void setReplicaInfo(ReplicaInfo value) override { replica_info = value; }
2023-03-03 19:30:43 +00:00
void setAsyncCallback(AsyncCallback async_callback) override;
private:
Packet receivePacketUnlocked(AsyncCallback async_callback) override;
/// Internal version of `dumpAddresses` function without locking.
std::string dumpAddressesUnlocked() const;
2017-03-09 04:26:17 +00:00
/// Description of a single replica.
2015-11-06 17:44:01 +00:00
struct ReplicaState
{
Connection * connection = nullptr;
ConnectionPool::Entry pool_entry;
2015-11-06 17:44:01 +00:00
};
2017-03-09 04:26:17 +00:00
/// Get a replica where you can read the data.
ReplicaState & getReplicaForReading();
2017-03-09 00:56:38 +00:00
/// Mark the replica as invalid.
void invalidateReplica(ReplicaState & replica_state);
const Settings & settings;
2015-11-06 17:44:01 +00:00
/// The current number of valid connections to the replicas of this shard.
size_t active_connection_count = 0;
std::vector<ReplicaState> replica_states;
std::unordered_map<int, size_t> fd_to_replica_state_idx;
2017-03-09 00:56:38 +00:00
/// Connection that received last block.
Connection * current_connection = nullptr;
/// Shared connection, may be empty. Used to keep object alive before draining.
std::shared_ptr<Connection> connection_ptr;
2015-10-12 14:53:16 +00:00
bool sent_query = false;
bool cancelled = false;
2023-02-03 13:34:18 +00:00
/// std::nullopt if parallel reading from replicas is not used
std::optional<ReplicaInfo> replica_info;
2017-03-09 00:56:38 +00:00
/// A mutex for the sendCancel function to execute safely
/// in separate thread.
mutable std::mutex cancel_mutex;
2020-12-04 13:35:24 +00:00
friend struct RemoteQueryExecutorRoutine;
};
}