ClickHouse/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h

97 lines
2.5 KiB
C++
Raw Normal View History

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>
#include <condition_variable>
#include <mutex>
#include <shared_mutex>
#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>
#include <Core/Types.h>
2014-04-11 13:05:17 +00:00
namespace DB
{
class BackgroundProcessingPool;
class BackgroundProcessingPoolTaskInfo;
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:
/// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task.
using Task = std::function<bool()>;
using TaskInfo = BackgroundProcessingPoolTaskInfo;
using TaskHandle = std::shared_ptr<TaskInfo>;
2014-07-03 08:51:11 +00:00
2014-04-11 13:05:17 +00:00
BackgroundProcessingPool(int size_);
2014-04-11 13:05:17 +00:00
size_t getNumberOfThreads() const
{
return size;
}
2014-04-11 13:05:17 +00:00
TaskHandle addTask(const Task & task);
void removeTask(const TaskHandle & task);
2014-04-11 13:05:17 +00:00
~BackgroundProcessingPool();
2014-04-11 13:05:17 +00:00
protected:
friend class BackgroundProcessingPoolTaskInfo;
using Tasks = std::multimap<Poco::Timestamp, TaskHandle>; /// key is desired next time to execute (priority).
using Threads = std::vector<std::thread>;
2014-04-11 13:05:17 +00:00
const size_t size;
static constexpr double sleep_seconds = 10;
static constexpr double sleep_seconds_random_part = 1.0;
Tasks tasks; /// Ordered in priority.
std::mutex tasks_mutex;
2016-09-01 02:26:45 +00:00
Threads threads;
std::atomic<bool> shutdown {false};
std::condition_variable wake_event;
2014-04-11 13:05:17 +00:00
void threadFunction();
2014-04-11 13:05:17 +00:00
};
using BackgroundProcessingPoolPtr = std::shared_ptr<BackgroundProcessingPool>;
2014-07-02 12:30:38 +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;
};
2014-04-11 13:05:17 +00:00
}