2014-04-11 13:05:17 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <thread>
|
|
|
|
#include <set>
|
|
|
|
#include <map>
|
2014-07-03 08:51:11 +00:00
|
|
|
#include <list>
|
2014-07-23 09:14:51 +00:00
|
|
|
#include <condition_variable>
|
2016-05-28 10:15:36 +00:00
|
|
|
#include <mutex>
|
2017-07-28 17:34:02 +00:00
|
|
|
#include <shared_mutex>
|
2017-02-02 22:08:19 +00:00
|
|
|
#include <atomic>
|
2017-09-18 18:52:12 +00:00
|
|
|
#include <functional>
|
2014-07-03 08:51:11 +00:00
|
|
|
#include <Poco/Event.h>
|
2016-10-14 02:51:03 +00:00
|
|
|
#include <Poco/Timestamp.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Types.h>
|
2019-12-12 22:08:54 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
2018-06-19 20:30:35 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2019-11-11 06:53:30 +00:00
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
|
2019-12-12 22:08:54 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric BackgroundPoolTask;
|
|
|
|
extern const Metric MemoryTrackingInBackgroundProcessingPool;
|
|
|
|
}
|
|
|
|
|
2014-04-11 13:05:17 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-03-26 14:12:07 +00:00
|
|
|
class BackgroundProcessingPool;
|
|
|
|
class BackgroundProcessingPoolTaskInfo;
|
|
|
|
|
2018-12-26 17:03:29 +00:00
|
|
|
enum class BackgroundProcessingPoolTaskResult
|
|
|
|
{
|
|
|
|
SUCCESS,
|
|
|
|
ERROR,
|
|
|
|
NOTHING_TO_DO,
|
|
|
|
};
|
2019-04-06 00:36:04 +00:00
|
|
|
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** 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.
|
2014-04-11 13:05:17 +00:00
|
|
|
*/
|
|
|
|
class BackgroundProcessingPool
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task.
|
2018-12-26 17:03:29 +00:00
|
|
|
using TaskResult = BackgroundProcessingPoolTaskResult;
|
|
|
|
using Task = std::function<TaskResult()>;
|
2018-03-26 14:12:07 +00:00
|
|
|
using TaskInfo = BackgroundProcessingPoolTaskInfo;
|
2017-04-01 07:20:54 +00:00
|
|
|
using TaskHandle = std::shared_ptr<TaskInfo>;
|
2014-07-03 08:51:11 +00:00
|
|
|
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2019-12-05 08:05:07 +00:00
|
|
|
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;
|
|
|
|
|
2019-12-12 22:08:54 +00:00
|
|
|
CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask;
|
|
|
|
CurrentMetrics::Metric memory_metric = CurrentMetrics::MemoryTrackingInBackgroundProcessingPool;
|
|
|
|
|
2019-12-05 08:05:07 +00:00
|
|
|
PoolSettings() noexcept {}
|
|
|
|
};
|
|
|
|
|
2019-11-17 21:00:10 +00:00
|
|
|
BackgroundProcessingPool(int size_,
|
2019-12-05 08:05:07 +00:00
|
|
|
const PoolSettings & pool_settings = {},
|
2019-11-17 21:00:10 +00:00
|
|
|
const char * log_name = "BackgroundProcessingPool",
|
|
|
|
const char * thread_name_ = "BackgrProcPool");
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t getNumberOfThreads() const
|
|
|
|
{
|
|
|
|
return size;
|
|
|
|
}
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2020-03-07 03:52:50 +00:00
|
|
|
/// Create task and start it.
|
2017-04-01 07:20:54 +00:00
|
|
|
TaskHandle addTask(const Task & task);
|
2019-02-12 16:41:06 +00:00
|
|
|
|
2020-03-07 03:52:50 +00:00
|
|
|
/// 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);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void removeTask(const TaskHandle & task);
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
~BackgroundProcessingPool();
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2018-03-26 14:12:07 +00:00
|
|
|
protected:
|
|
|
|
friend class BackgroundProcessingPoolTaskInfo;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using Tasks = std::multimap<Poco::Timestamp, TaskHandle>; /// key is desired next time to execute (priority).
|
2019-01-14 19:22:09 +00:00
|
|
|
using Threads = std::vector<ThreadFromGlobalPool>;
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const size_t size;
|
2019-11-14 11:10:17 +00:00
|
|
|
const char * thread_name;
|
|
|
|
Poco::Logger * logger;
|
2014-08-14 01:48:50 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Tasks tasks; /// Ordered in priority.
|
|
|
|
std::mutex tasks_mutex;
|
2016-09-01 02:26:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Threads threads;
|
2014-07-23 09:14:51 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::atomic<bool> shutdown {false};
|
|
|
|
std::condition_variable wake_event;
|
2014-04-11 13:05:17 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
/// Thread group used for profiling purposes
|
|
|
|
ThreadGroupStatusPtr thread_group;
|
2014-08-14 01:48:50 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void threadFunction();
|
2019-11-11 06:53:30 +00:00
|
|
|
|
|
|
|
private:
|
2019-12-05 08:05:07 +00:00
|
|
|
PoolSettings settings;
|
2014-04-11 13:05:17 +00:00
|
|
|
};
|
|
|
|
|
2018-03-26 14:12:07 +00:00
|
|
|
|
|
|
|
class BackgroundProcessingPoolTaskInfo
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
/// Wake up any thread.
|
|
|
|
void wake();
|
|
|
|
|
|
|
|
BackgroundProcessingPoolTaskInfo(BackgroundProcessingPool & pool_, const BackgroundProcessingPool::Task & function_)
|
|
|
|
: pool(pool_), function(function_) {}
|
|
|
|
|
|
|
|
protected:
|
|
|
|
friend class BackgroundProcessingPool;
|
|
|
|
|
|
|
|
BackgroundProcessingPool & pool;
|
|
|
|
BackgroundProcessingPool::Task function;
|
|
|
|
|
|
|
|
/// Read lock is hold when task is executed.
|
|
|
|
std::shared_mutex rwlock;
|
|
|
|
std::atomic<bool> removed {false};
|
|
|
|
|
|
|
|
std::multimap<Poco::Timestamp, std::shared_ptr<BackgroundProcessingPoolTaskInfo>>::iterator iterator;
|
2018-07-08 02:57:41 +00:00
|
|
|
|
|
|
|
/// For exponential backoff.
|
|
|
|
size_t count_no_work_done = 0;
|
2018-03-26 14:12:07 +00:00
|
|
|
};
|
|
|
|
|
2014-04-11 13:05:17 +00:00
|
|
|
}
|