mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: added simple priorities system [#METR-16911].
This commit is contained in:
parent
001ffdfa18
commit
6f3d8f0efc
@ -2,12 +2,13 @@
|
||||
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ProcessListEntry;
|
||||
|
||||
struct BlockIO
|
||||
{
|
||||
/** process_list_entry должен уничтожаться позже, чем in и out,
|
||||
@ -15,7 +16,7 @@ struct BlockIO
|
||||
* (MemoryTracker * current_memory_tracker),
|
||||
* которая может использоваться до уничтожения in и out.
|
||||
*/
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
std::shared_ptr<ProcessListEntry> process_list_entry;
|
||||
|
||||
BlockInputStreamPtr in;
|
||||
BlockOutputStreamPtr out;
|
||||
@ -38,6 +39,8 @@ struct BlockIO
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
~BlockIO();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/Core/Progress.h>
|
||||
|
||||
#include <DB/Interpreters/Limits.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
#include <DB/DataStreams/BlockStreamProfileInfo.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
@ -14,6 +13,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class QuotaForIntervals;
|
||||
class ProcessListElement;
|
||||
|
||||
|
||||
/** Смотрит за тем, как работает источник блоков.
|
||||
@ -82,7 +82,7 @@ public:
|
||||
* На основе этой информации будет проверяться квота, и некоторые ограничения.
|
||||
* Также эта информация будет доступна в запросе SHOW PROCESSLIST.
|
||||
*/
|
||||
void setProcessListElement(ProcessList::Element * elem);
|
||||
void setProcessListElement(ProcessListElement * elem);
|
||||
|
||||
/** Установить информацию о приблизительном общем количестве строк, которых нужно прочитать.
|
||||
*/
|
||||
@ -154,7 +154,7 @@ protected:
|
||||
BlockStreamProfileInfo info;
|
||||
std::atomic<bool> is_cancelled{false};
|
||||
ProgressCallback progress_callback;
|
||||
ProcessList::Element * process_list_elem = nullptr;
|
||||
ProcessListElement * process_list_elem = nullptr;
|
||||
|
||||
bool enabled_extremes = false;
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <memory>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/Condition.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
@ -13,6 +13,7 @@
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/Common/MemoryTracker.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Interpreters/QueryPriorities.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -36,13 +37,16 @@ struct ProcessListElement
|
||||
|
||||
MemoryTracker memory_tracker;
|
||||
|
||||
QueryPriorities::Handle priority_handle;
|
||||
|
||||
bool is_cancelled = false;
|
||||
|
||||
|
||||
ProcessListElement(const String & query_, const String & user_,
|
||||
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage)
|
||||
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
|
||||
size_t max_memory_usage, QueryPriorities::Handle && priority_handle_)
|
||||
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage),
|
||||
priority_handle(std::move(priority_handle_))
|
||||
{
|
||||
current_memory_tracker = &memory_tracker;
|
||||
}
|
||||
@ -55,126 +59,81 @@ struct ProcessListElement
|
||||
bool update(const Progress & value)
|
||||
{
|
||||
progress.incrementPiecewiseAtomically(value);
|
||||
|
||||
if (priority_handle)
|
||||
priority_handle->waitIfNeed(std::chrono::seconds(1)); /// NOTE Можно сделать настраиваемым таймаут.
|
||||
|
||||
return !is_cancelled;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ProcessList;
|
||||
|
||||
|
||||
/// Держит итератор на список, и удаляет элемент из списка в деструкторе.
|
||||
class ProcessListEntry
|
||||
{
|
||||
private:
|
||||
using Container = std::list<ProcessListElement>;
|
||||
|
||||
ProcessList & parent;
|
||||
Container::iterator it;
|
||||
public:
|
||||
ProcessListEntry(ProcessList & parent_, Container::iterator it_)
|
||||
: parent(parent_), it(it_) {}
|
||||
|
||||
~ProcessListEntry();
|
||||
|
||||
ProcessListElement * operator->() { return &*it; }
|
||||
const ProcessListElement * operator->() const { return &*it; }
|
||||
|
||||
ProcessListElement & get() { return *it; }
|
||||
const ProcessListElement & get() const { return *it; }
|
||||
};
|
||||
|
||||
|
||||
class ProcessList
|
||||
{
|
||||
friend class Entry;
|
||||
friend class ProcessListEntry;
|
||||
public:
|
||||
using Element = ProcessListElement;
|
||||
using Entry = ProcessListEntry;
|
||||
|
||||
/// list, чтобы итераторы не инвалидировались. NOTE: можно заменить на cyclic buffer, но почти незачем.
|
||||
typedef std::list<Element> Containter;
|
||||
using Container = std::list<Element>;
|
||||
/// Query_id -> Element *
|
||||
typedef std::unordered_map<String, Element *> QueryToElement;
|
||||
using QueryToElement = std::unordered_map<String, Element *>;
|
||||
/// User -> Query_id -> Element *
|
||||
typedef std::unordered_map<String, QueryToElement> UserToQueries;
|
||||
using UserToQueries = std::unordered_map<String, QueryToElement>;
|
||||
|
||||
private:
|
||||
mutable Poco::FastMutex mutex;
|
||||
mutable Poco::Condition have_space; /// Количество одновременно выполняющихся запросов стало меньше максимального.
|
||||
|
||||
Containter cont;
|
||||
Container cont;
|
||||
size_t cur_size; /// В C++03 std::list::size не O(1).
|
||||
size_t max_size; /// Если 0 - не ограничено. Иначе, если пытаемся добавить больше - кидается исключение.
|
||||
UserToQueries user_to_queries;
|
||||
|
||||
/// Держит итератор на список, и удаляет элемент из списка в деструкторе.
|
||||
class Entry
|
||||
{
|
||||
private:
|
||||
ProcessList & parent;
|
||||
Containter::iterator it;
|
||||
public:
|
||||
Entry(ProcessList & parent_, Containter::iterator it_)
|
||||
: parent(parent_), it(it_) {}
|
||||
|
||||
~Entry()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(parent.mutex);
|
||||
|
||||
/// В случае, если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
if (!it->is_cancelled && !it->query_id.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = parent.user_to_queries.find(it->user);
|
||||
if (queries != parent.user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(it->query_id);
|
||||
if (element != queries->second.end())
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
|
||||
parent.cont.erase(it);
|
||||
--parent.cur_size;
|
||||
parent.have_space.signal();
|
||||
}
|
||||
|
||||
Element * operator->() { return &*it; }
|
||||
const Element * operator->() const { return &*it; }
|
||||
|
||||
Element & get() { return *it; }
|
||||
const Element & get() const { return *it; }
|
||||
};
|
||||
QueryPriorities priorities;
|
||||
|
||||
public:
|
||||
ProcessList(size_t max_size_ = 0) : cur_size(0), max_size(max_size_) {}
|
||||
|
||||
typedef Poco::SharedPtr<Entry> EntryPtr;
|
||||
typedef std::shared_ptr<ProcessListEntry> EntryPtr;
|
||||
|
||||
/** Зарегистрировать выполняющийся запрос. Возвращает refcounted объект, который удаляет запрос из списка при уничтожении.
|
||||
* Если выполняющихся запросов сейчас слишком много - ждать не более указанного времени.
|
||||
* Если времени не хватило - кинуть исключение.
|
||||
*/
|
||||
EntryPtr insert(const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage = 0, size_t max_wait_milliseconds = DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS, bool replace_running_query = false)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size && (!max_wait_milliseconds || !have_space.tryWait(mutex, max_wait_milliseconds)))
|
||||
throw Exception("Too much simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
|
||||
if (queries != user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(query_id_);
|
||||
if (element != queries->second.end())
|
||||
{
|
||||
if (!replace_running_query)
|
||||
throw Exception("Query with id = " + query_id_ + " is already running.",
|
||||
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
|
||||
element->second->is_cancelled = true;
|
||||
/// В случае если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++cur_size;
|
||||
|
||||
res = new Entry(*this, cont.emplace(cont.end(), query_, user_, query_id_, ip_address_, max_memory_usage));
|
||||
|
||||
if (!query_id_.empty())
|
||||
user_to_queries[user_][query_id_] = &res->get();
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority);
|
||||
|
||||
/// Количество одновременно выполняющихся запросов.
|
||||
size_t size() const { return cur_size; }
|
||||
|
||||
/// Получить текущее состояние (копию) списка запросов.
|
||||
Containter get() const
|
||||
Container get() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
return cont;
|
||||
|
117
dbms/include/DB/Interpreters/QueryPriorities.h
Normal file
117
dbms/include/DB/Interpreters/QueryPriorities.h
Normal file
@ -0,0 +1,117 @@
|
||||
#pragma once
|
||||
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
|
||||
|
||||
/** Реализует приоритеты запросов.
|
||||
* Позволяет приостанавливать выполнение запроса, если выполняется хотя бы один более приоритетный запрос.
|
||||
*
|
||||
* Величина приоритета - целое число, чем меньше - тем больше приоритет.
|
||||
*
|
||||
* Приоритет 0 считается особенным - запросы с таким приоритетом выполняются всегда,
|
||||
* не зависят от других запросов и не влияют на другие запросы.
|
||||
* То есть 0 означает - не использовать приоритеты.
|
||||
*
|
||||
* NOTE Возможности сделать лучше:
|
||||
* - реализовать ограничение на максимальное количество запросов с таким приоритетом.
|
||||
*/
|
||||
class QueryPriorities
|
||||
{
|
||||
public:
|
||||
using Priority = int;
|
||||
|
||||
private:
|
||||
friend struct Handle;
|
||||
|
||||
using Count = int;
|
||||
|
||||
/// Количество выполняющихся сейчас запросов с заданным приоритетом.
|
||||
using Container = std::map<Priority, Count>;
|
||||
|
||||
std::mutex mutex;
|
||||
std::condition_variable condvar;
|
||||
Container container;
|
||||
|
||||
|
||||
/** Если есть более приоритетные запросы - спать, пока они не перестанут быть или не истечёт таймаут.
|
||||
* Возвращает true, если более приоритетные запросы исчезли на момент возврата из функции, false, если истёк таймаут.
|
||||
*/
|
||||
template <typename Duration>
|
||||
bool waitIfNeed(Priority priority, Duration timeout)
|
||||
{
|
||||
if (0 == priority)
|
||||
return true;
|
||||
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
|
||||
while (true)
|
||||
{
|
||||
/// Если ли хотя бы один более приоритетный запрос?
|
||||
bool found = false;
|
||||
for (const auto & value : container)
|
||||
{
|
||||
if (value.first >= priority)
|
||||
break;
|
||||
|
||||
if (value.second > 0)
|
||||
{
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!found)
|
||||
return true;
|
||||
|
||||
if (std::cv_status::timeout == condvar.wait_for(lock, timeout))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
struct HandleImpl
|
||||
{
|
||||
private:
|
||||
QueryPriorities & parent;
|
||||
QueryPriorities::Container::value_type & value;
|
||||
|
||||
public:
|
||||
HandleImpl(QueryPriorities & parent_, QueryPriorities::Container::value_type & value_)
|
||||
: parent(parent_), value(value_) {}
|
||||
|
||||
~HandleImpl()
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parent.mutex);
|
||||
--value.second;
|
||||
}
|
||||
parent.condvar.notify_all();
|
||||
}
|
||||
|
||||
template <typename Duration>
|
||||
bool waitIfNeed(Duration timeout)
|
||||
{
|
||||
return parent.waitIfNeed(value.first, timeout);
|
||||
}
|
||||
};
|
||||
|
||||
using Handle = std::shared_ptr<HandleImpl>;
|
||||
|
||||
/** Зарегистрировать, что запрос с заданным приоритетом выполняется.
|
||||
* Возвращается объект, в деструкторе которого, запись о запросе удаляется.
|
||||
*/
|
||||
Handle insert(Priority priority)
|
||||
{
|
||||
if (0 == priority)
|
||||
return {};
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = container.emplace(priority, 0).first;
|
||||
++it->second;
|
||||
return std::make_shared<HandleImpl>(*this, *it);
|
||||
}
|
||||
};
|
@ -132,6 +132,9 @@ struct Settings
|
||||
\
|
||||
/** Позволяет выбирать метод сжатия данных при записи */\
|
||||
M(SettingCompressionMethod, network_compression_method, CompressionMethod::LZ4) \
|
||||
\
|
||||
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
|
||||
M(SettingUInt64, priority, 0) \
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
|
9
dbms/src/DataStreams/BlockIO.cpp
Normal file
9
dbms/src/DataStreams/BlockIO.cpp
Normal file
@ -0,0 +1,9 @@
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BlockIO::~BlockIO() = default;
|
||||
|
||||
}
|
@ -5,6 +5,7 @@
|
||||
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Interpreters/Quota.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
@ -320,7 +321,7 @@ void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
|
||||
}
|
||||
|
||||
|
||||
void IProfilingBlockInputStream::setProcessListElement(ProcessList::Element * elem)
|
||||
void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem)
|
||||
{
|
||||
process_list_elem = elem;
|
||||
|
||||
|
72
dbms/src/Interpreters/ProcessList.cpp
Normal file
72
dbms/src/Interpreters/ProcessList.cpp
Normal file
@ -0,0 +1,72 @@
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
ProcessList::EntryPtr ProcessList::insert(
|
||||
const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size && (!max_wait_milliseconds || !have_space.tryWait(mutex, max_wait_milliseconds)))
|
||||
throw Exception("Too much simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
|
||||
if (queries != user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(query_id_);
|
||||
if (element != queries->second.end())
|
||||
{
|
||||
if (!replace_running_query)
|
||||
throw Exception("Query with id = " + query_id_ + " is already running.",
|
||||
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
|
||||
element->second->is_cancelled = true;
|
||||
/// В случае если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++cur_size;
|
||||
|
||||
res.reset(new Entry(*this, cont.emplace(cont.end(),
|
||||
query_, user_, query_id_, ip_address_, max_memory_usage, priorities.insert(priority))));
|
||||
|
||||
if (!query_id_.empty())
|
||||
user_to_queries[user_][query_id_] = &res->get();
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ProcessListEntry::~ProcessListEntry()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(parent.mutex);
|
||||
|
||||
/// В случае, если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
if (!it->is_cancelled && !it->query_id.empty())
|
||||
{
|
||||
ProcessList::UserToQueries::iterator queries = parent.user_to_queries.find(it->user);
|
||||
if (queries != parent.user_to_queries.end())
|
||||
{
|
||||
ProcessList::QueryToElement::iterator element = queries->second.find(it->query_id);
|
||||
if (element != queries->second.end())
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
|
||||
parent.cont.erase(it);
|
||||
--parent.cur_size;
|
||||
parent.have_space.signal();
|
||||
}
|
||||
|
||||
}
|
@ -14,6 +14,7 @@
|
||||
|
||||
#include <DB/Interpreters/Quota.h>
|
||||
#include <DB/Interpreters/InterpreterFactory.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Interpreters/executeQuery.h>
|
||||
|
||||
|
||||
@ -88,11 +89,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
|
||||
process_list_entry = context.getProcessList().insert(
|
||||
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
|
||||
context.getSettingsRef().limits.max_memory_usage,
|
||||
context.getSettingsRef().queue_max_wait_ms.totalMilliseconds(),
|
||||
context.getSettingsRef().replace_running_query);
|
||||
settings.limits.max_memory_usage,
|
||||
settings.queue_max_wait_ms.totalMilliseconds(),
|
||||
settings.replace_running_query,
|
||||
settings.priority);
|
||||
|
||||
context.setProcessListElement(&process_list_entry->get());
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/Common/getFQDNOrHostName.h>
|
||||
#include <DB/Interpreters/loadMetadata.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
#include <DB/Storages/StorageSystemTables.h>
|
||||
#include <DB/Storages/StorageSystemParts.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Storages/StorageSystemProcesses.h>
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user