ClickHouse/dbms/include/DB/Storages/MergeTree/BackgroundProcessingPool.h

123 lines
3.9 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>
2014-04-11 13:05:17 +00:00
#include <Poco/RWLock.h>
2014-07-03 08:51:11 +00:00
#include <Poco/Event.h>
2016-10-14 02:51:03 +00:00
#include <Poco/Timestamp.h>
2014-04-11 13:05:17 +00:00
#include <DB/Core/Types.h>
namespace DB
{
/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле.
* При этом, одна задача может выполняться одновременно из разных потоков.
* Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния).
* Задача - функция, возвращающая bool - сделала ли она какую-либо работу.
* Если не сделала, то в следующий раз будет выполнена позже.
*
* Также, задача во время выполнения может временно увеличить какой-либо счётчик, относящийся ко всем задачам
* - например, число одновременно идующих слияний.
2014-04-11 13:05:17 +00:00
*/
class BackgroundProcessingPool
{
public:
using Counters = std::map<String, int>;
2014-04-11 13:05:17 +00:00
/** Используется изнутри задачи. Позволяет инкрементировать какие-нибудь счетчики.
* После завершения задачи, все изменения откатятся.
2014-04-11 13:05:17 +00:00
* Например, чтобы можно было узнавать количество потоков, выполняющих большое слияние,
* можно в таске, выполняющей большое слияние, инкрементировать счетчик. Декрементировать обратно его не нужно.
*/
class Context
{
public:
void incrementCounter(const String & name, int value = 1)
{
2016-09-01 02:26:45 +00:00
std::unique_lock<std::mutex> lock(pool.counters_mutex);
2014-04-11 13:05:17 +00:00
local_counters[name] += value;
pool.counters[name] += value;
}
private:
friend class BackgroundProcessingPool;
Context(BackgroundProcessingPool & pool_, Counters & local_counters_) : pool(pool_), local_counters(local_counters_) {}
BackgroundProcessingPool & pool;
Counters & local_counters;
};
2016-10-14 02:51:03 +00:00
/// 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 (Context & context)>;
2014-04-11 13:05:17 +00:00
2014-07-03 08:51:11 +00:00
class TaskInfo
{
public:
2016-10-14 02:51:03 +00:00
/// Wake up any thread.
void wake();
2014-07-03 08:51:11 +00:00
2016-10-14 02:51:03 +00:00
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_) {}
2014-07-03 08:51:11 +00:00
private:
friend class BackgroundProcessingPool;
BackgroundProcessingPool & pool;
Task function;
2016-10-14 02:51:03 +00:00
/// Read lock is hold when task is executed.
Poco::RWLock rwlock;
2016-07-31 03:53:16 +00:00
std::atomic<bool> removed {false};
2016-10-14 02:51:03 +00:00
std::multimap<Poco::Timestamp, std::shared_ptr<TaskInfo>>::iterator iterator;
2014-07-03 08:51:11 +00:00
};
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
2014-04-11 13:05:17 +00:00
{
return size;
2014-04-11 13:05:17 +00:00
}
int getCounter(const String & name);
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
private:
2016-10-14 02:51:03 +00:00
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;
2016-10-14 02:51:03 +00:00
Tasks tasks; /// Ordered in priority.
2016-09-01 02:26:45 +00:00
std::mutex tasks_mutex;
2014-04-11 13:05:17 +00:00
Counters counters;
2016-09-01 02:26:45 +00:00
std::mutex counters_mutex;
Threads threads;
2016-07-31 03:53:16 +00:00
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
2014-04-11 13:05:17 +00:00
}