diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 8f6d7c19549..30c789ccf6f 100644 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -19,27 +19,21 @@ namespace DB void BackgroundProcessingPoolTaskInfo::wake() { Poco::Timestamp current_time; - { std::unique_lock lock(pool.tasks_mutex); - /// This will ensure that iterator is valid. Must be done under the same mutex when the iterator is invalidated. + /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. if (removed) return; - auto next_time_to_execute = iterator->first; - auto this_task_handle = iterator->second; + /// If this task did nothing the previous time and still should sleep, then reschedule to cancel the sleep. + const auto & scheduled_time = iterator->first; + if (scheduled_time > current_time) + pool.rescheduleTask(iterator, current_time); - /// If this task was done nothing at previous time and it has to sleep, then cancel sleep time. - if (next_time_to_execute > current_time) - next_time_to_execute = current_time; - - pool.tasks.erase(iterator); - iterator = pool.tasks.emplace(next_time_to_execute, this_task_handle); + /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. + pool.wake_event.notify_one(); } - - /// Note that if all threads are currently do some work, this call will not wakeup any thread. - pool.wake_event.notify_one(); } @@ -51,12 +45,12 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_, , thread_name(thread_name_) , settings(pool_settings) { - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); + logger = &Logger::get(log_name); + LOG_INFO(logger, "Create " << log_name << " with " << size << " threads"); threads.resize(size); for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { threadFunction(); }); + thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); } @@ -65,16 +59,19 @@ BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const return std::make_shared(*this, task); } -void BackgroundProcessingPool::startTask(const TaskHandle & task) +void BackgroundProcessingPool::startTask(const TaskHandle & task, bool allow_execute_in_parallel) { Poco::Timestamp current_time; + task->allow_execute_in_parallel = allow_execute_in_parallel; + { std::unique_lock lock(tasks_mutex); task->iterator = tasks.emplace(current_time, task); + + wake_event.notify_all(); } - wake_event.notify_all(); } BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) @@ -105,8 +102,12 @@ BackgroundProcessingPool::~BackgroundProcessingPool() { try { - shutdown = true; - wake_event.notify_all(); + { + std::lock_guard lock(tasks_mutex); + shutdown = true; + wake_event.notify_all(); + } + for (auto & thread : threads) thread.join(); } @@ -117,7 +118,7 @@ BackgroundProcessingPool::~BackgroundProcessingPool() } -void BackgroundProcessingPool::threadFunction() +void BackgroundProcessingPool::workLoopFunc() { setThreadName(thread_name); @@ -137,80 +138,82 @@ void BackgroundProcessingPool::threadFunction() } SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - if (auto * memory_tracker = CurrentThread::getMemoryTracker()) + if (const auto memory_tracker = CurrentThread::getMemoryTracker()) memory_tracker->setMetric(settings.memory_metric); pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - while (!shutdown) + Poco::Timestamp scheduled_task_start_time; + + while (true) { TaskResult task_result = TaskResult::ERROR; TaskHandle task; - try { - Poco::Timestamp min_time; + std::unique_lock lock(tasks_mutex); + while (!task && !shutdown) { - std::unique_lock lock(tasks_mutex); - - if (!tasks.empty()) + for (const auto & [time, handle] : tasks) { - for (const auto & time_handle : tasks) + if (!handle->removed + && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) { - if (!time_handle.second->removed) - { - min_time = time_handle.first; - task = time_handle.second; - break; - } + task = handle; + scheduled_task_start_time = time; + ++task->concurrent_executors; + break; } } + + if (task) + { + Poco::Timestamp current_time; + + if (scheduled_task_start_time <= current_time) + continue; + + wake_event.wait_for(lock, + std::chrono::microseconds(scheduled_task_start_time - current_time + + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); + } + else + { + wake_event.wait_for(lock, + std::chrono::duration(settings.thread_sleep_seconds + + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); + } } if (shutdown) break; + } - if (!task) - { - std::unique_lock lock(tasks_mutex); - wake_event.wait_for(lock, - std::chrono::duration(settings.thread_sleep_seconds - + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - continue; - } + std::shared_lock rlock(task->rwlock); - /// No tasks ready for execution. - Poco::Timestamp current_time; - if (min_time > current_time) - { - std::unique_lock lock(tasks_mutex); - wake_event.wait_for(lock, std::chrono::microseconds( - min_time - current_time + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); - } + if (task->removed) + continue; - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->function(); - } + try + { + CurrentMetrics::Increment metric_increment{settings.tasks_metric}; + task_result = task->task_function(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - if (shutdown) - break; - { std::unique_lock lock(tasks_mutex); + if (shutdown) + break; + + --task->concurrent_executors; + if (task->removed) continue; @@ -231,8 +234,7 @@ void BackgroundProcessingPool::threadFunction() else if (task_result == TaskResult::NOTHING_TO_DO) next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - tasks.erase(task->iterator); - task->iterator = tasks.emplace(next_time_to_execute, task); + rescheduleTask(task->iterator, next_time_to_execute); } } } diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.h b/src/Storages/MergeTree/BackgroundProcessingPool.h index 526cab0800e..02ba65376c5 100644 --- a/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -82,14 +82,16 @@ public: return size; } - /// Create task and start it. + /// Create task and start it. It is used internally. TaskHandle addTask(const Task & task); + /// The following two methods are invoked by Storage*MergeTree at startup /// Create task but not start it. TaskHandle createTask(const Task & task); /// Start the task that was created but not started. Precondition: task was not started. - void startTask(const TaskHandle & task); + void startTask(const TaskHandle & task, bool allow_execute_in_parallel = true); + /// Invoked by Storage*MergeTree at shutdown void removeTask(const TaskHandle & task); ~BackgroundProcessingPool(); @@ -109,13 +111,20 @@ protected: Threads threads; - std::atomic shutdown {false}; + bool shutdown{false}; std::condition_variable wake_event; /// Thread group used for profiling purposes ThreadGroupStatusPtr thread_group; - void threadFunction(); + void workLoopFunc(); + + void rescheduleTask(Tasks::iterator & task_it, const Poco::Timestamp & new_scheduled_ts) + { + auto node_handle = tasks.extract(task_it); + node_handle.key() = new_scheduled_ts; + task_it = tasks.insert(std::move(node_handle)); + } private: PoolSettings settings; @@ -125,23 +134,30 @@ private: class BackgroundProcessingPoolTaskInfo { public: - /// Wake up any thread. + /// Signals random idle thread from the pool that this task is ready to be executed. void wake(); + void signalReadyToRun(); /// TODO: Rename this properly BackgroundProcessingPoolTaskInfo(BackgroundProcessingPool & pool_, const BackgroundProcessingPool::Task & function_) - : pool(pool_), function(function_) {} + : pool(pool_), task_function(function_) {} protected: friend class BackgroundProcessingPool; BackgroundProcessingPool & pool; - BackgroundProcessingPool::Task function; + BackgroundProcessingPool::Task task_function; - /// Read lock is hold when task is executed. + /// Read lock is held while task is being executed. + /// Write lock is used for stopping BGProcPool std::shared_mutex rwlock; + + bool allow_execute_in_parallel = false; + size_t concurrent_executors = 0; + + /// Signals that this task must no longer be planned for execution and is about to be removed std::atomic removed {false}; - std::multimap>::iterator iterator; + BackgroundProcessingPool::Tasks::iterator iterator; /// For exponential backoff. size_t count_no_work_done = 0;