ClickHouse/src/Storages/MergeTree/MergeMutateExecutor.cpp

161 lines
3.7 KiB
C++
Raw Normal View History

2021-08-30 19:37:03 +00:00
#include <Storages/MergeTree/MergeMutateExecutor.h>
#include <Common/setThreadName.h>
2021-08-30 19:37:03 +00:00
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
namespace DB
{
2021-08-31 12:09:35 +00:00
2021-09-02 10:39:27 +00:00
String MergeTreeBackgroundExecutor::toString(Type type)
{
switch (type)
{
case Type::MERGE_MUTATE:
return "MergeMutate";
case Type::FETCH:
return "Fetch";
case Type::MOVE:
return "Move";
}
}
2021-08-31 11:02:39 +00:00
void MergeTreeBackgroundExecutor::removeTasksCorrespondingToStorage(StorageID id)
{
std::lock_guard remove_lock(remove_mutex);
2021-09-03 12:27:49 +00:00
std::vector<ActiveMeta> tasks_to_wait;
2021-08-31 11:02:39 +00:00
{
2021-08-31 23:20:23 +00:00
std::lock_guard lock(mutex);
/// Mark this StorageID as deleting
currently_deleting.emplace(id);
2021-09-02 18:49:37 +00:00
/// Erase storage related tasks from pending and select active tasks to wait for
pending.eraseAll([&] (auto item) -> bool { return item->task->getStorageID() == id; });
2021-09-03 12:27:49 +00:00
tasks_to_wait = active.getAll([&] (auto elem) -> bool { return elem.item->task->getStorageID() == id; });
2021-08-31 11:02:39 +00:00
}
2021-09-03 12:27:49 +00:00
for (auto & [item, future] : tasks_to_wait)
2021-08-31 11:02:39 +00:00
{
2021-09-03 12:27:49 +00:00
assert(future.valid());
try
{
future.wait();
}
catch (...) {}
2021-08-31 23:20:23 +00:00
}
2021-08-31 11:02:39 +00:00
2021-08-31 23:20:23 +00:00
{
std::lock_guard lock(mutex);
2021-09-03 12:27:49 +00:00
for (auto & [item, future] : tasks_to_wait)
{
assert(item.use_count() == 1);
item.reset();
}
2021-08-31 23:20:23 +00:00
currently_deleting.erase(id);
2021-08-31 11:02:39 +00:00
}
}
2021-08-30 19:37:03 +00:00
2021-09-03 12:27:49 +00:00
void MergeTreeBackgroundExecutor::routine(ItemPtr item)
{
setThreadName(name.c_str());
bool checked{false};
auto check_if_currently_deleting = [&] ()
{
checked = true;
return active.eraseAll([&] (auto & x) { return x.item == item; });
};
SCOPE_EXIT({
if (checked)
return;
std::lock_guard guard(mutex);
check_if_currently_deleting();
});
try
{
if (item->task->execute())
{
std::lock_guard guard(mutex);
if (check_if_currently_deleting())
return;
pending.tryPush(item);
has_tasks.notify_one();
return;
}
/// In a situation of a lack of memory this method can throw an exception,
/// because it may interact somehow with BackgroundSchedulePool, which may allocate memory
/// But it is rather safe, because we have try...catch block here, and another one in ThreadPool.
item->task->onCompleted();
std::lock_guard guard(mutex);
has_tasks.notify_one();
}
catch(...)
{
std::lock_guard guard(mutex);
has_tasks.notify_one();
try
{
item->task->onCompleted();
}
catch (...) {}
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
2021-08-30 19:37:03 +00:00
void MergeTreeBackgroundExecutor::schedulerThreadFunction()
{
DENY_ALLOCATIONS_IN_SCOPE;
bool status;
2021-08-30 19:37:03 +00:00
while (true)
{
2021-08-31 23:20:23 +00:00
std::unique_lock lock(mutex);
2021-08-30 19:37:03 +00:00
2021-08-31 23:20:23 +00:00
has_tasks.wait(lock, [this](){ return !pending.empty() || shutdown_suspend; });
2021-08-30 19:37:03 +00:00
2021-08-31 23:20:23 +00:00
if (shutdown_suspend)
break;
2021-08-30 19:37:03 +00:00
2021-09-02 17:40:29 +00:00
ItemPtr item;
if (!pending.tryPop(&item))
continue;
2021-08-30 19:37:03 +00:00
2021-09-03 12:27:49 +00:00
auto thread_pool_job = std::make_shared<std::packaged_task<void()>>([this, item] { routine(item); });
auto future = thread_pool_job->get_future();
bool res = pool.trySchedule([thread_pool_job] { (*thread_pool_job)(); });
2021-08-30 19:37:03 +00:00
if (!res)
{
2021-09-03 12:27:49 +00:00
active.eraseAll([&] (auto x) { return x.item == item; });
status = pending.tryPush(item);
assert(status);
2021-09-03 12:27:49 +00:00
continue;
2021-08-30 19:37:03 +00:00
}
2021-08-31 23:20:23 +00:00
2021-09-03 12:27:49 +00:00
status = active.tryPush({std::move(item), std::move(future)});
assert(status);
2021-08-30 19:37:03 +00:00
}
}
}