#pragma once #include #include #include #include #include #include #include #include #include namespace DB { class LambdaAdapter : public shared_ptr_helper, public ExecutableTask { public: template explicit LambdaAdapter(T && inner_, MergeTreeData & data_) : inner(inner_), data(data_) {} bool execute() override { res = inner(); inner = {}; return false; } void onCompleted() override { data.triggerBackgroundOperationTask(!res); } StorageID getStorageID() override { return data.getStorageID(); } private: bool res = false; std::function inner; MergeTreeData & data; }; class MergeTreeBackgroundExecutor : public shared_ptr_helper { public: using CountGetter = std::function; using Callback = std::function; MergeTreeBackgroundExecutor() { scheduler = ThreadFromGlobalPool([this]() { schedulerThreadFunction(); }); } ~MergeTreeBackgroundExecutor() { wait(); } void setThreadsCount(CountGetter && getter) { threads_count_getter = getter; } void setTasksCount(CountGetter && getter) { max_task_count_getter = getter; } void setMetric(CurrentMetrics::Metric metric_) { metric = metric_; } bool trySchedule(ExecutableTaskPtr task) { std::lock_guard lock(mutex); if (shutdown_suspend) return false; auto & value = CurrentMetrics::values[metric]; if (value.load() >= static_cast(max_task_count_getter())) return false; CurrentMetrics::add(metric); tasks.emplace_back(task); has_tasks.notify_one(); return true; } void removeTasksCorrespondingToStorage(StorageID id); void wait() { { std::lock_guard lock(mutex); shutdown_suspend = true; has_tasks.notify_all(); } if (scheduler.joinable()) scheduler.join(); pool.wait(); } size_t active() { return pool.active(); } size_t pending() { std::lock_guard lock(mutex); return tasks.size(); } private: void updatePoolConfiguration() { const auto max_threads = threads_count_getter(); pool.setMaxFreeThreads(0); pool.setMaxThreads(max_threads); pool.setQueueSize(max_threads); } void schedulerThreadFunction(); CountGetter threads_count_getter; CountGetter max_task_count_getter; CurrentMetrics::Metric metric; std::deque tasks; std::mutex remove_mutex; std::mutex currently_executing_mutex; std::map> currently_executing; std::mutex mutex; std::condition_variable has_tasks; std::atomic_bool shutdown_suspend{false}; ThreadPool pool; ThreadFromGlobalPool scheduler; }; }