2018-11-27 16:45:45 +00:00
|
|
|
#include "RWLock.h"
|
2017-09-26 17:19:16 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2017-09-01 15:05:23 +00:00
|
|
|
#include <Common/Exception.h>
|
2017-09-26 17:19:16 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
|
|
|
#include <Common/ProfileEvents.h>
|
|
|
|
|
2019-09-01 01:32:44 +00:00
|
|
|
#include <cassert>
|
|
|
|
|
2017-09-26 17:19:16 +00:00
|
|
|
|
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event RWLockAcquiredReadLocks;
|
|
|
|
extern const Event RWLockAcquiredWriteLocks;
|
|
|
|
extern const Event RWLockReadersWaitMilliseconds;
|
|
|
|
extern const Event RWLockWritersWaitMilliseconds;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric RWLockWaitingReaders;
|
|
|
|
extern const Metric RWLockWaitingWriters;
|
|
|
|
extern const Metric RWLockActiveReaders;
|
|
|
|
extern const Metric RWLockActiveWriters;
|
|
|
|
}
|
2017-09-01 15:05:23 +00:00
|
|
|
|
2017-08-31 21:11:25 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-09-04 12:49:49 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
2019-09-01 01:32:44 +00:00
|
|
|
extern const int DEADLOCK_AVOIDED;
|
2017-09-04 12:49:49 +00:00
|
|
|
}
|
|
|
|
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
class RWLockImpl::LockHolderImpl
|
2017-08-31 21:11:25 +00:00
|
|
|
{
|
2018-11-27 16:45:45 +00:00
|
|
|
RWLock parent;
|
2017-08-31 21:11:25 +00:00
|
|
|
GroupsContainer::iterator it_group;
|
|
|
|
ClientsContainer::iterator it_client;
|
2019-04-27 19:12:35 +00:00
|
|
|
ThreadToHolder::key_type thread_id;
|
|
|
|
QueryIdToHolder::key_type query_id;
|
2017-09-26 17:19:16 +00:00
|
|
|
CurrentMetrics::Increment active_client_increment;
|
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
LockHolderImpl(RWLock && parent, GroupsContainer::iterator it_group, ClientsContainer::iterator it_client);
|
2017-09-26 17:19:16 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
LockHolderImpl(const LockHolderImpl & other) = delete;
|
2017-09-26 17:19:16 +00:00
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
~LockHolderImpl();
|
2017-09-26 17:19:16 +00:00
|
|
|
|
2018-11-27 16:45:45 +00:00
|
|
|
friend class RWLockImpl;
|
2017-09-26 17:19:16 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2019-09-01 01:32:44 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
/// Global information about all read locks that query has. It is needed to avoid some type of deadlocks.
|
|
|
|
|
|
|
|
class QueryLockInfo
|
|
|
|
{
|
|
|
|
private:
|
|
|
|
std::mutex mutex;
|
|
|
|
std::map<std::string, size_t> queries;
|
|
|
|
|
|
|
|
public:
|
|
|
|
void add(const String & query_id)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
++queries[query_id];
|
|
|
|
}
|
|
|
|
|
|
|
|
void remove(const String & query_id)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
auto it = queries.find(query_id);
|
|
|
|
assert(it != queries.end());
|
|
|
|
if (--it->second == 0)
|
|
|
|
queries.erase(it);
|
|
|
|
}
|
|
|
|
|
|
|
|
void check(const String & query_id)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
if (queries.count(query_id))
|
2019-09-01 19:21:00 +00:00
|
|
|
throw Exception("Possible deadlock avoided. Client should retry.", ErrorCodes::DEADLOCK_AVOIDED);
|
2019-09-01 01:32:44 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
QueryLockInfo all_read_locks;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id)
|
2017-09-26 17:19:16 +00:00
|
|
|
{
|
|
|
|
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
|
|
|
CurrentMetrics::Increment waiting_client_increment((type == Read) ? CurrentMetrics::RWLockWaitingReaders
|
|
|
|
: CurrentMetrics::RWLockWaitingWriters);
|
|
|
|
auto finalize_metrics = [type, &watch] ()
|
|
|
|
{
|
|
|
|
ProfileEvents::increment((type == Read) ? ProfileEvents::RWLockAcquiredReadLocks
|
|
|
|
: ProfileEvents::RWLockAcquiredWriteLocks);
|
|
|
|
ProfileEvents::increment((type == Read) ? ProfileEvents::RWLockReadersWaitMilliseconds
|
|
|
|
: ProfileEvents::RWLockWritersWaitMilliseconds, watch.elapsedMilliseconds());
|
|
|
|
};
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2017-09-26 17:19:16 +00:00
|
|
|
GroupsContainer::iterator it_group;
|
|
|
|
ClientsContainer::iterator it_client;
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-09-02 00:12:01 +00:00
|
|
|
/// This object is placed above unique_lock, because it may lock in destructor.
|
|
|
|
LockHolder res;
|
|
|
|
|
2019-01-02 06:44:36 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2019-02-27 18:26:24 +00:00
|
|
|
/// Check if the same query is acquiring previously acquired lock
|
|
|
|
auto this_thread_id = std::this_thread::get_id();
|
2019-02-27 18:34:10 +00:00
|
|
|
auto it_thread = thread_to_holder.find(this_thread_id);
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
auto it_query = query_id_to_holder.end();
|
2019-02-28 09:37:05 +00:00
|
|
|
if (query_id != RWLockImpl::NO_QUERY)
|
2019-02-27 18:34:10 +00:00
|
|
|
it_query = query_id_to_holder.find(query_id);
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-09-02 01:04:41 +00:00
|
|
|
bool recursive_by_query_id = false;
|
2019-02-27 18:34:10 +00:00
|
|
|
if (it_thread != thread_to_holder.end())
|
2019-09-02 01:04:41 +00:00
|
|
|
{
|
2019-09-02 00:12:01 +00:00
|
|
|
res = it_thread->second.lock();
|
2019-09-02 01:04:41 +00:00
|
|
|
}
|
2019-02-27 18:34:10 +00:00
|
|
|
else if (it_query != query_id_to_holder.end())
|
2019-09-02 01:04:41 +00:00
|
|
|
{
|
|
|
|
recursive_by_query_id = true;
|
2019-09-02 00:12:01 +00:00
|
|
|
res = it_query->second.lock();
|
2019-09-02 01:04:41 +00:00
|
|
|
}
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-09-02 00:12:01 +00:00
|
|
|
if (res)
|
2019-02-27 18:26:24 +00:00
|
|
|
{
|
|
|
|
/// XXX: it means we can't upgrade lock from read to write - with proper waiting!
|
2019-09-02 00:12:01 +00:00
|
|
|
if (type != Read || res->it_group->type != Read)
|
2019-09-02 01:04:41 +00:00
|
|
|
{
|
|
|
|
if (recursive_by_query_id)
|
|
|
|
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
|
2019-02-27 18:26:24 +00:00
|
|
|
|
2019-09-02 01:04:41 +00:00
|
|
|
/// threads are reused between queries. If lock found by thread_id, it does not necessarily means that it's recursive.
|
|
|
|
res.reset();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return res;
|
2017-09-04 12:49:49 +00:00
|
|
|
}
|
|
|
|
|
2019-09-01 01:32:44 +00:00
|
|
|
/** If the query already has any active read lock and tries to acquire another read lock
|
|
|
|
* but it is not in front of the queue and has to wait, deadlock is possible:
|
|
|
|
*
|
|
|
|
* Example (four queries, two RWLocks - 'a' and 'b'):
|
|
|
|
*
|
|
|
|
* --> time -->
|
|
|
|
*
|
|
|
|
* q1: ra rb
|
|
|
|
* q2: wa
|
|
|
|
* q3: rb ra
|
|
|
|
* q4: wb
|
|
|
|
*
|
|
|
|
* We will throw an exception instead.
|
|
|
|
*/
|
|
|
|
|
2017-08-31 21:11:25 +00:00
|
|
|
if (type == Type::Write || queue.empty() || queue.back().type == Type::Write)
|
|
|
|
{
|
2019-09-01 12:16:31 +00:00
|
|
|
if (type == Type::Read && !queue.empty() && queue.back().type == Type::Write && query_id != RWLockImpl::NO_QUERY)
|
2019-09-01 01:32:44 +00:00
|
|
|
all_read_locks.check(query_id);
|
|
|
|
|
2017-08-31 21:11:25 +00:00
|
|
|
/// Create new group of clients
|
|
|
|
it_group = queue.emplace(queue.end(), type);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Will append myself to last group
|
|
|
|
it_group = std::prev(queue.end());
|
2019-09-01 01:32:44 +00:00
|
|
|
|
|
|
|
if (it_group != queue.begin() && query_id != RWLockImpl::NO_QUERY)
|
|
|
|
all_read_locks.check(query_id);
|
2017-08-31 21:11:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Append myself to the end of chosen group
|
|
|
|
auto & clients = it_group->clients;
|
|
|
|
try
|
|
|
|
{
|
2018-11-28 15:50:52 +00:00
|
|
|
it_client = clients.emplace(clients.end(), type);
|
2017-08-31 21:11:25 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// Remove group if it was the first client in the group and an error occurred
|
|
|
|
if (clients.empty())
|
|
|
|
queue.erase(it_group);
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2019-09-02 00:12:01 +00:00
|
|
|
res.reset(new LockHolderImpl(shared_from_this(), it_group, it_client));
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-05-05 02:01:23 +00:00
|
|
|
/// Wait a notification until we will be the only in the group.
|
|
|
|
it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); });
|
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
/// Insert myself (weak_ptr to the holder) to threads set to implement recursive lock
|
2019-04-27 19:12:35 +00:00
|
|
|
thread_to_holder.emplace(this_thread_id, res);
|
|
|
|
res->thread_id = this_thread_id;
|
2019-02-27 18:26:24 +00:00
|
|
|
|
2019-02-28 09:37:05 +00:00
|
|
|
if (query_id != RWLockImpl::NO_QUERY)
|
2019-09-01 01:32:44 +00:00
|
|
|
{
|
2019-04-27 19:12:35 +00:00
|
|
|
query_id_to_holder.emplace(query_id, res);
|
2019-09-01 01:32:44 +00:00
|
|
|
|
|
|
|
if (type == Type::Read)
|
|
|
|
all_read_locks.add(query_id);
|
|
|
|
}
|
2019-04-27 19:12:35 +00:00
|
|
|
res->query_id = query_id;
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2017-09-26 17:19:16 +00:00
|
|
|
finalize_metrics();
|
2017-09-01 15:05:23 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
RWLockImpl::LockHolderImpl::~LockHolderImpl()
|
2017-08-31 21:11:25 +00:00
|
|
|
{
|
2019-09-01 01:32:44 +00:00
|
|
|
std::lock_guard lock(parent->mutex);
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2019-02-27 18:34:10 +00:00
|
|
|
/// Remove weak_ptrs to the holder, since there are no owners of the current lock
|
2019-04-27 19:12:35 +00:00
|
|
|
parent->thread_to_holder.erase(thread_id);
|
|
|
|
parent->query_id_to_holder.erase(query_id);
|
2017-09-04 12:49:49 +00:00
|
|
|
|
2019-09-01 01:32:44 +00:00
|
|
|
if (*it_client == RWLockImpl::Read && query_id != RWLockImpl::NO_QUERY)
|
|
|
|
all_read_locks.remove(query_id);
|
|
|
|
|
2017-09-04 12:49:49 +00:00
|
|
|
/// Removes myself from client list of our group
|
|
|
|
it_group->clients.erase(it_client);
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2017-09-04 12:49:49 +00:00
|
|
|
/// Remove the group if we were the last client and notify the next group
|
|
|
|
if (it_group->clients.empty())
|
2017-08-31 21:11:25 +00:00
|
|
|
{
|
2018-08-26 02:08:35 +00:00
|
|
|
auto & parent_queue = parent->queue;
|
|
|
|
parent_queue.erase(it_group);
|
2017-08-31 21:11:25 +00:00
|
|
|
|
2018-08-26 02:08:35 +00:00
|
|
|
if (!parent_queue.empty())
|
|
|
|
parent_queue.front().cv.notify_all();
|
2017-08-31 21:11:25 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
RWLockImpl::LockHolderImpl::LockHolderImpl(RWLock && parent_, RWLockImpl::GroupsContainer::iterator it_group_,
|
|
|
|
RWLockImpl::ClientsContainer::iterator it_client_)
|
|
|
|
: parent{std::move(parent_)}, it_group{it_group_}, it_client{it_client_},
|
2018-11-28 15:50:52 +00:00
|
|
|
active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders
|
|
|
|
: CurrentMetrics::RWLockActiveWriters}
|
2019-09-01 01:32:44 +00:00
|
|
|
{
|
|
|
|
}
|
2017-08-31 21:11:25 +00:00
|
|
|
|
|
|
|
}
|