mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
added a test
This commit is contained in:
parent
e39f9aabde
commit
dbc950caa4
@ -71,6 +71,7 @@ void MergeTreeBackgroundExecutor::schedulerThreadFunction()
|
|||||||
{
|
{
|
||||||
ExecutableTaskPtr current;
|
ExecutableTaskPtr current;
|
||||||
auto current_promise = std::make_shared<std::promise<void>>();
|
auto current_promise = std::make_shared<std::promise<void>>();
|
||||||
|
|
||||||
{
|
{
|
||||||
std::unique_lock lock(mutex);
|
std::unique_lock lock(mutex);
|
||||||
has_tasks.wait(lock, [this](){ return !tasks.empty() || shutdown_suspend; });
|
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());
|
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);
|
auto metric_decrementor = std::make_shared<ParanoidMetricDecrementor>(metric);
|
||||||
metric_decrementor->alarm();
|
metric_decrementor->alarm();
|
||||||
|
@ -25,6 +25,7 @@ public:
|
|||||||
bool execute() override
|
bool execute() override
|
||||||
{
|
{
|
||||||
res = inner();
|
res = inner();
|
||||||
|
inner = {};
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -112,6 +113,17 @@ public:
|
|||||||
pool.wait();
|
pool.wait();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
size_t active()
|
||||||
|
{
|
||||||
|
return pool.active();
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t pending()
|
||||||
|
{
|
||||||
|
std::lock_guard lock(mutex);
|
||||||
|
return tasks.size();
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
|
||||||
void updatePoolConfiguration()
|
void updatePoolConfiguration()
|
||||||
|
85
src/Storages/MergeTree/tests/gtest_executor.cpp
Normal file
85
src/Storages/MergeTree/tests/gtest_executor.cpp
Normal 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();
|
||||||
|
|
||||||
|
}
|
@ -122,8 +122,6 @@ private:
|
|||||||
MergeTreeDataSelectExecutor reader;
|
MergeTreeDataSelectExecutor reader;
|
||||||
MergeTreeDataWriter writer;
|
MergeTreeDataWriter writer;
|
||||||
MergeTreeDataMergerMutator merger_mutator;
|
MergeTreeDataMergerMutator merger_mutator;
|
||||||
BackgroundJobAssignee background_executor;
|
|
||||||
BackgroundJobAssignee background_moves_executor;
|
|
||||||
|
|
||||||
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;
|
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;
|
||||||
|
|
||||||
@ -150,6 +148,10 @@ private:
|
|||||||
|
|
||||||
std::atomic<bool> shutdown_called {false};
|
std::atomic<bool> shutdown_called {false};
|
||||||
|
|
||||||
|
/// Must be the last to be destroyed first
|
||||||
|
BackgroundJobAssignee background_executor;
|
||||||
|
BackgroundJobAssignee background_moves_executor;
|
||||||
|
|
||||||
void loadMutations();
|
void loadMutations();
|
||||||
|
|
||||||
/// Load and initialize deduplication logs. Even if deduplication setting
|
/// Load and initialize deduplication logs. Even if deduplication setting
|
||||||
|
@ -285,8 +285,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|||||||
, merge_strategy_picker(*this)
|
, merge_strategy_picker(*this)
|
||||||
, queue(*this, merge_strategy_picker)
|
, queue(*this, merge_strategy_picker)
|
||||||
, fetcher(*this)
|
, fetcher(*this)
|
||||||
, background_executor(*this, BackgroundJobAssignee::Type::DataProcessing, getContext())
|
|
||||||
, background_moves_executor(*this, BackgroundJobAssignee::Type::Moving, getContext())
|
|
||||||
, cleanup_thread(*this)
|
, cleanup_thread(*this)
|
||||||
, part_check_thread(*this)
|
, part_check_thread(*this)
|
||||||
, restarting_thread(*this)
|
, restarting_thread(*this)
|
||||||
@ -295,6 +293,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|||||||
, replicated_fetches_pool_size(getContext()->getSettingsRef().background_fetches_pool_size)
|
, 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_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()))
|
, 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(
|
queue_updating_task = getContext()->getSchedulePool().createTask(
|
||||||
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });
|
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });
|
||||||
|
@ -370,9 +370,6 @@ private:
|
|||||||
int metadata_version = 0;
|
int metadata_version = 0;
|
||||||
/// Threads.
|
/// 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.
|
/// 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;
|
bool queue_update_in_progress = false;
|
||||||
BackgroundSchedulePool::TaskHolder queue_updating_task;
|
BackgroundSchedulePool::TaskHolder queue_updating_task;
|
||||||
@ -411,6 +408,10 @@ private:
|
|||||||
ThrottlerPtr replicated_fetches_throttler;
|
ThrottlerPtr replicated_fetches_throttler;
|
||||||
ThrottlerPtr replicated_sends_throttler;
|
ThrottlerPtr replicated_sends_throttler;
|
||||||
|
|
||||||
|
/// Must be the last to be destroyed first
|
||||||
|
BackgroundJobAssignee background_executor;
|
||||||
|
BackgroundJobAssignee background_moves_executor;
|
||||||
|
|
||||||
template <class Func>
|
template <class Func>
|
||||||
void foreachCommittedParts(Func && func, bool select_sequential_consistency) const;
|
void foreachCommittedParts(Func && func, bool select_sequential_consistency) const;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user