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

252 lines
7.0 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>
2014-04-11 13:05:17 +00:00
#include <Poco/Mutex.h>
#include <Poco/RWLock.h>
2014-07-03 08:51:11 +00:00
#include <Poco/Event.h>
2014-04-11 13:05:17 +00:00
#include <DB/Core/Types.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
2014-07-02 12:30:38 +00:00
#include <DB/IO/WriteHelpers.h>
2014-04-11 13:05:17 +00:00
namespace DB
{
/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле.
* Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния).
* Задача - функция, возвращающая bool - сделала ли она какую-либо работу.
* Если сделала - надо выполнить ещё раз. Если нет - надо подождать несколько секунд, или до события wake, и выполнить ещё раз.
*
* Также, задача во время выполнения может временно увеличить какой-либо счётчик, относящийся ко всем задачам
* - например, число одновременно идующих слияний.
2014-04-11 13:05:17 +00:00
*/
class BackgroundProcessingPool
{
public:
typedef std::map<String, int> Counters;
/** Используется изнутри задачи. Позволяет инкрементировать какие-нибудь счетчики.
* После завершения задачи, все изменения откатятся.
2014-04-11 13:05:17 +00:00
* Например, чтобы можно было узнавать количество потоков, выполняющих большое слияние,
* можно в таске, выполняющей большое слияние, инкрементировать счетчик. Декрементировать обратно его не нужно.
*/
class Context
{
public:
void incrementCounter(const String & name, int value = 1)
{
std::unique_lock<std::mutex> lock(pool.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;
};
/// Возвращает true, если что-то получилось сделать. В таком случае поток не будет спать перед следующим вызовом.
typedef std::function<bool (Context & context)> Task;
2014-07-03 08:51:11 +00:00
class TaskInfo
{
public:
/// Переставить таск в начало очереди и разбудить какой-нибудь поток.
void wake()
{
Poco::ScopedReadRWLock rlock(rwlock);
if (removed)
return;
std::unique_lock<std::mutex> lock(pool.mutex);
2014-07-03 08:51:11 +00:00
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
/// Не очень надёжно: если все потоки сейчас выполняют работу, этот вызов никого не разбудит,
/// и все будут спать в конце итерации.
pool.wake_event.notify_one();
2014-07-03 08:51:11 +00:00
}
private:
friend class BackgroundProcessingPool;
BackgroundProcessingPool & pool;
Task function;
/// При выполнении задачи, держится read lock. Переменная removed меняется под write lock-ом.
Poco::RWLock rwlock;
volatile bool removed = false;
2014-07-03 08:51:11 +00:00
std::list<std::shared_ptr<TaskInfo>>::iterator iterator;
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_) {}
2014-07-03 08:51:11 +00:00
};
typedef std::shared_ptr<TaskInfo> TaskHandle;
2014-04-11 13:05:17 +00:00
BackgroundProcessingPool(int size_) : size(size_)
2014-04-11 13:05:17 +00:00
{
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
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)
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
return counters[name];
}
TaskHandle addTask(const Task & task)
{
2014-07-03 08:51:11 +00:00
TaskHandle res(new TaskInfo(*this, task));
2014-04-11 13:05:17 +00:00
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
tasks.push_back(res);
2014-07-03 08:51:11 +00:00
res->iterator = --tasks.end();
2014-04-11 13:05:17 +00:00
}
wake_event.notify_all();
2014-04-11 13:05:17 +00:00
return res;
}
2014-07-03 08:51:11 +00:00
void removeTask(const TaskHandle & task)
2014-04-11 13:05:17 +00:00
{
/// Дождёмся завершения всех выполнений этой задачи.
2014-04-11 13:05:17 +00:00
{
Poco::ScopedWriteRWLock wlock(task->rwlock);
2014-04-11 13:05:17 +00:00
task->removed = true;
}
{
std::unique_lock<std::mutex> lock(mutex);
tasks.erase(task->iterator);
2014-04-11 13:05:17 +00:00
}
}
~BackgroundProcessingPool()
{
try
{
shutdown = true;
wake_event.notify_all();
for (std::thread & thread : threads)
thread.join();
2014-04-11 13:05:17 +00:00
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
private:
2014-07-03 08:51:11 +00:00
typedef std::list<TaskHandle> Tasks;
2014-04-11 13:05:17 +00:00
typedef std::vector<std::thread> Threads;
const size_t size;
static constexpr double sleep_seconds = 10;
Tasks tasks; /// Задачи в порядке, в котором мы планируем их выполнять.
2014-04-11 13:05:17 +00:00
Counters counters;
std::mutex mutex; /// Для работы со списком tasks, а также с counters (когда threads не пустой).
Threads threads;
volatile bool shutdown = false;
std::condition_variable wake_event;
2014-04-11 13:05:17 +00:00
2014-04-11 13:05:17 +00:00
void threadFunction()
{
while (!shutdown)
{
Counters counters_diff;
bool need_sleep = false;
try
{
2014-07-03 08:51:11 +00:00
TaskHandle task;
2014-04-11 13:05:17 +00:00
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
if (!tasks.empty())
{
need_sleep = true;
2014-07-03 08:51:11 +00:00
task = tasks.front();
tasks.splice(tasks.end(), tasks, tasks.begin());
2014-04-11 13:05:17 +00:00
}
}
if (shutdown)
break;
if (!task)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
2014-04-11 13:05:17 +00:00
continue;
}
Poco::ScopedReadRWLock rlock(task->rwlock);
2014-04-11 13:05:17 +00:00
if (task->removed)
continue;
Context context(*this, counters_diff);
if (task->function(context))
{
/// Если у задачи получилось выполнить какую-то работу, запустим её снова без паузы.
need_sleep = false;
2014-07-03 08:51:11 +00:00
std::unique_lock<std::mutex> lock(mutex);
tasks.splice(tasks.begin(), tasks, task->iterator);
2014-04-11 13:05:17 +00:00
}
}
catch (...)
{
need_sleep = true;
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// Вычтем все счётчики обратно.
2014-04-11 13:05:17 +00:00
if (!counters_diff.empty())
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
for (const auto & it : counters_diff)
counters[it.first] -= it.second;
}
if (shutdown)
break;
if (need_sleep)
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
2014-04-11 13:05:17 +00:00
}
}
}
};
2014-07-02 12:30:38 +00:00
typedef Poco::SharedPtr<BackgroundProcessingPool> BackgroundProcessingPoolPtr;
2014-04-11 13:05:17 +00:00
}