mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #10239 from ClickHouse/fix-cancel-race-in-remote
Fix race after cancel of RemoteBlockInputStream.
This commit is contained in:
commit
7587bd8cfc
@ -359,12 +359,17 @@ void RemoteBlockInputStream::sendQuery()
|
||||
|
||||
void RemoteBlockInputStream::tryCancel(const char * reason)
|
||||
{
|
||||
bool old_val = false;
|
||||
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
|
||||
return;
|
||||
{
|
||||
std::lock_guard guard(was_cancelled_mutex);
|
||||
|
||||
if (was_cancelled)
|
||||
return;
|
||||
|
||||
was_cancelled = true;
|
||||
multiplexed_connections->sendCancel();
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "(" << multiplexed_connections->dumpAddresses() << ") " << reason);
|
||||
multiplexed_connections->sendCancel();
|
||||
}
|
||||
|
||||
bool RemoteBlockInputStream::isQueryPending() const
|
||||
|
@ -135,7 +135,8 @@ private:
|
||||
* - data size is already satisfactory (when using LIMIT, for example)
|
||||
* - an exception was thrown from client side
|
||||
*/
|
||||
std::atomic<bool> was_cancelled { false };
|
||||
bool was_cancelled { false };
|
||||
std::mutex was_cancelled_mutex;
|
||||
|
||||
/** An exception from replica was received. No need in receiving more packets or
|
||||
* requesting to cancel query execution
|
||||
|
Loading…
Reference in New Issue
Block a user