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

280 lines
8.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>
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>
#include <Yandex/logger_useful.h>
2014-04-11 13:05:17 +00:00
namespace DB
{
/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле.
* При этом, одна задача может выполняться одновременно из разных потоков.
* Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния).
* Задача - функция, возвращающая bool - сделала ли она какую-либо работу.
* Если не сделала, то в следующий раз будет выполнена позже.
*
* Также, задача во время выполнения может временно увеличить какой-либо счётчик, относящийся ко всем задачам
* - например, число одновременно идующих слияний.
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:
/// Разбудить какой-нибудь поток.
2014-07-03 08:51:11 +00:00
void wake()
{
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);
/// Если эта задача в прошлый раз ничего не сделала, и ей было назначено спать, то отменим время сна.
time_t current_time = time(0);
if (next_time_to_execute > current_time)
next_time_to_execute = current_time;
/// Если все потоки сейчас выполняют работу, этот вызов никого не разбудит.
pool.wake_event.notify_one();
2014-07-03 08:51:11 +00:00
}
private:
friend class BackgroundProcessingPool;
BackgroundProcessingPool & pool;
Task function;
/// При выполнении задачи, держится read lock.
Poco::RWLock rwlock;
volatile bool removed = false;
volatile time_t next_time_to_execute = 0; /// Приоритет задачи. Для совпадающего времени в секундах берётся первая по списку задача.
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
{
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
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);
res->iterator = tasks.insert(tasks.begin(), res);
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
{
task->removed = true;
/// Дождёмся завершения всех выполнений этой задачи.
2014-04-11 13:05:17 +00:00
{
Poco::ScopedWriteRWLock wlock(task->rwlock);
2014-04-11 13:05:17 +00:00
}
{
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;
enum { 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 has_exception = false;
2014-04-11 13:05:17 +00:00
try
{
2014-07-03 08:51:11 +00:00
TaskHandle task;
time_t min_time = std::numeric_limits<time_t>::max();
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())
{
/// O(n), n - число задач. По сути, количество таблиц. Обычно их мало.
for (const auto & handle : tasks)
{
time_t next_time_to_execute = handle->next_time_to_execute;
if (next_time_to_execute < min_time)
{
min_time = next_time_to_execute;
task = handle;
}
}
if (task) /// Переложим в конец очереди (уменьшим приоритет среди задач с одинаковым next_time_to_execute).
tasks.splice(tasks.end(), tasks, task->iterator);
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;
}
if (task->removed)
continue;
/// Лучшей задачи не нашлось, а эта задача в прошлый раз ничего не сделала, и поэтому ей назначено некоторое время спать.
time_t current_time = time(0);
if (min_time > current_time)
2014-04-11 13:05:17 +00:00
{
std::unique_lock<std::mutex> lock(mutex);
wake_event.wait_for(lock, std::chrono::duration<double>(min_time - current_time));
2014-04-11 13:05:17 +00:00
}
Poco::ScopedReadRWLock rlock(task->rwlock);
if (task->removed)
continue;
Context context(*this, counters_diff);
bool done_work = task->function(context);
/// Если задача сделала полезную работу, то она сможет выполняться в следующий раз хоть сразу.
/// Если нет - добавляем задержку перед повторным исполнением.
task->next_time_to_execute = time(0) + (done_work ? 0 : sleep_seconds);
2014-04-11 13:05:17 +00:00
}
catch (...)
{
has_exception = true;
2014-04-11 13:05:17 +00:00
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 (has_exception)
2014-04-11 13:05:17 +00:00
{
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
}