added a test

This commit is contained in:
Nikita Mikhaylov 2021-08-31 14:54:24 +00:00
parent e39f9aabde
commit dbc950caa4
6 changed files with 109 additions and 8 deletions

View File

@ -71,6 +71,7 @@ void MergeTreeBackgroundExecutor::schedulerThreadFunction()
{
ExecutableTaskPtr current;
auto current_promise = std::make_shared<std::promise<void>>();
{
std::unique_lock lock(mutex);
has_tasks.wait(lock, [this](){ return !tasks.empty() || shutdown_suspend; });
@ -90,7 +91,7 @@ void MergeTreeBackgroundExecutor::schedulerThreadFunction()
currently_executing.emplace(current, current_promise->get_future());
}
bool res = pool.trySchedule([this, task = current, promise = current_promise] () mutable
bool res = pool.trySchedule([this, task = current, promise = current_promise] ()
{
auto metric_decrementor = std::make_shared<ParanoidMetricDecrementor>(metric);
metric_decrementor->alarm();

View File

@ -25,6 +25,7 @@ public:
bool execute() override
{
res = inner();
inner = {};
return false;
}
@ -112,6 +113,17 @@ public:
pool.wait();
}
size_t active()
{
return pool.active();
}
size_t pending()
{
std::lock_guard lock(mutex);
return tasks.size();
}
private:
void updatePoolConfiguration()

View File

@ -0,0 +1,85 @@
#include <gtest/gtest.h>
#include <atomic>
#include <memory>
#include <Storages/MergeTree/ExecutableTask.h>
#include <Storages/MergeTree/MergeMutateExecutor.h>
using namespace DB;
namespace CurrentMetrics
{
extern const Metric BackgroundPoolTask;
}
class FakeExecutableTask : public ExecutableTask
{
public:
explicit FakeExecutableTask(String name_, std::function<void()> on_completed_) : name(name_), on_completed(on_completed_)
{
}
bool execute() override
{
std::this_thread::sleep_for(std::chrono::milliseconds(5));
return false;
}
StorageID getStorageID() override
{
return {"test", name};
}
void onCompleted() override
{
on_completed();
}
private:
String name;
std::function<void()> on_completed;
};
TEST(Executor, Simple)
{
auto executor = DB::MergeTreeBackgroundExecutor::create();
const size_t tasks_kinds = 25;
const size_t batch = 100;
executor->setThreadsCount([]() { return 25; });
executor->setTasksCount([] () { return tasks_kinds * batch; });
executor->setMetric(CurrentMetrics::BackgroundPoolTask);
for (size_t i = 0; i < 4; ++i)
{
for (size_t j = 0; j < tasks_kinds; ++j)
{
bool res = executor->trySchedule(std::make_shared<FakeExecutableTask>(std::to_string(j), [](){}));
ASSERT_TRUE(res);
}
}
std::vector<std::thread> threads(batch);
for (auto & thread : threads)
thread = std::thread([&] ()
{
for (size_t j = 0; j < tasks_kinds; ++j)
executor->removeTasksCorrespondingToStorage({"test", std::to_string(j)});
});
for (auto & thread : threads)
thread.join();
ASSERT_EQ(executor->active(), 0);
ASSERT_EQ(executor->pending(), 0);
ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], 0);
executor->wait();
}

View File

@ -122,8 +122,6 @@ private:
MergeTreeDataSelectExecutor reader;
MergeTreeDataWriter writer;
MergeTreeDataMergerMutator merger_mutator;
BackgroundJobAssignee background_executor;
BackgroundJobAssignee background_moves_executor;
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;
@ -150,6 +148,10 @@ private:
std::atomic<bool> shutdown_called {false};
/// Must be the last to be destroyed first
BackgroundJobAssignee background_executor;
BackgroundJobAssignee background_moves_executor;
void loadMutations();
/// Load and initialize deduplication logs. Even if deduplication setting

View File

@ -285,8 +285,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, merge_strategy_picker(*this)
, queue(*this, merge_strategy_picker)
, fetcher(*this)
, background_executor(*this, BackgroundJobAssignee::Type::DataProcessing, getContext())
, background_moves_executor(*this, BackgroundJobAssignee::Type::Moving, getContext())
, cleanup_thread(*this)
, part_check_thread(*this)
, restarting_thread(*this)
@ -295,6 +293,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size)
, replicated_fetches_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler()))
, replicated_sends_throttler(std::make_shared<Throttler>(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler()))
, background_executor(*this, BackgroundJobAssignee::Type::DataProcessing, getContext())
, background_moves_executor(*this, BackgroundJobAssignee::Type::Moving, getContext())
{
queue_updating_task = getContext()->getSchedulePool().createTask(
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });

View File

@ -370,9 +370,6 @@ private:
int metadata_version = 0;
/// Threads.
BackgroundJobAssignee background_executor;
BackgroundJobAssignee background_moves_executor;
/// A task that keeps track of the updates in the logs of all replicas and loads them into the queue.
bool queue_update_in_progress = false;
BackgroundSchedulePool::TaskHolder queue_updating_task;
@ -411,6 +408,10 @@ private:
ThrottlerPtr replicated_fetches_throttler;
ThrottlerPtr replicated_sends_throttler;
/// Must be the last to be destroyed first
BackgroundJobAssignee background_executor;
BackgroundJobAssignee background_moves_executor;
template <class Func>
void foreachCommittedParts(Func && func, bool select_sequential_consistency) const;