Updated startTask() method + reworked some of the worker loop

This commit is contained in:
Alexander Kazakov 2020-06-02 17:37:10 +03:00
parent 5088f409ca
commit 013dc2d7c2
2 changed files with 92 additions and 74 deletions

View File

@ -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<TaskInfo>(*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<double>(std::uniform_real_distribution<double>(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<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;
}
if (!task)
{
std::unique_lock lock(tasks_mutex);
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)));
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<uint64_t>(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);
}
}
}

View File

@ -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<bool> 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<bool> removed {false};
std::multimap<Poco::Timestamp, std::shared_ptr<BackgroundProcessingPoolTaskInfo>>::iterator iterator;
BackgroundProcessingPool::Tasks::iterator iterator;
/// For exponential backoff.
size_t count_no_work_done = 0;