Fix possible hung/deadlock on query cancellation

Right due to graceful cancellation of the query, it is possible to hung
accepting new queries or even have a deadlock, this is because
cancellation is done while acquiring ProcessListBase::mutex.

So this patch makes query cancellation lock-free, and now the lock will
be acquired only for preparing the query and after cancel is done.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-09-18 15:46:14 +02:00
parent e26976fff2
commit f76f14a99c
2 changed files with 59 additions and 7 deletions

View File

@ -297,6 +297,9 @@ ProcessListEntry::~ProcessListEntry()
}
}
/// Wait for the query if it is in the cancellation right now.
parent.cancelled_cv.wait(lock.lock, [&]() { return it->is_cancelling == false; });
/// This removes the memory_tracker of one request.
parent.processes.erase(it);
@ -430,12 +433,35 @@ QueryStatus * ProcessList::tryGetProcessListElement(const String & current_query
CancellationCode ProcessList::sendCancelToQuery(const String & current_query_id, const String & current_user, bool kill)
{
auto lock = safeLock();
QueryStatus * elem;
QueryStatus * elem = tryGetProcessListElement(current_query_id, current_user);
/// Cancelling the query should be done without the lock.
///
/// Since it may be not that trivial, for example in case of distributed
/// queries it tries to cancel the query gracefully on shards and this can
/// take a while, so acquiring a lock during this time will lead to wait
/// all new queries for this cancellation.
///
/// Another problem is that it can lead to a deadlock, because of
/// OvercommitTracker.
///
/// So here we first set is_cancelling, and later reset it.
/// The ProcessListEntry cannot be destroy if is_cancelling is true.
{
auto lock = safeLock();
elem = tryGetProcessListElement(current_query_id, current_user);
if (!elem)
return CancellationCode::NotFound;
elem->is_cancelling = true;
}
if (!elem)
return CancellationCode::NotFound;
SCOPE_EXIT({
DENY_ALLOCATIONS_IN_SCOPE;
auto lock = unsafeLock();
elem->is_cancelling = false;
cancelled_cv.notify_all();
});
return elem->cancelQuery(kill);
}
@ -443,10 +469,28 @@ CancellationCode ProcessList::sendCancelToQuery(const String & current_query_id,
void ProcessList::killAllQueries()
{
auto lock = safeLock();
std::vector<QueryStatus *> cancelled_processes;
SCOPE_EXIT({
auto lock = safeLock();
for (auto & cancelled_process : cancelled_processes)
cancelled_process->is_cancelling = false;
cancelled_cv.notify_all();
});
{
auto lock = safeLock();
cancelled_processes.reserve(processes.size());
for (auto & process : processes)
{
cancelled_processes.push_back(&process);
process.is_cancelling = true;
}
}
for (auto & cancelled_process : cancelled_processes)
cancelled_process->cancelQuery(true);
for (auto & process : processes)
process.cancelQuery(true);
}

View File

@ -100,6 +100,11 @@ protected:
QueryPriorities::Handle priority_handle = nullptr;
/// True if query cancellation is in progress right now
/// ProcessListEntry should not be destroyed if is_cancelling is true
/// Flag changes is synced with ProcessListBase::mutex and notified with ProcessList::cancelled_cv
bool is_cancelling { false };
/// KILL was send to the query
std::atomic<bool> is_killed { false };
/// All data to the client already had been sent.
@ -331,6 +336,9 @@ protected:
/// List of queries
Container processes;
/// Notify about cancelled queries (done with ProcessListBase::mutex acquired).
mutable std::condition_variable cancelled_cv;
size_t max_size = 0; /// 0 means no limit. Otherwise, when limit exceeded, an exception is thrown.
/// Stores per-user info: queries, statistics and limits