mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into add-test-for-deadlock-system-tables
This commit is contained in:
commit
dcdd8c6e3c
@ -40,7 +40,6 @@ class RWLockImpl::LockHolderImpl
|
||||
RWLock parent;
|
||||
GroupsContainer::iterator it_group;
|
||||
ClientsContainer::iterator it_client;
|
||||
ThreadToHolder::key_type thread_id;
|
||||
QueryIdToHolder::key_type query_id;
|
||||
CurrentMetrics::Increment active_client_increment;
|
||||
|
||||
@ -116,35 +115,18 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
/// Check if the same query is acquiring previously acquired lock
|
||||
auto this_thread_id = std::this_thread::get_id();
|
||||
auto it_thread = thread_to_holder.find(this_thread_id);
|
||||
|
||||
auto it_query = query_id_to_holder.end();
|
||||
if (query_id != RWLockImpl::NO_QUERY)
|
||||
it_query = query_id_to_holder.find(query_id);
|
||||
|
||||
bool recursive_by_query_id = false;
|
||||
if (it_thread != thread_to_holder.end())
|
||||
{
|
||||
res = it_thread->second.lock();
|
||||
}
|
||||
else if (it_query != query_id_to_holder.end())
|
||||
{
|
||||
recursive_by_query_id = true;
|
||||
res = it_query->second.lock();
|
||||
auto it_query = query_id_to_holder.find(query_id);
|
||||
if (it_query != query_id_to_holder.end())
|
||||
res = it_query->second.lock();
|
||||
}
|
||||
|
||||
if (res)
|
||||
{
|
||||
/// XXX: it means we can't upgrade lock from read to write - with proper waiting!
|
||||
if (type != Read || res->it_group->type != Read)
|
||||
{
|
||||
if (recursive_by_query_id)
|
||||
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// threads are reused between queries. If lock found by thread_id, it does not necessarily means that it's recursive.
|
||||
res.reset();
|
||||
}
|
||||
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
|
||||
else
|
||||
return res;
|
||||
}
|
||||
@ -200,10 +182,7 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
|
||||
/// Wait a notification until we will be the only in the group.
|
||||
it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); });
|
||||
|
||||
/// Insert myself (weak_ptr to the holder) to threads set to implement recursive lock
|
||||
thread_to_holder.emplace(this_thread_id, res);
|
||||
res->thread_id = this_thread_id;
|
||||
|
||||
/// Insert myself (weak_ptr to the holder) to queries set to implement recursive lock
|
||||
if (query_id != RWLockImpl::NO_QUERY)
|
||||
{
|
||||
query_id_to_holder.emplace(query_id, res);
|
||||
@ -223,7 +202,6 @@ RWLockImpl::LockHolderImpl::~LockHolderImpl()
|
||||
std::lock_guard lock(parent->mutex);
|
||||
|
||||
/// Remove weak_ptrs to the holder, since there are no owners of the current lock
|
||||
parent->thread_to_holder.erase(thread_id);
|
||||
parent->query_id_to_holder.erase(query_id);
|
||||
|
||||
if (*it_client == RWLockImpl::Read && query_id != RWLockImpl::NO_QUERY)
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <vector>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <thread>
|
||||
#include <map>
|
||||
#include <string>
|
||||
|
||||
@ -19,7 +18,7 @@ using RWLock = std::shared_ptr<RWLockImpl>;
|
||||
|
||||
|
||||
/// Implements shared lock with FIFO service
|
||||
/// Can be acquired recursively (several calls for the same query or the same OS thread) in Read mode
|
||||
/// Can be acquired recursively (several calls for the same query) in Read mode
|
||||
///
|
||||
/// NOTE: it is important to allow acquiring the same lock in Read mode without waiting if it is already
|
||||
/// acquired by another thread of the same query. Otherwise the following deadlock is possible:
|
||||
@ -55,7 +54,6 @@ private:
|
||||
struct Group;
|
||||
using GroupsContainer = std::list<Group>;
|
||||
using ClientsContainer = std::list<Type>;
|
||||
using ThreadToHolder = std::map<std::thread::id, std::weak_ptr<LockHolderImpl>>;
|
||||
using QueryIdToHolder = std::map<String, std::weak_ptr<LockHolderImpl>>;
|
||||
|
||||
/// Group of clients that should be executed concurrently
|
||||
@ -73,7 +71,6 @@ private:
|
||||
|
||||
mutable std::mutex mutex;
|
||||
GroupsContainer queue;
|
||||
ThreadToHolder thread_to_holder;
|
||||
QueryIdToHolder query_id_to_holder;
|
||||
};
|
||||
|
||||
|
@ -102,7 +102,7 @@ TEST(Common, RWLock_Recursive)
|
||||
{
|
||||
for (int i = 0; i < 2 * cycles; ++i)
|
||||
{
|
||||
auto lock = fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY);
|
||||
auto lock = fifo_lock->getLock(RWLockImpl::Write, "q1");
|
||||
|
||||
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));
|
||||
std::this_thread::sleep_for(sleep_for);
|
||||
@ -113,17 +113,17 @@ TEST(Common, RWLock_Recursive)
|
||||
{
|
||||
for (int i = 0; i < cycles; ++i)
|
||||
{
|
||||
auto lock1 = fifo_lock->getLock(RWLockImpl::Read, RWLockImpl::NO_QUERY);
|
||||
auto lock1 = fifo_lock->getLock(RWLockImpl::Read, "q2");
|
||||
|
||||
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));
|
||||
std::this_thread::sleep_for(sleep_for);
|
||||
|
||||
auto lock2 = fifo_lock->getLock(RWLockImpl::Read, RWLockImpl::NO_QUERY);
|
||||
auto lock2 = fifo_lock->getLock(RWLockImpl::Read, "q2");
|
||||
|
||||
EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY);});
|
||||
EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write, "q2");});
|
||||
}
|
||||
|
||||
fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY);
|
||||
fifo_lock->getLock(RWLockImpl::Write, "q2");
|
||||
});
|
||||
|
||||
t1.join();
|
||||
|
Loading…
Reference in New Issue
Block a user