mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-09-28 04:30:53 +00:00
Send cancel packet and cancel read_context before retrying the query
This commit is contained in:
parent
921518db0a
commit
d05c6446b9
@ -244,7 +244,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
|||||||
return Block();
|
return Block();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!read_context)
|
if (!read_context || resent_query)
|
||||||
{
|
{
|
||||||
std::lock_guard lock(was_cancelled_mutex);
|
std::lock_guard lock(was_cancelled_mutex);
|
||||||
if (was_cancelled)
|
if (was_cancelled)
|
||||||
@ -270,7 +270,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
|
|||||||
if (got_duplicated_part_uuids)
|
if (got_duplicated_part_uuids)
|
||||||
{
|
{
|
||||||
/// Cancel previous query and disconnect before retry.
|
/// Cancel previous query and disconnect before retry.
|
||||||
cancel();
|
cancel(&read_context);
|
||||||
multiplexed_connections->disconnect();
|
multiplexed_connections->disconnect();
|
||||||
|
|
||||||
/// Only resend once, otherwise throw an exception
|
/// Only resend once, otherwise throw an exception
|
||||||
@ -523,7 +523,7 @@ bool RemoteQueryExecutor::isQueryPending() const
|
|||||||
|
|
||||||
bool RemoteQueryExecutor::hasThrownException() const
|
bool RemoteQueryExecutor::hasThrownException() const
|
||||||
{
|
{
|
||||||
return got_exception_from_replica || got_unknown_packet_from_replica || got_duplicated_part_uuids;
|
return got_exception_from_replica || got_unknown_packet_from_replica;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user