Merge pull request #10239 from ClickHouse/fix-cancel-race-in-remote

Fix race after cancel of RemoteBlockInputStream.
This commit is contained in:
alexey-milovidov 2020-04-14 05:04:38 +03:00 committed by GitHub
commit 7587bd8cfc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 11 additions and 5 deletions

View File

@ -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

View File

@ -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