2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Common/setThreadName.h>
|
|
|
|
#include <Common/CurrentMetrics.h>
|
|
|
|
#include <Common/MemoryTracker.h>
|
2017-09-09 23:17:38 +00:00
|
|
|
#include <Common/randomSeed.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2015-09-29 19:19:54 +00:00
|
|
|
#include <common/logger_useful.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
2018-05-29 18:14:31 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2018-03-26 14:12:07 +00:00
|
|
|
#include <Interpreters/DNSCacheUpdater.h>
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
#include <ext/scope_guard.h>
|
2017-09-09 23:17:38 +00:00
|
|
|
#include <pcg_random.hpp>
|
2016-05-23 00:40:28 +00:00
|
|
|
#include <random>
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
|
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const Metric BackgroundPoolTask;
|
|
|
|
extern const Metric MemoryTrackingInBackgroundProcessingPool;
|
2016-10-24 04:06:27 +00:00
|
|
|
}
|
|
|
|
|
2015-09-24 04:13:06 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-07-08 02:57:41 +00:00
|
|
|
static constexpr double thread_sleep_seconds = 10;
|
|
|
|
static constexpr double thread_sleep_seconds_random_part = 1.0;
|
2018-12-26 17:03:29 +00:00
|
|
|
static constexpr double thread_sleep_seconds_if_nothing_to_do = 0.1;
|
2015-09-24 07:03:20 +00:00
|
|
|
|
2018-07-08 02:57:41 +00:00
|
|
|
/// For exponential backoff.
|
|
|
|
static constexpr double task_sleep_seconds_when_no_work_min = 10;
|
|
|
|
static constexpr double task_sleep_seconds_when_no_work_max = 600;
|
|
|
|
static constexpr double task_sleep_seconds_when_no_work_multiplier = 1.1;
|
|
|
|
static constexpr double task_sleep_seconds_when_no_work_random_part = 1.0;
|
2015-09-24 07:03:20 +00:00
|
|
|
|
|
|
|
|
2018-03-26 14:12:07 +00:00
|
|
|
void BackgroundProcessingPoolTaskInfo::wake()
|
2015-09-24 04:13:06 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::Timestamp current_time;
|
2016-09-01 02:26:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(pool.tasks_mutex);
|
2016-09-01 02:26:45 +00:00
|
|
|
|
2018-11-27 14:59:10 +00:00
|
|
|
/// This will ensure that iterator is valid. Must be done under the same mutex when the iterator is invalidated.
|
|
|
|
if (removed)
|
|
|
|
return;
|
2016-09-01 02:26:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto next_time_to_execute = iterator->first;
|
2018-03-26 14:12:07 +00:00
|
|
|
auto this_task_handle = iterator->second;
|
2016-10-14 02:51:03 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// If this task was done nothing at previous time and it has to sleep, then cancel sleep time.
|
|
|
|
if (next_time_to_execute > current_time)
|
|
|
|
next_time_to_execute = current_time;
|
2016-10-14 02:51:03 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
pool.tasks.erase(iterator);
|
|
|
|
iterator = pool.tasks.emplace(next_time_to_execute, this_task_handle);
|
|
|
|
}
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Note that if all threads are currently do some work, this call will not wakeup any thread.
|
|
|
|
pool.wake_event.notify_one();
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
threads.resize(size);
|
|
|
|
for (auto & thread : threads)
|
2019-01-14 19:22:09 +00:00
|
|
|
thread = ThreadFromGlobalPool([this] { threadFunction(); });
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
TaskHandle res = std::make_shared<TaskInfo>(*this, task);
|
2016-10-14 02:51:03 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::Timestamp current_time;
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
res->iterator = tasks.emplace(current_time, res);
|
|
|
|
}
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
wake_event.notify_all();
|
2015-09-24 04:13:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return res;
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void BackgroundProcessingPool::removeTask(const TaskHandle & task)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (task->removed.exchange(true))
|
|
|
|
return;
|
|
|
|
|
2017-07-28 17:34:02 +00:00
|
|
|
/// Wait for all executions of this task.
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock wlock(task->rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
tasks.erase(task->iterator);
|
2018-11-27 14:59:10 +00:00
|
|
|
/// Note that the task may be still accessible through TaskHandle (shared_ptr).
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
BackgroundProcessingPool::~BackgroundProcessingPool()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
shutdown = true;
|
|
|
|
wake_event.notify_all();
|
2019-01-14 19:22:09 +00:00
|
|
|
for (auto & thread : threads)
|
2017-04-01 07:20:54 +00:00
|
|
|
thread.join();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void BackgroundProcessingPool::threadFunction()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
setThreadName("BackgrProcPool");
|
|
|
|
|
2018-08-22 03:58:19 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(tasks_mutex);
|
|
|
|
|
|
|
|
if (thread_group)
|
|
|
|
{
|
|
|
|
/// Put all threads to one thread pool
|
|
|
|
CurrentThread::attachTo(thread_group);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
CurrentThread::initializeQuery();
|
|
|
|
thread_group = CurrentThread::getGroup();
|
|
|
|
}
|
|
|
|
}
|
2018-06-19 20:30:35 +00:00
|
|
|
|
2018-08-22 03:58:19 +00:00
|
|
|
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
|
2019-03-14 18:03:35 +00:00
|
|
|
if (auto memory_tracker = CurrentThread::getMemoryTracker())
|
|
|
|
memory_tracker->setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-09-09 23:17:38 +00:00
|
|
|
pcg64 rng(randomSeed());
|
2018-07-08 02:57:41 +00:00
|
|
|
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
while (!shutdown)
|
|
|
|
{
|
2018-12-26 17:03:29 +00:00
|
|
|
TaskResult task_result = TaskResult::ERROR;
|
2017-04-01 07:20:54 +00:00
|
|
|
TaskHandle task;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
Poco::Timestamp min_time;
|
|
|
|
|
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!tasks.empty())
|
|
|
|
{
|
|
|
|
for (const auto & time_handle : tasks)
|
|
|
|
{
|
|
|
|
if (!time_handle.second->removed)
|
|
|
|
{
|
|
|
|
min_time = time_handle.first;
|
|
|
|
task = time_handle.second;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (shutdown)
|
|
|
|
break;
|
|
|
|
|
|
|
|
if (!task)
|
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
wake_event.wait_for(lock,
|
2018-07-08 02:57:41 +00:00
|
|
|
std::chrono::duration<double>(thread_sleep_seconds
|
|
|
|
+ std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
|
2017-04-01 07:20:54 +00:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// No tasks ready for execution.
|
|
|
|
Poco::Timestamp current_time;
|
|
|
|
if (min_time > current_time)
|
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
wake_event.wait_for(lock, std::chrono::microseconds(
|
2018-07-08 02:57:41 +00:00
|
|
|
min_time - current_time + std::uniform_int_distribution<uint64_t>(0, thread_sleep_seconds_random_part * 1000000)(rng)));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-08-22 03:58:19 +00:00
|
|
|
std::shared_lock rlock(task->rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (task->removed)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
{
|
|
|
|
CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask};
|
2018-12-26 17:03:29 +00:00
|
|
|
task_result = task->function();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (shutdown)
|
|
|
|
break;
|
|
|
|
|
|
|
|
{
|
2018-08-22 03:58:19 +00:00
|
|
|
std::unique_lock lock(tasks_mutex);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (task->removed)
|
|
|
|
continue;
|
|
|
|
|
2018-12-26 17:03:29 +00:00
|
|
|
if (task_result == TaskResult::SUCCESS)
|
2018-07-08 02:57:41 +00:00
|
|
|
task->count_no_work_done = 0;
|
|
|
|
else
|
|
|
|
++task->count_no_work_done;
|
|
|
|
|
|
|
|
/// If task has done work, it could be executed again immediately.
|
|
|
|
/// If not, add delay before next run.
|
|
|
|
|
|
|
|
Poco::Timestamp next_time_to_execute; /// current time
|
2018-12-26 17:03:29 +00:00
|
|
|
if (task_result == TaskResult::ERROR)
|
2018-07-08 02:57:41 +00:00
|
|
|
next_time_to_execute += 1000000 * (std::min(
|
|
|
|
task_sleep_seconds_when_no_work_max,
|
|
|
|
task_sleep_seconds_when_no_work_min * std::pow(task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done))
|
|
|
|
+ std::uniform_real_distribution<double>(0, task_sleep_seconds_when_no_work_random_part)(rng));
|
2018-12-26 17:03:29 +00:00
|
|
|
else if (task_result == TaskResult::NOTHING_TO_DO)
|
|
|
|
next_time_to_execute += 1000000 * thread_sleep_seconds_if_nothing_to_do;
|
2018-07-08 02:57:41 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
tasks.erase(task->iterator);
|
|
|
|
task->iterator = tasks.emplace(next_time_to_execute, task);
|
|
|
|
}
|
|
|
|
}
|
2015-09-24 04:13:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|