ClickHouse/dbms/include/DB/Client/MultiplexedConnections.h

166 lines
5.4 KiB
C++
Raw Normal View History

#pragma once
#include <DB/Common/Throttler.h>
#include <DB/Client/Connection.h>
#include <DB/Client/ConnectionPool.h>
#include <Poco/ScopedLock.h>
#include <mutex>
namespace DB
{
2017-03-09 00:56:38 +00:00
/** To retrieve data directly from multiple replicas (connections) from one or several shards
* 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.
*/
2015-11-06 17:44:01 +00:00
class MultiplexedConnections final : private boost::noncopyable
{
public:
2017-03-09 00:56:38 +00:00
/// Accepts ready connection.
2015-11-06 17:44:01 +00:00
MultiplexedConnections(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
2017-03-09 00:56:38 +00:00
/** Accepts a pool from which it will be necessary to get one or more connections.
* If the append_extra_info flag is set, additional information appended to each received block.
* If the get_all_replicas flag is set, all connections are selected.
2015-10-12 14:53:16 +00:00
*/
2015-11-06 17:44:01 +00:00
MultiplexedConnections(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_,
2016-03-01 17:47:53 +00:00
bool append_extra_info = false, PoolMode pool_mode_ = PoolMode::GET_MANY);
2015-11-06 17:44:01 +00:00
2017-03-09 00:56:38 +00:00
/** Accepts pools, one for each shard, from which one will need to get one or more connections.
* If the append_extra_info flag is set, additional information appended to each received block.
* If the do_broadcast flag is set, all connections are received.
2015-11-06 17:44:01 +00:00
*/
MultiplexedConnections(ConnectionPools & pools_, const Settings * settings_, ThrottlerPtr throttler_,
2016-03-01 17:47:53 +00:00
bool append_extra_info = false, PoolMode pool_mode_ = PoolMode::GET_MANY);
2017-03-09 00:56:38 +00:00
/// Send all content of external tables to replicas.
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
2017-03-09 00:56:38 +00:00
/// Send request to replicas.
void sendQuery(
const String & query,
const String & query_id = "",
UInt64 stage = QueryProcessingStage::Complete,
const ClientInfo * client_info = nullptr,
bool with_pending_data = false);
2017-03-09 00:56:38 +00:00
/// Get package from any replica.
Connection::Packet receivePacket();
2017-03-09 00:56:38 +00:00
/// Get information about the last received package.
2015-10-12 14:53:16 +00:00
BlockExtraInfo getBlockExtraInfo() const;
2017-03-09 00:56:38 +00:00
/// Break all active connections.
void disconnect();
2017-03-09 00:56:38 +00:00
/// Send a request to the replica to cancel the request
void sendCancel();
2017-03-09 00:56:38 +00:00
/** 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.
*/
Connection::Packet drain();
2017-03-09 00:56:38 +00:00
/// Get the replica addresses as a string.
std::string dumpAddresses() const;
2017-03-09 00:56:38 +00:00
/// Returns the number of replicas.
/// Without locking, because sendCancel() does not change this number.
size_t size() const { return replica_map.size(); }
2017-03-09 00:56:38 +00:00
/// Check if there are any valid replicas.
/// Without locking, because sendCancel() does not change the state of the replicas.
2015-11-06 17:44:01 +00:00
bool hasActiveConnections() const { return active_connection_total_count > 0; }
private:
2017-03-09 00:56:38 +00:00
/// Connections of the 1st shard, then the connections of the 2nd shard, etc.
2015-11-06 17:44:01 +00:00
using Connections = std::vector<Connection *>;
2017-03-09 00:56:38 +00:00
/// The state of the connections of one shard.
2015-11-06 17:44:01 +00:00
struct ShardState
{
2017-03-09 00:56:38 +00:00
/// The number of connections allocated, i.e. replicas for this shard.
2015-11-06 17:44:01 +00:00
size_t allocated_connection_count;
2017-03-09 00:56:38 +00:00
/// The current number of valid connections to the replicas of this shard.
2015-11-06 17:44:01 +00:00
size_t active_connection_count;
};
2017-03-09 00:56:38 +00:00
/// Description of a single replica.
2015-11-06 17:44:01 +00:00
struct ReplicaState
{
size_t connection_index;
2017-03-09 00:56:38 +00:00
/// The owner of this replica.
2015-11-06 17:44:01 +00:00
ShardState * shard_state;
};
2017-03-09 00:56:38 +00:00
/// Replicas hashed by id of the socket.
2015-11-06 17:44:01 +00:00
using ReplicaMap = std::unordered_map<int, ReplicaState>;
2017-03-09 00:56:38 +00:00
/// The state of each shard.
2015-11-06 17:44:01 +00:00
using ShardStates = std::vector<ShardState>;
private:
2015-11-06 17:44:01 +00:00
void initFromShard(IConnectionPool * pool);
2017-03-09 00:56:38 +00:00
/// Register shards.
2015-11-06 17:44:01 +00:00
void registerShards();
2017-03-09 00:56:38 +00:00
/// Register replicas of one shard.
2015-11-06 17:44:01 +00:00
void registerReplicas(size_t index_begin, size_t index_end, ShardState & shard_state);
2017-03-09 00:56:38 +00:00
/// Interval version of `receivePacket` function without blocking.
Connection::Packet receivePacketUnlocked();
2017-03-09 00:56:38 +00:00
/// Interval version of `dumpAddresses` function without blocking.
2015-09-17 20:00:19 +00:00
std::string dumpAddressesUnlocked() const;
2017-03-09 00:56:38 +00:00
/// Get a replica where you can read the data.
ReplicaMap::iterator getReplicaForReading();
2017-03-09 00:56:38 +00:00
/** Check if there are any data that can be read on any of the replicas.
* Returns one such replica if it exists.
*/
ReplicaMap::iterator waitForReadEvent();
2017-03-09 00:56:38 +00:00
/// Mark the replica as invalid.
void invalidateReplica(ReplicaMap::iterator it);
private:
const Settings * settings;
2015-11-06 17:44:01 +00:00
Connections connections;
ReplicaMap replica_map;
2015-11-06 17:44:01 +00:00
ShardStates shard_states;
2017-03-09 00:56:38 +00:00
/// If not nullptr, then it is used to restrict network traffic.
ThrottlerPtr throttler;
std::vector<ConnectionPool::Entry> pool_entries;
2017-03-09 00:56:38 +00:00
/// Connection that received last block.
2015-10-12 14:53:16 +00:00
Connection * current_connection;
2017-03-09 00:56:38 +00:00
/// Information about the last received block, if supported.
2015-10-12 14:53:16 +00:00
std::unique_ptr<BlockExtraInfo> block_extra_info;
2017-03-09 00:56:38 +00:00
/// The current number of valid connections to replicas.
2015-11-06 17:44:01 +00:00
size_t active_connection_total_count = 0;
2017-03-09 00:56:38 +00:00
/// The query is run in parallel on multiple replicas.
bool supports_parallel_execution;
2017-03-09 00:56:38 +00:00
/// Send the request
bool sent_query = false;
2017-03-09 00:56:38 +00:00
/// Cancel request
bool cancelled = false;
2016-03-01 17:47:53 +00:00
PoolMode pool_mode = PoolMode::GET_MANY;
2015-11-06 17:44:01 +00:00
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;
};
}