ClickHouse/src/Common/RWLock.h

79 lines
2.1 KiB
C++
Raw Normal View History

#pragma once
#include <Core/Types.h>
#include <list>
#include <vector>
#include <mutex>
#include <condition_variable>
#include <map>
#include <string>
2019-09-05 18:09:33 +00:00
#include <unordered_map>
namespace DB
{
2018-11-27 16:45:45 +00:00
class RWLockImpl;
using RWLock = std::shared_ptr<RWLockImpl>;
/// Implements shared lock with FIFO service
/// Can be acquired recursively (several calls for the same query) in Read mode
2019-02-27 18:39:55 +00:00
///
/// 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:
/// - SELECT thread 1 locks in the Read mode
/// - ALTER tries to lock in the Write mode (waits for SELECT thread 1)
/// - SELECT thread 2 tries to lock in the Read mode (waits for ALTER)
2018-11-27 16:45:45 +00:00
class RWLockImpl : public std::enable_shared_from_this<RWLockImpl>
{
public:
enum Type
{
Read,
2018-11-27 16:45:45 +00:00
Write,
};
2018-11-27 16:45:45 +00:00
static RWLock create() { return RWLock(new RWLockImpl); }
/// Just use LockHolder::reset() to release the lock
class LockHolderImpl;
friend class LockHolderImpl;
using LockHolder = std::shared_ptr<LockHolderImpl>;
/// Waits in the queue and returns appropriate lock
/// Empty query_id means the lock is acquired out of the query context (e.g. in a background thread).
LockHolder getLock(Type type, const String & query_id);
/// Use as query_id to acquire a lock outside the query context.
inline static const String NO_QUERY = String();
private:
2018-11-27 16:45:45 +00:00
RWLockImpl() = default;
struct Group;
using GroupsContainer = std::list<Group>;
2019-09-05 18:09:33 +00:00
using OwnerQueryIds = std::unordered_map<String, size_t>;
2019-09-05 15:14:17 +00:00
/// Group of locking requests that should be granted concurrently
/// i.e. a group can contain several readers, but only one writer
struct Group
{
const Type type;
size_t refererrs;
2019-09-05 15:14:17 +00:00
std::condition_variable cv; /// all locking requests of the group wait on this condvar
explicit Group(Type type_) : type{type_}, refererrs{0} {}
};
GroupsContainer queue;
2019-09-05 18:09:33 +00:00
OwnerQueryIds owner_queries;
mutable std::mutex mutex;
};
}