mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
dbms: addition to prev. revision [#CONV-2944].
This commit is contained in:
parent
bb3fedcd55
commit
d2ed0ae566
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
#include <DB/Client/Connection.h>
|
||||
@ -14,7 +16,8 @@ class RemoteBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
RemoteBlockInputStream(Connection & connection_, const String & query_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||
: connection(connection_), query(query_), stage(stage_), sent_query(false), finished(false), cancelled(false)
|
||||
: connection(connection_), query(query_), stage(stage_), sent_query(false), finished(false), cancelled(false),
|
||||
log(&Logger::get("RemoteBlockInputStream (" + connection.getServerAddress() + ")"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -33,6 +36,8 @@ public:
|
||||
{
|
||||
if (is_cancelled_callback && is_cancelled_callback())
|
||||
{
|
||||
LOG_TRACE(log, "Cancelling query");
|
||||
|
||||
/// Если да - запросим удалённый сервер тоже прервать запрос.
|
||||
cancelled = true;
|
||||
connection.sendCancel();
|
||||
@ -83,6 +88,8 @@ public:
|
||||
{
|
||||
if (!cancelled)
|
||||
{
|
||||
LOG_TRACE(log, "Cancelling query because enough data has been read");
|
||||
|
||||
cancelled = true;
|
||||
connection.sendCancel();
|
||||
}
|
||||
@ -122,6 +129,8 @@ private:
|
||||
bool sent_query;
|
||||
bool finished;
|
||||
bool cancelled;
|
||||
|
||||
Logger * log;
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user