2012-05-17 19:15:53 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2012-10-16 18:12:29 +00:00
|
|
|
|
#include <Yandex/logger_useful.h>
|
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
2014-01-22 14:24:05 +00:00
|
|
|
|
#include <DB/Common/VirtualColumnUtils.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-05-17 19:15:53 +00:00
|
|
|
|
|
2012-10-22 19:55:19 +00:00
|
|
|
|
#include <DB/Client/ConnectionPool.h>
|
2015-02-04 10:27:06 +00:00
|
|
|
|
#include <DB/Client/ParallelReplicas.h>
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2015-02-07 17:12:29 +00:00
|
|
|
|
/** Позволяет выполнить запрос (SELECT) на удалённых репликах одного шарда и получить результат.
|
2012-05-17 19:15:53 +00:00
|
|
|
|
*/
|
|
|
|
|
class RemoteBlockInputStream : public IProfilingBlockInputStream
|
|
|
|
|
{
|
2014-04-07 00:09:19 +00:00
|
|
|
|
private:
|
|
|
|
|
void init(const Settings * settings_)
|
2014-04-07 00:00:23 +00:00
|
|
|
|
{
|
|
|
|
|
if (settings_)
|
|
|
|
|
{
|
|
|
|
|
send_settings = true;
|
|
|
|
|
settings = *settings_;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
send_settings = false;
|
|
|
|
|
}
|
2015-01-14 10:06:30 +00:00
|
|
|
|
|
2014-04-07 00:09:19 +00:00
|
|
|
|
public:
|
|
|
|
|
/// Принимает готовое соединение.
|
2015-02-10 20:48:17 +00:00
|
|
|
|
RemoteBlockInputStream(Connection & connection_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
2014-12-19 14:56:18 +00:00
|
|
|
|
const Context & context = getDefaultContext())
|
2015-02-10 20:48:17 +00:00
|
|
|
|
: connection(&connection_), query(query_), throttler(throttler_), external_tables(external_tables_), stage(stage_), context(context)
|
2014-04-07 00:09:19 +00:00
|
|
|
|
{
|
|
|
|
|
init(settings_);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Принимает готовое соединение. Захватывает владение соединением из пула.
|
2015-02-10 20:48:17 +00:00
|
|
|
|
RemoteBlockInputStream(ConnectionPool::Entry & pool_entry_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
2014-12-19 14:56:18 +00:00
|
|
|
|
const Context & context = getDefaultContext())
|
2015-02-10 20:48:17 +00:00
|
|
|
|
: pool_entry(pool_entry_), connection(&*pool_entry_), query(query_), throttler(throttler_),
|
2014-12-17 11:53:17 +00:00
|
|
|
|
external_tables(external_tables_), stage(stage_), context(context)
|
2014-04-07 00:09:19 +00:00
|
|
|
|
{
|
|
|
|
|
init(settings_);
|
|
|
|
|
}
|
2014-04-07 00:00:23 +00:00
|
|
|
|
|
2015-02-06 22:32:54 +00:00
|
|
|
|
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
|
2015-02-10 20:48:17 +00:00
|
|
|
|
RemoteBlockInputStream(IConnectionPool * pool_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
|
2014-12-19 14:56:18 +00:00
|
|
|
|
const Context & context = getDefaultContext())
|
2015-02-10 20:48:17 +00:00
|
|
|
|
: pool(pool_), query(query_), throttler(throttler_), external_tables(external_tables_), stage(stage_), context(context)
|
2012-10-22 19:55:19 +00:00
|
|
|
|
{
|
2014-04-07 00:09:19 +00:00
|
|
|
|
init(settings_);
|
2012-10-22 19:55:19 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-10-18 19:24:46 +00:00
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
String getName() const override { return "RemoteBlockInputStream"; }
|
2012-05-17 19:15:53 +00:00
|
|
|
|
|
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
String getID() const override
|
2013-05-03 10:20:53 +00:00
|
|
|
|
{
|
2013-05-04 04:05:15 +00:00
|
|
|
|
std::stringstream res;
|
|
|
|
|
res << this;
|
|
|
|
|
return res.str();
|
2013-05-03 10:20:53 +00:00
|
|
|
|
}
|
|
|
|
|
|
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
|
|
|
|
|
2012-10-18 19:24:46 +00:00
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
void cancel() override
|
2012-10-18 19:24:46 +00:00
|
|
|
|
{
|
2015-03-20 16:20:47 +00:00
|
|
|
|
bool old_val = false;
|
|
|
|
|
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
|
2012-11-10 05:13:46 +00:00
|
|
|
|
return;
|
|
|
|
|
|
2015-02-25 12:43:06 +00:00
|
|
|
|
if (hasNoQueryInProgress() || hasThrownException())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-03-06 01:06:11 +00:00
|
|
|
|
tryCancel("Cancelling query");
|
2012-10-18 19:24:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
~RemoteBlockInputStream() override
|
2012-10-12 18:57:10 +00:00
|
|
|
|
{
|
2015-02-08 21:34:43 +00:00
|
|
|
|
/** Если прервались в середине цикла общения с репликами, то прервываем
|
|
|
|
|
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
|
|
|
|
|
* эти соединения не остались висеть в рассихронизированном состоянии.
|
2013-01-28 20:32:21 +00:00
|
|
|
|
*/
|
2015-03-06 00:01:59 +00:00
|
|
|
|
if (established || isQueryInProgress())
|
2015-02-20 14:43:22 +00:00
|
|
|
|
parallel_replicas->disconnect();
|
2012-10-12 18:57:10 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-10-20 02:10:47 +00:00
|
|
|
|
protected:
|
2015-02-07 17:12:29 +00:00
|
|
|
|
/// Отправить на удаленные реплики все временные таблицы
|
2014-03-13 15:00:06 +00:00
|
|
|
|
void sendExternalTables()
|
|
|
|
|
{
|
2015-02-05 22:31:03 +00:00
|
|
|
|
size_t count = parallel_replicas->size();
|
2015-01-15 15:05:03 +00:00
|
|
|
|
|
|
|
|
|
std::vector<ExternalTablesData> instances;
|
|
|
|
|
instances.reserve(count);
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < count; ++i)
|
2014-03-13 15:00:06 +00:00
|
|
|
|
{
|
2015-01-15 15:05:03 +00:00
|
|
|
|
ExternalTablesData res;
|
|
|
|
|
for (const auto & table : external_tables)
|
|
|
|
|
{
|
|
|
|
|
StoragePtr cur = table.second;
|
|
|
|
|
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
|
|
|
|
|
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
|
|
|
|
|
stage, DEFAULT_BLOCK_SIZE, 1);
|
|
|
|
|
if (input.size() == 0)
|
|
|
|
|
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
|
|
|
|
|
else
|
|
|
|
|
res.push_back(std::make_pair(input[0], table.first));
|
|
|
|
|
}
|
|
|
|
|
instances.push_back(std::move(res));
|
2014-03-13 15:00:06 +00:00
|
|
|
|
}
|
2015-01-15 15:05:03 +00:00
|
|
|
|
|
2015-02-05 22:31:03 +00:00
|
|
|
|
parallel_replicas->sendExternalTablesData(instances);
|
2014-03-13 15:00:06 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
Block readImpl() override
|
2012-10-20 02:10:47 +00:00
|
|
|
|
{
|
2015-03-06 00:01:59 +00:00
|
|
|
|
if (!sent_query)
|
2012-10-20 02:10:47 +00:00
|
|
|
|
{
|
2015-02-06 14:46:04 +00:00
|
|
|
|
createParallelReplicas();
|
2015-02-27 12:00:02 +00:00
|
|
|
|
|
2015-03-06 00:12:08 +00:00
|
|
|
|
established = true;
|
2015-02-27 12:00:02 +00:00
|
|
|
|
|
2015-02-05 22:31:03 +00:00
|
|
|
|
parallel_replicas->sendQuery(query, "", stage, true);
|
2015-02-27 12:00:02 +00:00
|
|
|
|
|
2015-03-06 00:12:08 +00:00
|
|
|
|
established = false;
|
|
|
|
|
sent_query = true;
|
2015-02-27 12:00:02 +00:00
|
|
|
|
|
2014-03-13 15:00:06 +00:00
|
|
|
|
sendExternalTables();
|
2012-10-20 02:10:47 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
while (true)
|
|
|
|
|
{
|
2015-03-20 16:20:47 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return Block();
|
|
|
|
|
|
2015-02-05 22:31:03 +00:00
|
|
|
|
Connection::Packet packet = parallel_replicas->receivePacket();
|
2012-10-20 02:10:47 +00:00
|
|
|
|
|
|
|
|
|
switch (packet.type)
|
|
|
|
|
{
|
|
|
|
|
case Protocol::Server::Data:
|
2013-06-24 13:38:50 +00:00
|
|
|
|
/// Если блок не пуст и не является заголовочным блоком
|
|
|
|
|
if (packet.block && packet.block.rows() > 0)
|
2012-10-20 02:10:47 +00:00
|
|
|
|
return packet.block;
|
|
|
|
|
break; /// Если блок пустой - получим другие пакеты до EndOfStream.
|
|
|
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
2015-03-06 00:12:08 +00:00
|
|
|
|
got_exception_from_replica = true;
|
2012-10-20 02:10:47 +00:00
|
|
|
|
packet.exception->rethrow();
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
case Protocol::Server::EndOfStream:
|
2015-02-07 17:12:29 +00:00
|
|
|
|
if (!parallel_replicas->hasActiveReplicas())
|
2015-02-05 21:10:29 +00:00
|
|
|
|
{
|
2015-03-06 00:12:08 +00:00
|
|
|
|
finished = true;
|
2015-02-05 21:10:29 +00:00
|
|
|
|
return Block();
|
|
|
|
|
}
|
|
|
|
|
break;
|
2012-10-20 02:10:47 +00:00
|
|
|
|
|
|
|
|
|
case Protocol::Server::Progress:
|
2013-11-03 05:32:42 +00:00
|
|
|
|
/** Используем прогресс с удалённого сервера.
|
|
|
|
|
* В том числе, запишем его в ProcessList,
|
|
|
|
|
* и будем использовать его для проверки
|
|
|
|
|
* ограничений (например, минимальная скорость выполнения запроса)
|
|
|
|
|
* и квот (например, на количество строчек для чтения).
|
|
|
|
|
*/
|
2014-10-25 18:33:52 +00:00
|
|
|
|
progressImpl(packet.progress);
|
2012-10-20 02:10:47 +00:00
|
|
|
|
break;
|
|
|
|
|
|
2013-06-01 18:57:04 +00:00
|
|
|
|
case Protocol::Server::ProfileInfo:
|
2013-09-06 19:33:07 +00:00
|
|
|
|
info = packet.profile_info;
|
2013-06-01 18:57:04 +00:00
|
|
|
|
break;
|
|
|
|
|
|
2013-09-05 20:22:43 +00:00
|
|
|
|
case Protocol::Server::Totals:
|
|
|
|
|
totals = packet.block;
|
|
|
|
|
break;
|
|
|
|
|
|
2013-09-07 02:03:13 +00:00
|
|
|
|
case Protocol::Server::Extremes:
|
|
|
|
|
extremes = packet.block;
|
|
|
|
|
break;
|
|
|
|
|
|
2012-10-20 02:10:47 +00:00
|
|
|
|
default:
|
2015-03-06 00:12:08 +00:00
|
|
|
|
got_unknown_packet_from_replica = true;
|
2012-10-20 02:10:47 +00:00
|
|
|
|
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-11-08 23:52:18 +00:00
|
|
|
|
void readSuffixImpl() override
|
2013-09-13 20:33:09 +00:00
|
|
|
|
{
|
|
|
|
|
/** Если одно из:
|
|
|
|
|
* - ничего не начинали делать;
|
|
|
|
|
* - получили все пакеты до EndOfStream;
|
2015-02-08 21:34:43 +00:00
|
|
|
|
* - получили с одной реплики эксепшен;
|
|
|
|
|
* - получили с одной реплики неизвестный пакет;
|
2013-09-13 20:33:09 +00:00
|
|
|
|
* - то больше читать ничего не нужно.
|
|
|
|
|
*/
|
2015-02-09 10:51:36 +00:00
|
|
|
|
if (hasNoQueryInProgress() || hasThrownException())
|
2013-09-13 20:33:09 +00:00
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
/** Если ещё прочитали не все данные, но они больше не нужны.
|
|
|
|
|
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT).
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
|
2015-03-06 01:06:11 +00:00
|
|
|
|
tryCancel("Cancelling query because enough data has been read");
|
2013-09-13 20:33:09 +00:00
|
|
|
|
|
2015-02-07 17:12:29 +00:00
|
|
|
|
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами.
|
2015-02-05 22:31:03 +00:00
|
|
|
|
Connection::Packet packet = parallel_replicas->drain();
|
|
|
|
|
switch (packet.type)
|
2015-01-22 21:54:16 +00:00
|
|
|
|
{
|
2015-02-05 22:31:03 +00:00
|
|
|
|
case Protocol::Server::EndOfStream:
|
2015-03-06 00:12:08 +00:00
|
|
|
|
finished = true;
|
2015-02-05 22:31:03 +00:00
|
|
|
|
break;
|
2015-01-22 21:54:16 +00:00
|
|
|
|
|
2015-02-05 22:31:03 +00:00
|
|
|
|
case Protocol::Server::Exception:
|
2015-03-06 00:12:08 +00:00
|
|
|
|
got_exception_from_replica = true;
|
2015-02-05 22:31:03 +00:00
|
|
|
|
packet.exception->rethrow();
|
|
|
|
|
break;
|
2015-01-22 21:54:16 +00:00
|
|
|
|
|
2015-02-05 22:31:03 +00:00
|
|
|
|
default:
|
2015-03-06 00:12:08 +00:00
|
|
|
|
got_unknown_packet_from_replica = true;
|
2015-02-05 22:31:03 +00:00
|
|
|
|
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
|
2013-09-13 20:33:09 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-02-07 19:39:16 +00:00
|
|
|
|
/// Создать объект для общения с репликами одного шарда, на которых должен выполниться запрос.
|
2015-02-06 22:32:54 +00:00
|
|
|
|
void createParallelReplicas()
|
|
|
|
|
{
|
|
|
|
|
Settings * parallel_replicas_settings = send_settings ? &settings : nullptr;
|
|
|
|
|
if (connection != nullptr)
|
2015-02-10 21:10:58 +00:00
|
|
|
|
parallel_replicas = std::make_unique<ParallelReplicas>(connection, parallel_replicas_settings, throttler);
|
2015-02-06 22:32:54 +00:00
|
|
|
|
else
|
2015-02-10 21:10:58 +00:00
|
|
|
|
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
|
2015-02-06 22:32:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-02-09 10:51:36 +00:00
|
|
|
|
/// Возвращает true, если запрос отправлен, а ещё не выполнен.
|
|
|
|
|
bool isQueryInProgress() const
|
2015-02-06 12:33:15 +00:00
|
|
|
|
{
|
2015-03-06 00:01:59 +00:00
|
|
|
|
return sent_query && !finished && !was_cancelled;
|
2015-02-08 21:34:43 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-02-09 10:51:36 +00:00
|
|
|
|
/// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен.
|
|
|
|
|
bool hasNoQueryInProgress() const
|
2015-02-08 21:34:43 +00:00
|
|
|
|
{
|
2015-03-06 00:01:59 +00:00
|
|
|
|
return !sent_query || finished;
|
2015-02-08 21:34:43 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Возвращает true, если исключение было выкинуто.
|
|
|
|
|
bool hasThrownException() const
|
|
|
|
|
{
|
2015-03-06 00:01:59 +00:00
|
|
|
|
return got_exception_from_replica || got_unknown_packet_from_replica;
|
2015-02-06 12:33:15 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-02-21 13:59:15 +00:00
|
|
|
|
private:
|
|
|
|
|
/// 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;
|
|
|
|
|
}
|
|
|
|
|
|
2015-02-25 12:43:06 +00:00
|
|
|
|
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
|
2015-03-06 01:06:11 +00:00
|
|
|
|
void tryCancel(const char * reason)
|
2015-02-21 13:59:15 +00:00
|
|
|
|
{
|
2015-02-25 12:43:06 +00:00
|
|
|
|
bool old_val = false;
|
2015-03-20 16:20:47 +00:00
|
|
|
|
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
|
2015-03-06 01:06:11 +00:00
|
|
|
|
return;
|
2015-03-06 00:01:59 +00:00
|
|
|
|
|
2015-03-06 01:06:11 +00:00
|
|
|
|
LOG_TRACE(log, "(" << parallel_replicas->dumpAddresses() << ") " << reason);
|
|
|
|
|
parallel_replicas->sendCancel();
|
2015-02-21 13:59:15 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
private:
|
2014-04-07 00:00:23 +00:00
|
|
|
|
IConnectionPool * pool = nullptr;
|
2015-02-05 22:31:03 +00:00
|
|
|
|
|
2014-04-07 00:09:19 +00:00
|
|
|
|
ConnectionPool::Entry pool_entry;
|
|
|
|
|
Connection * connection = nullptr;
|
2015-02-04 10:27:06 +00:00
|
|
|
|
std::unique_ptr<ParallelReplicas> parallel_replicas;
|
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-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
|
|
|
|
|
2014-04-07 00:00:23 +00:00
|
|
|
|
Logger * log = &Logger::get("RemoteBlockInputStream");
|
2012-05-17 19:15:53 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
}
|