mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
dbms: cancelling of remote/distributed queries [#CONV-2944].
This commit is contained in:
parent
41c04b36d8
commit
7f4d9e188c
@ -14,7 +14,7 @@ class RemoteBlockInputStream : public IProfilingBlockInputStream
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
RemoteBlockInputStream(Connection & connection_, const String & query_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
RemoteBlockInputStream(Connection & connection_, const String & query_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete)
|
||||||
: connection(connection_), query(query_), stage(stage_), sent_query(false), finished(false)
|
: connection(connection_), query(query_), stage(stage_), sent_query(false), finished(false), cancelled(false)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -28,6 +28,17 @@ public:
|
|||||||
|
|
||||||
while (true)
|
while (true)
|
||||||
{
|
{
|
||||||
|
/// Периодически (каждую секунду) проверяем, не запрошено ли прервать запрос.
|
||||||
|
while (!cancelled && !connection.poll(1000000))
|
||||||
|
{
|
||||||
|
if (is_cancelled_callback && is_cancelled_callback())
|
||||||
|
{
|
||||||
|
/// Если да - запросим удалённый сервер тоже прервать запрос.
|
||||||
|
cancelled = true;
|
||||||
|
connection.sendCancel();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Connection::Packet packet = connection.receivePacket();
|
Connection::Packet packet = connection.receivePacket();
|
||||||
|
|
||||||
switch (packet.type)
|
switch (packet.type)
|
||||||
@ -106,6 +117,7 @@ private:
|
|||||||
|
|
||||||
bool sent_query;
|
bool sent_query;
|
||||||
bool finished;
|
bool finished;
|
||||||
|
bool cancelled;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user