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

293 lines
7.2 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
{
/** В нескольких потоках в бесконечном цикле выполняет указанные функции.
*/
class BackgroundProcessingPool
{
public:
typedef std::map<String, int> Counters;
/** Используется изнутри таски. Позволяет инкрементировать какие-нибудь счетчики.
* После завершения таски, все изменения откатятся.
* Например, чтобы можно было узнавать количество потоков, выполняющих большое слияние,
* можно в таске, выполняющей большое слияние, инкрементировать счетчик. Декрементировать обратно его не нужно.
*/
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()
{
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;
Poco::RWLock lock;
volatile bool removed;
std::list<std::shared_ptr<TaskInfo>>::iterator iterator;
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_), removed(false) {}
};
typedef std::shared_ptr<TaskInfo> TaskHandle;
2014-04-11 13:05:17 +00:00
2014-07-02 12:30:38 +00:00
BackgroundProcessingPool(int size_) : size(size_), sleep_seconds(10), shutdown(false) {}
2014-04-11 13:05:17 +00:00
void setNumberOfThreads(int size_)
{
2014-07-02 12:30:38 +00:00
if (size_ <= 0)
throw Exception("Invalid number of threads: " + toString(size_), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
std::unique_lock<std::mutex> tlock(threads_mutex);
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
if (size_ == size)
return;
if (threads.empty())
{
size = size_;
return;
}
throw Exception("setNumberOfThreads is not implemented for non-empty pool", ErrorCodes::NOT_IMPLEMENTED);
}
int getNumberOfThreads()
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
return size;
}
void setSleepTime(double seconds)
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
sleep_seconds = seconds;
}
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)
{
std::unique_lock<std::mutex> lock(threads_mutex);
2014-04-11 13:05:17 +00:00
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
}
if (threads.empty())
{
shutdown = false;
counters.clear();
threads.resize(size);
for (std::thread & thread : threads)
thread = std::thread(std::bind(&BackgroundProcessingPool::threadFunction, this));
}
return res;
}
2014-07-03 08:51:11 +00:00
void removeTask(const TaskHandle & task)
2014-04-11 13:05:17 +00:00
{
std::unique_lock<std::mutex> tlock(threads_mutex);
2014-04-11 13:05:17 +00:00
/// Дождемся завершения всех выполнений этой задачи.
{
Poco::ScopedWriteRWLock wlock(task->lock);
task->removed = true;
}
{
std::unique_lock<std::mutex> lock(mutex);
2014-04-11 13:05:17 +00:00
auto it = std::find(tasks.begin(), tasks.end(), task);
if (it == tasks.end())
throw Exception("Task not found", ErrorCodes::LOGICAL_ERROR);
tasks.erase(it);
}
if (tasks.empty())
{
shutdown = true;
wake_event.notify_all();
2014-04-11 13:05:17 +00:00
for (std::thread & thread : threads)
thread.join();
threads.clear();
counters.clear();
}
}
~BackgroundProcessingPool()
{
try
{
std::unique_lock<std::mutex> lock(threads_mutex);
2014-04-11 13:05:17 +00:00
if (!threads.empty())
{
LOG_ERROR(&Logger::get("~BackgroundProcessingPool"), "Destroying non-empty BackgroundProcessingPool");
shutdown = true;
wake_event.notify_all();
2014-04-11 13:05:17 +00:00
for (std::thread & thread : threads)
thread.join();
}
}
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;
std::mutex threads_mutex;
std::mutex mutex;
2014-04-11 13:05:17 +00:00
int size;
2014-07-03 08:51:11 +00:00
Tasks tasks; /// Таски в порядке, в котором мы планируем их выполнять.
2014-04-11 13:05:17 +00:00
Threads threads;
Counters counters;
double sleep_seconds;
volatile bool shutdown;
std::condition_variable wake_event;
2014-04-11 13:05:17 +00:00
void threadFunction()
{
while (!shutdown)
{
Counters counters_diff;
bool need_sleep = false;
size_t tasks_count = 1;
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::this_thread::sleep_for(std::chrono::duration<double>(sleep_seconds));
continue;
}
Poco::ScopedReadRWLock rlock(task->lock);
if (task->removed)
continue;
Context context(*this, counters_diff);
if (task->function(context))
{
2014-07-03 08:51:11 +00:00
/// Если у таска получилось выполнить какую-то работу, запустим его снова без паузы.
std::unique_lock<std::mutex> lock(mutex);
2014-07-03 08:51:11 +00:00
auto it = std::find(tasks.begin(), tasks.end(), task);
if (it != tasks.end())
{
need_sleep = false;
tasks.splice(tasks.begin(), tasks, it);
}
2014-04-11 13:05:17 +00:00
}
}
catch (...)
{
need_sleep = true;
tryLogCurrentException(__PRETTY_FUNCTION__);
}
/// Вычтем все счетчики обратно.
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 / tasks_count));
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
}