2012-05-17 19:15:53 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2015-09-29 19:19:54 +00:00
|
|
|
|
#include <common/logger_useful.h>
|
2012-10-16 18:12:29 +00:00
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
2015-02-10 20:48:17 +00:00
|
|
|
|
#include <DB/Common/Throttler.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
#include <DB/Interpreters/Context.h>
|
2012-10-22 19:55:19 +00:00
|
|
|
|
#include <DB/Client/ConnectionPool.h>
|
2015-11-06 17:44:01 +00:00
|
|
|
|
#include <DB/Client/MultiplexedConnections.h>
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/** Позволяет выполнить запрос на удалённых репликах одного шарда и получить результат.
|
2012-05-17 19:15:53 +00:00
|
|
|
|
*/
|
|
|
|
|
class RemoteBlockInputStream : public IProfilingBlockInputStream
|
|
|
|
|
{
|
2014-04-07 00:09:19 +00:00
|
|
|
|
public:
|
|
|
|
|
/// Принимает готовое соединение.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
RemoteBlockInputStream(Connection & connection_, const String & query_, const Settings * settings_,
|
|
|
|
|
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
|
|
|
|
|
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
|
|
|
|
const Context & context_ = getDefaultContext());
|
2014-04-07 00:09:19 +00:00
|
|
|
|
|
|
|
|
|
/// Принимает готовое соединение. Захватывает владение соединением из пула.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
RemoteBlockInputStream(ConnectionPool::Entry & pool_entry_, const String & query_, const Settings * settings_,
|
|
|
|
|
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
|
|
|
|
|
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
|
|
|
|
const Context & context_ = getDefaultContext());
|
2014-04-07 00:00:23 +00:00
|
|
|
|
|
2015-02-06 22:32:54 +00:00
|
|
|
|
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
RemoteBlockInputStream(IConnectionPool * pool_, const String & query_, const Settings * settings_,
|
|
|
|
|
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
|
|
|
|
|
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
|
|
|
|
const Context & context_ = getDefaultContext());
|
2012-10-22 19:55:19 +00:00
|
|
|
|
|
2015-11-11 02:13:24 +00:00
|
|
|
|
/// Принимает пулы - один для каждого шарда, из которых нужно будет достать одно или несколько соединений.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
RemoteBlockInputStream(ConnectionPoolsPtr & pools_, const String & query_, const Settings * settings_,
|
|
|
|
|
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
|
|
|
|
|
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
|
|
|
|
const Context & context_ = getDefaultContext());
|
2012-10-18 19:24:46 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
~RemoteBlockInputStream() override;
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Отправить запрос на все существующие реплики.
|
|
|
|
|
void doBroadcast();
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Кроме блоков, получить информацию о блоках.
|
|
|
|
|
void appendExtraInfo();
|
2013-05-03 10:20:53 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Отправляет запрос (инициирует вычисления) раньше, чем read.
|
|
|
|
|
void readPrefix() override;
|
2012-10-18 19:24:46 +00:00
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
/** Отменяем умолчальное уведомление о прогрессе,
|
|
|
|
|
* так как колбэк прогресса вызывается самостоятельно.
|
|
|
|
|
*/
|
2014-10-25 18:33:52 +00:00
|
|
|
|
void progress(const Progress & value) override {}
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void cancel() override;
|
2012-10-18 19:24:46 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
String getName() const override { return "Remote"; }
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
String getID() const override
|
2015-10-12 14:53:16 +00:00
|
|
|
|
{
|
2015-11-06 17:44:01 +00:00
|
|
|
|
std::stringstream res;
|
|
|
|
|
res << this;
|
|
|
|
|
return res.str();
|
2015-10-12 14:53:16 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
BlockExtraInfo getBlockExtraInfo() const override
|
|
|
|
|
{
|
2015-11-06 17:44:01 +00:00
|
|
|
|
return multiplexed_connections->getBlockExtraInfo();
|
2015-10-12 14:53:16 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-10-20 02:10:47 +00:00
|
|
|
|
protected:
|
2015-09-05 01:22:09 +00:00
|
|
|
|
/// Отправить на удаленные серверы все временные таблицы.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void sendExternalTables();
|
2014-03-13 15:00:06 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
Block readImpl() override;
|
2015-09-05 01:22:09 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void readSuffixImpl() override;
|
2013-09-13 20:33:09 +00:00
|
|
|
|
|
2015-02-07 19:39:16 +00:00
|
|
|
|
/// Создать объект для общения с репликами одного шарда, на которых должен выполниться запрос.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void createMultiplexedConnections();
|
2015-02-06 22:32:54 +00:00
|
|
|
|
|
2015-04-16 09:29:40 +00:00
|
|
|
|
/// Возвращает true, если запрос отправлен.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
bool isQueryPending() const;
|
2015-02-08 21:34:43 +00:00
|
|
|
|
|
|
|
|
|
/// Возвращает true, если исключение было выкинуто.
|
2015-11-06 17:44:01 +00:00
|
|
|
|
bool hasThrownException() const;
|
2015-02-06 12:33:15 +00:00
|
|
|
|
|
2015-02-21 13:59:15 +00:00
|
|
|
|
private:
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void init(const Settings * settings_);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
void sendQuery();
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
|
|
|
|
|
void tryCancel(const char * reason);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-02-21 13:59:15 +00:00
|
|
|
|
/// ITable::read requires a Context, therefore we should create one if the user can't supply it
|
|
|
|
|
static Context & getDefaultContext()
|
|
|
|
|
{
|
|
|
|
|
static Context instance;
|
|
|
|
|
return instance;
|
|
|
|
|
}
|
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
private:
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Готовое соединение.
|
2014-04-07 00:09:19 +00:00
|
|
|
|
ConnectionPool::Entry pool_entry;
|
|
|
|
|
Connection * connection = nullptr;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
|
|
|
|
/// Пул соединений одного шарда.
|
|
|
|
|
IConnectionPool * pool = nullptr;
|
|
|
|
|
|
|
|
|
|
/// Пулы соединений одного или нескольких шардов.
|
|
|
|
|
ConnectionPoolsPtr pools;
|
|
|
|
|
|
|
|
|
|
std::unique_ptr<MultiplexedConnections> multiplexed_connections;
|
2015-01-14 10:06:30 +00:00
|
|
|
|
|
2012-05-23 19:51:30 +00:00
|
|
|
|
const String query;
|
2013-02-01 19:02:04 +00:00
|
|
|
|
bool send_settings;
|
|
|
|
|
Settings settings;
|
2015-02-10 20:48:17 +00:00
|
|
|
|
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.
|
|
|
|
|
ThrottlerPtr throttler;
|
2014-03-14 15:42:30 +00:00
|
|
|
|
/// Временные таблицы, которые необходимо переслать на удаленные сервера.
|
2014-03-13 15:00:06 +00:00
|
|
|
|
Tables external_tables;
|
2012-05-17 19:15:53 +00:00
|
|
|
|
QueryProcessingStage::Enum stage;
|
2014-12-17 11:53:17 +00:00
|
|
|
|
Context context;
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
2015-07-27 17:36:52 +00:00
|
|
|
|
/// Потоки для чтения из временных таблиц - для последующей отправки данных на удалённые серверы для GLOBAL-подзапросов.
|
|
|
|
|
std::vector<ExternalTablesData> external_tables_data;
|
|
|
|
|
std::mutex external_tables_mutex;
|
|
|
|
|
|
2015-02-21 13:59:15 +00:00
|
|
|
|
/// Установили соединения с репликами, но ещё не отправили запрос.
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> established { false };
|
2015-02-21 13:59:15 +00:00
|
|
|
|
|
2012-10-18 19:00:49 +00:00
|
|
|
|
/// Отправили запрос (это делается перед получением первого блока).
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> sent_query { false };
|
2014-07-06 21:59:20 +00:00
|
|
|
|
|
2015-02-07 17:12:29 +00:00
|
|
|
|
/** Получили все данные от всех реплик, до пакета EndOfStream.
|
2012-10-18 19:00:49 +00:00
|
|
|
|
* Если при уничтожении объекта, ещё не все данные считаны,
|
2015-02-07 17:12:29 +00:00
|
|
|
|
* то для того, чтобы не было рассинхронизации, на реплики отправляются просьбы прервать выполнение запроса,
|
2012-10-18 19:00:49 +00:00
|
|
|
|
* и после этого считываются все пакеты до EndOfStream.
|
|
|
|
|
*/
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> finished { false };
|
2014-07-06 21:59:20 +00:00
|
|
|
|
|
2015-02-07 19:39:16 +00:00
|
|
|
|
/** На каждую реплику была отправлена просьба прервать выполнение запроса, так как данные больше не нужны.
|
2012-10-18 19:00:49 +00:00
|
|
|
|
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT),
|
|
|
|
|
* или если на стороне клиента произошло исключение.
|
|
|
|
|
*/
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> was_cancelled { false };
|
2012-10-16 18:12:29 +00:00
|
|
|
|
|
2015-02-07 19:39:16 +00:00
|
|
|
|
/** С одной репилки было получено исключение. В этом случае получать больше пакетов или
|
|
|
|
|
* просить прервать запрос на этой реплике не нужно.
|
|
|
|
|
*/
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> got_exception_from_replica { false };
|
2015-02-07 19:39:16 +00:00
|
|
|
|
|
|
|
|
|
/** С одной реплики был получен неизвестный пакет. В этом случае получать больше пакетов или
|
|
|
|
|
* просить прервать запрос на этой реплике не нужно.
|
|
|
|
|
*/
|
2015-03-06 00:01:59 +00:00
|
|
|
|
std::atomic<bool> got_unknown_packet_from_replica { false };
|
2012-10-18 19:00:49 +00:00
|
|
|
|
|
2015-10-12 14:53:16 +00:00
|
|
|
|
bool append_extra_info = false;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
bool do_broadcast = false;
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2014-04-07 00:00:23 +00:00
|
|
|
|
Logger * log = &Logger::get("RemoteBlockInputStream");
|
2012-05-17 19:15:53 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
}
|