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>
|
2014-07-23 09:14:51 +00:00
|
|
|
|
#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)
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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()
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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);
|
2014-07-23 09:14:51 +00:00
|
|
|
|
|
|
|
|
|
/// Не очень надежно: если все потоки сейчас выполняют работу, этот вызов никого не разбудит,
|
|
|
|
|
/// и все будут спать в конце итерации.
|
|
|
|
|
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);
|
|
|
|
|
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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()
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
std::unique_lock<std::mutex> lock(mutex);
|
2014-04-11 13:05:17 +00:00
|
|
|
|
return size;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void setSleepTime(double seconds)
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
std::unique_lock<std::mutex> lock(mutex);
|
2014-04-11 13:05:17 +00:00
|
|
|
|
sleep_seconds = seconds;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
int getCounter(const String & name)
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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-23 09:14:51 +00:00
|
|
|
|
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
|
|
|
|
|
|
|
|
|
{
|
2014-07-23 09:14:51 +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
|
|
|
|
{
|
2014-07-23 09:14:51 +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;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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;
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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;
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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;
|
|
|
|
|
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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;
|
2014-07-23 09:14:51 +00:00
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
|
{
|
2014-07-23 09:14:51 +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
|
|
|
|
/// Если у таска получилось выполнить какую-то работу, запустим его снова без паузы.
|
2014-07-23 09:14:51 +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())
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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)
|
|
|
|
|
{
|
2014-07-23 09:14:51 +00:00
|
|
|
|
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
|
|
|
|
}
|