Better backoff policy

This commit is contained in:
alesapin 2020-10-15 16:57:50 +03:00
parent 60d14352f8
commit 629506445d
4 changed files with 19 additions and 264 deletions

View File

@ -39,21 +39,17 @@ bool incrementIfLess(std::atomic<Int64> & 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<double>(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<double>(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()

View File

@ -57,7 +57,7 @@ private:
TaskSleepSettings sleep_settings;
pcg64 rng;
std::atomic<size_t> errors_count{0};
std::atomic<size_t> no_work_done_count{0};
std::unordered_map<PoolType, ThreadPool> pools;
std::unordered_map<PoolType, PoolConfig> pools_configs;
@ -83,7 +83,7 @@ protected:
private:
void jobExecutingTask();
void scheduleTask(bool nothing_to_do);
void scheduleTask();
};
class BackgroundJobsExecutor final : public IBackgroundJobExecutor

View File

@ -1,243 +0,0 @@
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/MemoryTracker.h>
#include <Common/randomSeed.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Common/CurrentThread.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <ext/scope_guard.h>
#include <pcg_random.hpp>
#include <random>
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<TaskInfo>(*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<double>(std::uniform_real_distribution<double>(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<uint64_t>(0, settings.thread_sleep_seconds_random_part * 1000000)(rng)));
}
else
{
wake_event.wait_for(lock,
std::chrono::duration<double>(settings.thread_sleep_seconds
+ std::uniform_real_distribution<double>(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<double>(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);
}
}
}
}

View File

@ -17,7 +17,7 @@ static std::atomic<Int64> 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<TestJobExecutor>;
TEST(BackgroundExecutor, TestMetric)
{
auto & context_holder = getContext();
const auto & context_holder = getContext();
std::vector<TestExecutorPtr> executors;
for (size_t i = 0; i < 100; ++i)
executors.emplace_back(std::make_unique<TestJobExecutor>(const_cast<Context &>(context_holder.context)));