ClickHouse/src/Storages/MergeTree/MergeMutateExecutor.cpp

151 lines
3.3 KiB
C++
Raw Normal View History

2021-08-30 19:37:03 +00:00
#include <Storages/MergeTree/MergeMutateExecutor.h>
#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-08-31 23:20:23 +00:00
std::vector<ItemPtr> 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 17:40:29 +00:00
pending.removeElements([&] (auto item) -> bool { return item->task->getStorageID() == id; });
2021-08-31 23:20:23 +00:00
/// Find pending to wait
2021-09-02 17:40:29 +00:00
tasks_to_wait = active.getAll([&] (auto item) -> bool { return item->task->getStorageID() == id; });
2021-08-31 11:02:39 +00:00
}
2021-09-02 10:39:27 +00:00
for (const auto & item : tasks_to_wait)
2021-08-31 11:02:39 +00:00
{
2021-08-31 23:20:23 +00:00
assert(item->future.valid());
item->future.wait();
}
2021-08-31 11:02:39 +00:00
2021-08-31 23:20:23 +00:00
{
std::lock_guard lock(mutex);
currently_deleting.erase(id);
2021-08-31 11:02:39 +00:00
}
}
2021-08-30 19:37:03 +00:00
void MergeTreeBackgroundExecutor::schedulerThreadFunction()
{
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-02 17:40:29 +00:00
active.tryPush(item);
try
{
/// This is needed to increase / decrease the number of threads at runtime
if (update_timer.compareAndRestartDeferred(1.))
updateConfiguration();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
2021-08-31 23:20:23 +00:00
2021-08-31 11:02:39 +00:00
2021-08-31 18:07:24 +00:00
bool res = pool.trySchedule([this, item] ()
2021-08-31 11:02:39 +00:00
{
2021-09-02 10:39:27 +00:00
setThreadName(name.c_str());
2021-08-31 23:20:23 +00:00
auto check_if_deleting = [&] () -> bool
2021-08-31 11:02:39 +00:00
{
2021-09-02 17:40:29 +00:00
active.tryErase(item);
2021-08-31 23:20:23 +00:00
for (auto & id : currently_deleting)
2021-08-31 11:02:39 +00:00
{
2021-08-31 23:20:23 +00:00
if (item->task->getStorageID() == id)
{
item->promise.set_value();
return true;
}
2021-08-31 11:02:39 +00:00
}
2021-08-31 23:20:23 +00:00
return false;
2021-08-31 11:02:39 +00:00
};
2021-08-31 23:20:23 +00:00
SCOPE_EXIT({
std::lock_guard guard(mutex);
check_if_deleting();
});
2021-08-31 11:02:39 +00:00
2021-08-30 19:37:03 +00:00
try
{
2021-08-31 23:20:23 +00:00
if (item->task->execute())
2021-08-30 19:37:03 +00:00
{
std::lock_guard guard(mutex);
2021-08-31 23:20:23 +00:00
if (check_if_deleting())
return;
2021-09-02 17:40:29 +00:00
pending.tryPush(item);
2021-08-30 19:37:03 +00:00
has_tasks.notify_one();
return;
}
2021-08-31 18:07:24 +00:00
item->task->onCompleted();
2021-08-30 19:37:03 +00:00
std::lock_guard guard(mutex);
has_tasks.notify_one();
}
catch(...)
{
2021-08-31 18:07:24 +00:00
item->task->onCompleted();
2021-08-30 19:37:03 +00:00
std::lock_guard guard(mutex);
has_tasks.notify_one();
tryLogCurrentException(__PRETTY_FUNCTION__);
}
2021-08-31 23:20:23 +00:00
2021-08-30 19:37:03 +00:00
});
if (!res)
{
2021-09-02 17:40:29 +00:00
active.tryErase(item);
pending.tryPush(item);
2021-08-30 19:37:03 +00:00
}
2021-08-31 23:20:23 +00:00
2021-08-30 19:37:03 +00:00
}
}
}