diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 873d7fd8aa0..57003bbfaa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -39,21 +39,17 @@ bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) } -void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) +void IBackgroundJobExecutor::scheduleTask() { - auto errors = errors_count.load(std::memory_order_relaxed); - size_t next_time_to_execute = 0; - if (errors != 0) + auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); + if (no_work_done_times != 0) { - next_time_to_execute += 1000 * (std::min( + auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); - } - else if (nothing_to_do) - { - next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + scheduling_task->scheduleAfter(next_time_to_execute); } else { @@ -61,7 +57,6 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) return; } - scheduling_task->scheduleAfter(next_time_to_execute); } void IBackgroundJobExecutor::jobExecutingTask() @@ -82,11 +77,11 @@ try { job(); CurrentMetrics::values[pool_config.tasks_metric]--; - errors_count = 0; + no_work_done_count = 0; } catch (...) { - errors_count++; + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } @@ -94,21 +89,24 @@ try } catch (...) { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } } - scheduleTask(false); } else /// Nothing to do, no jobs { - scheduleTask(true); + no_work_done_count++; } + + scheduleTask(); } catch (...) /// Exception while we looking for a task { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - scheduleTask(true); + scheduleTask(); } void IBackgroundJobExecutor::start() diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 03d3823bcab..d12124492f9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -57,7 +57,7 @@ private: TaskSleepSettings sleep_settings; pcg64 rng; - std::atomic errors_count{0}; + std::atomic no_work_done_count{0}; std::unordered_map pools; std::unordered_map pools_configs; @@ -83,7 +83,7 @@ protected: private: void jobExecutingTask(); - void scheduleTask(bool nothing_to_do); + void scheduleTask(); }; class BackgroundJobsExecutor final : public IBackgroundJobExecutor diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp deleted file mode 100644 index d779bd16814..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ /dev/null @@ -1,243 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -void BackgroundProcessingPoolTaskInfo::signalReadyToRun() -{ - Poco::Timestamp current_time; - { - std::unique_lock lock(pool.tasks_mutex); - - /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. - if (removed) - return; - - /// 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); - - /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. - pool.wake_event.notify_one(); - } -} - - -BackgroundProcessingPool::BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings, - const char * log_name, - const char * thread_name_) - : size(size_) - , thread_name(thread_name_) - , settings(pool_settings) -{ - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); - - threads.resize(size); - for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); -} - - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const Task & task) -{ - return std::make_shared(*this, 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(); - } - -} - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) -{ - TaskHandle res = createTask(task); - startTask(res); - return res; -} - -void BackgroundProcessingPool::removeTask(const TaskHandle & task) -{ - if (task->removed.exchange(true)) - return; - - /// Wait for all executions of this task. - { - std::unique_lock wlock(task->rwlock); - } - - { - std::unique_lock lock(tasks_mutex); - tasks.erase(task->iterator); - /// Note that the task may be still accessible through TaskHandle (shared_ptr). - } -} - -BackgroundProcessingPool::~BackgroundProcessingPool() -{ - try - { - { - std::lock_guard lock(tasks_mutex); - shutdown = true; - wake_event.notify_all(); - } - - for (auto & thread : threads) - thread.join(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -void BackgroundProcessingPool::workLoopFunc() -{ - setThreadName(thread_name); - - { - std::lock_guard lock(tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - } - - SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - - pcg64 rng(randomSeed()); - std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - - Poco::Timestamp scheduled_task_start_time; - - while (true) - { - TaskResult task_result = TaskResult::ERROR; - TaskHandle task; - - { - std::unique_lock lock(tasks_mutex); - - while (!task && !shutdown) - { - for (const auto & [time, handle] : tasks) - { - if (!handle->removed - && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) - { - 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; - } - - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - try - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->task_function(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - { - std::unique_lock lock(tasks_mutex); - - if (shutdown) - break; - - --task->concurrent_executors; - - if (task->removed) - continue; - - if (task_result == TaskResult::SUCCESS) - task->count_no_work_done = 0; - else if (task_result == TaskResult::ERROR) - ++task->count_no_work_done; - /// NOTHING_TO_DO should not increment count_no_work_done - /// otherwise error after period of inactivity (lot of NOTHING_TO_DO) - /// leads to 5-10 min replication hang - - /// If task has done work, it could be executed again immediately. - /// If not, add delay before next run. - - Poco::Timestamp next_time_to_execute; /// current time - if (task_result == TaskResult::ERROR) - next_time_to_execute += 1000000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - else if (task_result == TaskResult::NOTHING_TO_DO) - next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - - rescheduleTask(task->iterator, next_time_to_execute); - } - } -} - -} diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp index 1bc96ed0c42..77bc539b1b1 100644 --- a/src/Storages/tests/gtest_background_executor.cpp +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -17,7 +17,7 @@ static std::atomic counter{0}; class TestJobExecutor : public IBackgroundJobExecutor { public: - TestJobExecutor(Context & context) + explicit TestJobExecutor(Context & context) :IBackgroundJobExecutor( context, TaskSleepSettings{}, @@ -40,7 +40,7 @@ using TestExecutorPtr = std::unique_ptr; TEST(BackgroundExecutor, TestMetric) { - auto & context_holder = getContext(); + const auto & context_holder = getContext(); std::vector executors; for (size_t i = 0; i < 100; ++i) executors.emplace_back(std::make_unique(const_cast(context_holder.context)));