ClickHouse/src/Storages/MergeTree/MergeMutateExecutor.h

197 lines
4.6 KiB
C++
Raw Normal View History

2021-08-30 19:37:03 +00:00
#pragma once
#include <deque>
#include <functional>
#include <atomic>
#include <mutex>
#include <future>
2021-08-30 19:37:03 +00:00
#include <condition_variable>
#include <set>
2021-08-30 19:37:03 +00:00
#include <common/shared_ptr_helper.h>
2021-09-03 13:02:49 +00:00
#include <common/logger_useful.h>
2021-08-30 19:37:03 +00:00
#include <Common/ThreadPool.h>
2021-09-02 17:40:29 +00:00
#include <Common/Stopwatch.h>
#include <Common/RingBuffer.h>
2021-08-30 19:37:03 +00:00
#include <Storages/MergeTree/ExecutableTask.h>
namespace DB
{
class MergeTreeBackgroundExecutor : public shared_ptr_helper<MergeTreeBackgroundExecutor>
{
public:
using CountGetter = std::function<size_t()>;
using Callback = std::function<void()>;
2021-09-02 10:39:27 +00:00
enum class Type
{
MERGE_MUTATE,
FETCH,
MOVE
};
2021-08-30 19:37:03 +00:00
2021-09-02 17:40:29 +00:00
MergeTreeBackgroundExecutor(
Type type_,
CountGetter && threads_count_getter_,
CountGetter && max_task_count_getter_,
CurrentMetrics::Metric metric_)
: type(type_)
, threads_count_getter(threads_count_getter_)
, max_task_count_getter(max_task_count_getter_)
, metric(metric_)
2021-08-30 19:37:03 +00:00
{
2021-09-02 10:39:27 +00:00
name = toString(type);
2021-09-02 17:40:29 +00:00
updateConfiguration();
2021-08-30 19:37:03 +00:00
scheduler = ThreadFromGlobalPool([this]() { schedulerThreadFunction(); });
}
~MergeTreeBackgroundExecutor()
{
wait();
}
bool trySchedule(ExecutableTaskPtr task)
{
std::lock_guard lock(mutex);
if (shutdown_suspend)
return false;
2021-09-03 00:51:12 +00:00
try
{
/// This is needed to increase / decrease the number of threads at runtime
if (update_timer.compareAndRestartDeferred(10.))
updateConfiguration();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
2021-08-30 19:37:03 +00:00
auto & value = CurrentMetrics::values[metric];
2021-09-02 17:40:29 +00:00
if (value.load() >= static_cast<int64_t>(max_tasks_count))
return false;
2021-09-03 13:02:49 +00:00
if (!scheduler.joinable())
{
LOG_ERROR(&Poco::Logger::get("MergeTreeBackgroundExecutor"), "Scheduler thread is dead. Trying to alive..");
scheduler = ThreadFromGlobalPool([this]() { schedulerThreadFunction(); });
if (!scheduler.joinable())
LOG_FATAL(&Poco::Logger::get("MergeTreeBackgroundExecutor"), "Scheduler thread is dead permanently. Restart is needed");
}
2021-09-02 17:40:29 +00:00
if (!pending.tryPush(std::make_shared<Item>(std::move(task), metric)))
2021-08-30 19:37:03 +00:00
return false;
2021-09-02 17:40:29 +00:00
2021-08-30 19:37:03 +00:00
has_tasks.notify_one();
return true;
}
2021-08-31 11:02:39 +00:00
void removeTasksCorrespondingToStorage(StorageID id);
2021-08-30 19:37:03 +00:00
void wait()
{
{
std::lock_guard lock(mutex);
shutdown_suspend = true;
has_tasks.notify_all();
}
if (scheduler.joinable())
scheduler.join();
pool.wait();
}
2021-08-31 23:20:23 +00:00
size_t activeCount()
2021-08-31 14:54:24 +00:00
{
2021-08-31 23:20:23 +00:00
std::lock_guard lock(mutex);
return active.size();
2021-08-31 14:54:24 +00:00
}
2021-08-31 23:20:23 +00:00
size_t pendingCount()
2021-08-31 14:54:24 +00:00
{
std::lock_guard lock(mutex);
2021-08-31 23:20:23 +00:00
return pending.size();
2021-08-31 14:54:24 +00:00
}
2021-08-30 19:37:03 +00:00
private:
2021-09-02 17:40:29 +00:00
void updateConfiguration()
2021-08-30 19:37:03 +00:00
{
2021-09-02 17:40:29 +00:00
auto new_threads_count = threads_count_getter();
auto new_max_tasks_count = max_task_count_getter();
try
{
pending.resize(new_max_tasks_count);
2021-09-02 18:49:37 +00:00
active.resize(new_max_tasks_count);
2021-09-02 17:40:29 +00:00
2021-09-03 00:45:19 +00:00
pool.setMaxFreeThreads(new_threads_count);
2021-09-02 17:40:29 +00:00
pool.setMaxThreads(new_threads_count);
pool.setQueueSize(new_max_tasks_count);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
threads_count = new_threads_count;
max_tasks_count = new_max_tasks_count;
2021-08-30 19:37:03 +00:00
}
2021-09-02 10:39:27 +00:00
static String toString(Type type);
2021-08-30 19:37:03 +00:00
2021-09-02 10:39:27 +00:00
Type type;
String name;
2021-08-30 19:37:03 +00:00
CountGetter threads_count_getter;
CountGetter max_task_count_getter;
CurrentMetrics::Metric metric;
2021-09-02 17:40:29 +00:00
size_t threads_count{0};
size_t max_tasks_count{0};
AtomicStopwatch update_timer;
2021-08-31 18:07:24 +00:00
struct Item
{
2021-08-31 23:20:23 +00:00
explicit Item(ExecutableTaskPtr && task_, CurrentMetrics::Metric metric_)
: task(std::move(task_))
, increment(std::move(metric_))
{
}
2021-08-31 18:07:24 +00:00
ExecutableTaskPtr task;
CurrentMetrics::Increment increment;
2021-09-03 13:02:49 +00:00
Poco::Event is_done;
2021-08-31 18:07:24 +00:00
};
using ItemPtr = std::shared_ptr<Item>;
2021-09-03 12:27:49 +00:00
void routine(ItemPtr item);
void schedulerThreadFunction();
2021-09-02 17:40:29 +00:00
/// Initially it will be empty
RingBuffer<ItemPtr> pending{0};
2021-09-03 13:02:49 +00:00
RingBuffer<ItemPtr> active{0};
2021-08-31 23:20:23 +00:00
std::set<StorageID> currently_deleting;
2021-08-31 11:02:39 +00:00
std::mutex remove_mutex;
2021-08-30 19:37:03 +00:00
std::mutex mutex;
std::condition_variable has_tasks;
std::atomic_bool shutdown_suspend{false};
ThreadPool pool;
ThreadFromGlobalPool scheduler;
};
}