mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 09:52:38 +00:00
Remove background processing pool
This commit is contained in:
parent
6a4911dc57
commit
fc25b6cd72
@ -1,240 +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
|
||||
++task->count_no_work_done;
|
||||
|
||||
/// 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,163 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <thread>
|
||||
#include <set>
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <atomic>
|
||||
#include <functional>
|
||||
#include <Poco/Event.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <common/types.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric BackgroundPoolTask;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class BackgroundProcessingPool;
|
||||
class BackgroundProcessingPoolTaskInfo;
|
||||
|
||||
enum class BackgroundProcessingPoolTaskResult
|
||||
{
|
||||
SUCCESS,
|
||||
ERROR,
|
||||
NOTHING_TO_DO,
|
||||
};
|
||||
|
||||
|
||||
/** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop.
|
||||
* In this case, one task can run simultaneously from different threads.
|
||||
* Designed for tasks that perform continuous background work (for example, merge).
|
||||
* `Task` is a function that returns a bool - did it do any work.
|
||||
* If not, then the next time will be done later.
|
||||
*/
|
||||
class BackgroundProcessingPool
|
||||
{
|
||||
public:
|
||||
/// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task.
|
||||
using TaskResult = BackgroundProcessingPoolTaskResult;
|
||||
using Task = std::function<TaskResult()>;
|
||||
using TaskInfo = BackgroundProcessingPoolTaskInfo;
|
||||
using TaskHandle = std::shared_ptr<TaskInfo>;
|
||||
|
||||
|
||||
struct PoolSettings
|
||||
{
|
||||
double thread_sleep_seconds = 10;
|
||||
double thread_sleep_seconds_random_part = 1.0;
|
||||
double thread_sleep_seconds_if_nothing_to_do = 0.1;
|
||||
|
||||
/// For exponential backoff.
|
||||
double task_sleep_seconds_when_no_work_min = 10;
|
||||
double task_sleep_seconds_when_no_work_max = 600;
|
||||
double task_sleep_seconds_when_no_work_multiplier = 1.1;
|
||||
double task_sleep_seconds_when_no_work_random_part = 1.0;
|
||||
|
||||
CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask;
|
||||
|
||||
PoolSettings() noexcept {}
|
||||
};
|
||||
|
||||
BackgroundProcessingPool(int size_,
|
||||
const PoolSettings & pool_settings = {},
|
||||
const char * log_name = "BackgroundProcessingPool",
|
||||
const char * thread_name_ = "BackgrProcPool");
|
||||
|
||||
size_t getNumberOfThreads() const
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
/// Create task and start it.
|
||||
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, bool allow_execute_in_parallel = true);
|
||||
|
||||
/// Invoked by Storage*MergeTree at shutdown
|
||||
void removeTask(const TaskHandle & task);
|
||||
|
||||
~BackgroundProcessingPool();
|
||||
|
||||
protected:
|
||||
friend class BackgroundProcessingPoolTaskInfo;
|
||||
|
||||
using Tasks = std::multimap<Poco::Timestamp, TaskHandle>; /// key is desired next time to execute (priority).
|
||||
using Threads = std::vector<ThreadFromGlobalPool>;
|
||||
|
||||
const size_t size;
|
||||
const char * thread_name;
|
||||
Poco::Logger * logger;
|
||||
|
||||
Tasks tasks; /// Ordered in priority.
|
||||
std::mutex tasks_mutex;
|
||||
|
||||
Threads threads;
|
||||
|
||||
bool shutdown{false};
|
||||
std::condition_variable wake_event;
|
||||
|
||||
/// Thread group used for profiling purposes
|
||||
ThreadGroupStatusPtr thread_group;
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
|
||||
class BackgroundProcessingPoolTaskInfo
|
||||
{
|
||||
public:
|
||||
/// Signals random idle thread from the pool that this task is ready to be executed.
|
||||
void signalReadyToRun();
|
||||
|
||||
BackgroundProcessingPoolTaskInfo(BackgroundProcessingPool & pool_, const BackgroundProcessingPool::Task & function_)
|
||||
: pool(pool_), task_function(function_) {}
|
||||
|
||||
protected:
|
||||
friend class BackgroundProcessingPool;
|
||||
|
||||
BackgroundProcessingPool & pool;
|
||||
BackgroundProcessingPool::Task task_function;
|
||||
|
||||
/// 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};
|
||||
|
||||
BackgroundProcessingPool::Tasks::iterator iterator;
|
||||
|
||||
/// For exponential backoff.
|
||||
size_t count_no_work_done = 0;
|
||||
};
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user