diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index ce7db264eef..27b3de66497 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -244,7 +244,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr return Block(); } - if (!read_context) + if (!read_context || resent_query) { std::lock_guard lock(was_cancelled_mutex); if (was_cancelled) @@ -270,7 +270,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr if (got_duplicated_part_uuids) { /// Cancel previous query and disconnect before retry. - cancel(); + cancel(&read_context); multiplexed_connections->disconnect(); /// Only resend once, otherwise throw an exception @@ -523,7 +523,7 @@ bool RemoteQueryExecutor::isQueryPending() 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; } }