From 92777691b59d02654c09ab685124ead47852e5b9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 17 May 2022 10:16:40 +0000 Subject: [PATCH 001/628] fix check for non-deterministic functions in TTL expressions --- .../MergeTree/registerStorageMergeTree.cpp | 4 +-- src/Storages/TTLDescription.cpp | 26 ++++++------------- src/Storages/TTLDescription.h | 7 +++-- .../0_stateless/00933_ttl_simple.reference | 8 +++--- .../queries/0_stateless/00933_ttl_simple.sql | 10 ++++--- .../02296_ttl_non_deterministic.reference | 0 .../02296_ttl_non_deterministic.sql | 15 +++++++++++ 7 files changed, 38 insertions(+), 32 deletions(-) create mode 100644 tests/queries/0_stateless/02296_ttl_non_deterministic.reference create mode 100644 tests/queries/0_stateless/02296_ttl_non_deterministic.sql diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6d8c3b313d4..3219f26aeaa 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -570,7 +570,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->ttl_table) { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - args.storage_def->ttl_table->ptr(), metadata.columns, args.getContext(), metadata.primary_key); + args.storage_def->ttl_table->ptr(), metadata.columns, args.getContext(), metadata.primary_key, args.attach); } if (args.query.columns_list && args.query.columns_list->indices) @@ -593,7 +593,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key, args.attach); metadata.column_ttls_by_name[name] = new_ttl_entry; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index b745da13484..1479545e103 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -57,22 +57,9 @@ namespace void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) { - for (const auto & action : ttl_expression->getActions()) - { - if (action.node->type == ActionsDAG::ActionType::FUNCTION) - { - IFunctionBase & func = *action.node->function_base; - if (!func.isDeterministic()) - throw Exception( - "TTL expression cannot contain non-deterministic functions, " - "but contains function " - + func.getName(), - ErrorCodes::BAD_ARGUMENTS); - } - } + ttl_expression->getActionsDAG().assertDeterministic(); const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); - if (!typeid_cast(result_column.type.get()) && !typeid_cast(result_column.type.get())) { @@ -164,7 +151,8 @@ TTLDescription TTLDescription::getTTLFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, - const KeyDescription & primary_key) + const KeyDescription & primary_key, + bool is_attach) { TTLDescription result; const auto * ttl_element = definition_ast->as(); @@ -295,7 +283,8 @@ TTLDescription TTLDescription::getTTLFromAST( } } - checkTTLExpression(result.expression, result.result_column); + if (!is_attach) + checkTTLExpression(result.expression, result.result_column); return result; } @@ -333,7 +322,8 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, - const KeyDescription & primary_key) + const KeyDescription & primary_key, + bool is_attach) { TTLTableDescription result; if (!definition_ast) @@ -344,7 +334,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( bool have_unconditional_delete_ttl = false; for (const auto & ttl_element_ptr : definition_ast->children) { - auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); + auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key, is_attach); if (ttl.mode == TTLMode::DELETE) { if (!ttl.where_expression) diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 8f60eb604b5..91ef7b44d37 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -82,9 +82,8 @@ struct TTLDescription /// Codec name which will be used to recompress data ASTPtr recompression_codec; - /// Parse TTL structure from definition. Able to parse both column and table - /// TTLs. - static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); + /// Parse TTL structure from definition. Able to parse both column and table TTLs. + static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); TTLDescription() = default; TTLDescription(const TTLDescription & other); @@ -121,7 +120,7 @@ struct TTLTableDescription TTLTableDescription & operator=(const TTLTableDescription & other); static TTLTableDescription getTTLForTableFromAST( - const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); + const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); /// Parse description from string static TTLTableDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); diff --git a/tests/queries/0_stateless/00933_ttl_simple.reference b/tests/queries/0_stateless/00933_ttl_simple.reference index e3982814eab..72f5134e235 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.reference +++ b/tests/queries/0_stateless/00933_ttl_simple.reference @@ -6,11 +6,11 @@ 2000-10-10 00:00:00 0 2100-10-10 00:00:00 3 2100-10-10 2 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2000-10-10 00:00:00\', \'DateTime\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2100-10-10 00:00:00\', \'DateTime\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() - 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2000-10-10\', \'Date\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() + 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2100-10-10\', \'Date\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 2bf686822d5..1cd41c0a113 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -53,7 +53,9 @@ select * from ttl_00933_1 order by d; -- const DateTime TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2000-10-10 00:00:00'::DateTime) +engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; + show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -61,7 +63,7 @@ select * from ttl_00933_1; -- const DateTime TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2100-10-10 00:00:00'::DateTime) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -69,7 +71,7 @@ select * from ttl_00933_1; -- const Date TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2000-10-10'::Date) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -77,7 +79,7 @@ select * from ttl_00933_1; -- const Date TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2100-10-10'::Date) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; diff --git a/tests/queries/0_stateless/02296_ttl_non_deterministic.reference b/tests/queries/0_stateless/02296_ttl_non_deterministic.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02296_ttl_non_deterministic.sql b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql new file mode 100644 index 00000000000..b5667aafcab --- /dev/null +++ b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS t_ttl_non_deterministic; + +CREATE TABLE t_ttl_non_deterministic(A Int64) +ENGINE = MergeTree ORDER BY A TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} + +CREATE TABLE t_ttl_non_deterministic(A Int64) ENGINE = MergeTree ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; + +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64 TTL now() + toIntervalMonth(1)) +ENGINE = MergeTree ORDER BY A; -- {serverError BAD_ARGUMENTS} + +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64) ENGINE = MergeTree ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY COLUMN B Int64 TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; From ef08d24347ddb220d790469613c107d89aafd2b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 18 May 2022 12:10:52 +0000 Subject: [PATCH 002/628] retrun back some checks for TTL expressions --- src/Storages/TTLDescription.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 1479545e103..0ea9e378913 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -55,9 +55,11 @@ TTLAggregateDescription & TTLAggregateDescription::operator=(const TTLAggregateD namespace { -void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name, bool is_attach) { - ttl_expression->getActionsDAG().assertDeterministic(); + /// Do not apply this check in ATTACH queries for compatibility reasons. + if (!is_attach) + ttl_expression->getActionsDAG().assertDeterministic(); const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); if (!typeid_cast(result_column.type.get()) @@ -283,8 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( } } - if (!is_attach) - checkTTLExpression(result.expression, result.result_column); + checkTTLExpression(result.expression, result.result_column, is_attach); return result; } From ccf2c4ea433c6bb5d6f697a526ed9a930040dad8 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 9 Apr 2023 12:50:26 +0000 Subject: [PATCH 003/628] wip AsyncLoader --- src/Common/AsyncLoader.h | 556 ++++++++++++++++++++++++++++++++++++++ src/Common/ErrorCodes.cpp | 3 + 2 files changed, 559 insertions(+) create mode 100644 src/Common/AsyncLoader.h diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h new file mode 100644 index 00000000000..dd35621443d --- /dev/null +++ b/src/Common/AsyncLoader.h @@ -0,0 +1,556 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class LoadJob; +using LoadJobPtr = std::shared_ptr; +using LoadJobSet = std::unordered_set; +class LoadTask; +class AsyncLoader; + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ASYNC_LOAD_FAILED; + extern const int ASYNC_LOAD_CANCELED; + extern const int ASYNC_LOAD_DEPENDENCY_FAILED; +} + +enum class LoadStatus +{ + PENDING, // Load is not finished yet + SUCCESS, // Load was successful + FAILED // Load failed or canceled +}; + +class LoadJob : private boost::noncopyable +{ +public: + template + LoadJob(LoadJobSet && dependencies_, const String & name_, Func && func_) + : dependencies(std::move(dependencies_)) + , name(name_) + , func(std::forward(func_)) + {} + + LoadStatus status() const + { + std::unique_lock lock{mutex}; + return !is_finished ? LoadStatus::PENDING : (exception ? LoadStatus::FAILED : LoadStatus::SUCCESS); + } + + void wait() + { + std::unique_lock lock{mutex}; + finished.wait(lock, [this] { return is_finished; }); + } + + void get() + { + std::unique_lock lock{mutex}; + finished.wait(lock, [this] { return is_finished; }); + if (exception) + std::rethrow_exception(exception); + } + + const LoadJobSet dependencies; // jobs to be done before this one, with ownership + const String name; + std::atomic priority{0}; + +private: + friend class AsyncLoader; + + void setSuccess() + { + std::unique_lock lock{mutex}; + is_finished = true; + finished.notify_all(); + } + + void setFailure(const std::exception_ptr & ptr) + { + std::unique_lock lock{mutex}; + is_finished = true; + exception = ptr; + finished.notify_all(); + } + + std::function func; + + std::mutex mutex; + std::condition_variable finished; + bool is_finished = false; + std::exception_ptr exception; +}; + +template +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && func) +{ + return std::make_shared(std::move(dependencies), name, std::forward(func)); +} + +// TODO(serxa): write good comment +class AsyncLoader : private boost::noncopyable +{ +private: + // Key of a pending job in ready queue + struct ReadyKey { + ssize_t priority; + UInt64 ready_seqno; + + bool operator<(const ReadyKey & rhs) const + { + if (priority == rhs.priority) + return ready_seqno < rhs.ready_seqno; + return priority > rhs.priority; + } + }; + + // Scheduling information for a pending job + struct Info { + ssize_t priority = 0; + size_t dependencies_left = 0; + UInt64 ready_seqno = 0; // zero means that job is not in ready queue + LoadJobSet dependent_jobs; // TODO(serxa): clean it on remove jobs + + ReadyKey key() const + { + return {priority, ready_seqno}; + } + }; + +public: + using Metric = CurrentMetrics::Metric; + + // Helper class that removes all not started jobs in destructor and wait all executing jobs to finish + class Task + { + public: + Task() + : loader(nullptr) + {} + + Task(AsyncLoader * loader_, LoadJobSet && jobs_) + : loader(loader_) + , jobs(std::move(jobs_)) + {} + + Task(const Task & o) = delete; + Task & operator=(const Task & o) = delete; + + Task(Task && o) noexcept + : loader(std::exchange(o.loader, nullptr)) + , jobs(std::move(o.jobs)) + {} + + Task & operator=(Task && o) noexcept + { + loader = std::exchange(o.loader, nullptr); + jobs = std::move(o.jobs); + return *this; + } + + void merge(Task && o) + { + if (!loader) + { + *this = std::move(o); + } + else + { + chassert(loader == o.loader); + jobs.merge(o.jobs); + o.loader = nullptr; + } + } + + ~Task() + { + // Remove jobs that are not ready and wait for jobs that are in progress + if (loader) + loader->remove(jobs); + } + + // Do not track jobs in this task + void detach() + { + loader = nullptr; + jobs.clear(); + } + + private: + AsyncLoader * loader; + LoadJobSet jobs; + }; + + AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_) + : max_threads(max_threads_) + , pool(metric_threads, metric_active_threads, max_threads) + {} + + void start() + { + std::unique_lock lock{mutex}; + is_running = true; + for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) + spawn(lock); + } + + void stop() + { + std::unique_lock lock{mutex}; + is_stopping = true; + } + + [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0) + { + std::unique_lock lock{mutex}; + + // Sanity checks + for (const auto & job : jobs) + { + if (job->status() != LoadStatus::PENDING) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to schedule already finished load job '{}'", job->name); + if (scheduled_jobs.contains(job)) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Load job '{}' has been already scheduled", job->name); + } + + // TODO(serxa): ensure scheduled_jobs graph will have no cycles, otherwise we have infinite recursion and other strange stuff? + + // We do not want any exception to be throws after this point, because the following code is not exception-safe + DENY_ALLOCATIONS_IN_SCOPE; + + // Schedule all incoming jobs + for (const auto & job : jobs) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + scheduled_jobs.emplace(job, Info{.priority = priority}); + )}; + job->priority.store(priority); // Set user-facing priority + } + + // Process incoming dependencies + for (const auto & job : jobs) + { + Info & info = scheduled_jobs.find(job)->second; + for (const auto & dep : job->dependencies) + { + // Register every dependency on scheduled job with back-link to dependent job + if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + dep_info->second.dependent_jobs.insert(job); + }); + info.dependencies_left++; + } + else + { + // TODO(serxa): check status: (1) pending: it is wrong - throw? (2) success: good - no dep. (3) failed: propagate failure! + } + + // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion + prioritize(dep, priority, lock); + } + + // Place jobs w/o dependencies to ready queue + if (info.dependencies_left == 0) + enqueue(info, job, lock); + } + + return Task(this, std::move(jobs)); + } + + // Increase priority of a job and all its dependencies recursively + void prioritize(const LoadJobPtr & job, ssize_t new_priority) + { + DENY_ALLOCATIONS_IN_SCOPE; + std::unique_lock lock{mutex}; + prioritize(job, new_priority, lock); + } + + // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them + void remove(const LoadJobSet & jobs) + { + DENY_ALLOCATIONS_IN_SCOPE; + std::unique_lock lock{mutex}; + for (const auto & job : jobs) + { + if (auto it = finished_jobs.find(job); it != finished_jobs.end()) // Job is already finished + { + finished_jobs.erase(it); + } + else if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + if (info->second.dependencies_left > 0) // Job is not ready yet + canceled(job); + else if (info->second.ready_seqno) // Job is enqueued in ready queue + { + ready_queue.erase(info->second.key()); + info->second.ready_seqno = 0; + canceled(job); + } + else // Job is currently executing + { + lock.unlock(); + job->wait(); // Wait for job to finish + lock.lock(); + } + finished_jobs.erase(job); + } + } + } + +private: + void canceled(const LoadJobPtr & job, std::unique_lock &) + { + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr( + Exception(ASYNC_LOAD_CANCELED, + "Load job '{}' canceled", + job->name)); + )}; + failed(job, e, lock); + } + + void loaded(const LoadJobPtr & job, std::unique_lock & lock) + { + // Notify waiters + job->setSuccess(); + + // Update dependent jobs and enqueue if ready + chassert(scheduled_jobs.contains(job)); // Job was pending + for (const auto & dep : scheduled_jobs[job].dependent_jobs) + { + chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending + Info & dep_info = scheduled_jobs[dep]; + if (--dep_info.dependencies_left == 0) + enqueue(dep_info, dep, lock); + } + + finish(job); + } + + void failed(const LoadJobPtr & job, std::exception_ptr exception_from_job, std::unique_lock & lock) + { + // Notify waiters + job->setFailure(exception_from_job); + + // Recurse into all dependent jobs + chassert(scheduled_jobs.contains(job)); // Job was pending + Info & info = scheduled_jobs[job]; + LoadJobSet dependent; + dependent.swap(info.dependent_jobs); // To avoid container modification during recursion + for (const auto & dep : dependent) + { + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr( + Exception(ASYNC_LOAD_DEPENDENCY_FAILED, + "Load job '{}' -> {}", + dep->name, + getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); + }); + failed(dep, e, lock); + } + + // Clean dependency graph edges + for (const auto & dep : job->dependencies) + if (auto dep_info = scheduled_jobs.find(dep); info != scheduled_jobs.end()) + dep_info->second.dependent_jobs.erase(job); + + // Job became finished + finish(job); + } + + void finish(const LoadJobPtr & job, std::unique_lock &) + { + scheduled_jobs.erase(job); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + finished_jobs.insert(job); + )}; + } + + void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + if (info->second.priority >= new_priority) + return; // Never lower priority + + // Update priority and push job forward through ready queue if needed + if (info->second.ready_seqno) + ready_queue.erase(info->second.key()); + info->second.priority = new_priority; + job->priority.store(new_priority); // Set user-facing priority (may affect executing jobs) + if (info->second.ready_seqno) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + ready_queue.emplace(info->second.key(), job); + }); + } + + // Recurse into dependencies + for (const auto & dep : job->dependencies) + prioritize(dep, new_priority, lock); + } + } + + void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) + { + chassert(info.dependencies_left == 0); + chassert(info.ready_seqno == 0); + info.ready_seqno = ++last_ready_seqno; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + ready_queue.emplace(info.key(), job); + }); + + if (is_running && workers < max_threads) // TODO(serxa): Can we make max_thread changeable in runtime + spawn(lock); + } + + void spawn(std::unique_lock &) + { + // TODO(serxa): add metrics for number of active workers or executing jobs + workers++; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + pool.scheduleOrThrowOnError([this] { worker(); }); + }); + } + + void worker() + { + DENY_ALLOCATIONS_IN_SCOPE; + + LoadJobPtr job; + std::exception_ptr exception_from_job; + while (true) { // TODO(serxa): shutdown + + /// This is inside the loop to also reset previous thread names set inside the jobs + setThreadName("AsyncLoader"); + + { + std::unique_lock lock{mutex}; + + // Handle just executed job + if (exception_from_job) + failed(job, exception_from_job, lock); + else if (job) + loaded(job, lock); + + if (ready_queue.empty()) + { + workers--; + return; + } + + // Take next job to be executed from the ready queue + auto it = ready_queue.begin(); + job = it->second; + ready_queue.erase(it); + scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue + } + + try + { + ALLOW_ALLOCATIONS_IN_SCOPE; + job->func(*job); + exception_from_job = {}; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + exception_from_job = std::make_exception_ptr( + Exception(ASYNC_LOAD_FAILED, + "Load job '{}' failed: {}", + job->name, + getCurrentExceptionMessage(/* with_stacktrace = */ true))); + }); + } + } + } + + std::mutex mutex; + bool is_running = false; + bool is_stopping = false; + + // Full set of scheduled pending jobs along with scheduling info + std::unordered_map scheduled_jobs; + + // Subset of scheduled pending jobs with resolved dependencies (waiting for a thread to be executed) + // Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities + std::map ready_queue; + + // Set of finished jobs (for introspection only, until job is removed) + LoadJobSet finished_jobs; + + // Increasing counter for ReadyKey assignment (to preserve FIFO order of jobs with equal priority) + UInt64 last_ready_seqno = 0; + + // For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs + size_t max_threads; + size_t workers = 0; + ThreadPool pool; +}; + +} + +//////////////////////////////////////////////////////////////////////////////////////////////////////// + +namespace CurrentMetrics +{ + extern const Metric TablesLoaderThreads; + extern const Metric TablesLoaderThreadsActive; +} + +namespace DB +{ + +void test() +{ + AsyncLoader loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, 2); + + auto job_func = [] (const LoadJob & self) { + std::cout << self.name << " done with priority " << self.priority << std::endl; + }; + + auto job1 = makeLoadJob({}, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task1 = loader.schedule({ job1, job2 }); + + auto job3 = makeLoadJob({ job2 }, "job3", job_func); + auto job4 = makeLoadJob({ job2 }, "job4", job_func); + auto task2 = loader.schedule({ job3, job4 }); + auto job5 = makeLoadJob({ job3, job4 }, "job5", job_func); + task2.merge(loader.schedule({ job5 }, -1)); +} + +} diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9abf3bba8ff..fe99006eff4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -650,6 +650,9 @@ M(679, IO_URING_SUBMIT_ERROR) \ M(690, MIXED_ACCESS_PARAMETER_TYPES) \ M(691, UNKNOWN_ELEMENT_OF_ENUM) \ + M(692, ASYNC_LOAD_FAILED) \ + M(693, ASYNC_LOAD_CANCELED) \ + M(694, ASYNC_LOAD_DEPENDENCY_FAILED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ From df1f48d2b194d0991b656fa25b33d3dee0828544 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Apr 2023 23:14:20 +0000 Subject: [PATCH 004/628] make t work, add simple test --- src/Common/AsyncLoader.h | 87 ++++++++++--------------- src/Common/tests/gtest_async_loader.cpp | 66 +++++++++++++++++++ 2 files changed, 102 insertions(+), 51 deletions(-) create mode 100644 src/Common/tests/gtest_async_loader.cpp diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index dd35621443d..944251653e4 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -95,7 +95,7 @@ private: std::function func; - std::mutex mutex; + mutable std::mutex mutex; std::condition_variable finished; bool is_finished = false; std::exception_ptr exception; @@ -129,7 +129,7 @@ private: ssize_t priority = 0; size_t dependencies_left = 0; UInt64 ready_seqno = 0; // zero means that job is not in ready queue - LoadJobSet dependent_jobs; // TODO(serxa): clean it on remove jobs + LoadJobSet dependent_jobs; ReadyKey key() const { @@ -206,6 +206,7 @@ public: , pool(metric_threads, metric_active_threads, max_threads) {} + // Start workers to execute scheduled load jobs void start() { std::unique_lock lock{mutex}; @@ -214,10 +215,24 @@ public: spawn(lock); } + // Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs. + void wait() + { + pool.wait(); + } + + // Wait for currently executing jobs to finish, but do not run any other pending jobs. + // Not finished jobs are left in pending state: + // - they can be resumed by calling start() again; + // - or canceled using ~Task() or remove() later. void stop() { - std::unique_lock lock{mutex}; - is_stopping = true; + { + std::unique_lock lock{mutex}; + is_running = false; + // NOTE: there is no need to notify because workers never wait + } + pool.wait(); } [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0) @@ -244,7 +259,7 @@ public: NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; scheduled_jobs.emplace(job, Info{.priority = priority}); - )}; + }); job->priority.store(priority); // Set user-facing priority } @@ -302,12 +317,12 @@ public: else if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { if (info->second.dependencies_left > 0) // Job is not ready yet - canceled(job); + canceled(job, lock); else if (info->second.ready_seqno) // Job is enqueued in ready queue { ready_queue.erase(info->second.key()); info->second.ready_seqno = 0; - canceled(job); + canceled(job, lock); } else // Job is currently executing { @@ -321,16 +336,16 @@ public: } private: - void canceled(const LoadJobPtr & job, std::unique_lock &) + void canceled(const LoadJobPtr & job, std::unique_lock & lock) { std::exception_ptr e; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; e = std::make_exception_ptr( - Exception(ASYNC_LOAD_CANCELED, + Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); - )}; + }); failed(job, e, lock); } @@ -349,7 +364,7 @@ private: enqueue(dep_info, dep, lock); } - finish(job); + finish(job, lock); } void failed(const LoadJobPtr & job, std::exception_ptr exception_from_job, std::unique_lock & lock) @@ -368,7 +383,7 @@ private: NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; e = std::make_exception_ptr( - Exception(ASYNC_LOAD_DEPENDENCY_FAILED, + Exception(ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED, "Load job '{}' -> {}", dep->name, getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); @@ -378,11 +393,11 @@ private: // Clean dependency graph edges for (const auto & dep : job->dependencies) - if (auto dep_info = scheduled_jobs.find(dep); info != scheduled_jobs.end()) + if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) dep_info->second.dependent_jobs.erase(job); // Job became finished - finish(job); + finish(job, lock); } void finish(const LoadJobPtr & job, std::unique_lock &) @@ -391,7 +406,7 @@ private: NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; finished_jobs.insert(job); - )}; + }); } void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) @@ -430,7 +445,7 @@ private: ready_queue.emplace(info.key(), job); }); - if (is_running && workers < max_threads) // TODO(serxa): Can we make max_thread changeable in runtime + if (is_running && workers < max_threads) // TODO(serxa): Can we make max_thread changeable in runtime? spawn(lock); } @@ -450,7 +465,7 @@ private: LoadJobPtr job; std::exception_ptr exception_from_job; - while (true) { // TODO(serxa): shutdown + while (true) { /// This is inside the loop to also reset previous thread names set inside the jobs setThreadName("AsyncLoader"); @@ -464,6 +479,9 @@ private: else if (job) loaded(job, lock); + if (!is_running) + return; + if (ready_queue.empty()) { workers--; @@ -489,7 +507,7 @@ private: NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; exception_from_job = std::make_exception_ptr( - Exception(ASYNC_LOAD_FAILED, + Exception(ErrorCodes::ASYNC_LOAD_FAILED, "Load job '{}' failed: {}", job->name, getCurrentExceptionMessage(/* with_stacktrace = */ true))); @@ -500,7 +518,6 @@ private: std::mutex mutex; bool is_running = false; - bool is_stopping = false; // Full set of scheduled pending jobs along with scheduling info std::unordered_map scheduled_jobs; @@ -522,35 +539,3 @@ private: }; } - -//////////////////////////////////////////////////////////////////////////////////////////////////////// - -namespace CurrentMetrics -{ - extern const Metric TablesLoaderThreads; - extern const Metric TablesLoaderThreadsActive; -} - -namespace DB -{ - -void test() -{ - AsyncLoader loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, 2); - - auto job_func = [] (const LoadJob & self) { - std::cout << self.name << " done with priority " << self.priority << std::endl; - }; - - auto job1 = makeLoadJob({}, "job1", job_func); - auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task1 = loader.schedule({ job1, job2 }); - - auto job3 = makeLoadJob({ job2 }, "job3", job_func); - auto job4 = makeLoadJob({ job2 }, "job4", job_func); - auto task2 = loader.schedule({ job3, job4 }); - auto job5 = makeLoadJob({ job3, job4 }, "job5", job_func); - task2.merge(loader.schedule({ job5 }, -1)); -} - -} diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp new file mode 100644 index 00000000000..f8d51c0eebd --- /dev/null +++ b/src/Common/tests/gtest_async_loader.cpp @@ -0,0 +1,66 @@ +#include + +#include +#include +#include + +#include +#include +#include +#include + + +using namespace DB; + + +namespace CurrentMetrics +{ + extern const Metric TablesLoaderThreads; + extern const Metric TablesLoaderThreadsActive; +} + +struct AsyncLoaderTest +{ + AsyncLoader loader; + + explicit AsyncLoaderTest(size_t max_threads = 1) + : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads) + {} +}; + +TEST(AsyncLoader, Smoke) +{ + AsyncLoaderTest t(2); + + static constexpr ssize_t low_priority = -1; + + std::atomic jobs_done{0}; + std::atomic low_priority_jobs_done{0}; + + auto job_func = [&] (const LoadJob & self) { + jobs_done++; + if (self.priority == low_priority) + low_priority_jobs_done++; + }; + + { + auto job1 = makeLoadJob({}, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task1 = t.loader.schedule({ job1, job2 }); + + auto job3 = makeLoadJob({ job2 }, "job3", job_func); + auto job4 = makeLoadJob({ job2 }, "job4", job_func); + auto task2 = t.loader.schedule({ job3, job4 }); + auto job5 = makeLoadJob({ job3, job4 }, "job5", job_func); + task2.merge(t.loader.schedule({ job5 }, low_priority)); + + t.loader.start(); + + t.loader.wait(); + } + + ASSERT_EQ(jobs_done, 5); + ASSERT_EQ(low_priority_jobs_done, 1); + + t.loader.stop(); +} From dad9d2700e6f0c61d3a31c8251a2d86f46721788 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 11 Apr 2023 00:08:55 +0000 Subject: [PATCH 005/628] fix style --- src/Common/AsyncLoader.h | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 944251653e4..8b4fa0bde26 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -112,7 +112,8 @@ class AsyncLoader : private boost::noncopyable { private: // Key of a pending job in ready queue - struct ReadyKey { + struct ReadyKey + { ssize_t priority; UInt64 ready_seqno; @@ -125,7 +126,8 @@ private: }; // Scheduling information for a pending job - struct Info { + struct Info + { ssize_t priority = 0; size_t dependencies_left = 0; UInt64 ready_seqno = 0; // zero means that job is not in ready queue @@ -465,8 +467,8 @@ private: LoadJobPtr job; std::exception_ptr exception_from_job; - while (true) { - + while (true) + { /// This is inside the loop to also reset previous thread names set inside the jobs setThreadName("AsyncLoader"); From 53e5350b7de34de4180064fd42b9329111a46a79 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Apr 2023 18:43:58 +0000 Subject: [PATCH 006/628] add dependency cycle detection --- src/Common/AsyncLoader.h | 49 ++++++++++++++++++++++--- src/Common/ErrorCodes.cpp | 7 ++-- src/Common/tests/gtest_async_loader.cpp | 44 ++++++++++++++++++++++ 3 files changed, 91 insertions(+), 9 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 8b4fa0bde26..6ccf3552721 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -28,7 +28,7 @@ class AsyncLoader; namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int ASYNC_LOAD_SCHEDULE_FAILED; extern const int ASYNC_LOAD_FAILED; extern const int ASYNC_LOAD_CANCELED; extern const int ASYNC_LOAD_DEPENDENCY_FAILED; @@ -71,7 +71,7 @@ public: std::rethrow_exception(exception); } - const LoadJobSet dependencies; // jobs to be done before this one, with ownership + const LoadJobSet dependencies; // jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard const String name; std::atomic priority{0}; @@ -245,12 +245,13 @@ public: for (const auto & job : jobs) { if (job->status() != LoadStatus::PENDING) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to schedule already finished load job '{}'", job->name); + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Trying to schedule already finished load job '{}'", job->name); if (scheduled_jobs.contains(job)) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Load job '{}' has been already scheduled", job->name); + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job '{}' has been already scheduled", job->name); } - // TODO(serxa): ensure scheduled_jobs graph will have no cycles, otherwise we have infinite recursion and other strange stuff? + // Ensure scheduled_jobs graph will have no cycles. The only way to get a cycle is to add a cycle, assuming old jobs cannot reference new ones. + checkCycle(jobs, lock); // We do not want any exception to be throws after this point, because the following code is not exception-safe DENY_ALLOCATIONS_IN_SCOPE; @@ -338,6 +339,41 @@ public: } private: + void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) + { + LoadJobSet left = jobs; + LoadJobSet visited; + visited.reserve(left.size()); + while (!left.empty()) + { + LoadJobPtr job = *left.begin(); + checkCycleImpl(job, left, visited, lock); + } + } + + String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock) + { + if (!left.contains(job)) + return {}; // Do not consider external dependencies and already processed jobs + if (auto [_, inserted] = visited.insert(job); !inserted) + { + visited.erase(job); // Mark where cycle ends + return job->name; + } + for (const auto & dep : job->dependencies) + { + if (auto chain = checkCycleImpl(dep, left, visited, lock); !chain.empty()) + { + if (!visited.contains(job)) // Check for cycle end + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job dependency cycle detected: {} -> {}", job->name, chain); + else + return fmt::format("{} -> {}", job->name, chain); // chain is not a cycle yet -- continue building + } + } + left.erase(job); + return {}; + } + void canceled(const LoadJobPtr & job, std::unique_lock & lock) { std::exception_ptr e; @@ -453,7 +489,6 @@ private: void spawn(std::unique_lock &) { - // TODO(serxa): add metrics for number of active workers or executing jobs workers++; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; @@ -521,6 +556,8 @@ private: std::mutex mutex; bool is_running = false; + // TODO(serxa): add metrics for number of jobs in every state + // Full set of scheduled pending jobs along with scheduling info std::unordered_map scheduled_jobs; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index fe99006eff4..0d203fc799c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -650,9 +650,10 @@ M(679, IO_URING_SUBMIT_ERROR) \ M(690, MIXED_ACCESS_PARAMETER_TYPES) \ M(691, UNKNOWN_ELEMENT_OF_ENUM) \ - M(692, ASYNC_LOAD_FAILED) \ - M(693, ASYNC_LOAD_CANCELED) \ - M(694, ASYNC_LOAD_DEPENDENCY_FAILED) \ + M(692, ASYNC_LOAD_SCHEDULE_FAILED) \ + M(693, ASYNC_LOAD_FAILED) \ + M(694, ASYNC_LOAD_CANCELED) \ + M(695, ASYNC_LOAD_DEPENDENCY_FAILED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index f8d51c0eebd..44c97bf0158 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -64,3 +64,47 @@ TEST(AsyncLoader, Smoke) t.loader.stop(); } + +TEST(AsyncLoader, CycleDetection) +{ + AsyncLoaderTest t; + + auto job_func = [&] (const LoadJob &) {}; + + LoadJobPtr cycle_breaker; // To avoid memleak we introduce with a cycle + + try + { + std::vector jobs; + jobs.push_back(makeLoadJob({}, "job0", job_func)); + jobs.push_back(makeLoadJob({ jobs[0] }, "job1", job_func)); + jobs.push_back(makeLoadJob({ jobs[0], jobs[1] }, "job2", job_func)); + jobs.push_back(makeLoadJob({ jobs[0], jobs[2] }, "job3", job_func)); + + // Actually it is hard to construct a cycle, but suppose someone was able to succeed violating constness + const_cast(jobs[1]->dependencies).insert(jobs[3]); + cycle_breaker = jobs[1]; + + // Add couple unrelated jobs + jobs.push_back(makeLoadJob({ jobs[1] }, "job4", job_func)); + jobs.push_back(makeLoadJob({ jobs[4] }, "job5", job_func)); + jobs.push_back(makeLoadJob({ jobs[3] }, "job6", job_func)); + jobs.push_back(makeLoadJob({ jobs[1], jobs[2], jobs[3], jobs[4], jobs[5], jobs[6] }, "job7", job_func)); + + // Also add another not connected jobs + jobs.push_back(makeLoadJob({}, "job8", job_func)); + jobs.push_back(makeLoadJob({}, "job9", job_func)); + jobs.push_back(makeLoadJob({ jobs[9] }, "job10", job_func)); + + auto task1 = t.loader.schedule({ jobs.begin(), jobs.end()}); + FAIL(); + } + catch (Exception & e) + { + int present[] = { 0, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0 }; + for (int i = 0; i < std::size(present); i++) + ASSERT_EQ(e.message().find(fmt::format("job{}", i)) != String::npos, present[i]); + } + + const_cast(cycle_breaker->dependencies).clear(); +} From e59e12722efe0183c2e76280971eace7dc479787 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Apr 2023 22:48:46 +0000 Subject: [PATCH 007/628] add simple randomized test --- src/Common/AsyncLoader.h | 6 +++ src/Common/tests/gtest_async_loader.cpp | 57 +++++++++++++++++++++++++ 2 files changed, 63 insertions(+) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 6ccf3552721..49cecdbbb8e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -208,6 +208,12 @@ public: , pool(metric_threads, metric_active_threads, max_threads) {} + // WARNING: all Task instances returned by `schedule()` should be destructed before AsyncLoader + ~AsyncLoader() + { + stop(); + } + // Start workers to execute scheduled load jobs void start() { diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 44c97bf0158..3825f5d2592 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -1,5 +1,7 @@ #include +#include +#include #include #include #include @@ -23,6 +25,40 @@ struct AsyncLoaderTest { AsyncLoader loader; + std::mutex rng_mutex; + pcg64 rng{randomSeed()}; + + template + T randomInt(T from, T to) + { + std::uniform_int_distribution distribution(from, to); + std::scoped_lock lock(rng_mutex); + return distribution(rng); + } + + void randomSleepUs(UInt64 min_us, UInt64 max_us, int probability_percent) + { + if (randomInt(0, 99) < probability_percent) + std::this_thread::sleep_for(std::chrono::microseconds(randomInt(min_us, max_us))); + } + + template + LoadJobSet randomJobSet(int job_count, int dep_probability_percent, JobFunc job_func) + { + std::vector jobs; + for (int j = 0; j < job_count; j++) + { + LoadJobSet deps; + for (int d = 0; d < j; d++) + { + if (randomInt(0, 99) < dep_probability_percent) + deps.insert(jobs[d]); + } + jobs.push_back(makeLoadJob(std::move(deps), fmt::format("job{}", j), job_func)); + } + return {jobs.begin(), jobs.end()}; + } + explicit AsyncLoaderTest(size_t max_threads = 1) : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads) {} @@ -108,3 +144,24 @@ TEST(AsyncLoader, CycleDetection) const_cast(cycle_breaker->dependencies).clear(); } + +TEST(AsyncLoader, RandomTasks) +{ + AsyncLoaderTest t(16); + t.loader.start(); + + auto job_func = [&] (const LoadJob &) + { + t.randomSleepUs(100, 500, 5); + }; + + std::vector tasks; + for (int i = 0; i < 512; i++) + { + int job_count = t.randomInt(1, 32); + tasks.push_back(t.loader.schedule(t.randomJobSet(job_count, 5, job_func))); + t.randomSleepUs(100, 900, 20); // avg=100us + } + t.loader.wait(); +} + From 3e936cc4718c015e4c43ffbff56e17c47e12e829 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Apr 2023 23:18:38 +0000 Subject: [PATCH 008/628] add tests for job status/wait/cancel --- src/Common/AsyncLoader.h | 15 +++++----- src/Common/tests/gtest_async_loader.cpp | 40 +++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 7 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 49cecdbbb8e..5f4c67ab065 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -58,12 +58,6 @@ public: } void wait() - { - std::unique_lock lock{mutex}; - finished.wait(lock, [this] { return is_finished; }); - } - - void get() { std::unique_lock lock{mutex}; finished.wait(lock, [this] { return is_finished; }); @@ -186,9 +180,16 @@ public: ~Task() { - // Remove jobs that are not ready and wait for jobs that are in progress + remove(); + } + + void remove() + { if (loader) + { loader->remove(jobs); + detach(); + } } // Do not track jobs in this task diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 3825f5d2592..7e76ac0e07a 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -21,6 +21,14 @@ namespace CurrentMetrics extern const Metric TablesLoaderThreadsActive; } +namespace DB::ErrorCodes +{ + extern const int ASYNC_LOAD_SCHEDULE_FAILED; + extern const int ASYNC_LOAD_FAILED; + extern const int ASYNC_LOAD_CANCELED; + extern const int ASYNC_LOAD_DEPENDENCY_FAILED; +} + struct AsyncLoaderTest { AsyncLoader loader; @@ -90,9 +98,18 @@ TEST(AsyncLoader, Smoke) auto job5 = makeLoadJob({ job3, job4 }, "job5", job_func); task2.merge(t.loader.schedule({ job5 }, low_priority)); + std::thread waiter_thread([=] { job5->wait(); }); + t.loader.start(); + job3->wait(); t.loader.wait(); + job4->wait(); + + waiter_thread.join(); + + ASSERT_EQ(job1->status(), LoadStatus::SUCCESS); + ASSERT_EQ(job2->status(), LoadStatus::SUCCESS); } ASSERT_EQ(jobs_done, 5); @@ -145,6 +162,29 @@ TEST(AsyncLoader, CycleDetection) const_cast(cycle_breaker->dependencies).clear(); } +TEST(AsyncLoader, CancelPendingJob) +{ + AsyncLoaderTest t; + + auto job_func = [&] (const LoadJob &) {}; + + auto job = makeLoadJob({}, "job", job_func); + auto task = t.loader.schedule({job}); + + task.remove(); // this cancels pending the job (async loader was not started to execute it) + + ASSERT_EQ(job->status(), LoadStatus::FAILED); + try + { + job->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_CANCELED); + } +} + TEST(AsyncLoader, RandomTasks) { AsyncLoaderTest t(16); From 46a9b9ab78a18c420d4a1eaf43dce79388abd05c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 00:04:31 +0000 Subject: [PATCH 009/628] add more tests for cancel --- src/Common/AsyncLoader.h | 29 +++++-- src/Common/tests/gtest_async_loader.cpp | 109 +++++++++++++++++++++++- 2 files changed, 132 insertions(+), 6 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 5f4c67ab065..7bce762bb39 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -57,14 +57,30 @@ public: return !is_finished ? LoadStatus::PENDING : (exception ? LoadStatus::FAILED : LoadStatus::SUCCESS); } - void wait() + void wait() const { std::unique_lock lock{mutex}; + waiters++; finished.wait(lock, [this] { return is_finished; }); + waiters--; if (exception) std::rethrow_exception(exception); } + void waitNoThrow() const + { + std::unique_lock lock{mutex}; + waiters++; + finished.wait(lock, [this] { return is_finished; }); + waiters--; + } + + size_t waiters_count() const + { + std::unique_lock lock{mutex}; + return waiters; + } + const LoadJobSet dependencies; // jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard const String name; std::atomic priority{0}; @@ -76,7 +92,8 @@ private: { std::unique_lock lock{mutex}; is_finished = true; - finished.notify_all(); + if (waiters > 0) + finished.notify_all(); } void setFailure(const std::exception_ptr & ptr) @@ -84,13 +101,15 @@ private: std::unique_lock lock{mutex}; is_finished = true; exception = ptr; - finished.notify_all(); + if (waiters > 0) + finished.notify_all(); } std::function func; mutable std::mutex mutex; - std::condition_variable finished; + mutable std::condition_variable finished; + mutable size_t waiters = 0; bool is_finished = false; std::exception_ptr exception; }; @@ -337,7 +356,7 @@ public: else // Job is currently executing { lock.unlock(); - job->wait(); // Wait for job to finish + job->waitNoThrow(); // Wait for job to finish lock.lock(); } finished_jobs.erase(job); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 7e76ac0e07a..7ef831cbfd0 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -169,7 +170,7 @@ TEST(AsyncLoader, CancelPendingJob) auto job_func = [&] (const LoadJob &) {}; auto job = makeLoadJob({}, "job", job_func); - auto task = t.loader.schedule({job}); + auto task = t.loader.schedule({ job }); task.remove(); // this cancels pending the job (async loader was not started to execute it) @@ -185,6 +186,112 @@ TEST(AsyncLoader, CancelPendingJob) } } +TEST(AsyncLoader, CancelPendingTask) +{ + AsyncLoaderTest t; + + auto job_func = [&] (const LoadJob &) {}; + + auto job1 = makeLoadJob({}, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task = t.loader.schedule({ job1, job2 }); + + task.remove(); // this cancels both jobs (async loader was not started to execute it) + + ASSERT_EQ(job1->status(), LoadStatus::FAILED); + ASSERT_EQ(job2->status(), LoadStatus::FAILED); + + try + { + job1->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED); + } + + try + { + job2->wait(); + FAIL(); + } + catch (Exception & e) + { + // Result depend on non-deterministic cancel order + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED || e.code() == ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED); + } +} + +TEST(AsyncLoader, CancelPendingDependency) +{ + AsyncLoaderTest t; + + auto job_func = [&] (const LoadJob &) {}; + + auto job1 = makeLoadJob({}, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task1 = t.loader.schedule({ job1 }); + auto task2 = t.loader.schedule({ job2 }); + + task1.remove(); // this cancels both jobs, due to dependency (async loader was not started to execute it) + + ASSERT_EQ(job1->status(), LoadStatus::FAILED); + ASSERT_EQ(job2->status(), LoadStatus::FAILED); + + try + { + job1->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED); + } + + try + { + job2->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED); + } +} + +TEST(AsyncLoader, CancelExecutingJob) +{ + AsyncLoaderTest t; + t.loader.start(); + + std::barrier sync(2); + + auto job_func = [&] (const LoadJob &) + { + sync.arrive_and_wait(); // (A) sync with main thread + sync.arrive_and_wait(); // (B) wait for waiter + // signals (C) + }; + + auto job = makeLoadJob({}, "job", job_func); + auto task = t.loader.schedule({ job }); + + sync.arrive_and_wait(); // (A) wait for job to start executing + std::thread canceler([&] + { + task.remove(); // waits for (C) + }); + while (job->waiters_count() == 0) + std::this_thread::yield(); + ASSERT_EQ(job->status(), LoadStatus::PENDING); + sync.arrive_and_wait(); // (B) sync with job + canceler.join(); + + ASSERT_EQ(job->status(), LoadStatus::SUCCESS); + job->wait(); +} + TEST(AsyncLoader, RandomTasks) { AsyncLoaderTest t(16); From 569f7651ce4b27d308fd1a3fc298f78d622893a6 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 01:21:25 +0000 Subject: [PATCH 010/628] fix remove of executing tasks + add test --- src/Common/AsyncLoader.h | 33 +++++++++------ src/Common/tests/gtest_async_loader.cpp | 56 +++++++++++++++++++++++++ 2 files changed, 77 insertions(+), 12 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 7bce762bb39..999559169ab 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -337,13 +337,12 @@ public: { DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; + // On the first pass: + // - cancel all not executing jobs to avoid races + // - do not wait executing jobs (otherwise, on unlock a worker could start executing a dependent job, that should be canceled) for (const auto & job : jobs) { - if (auto it = finished_jobs.find(job); it != finished_jobs.end()) // Job is already finished - { - finished_jobs.erase(it); - } - else if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { if (info->second.dependencies_left > 0) // Job is not ready yet canceled(job, lock); @@ -353,15 +352,25 @@ public: info->second.ready_seqno = 0; canceled(job, lock); } - else // Job is currently executing - { - lock.unlock(); - job->waitNoThrow(); // Wait for job to finish - lock.lock(); - } - finished_jobs.erase(job); } } + // On the second pass wait for executing jobs to finish + for (const auto & job : jobs) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + // Job is currently executing + chassert(info->second.dependencies_left == 0); + chassert(info->second.ready_seqno == 0); + lock.unlock(); + job->waitNoThrow(); // Wait for job to finish + lock.lock(); + } + } + // On the third pass all jobs are finished - remove them all + // It is better to do it under one lock to avoid exposing intermediate states + for (const auto & job : jobs) + finished_jobs.erase(job); } private: diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 7ef831cbfd0..2ba27264460 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -292,6 +293,61 @@ TEST(AsyncLoader, CancelExecutingJob) job->wait(); } +TEST(AsyncLoader, CancelExecutingTask) +{ + AsyncLoaderTest t(16); + t.loader.start(); + std::barrier sync(2); + + auto blocker_job_func = [&] (const LoadJob &) + { + sync.arrive_and_wait(); // (A) sync with main thread + sync.arrive_and_wait(); // (B) wait for waiter + // signals (C) + }; + + auto job_to_cancel_func = [&] (const LoadJob &) + { + FAIL(); // this job should be canceled + }; + + auto job_to_succeed_func = [&] (const LoadJob &) + { + }; + + // Make several iterations to catch the race (if any) + for (int iteration = 0; iteration < 10; iteration++) { + std::vector task1_jobs; + auto blocker_job = makeLoadJob({}, "blocker_job", blocker_job_func); + task1_jobs.push_back(blocker_job); + for (int i = 0; i < 100; i++) + task1_jobs.push_back(makeLoadJob({ blocker_job }, "job_to_cancel", job_to_cancel_func)); + auto task1 = t.loader.schedule({ task1_jobs.begin(), task1_jobs.end() }); + auto job_to_succeed = makeLoadJob({ blocker_job }, "job_to_succeed", job_to_succeed_func); + auto task2 = t.loader.schedule({ job_to_succeed }); + + sync.arrive_and_wait(); // (A) wait for job to start executing + std::thread canceler([&] + { + task1.remove(); // waits for (C) + }); + while (blocker_job->waiters_count() == 0) + std::this_thread::yield(); + ASSERT_EQ(blocker_job->status(), LoadStatus::PENDING); + sync.arrive_and_wait(); // (B) sync with job + canceler.join(); + t.loader.wait(); + + ASSERT_EQ(blocker_job->status(), LoadStatus::SUCCESS); + ASSERT_EQ(job_to_succeed->status(), LoadStatus::SUCCESS); + for (const auto & job : task1_jobs) + { + if (job != blocker_job) + ASSERT_EQ(job->status(), LoadStatus::FAILED); + } + } +} + TEST(AsyncLoader, RandomTasks) { AsyncLoaderTest t(16); From 15f1962f9fab8bcf6015f514a45ab5c7bc066a73 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 01:23:40 +0000 Subject: [PATCH 011/628] allow cancels on RandomTasks test shutdown --- src/Common/tests/gtest_async_loader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 2ba27264460..ae0929e6e99 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -365,6 +365,5 @@ TEST(AsyncLoader, RandomTasks) tasks.push_back(t.loader.schedule(t.randomJobSet(job_count, 5, job_func))); t.randomSleepUs(100, 900, 20); // avg=100us } - t.loader.wait(); } From 7605f51de9f2e964b2f0f4c70e1228b31eff4057 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 12:46:09 +0000 Subject: [PATCH 012/628] add concurrency test --- src/Common/tests/gtest_async_loader.cpp | 44 ++++++++++++++++++++++--- 1 file changed, 39 insertions(+), 5 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index ae0929e6e99..4a86938817c 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -38,6 +38,10 @@ struct AsyncLoaderTest std::mutex rng_mutex; pcg64 rng{randomSeed()}; + explicit AsyncLoaderTest(size_t max_threads = 1) + : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads) + {} + template T randomInt(T from, T to) { @@ -53,7 +57,7 @@ struct AsyncLoaderTest } template - LoadJobSet randomJobSet(int job_count, int dep_probability_percent, JobFunc job_func) + LoadJobSet randomJobSet(int job_count, int dep_probability_percent, JobFunc job_func, std::string_view name_prefix = "job") { std::vector jobs; for (int j = 0; j < job_count; j++) @@ -64,14 +68,20 @@ struct AsyncLoaderTest if (randomInt(0, 99) < dep_probability_percent) deps.insert(jobs[d]); } - jobs.push_back(makeLoadJob(std::move(deps), fmt::format("job{}", j), job_func)); + jobs.push_back(makeLoadJob(std::move(deps), fmt::format("{}{}", name_prefix, j), job_func)); } return {jobs.begin(), jobs.end()}; } - explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads) - {} + template + LoadJobSet chainJobSet(int job_count, JobFunc job_func, std::string_view name_prefix = "job") + { + std::vector jobs; + jobs.push_back(makeLoadJob({}, fmt::format("{}{}", name_prefix, 0), job_func)); + for (int j = 1; j < job_count; j++) + jobs.push_back(makeLoadJob({ jobs[j - 1] }, fmt::format("{}{}", name_prefix, j), job_func)); + return {jobs.begin(), jobs.end()}; + } }; TEST(AsyncLoader, Smoke) @@ -367,3 +377,27 @@ TEST(AsyncLoader, RandomTasks) } } +TEST(AsyncLoader, TestConcurrency) +{ + AsyncLoaderTest t(10); + t.loader.start(); + + for (int concurrency = 1; concurrency <= 10; concurrency++) + { + std::barrier sync(concurrency); + + std::atomic executing{0}; + auto job_func = [&] (const LoadJob &) + { + executing++; + ASSERT_LE(executing, concurrency); + sync.arrive_and_wait(); + executing--; + }; + + std::vector tasks; + for (int i = 0; i < concurrency; i++) + tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); + t.loader.wait(); + } +} From 0024febe70aa27484921d818e66f18f52071bbd3 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 12:57:51 +0000 Subject: [PATCH 013/628] add overload test --- src/Common/AsyncLoader.h | 8 ++++++- src/Common/tests/gtest_async_loader.cpp | 29 +++++++++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 999559169ab..9467a705c10 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -373,6 +373,12 @@ public: finished_jobs.erase(job); } + size_t getMaxThreads() const + { + std::unique_lock lock{mutex}; + return pool.getMaxThreads(); + } + private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) { @@ -588,7 +594,7 @@ private: } } - std::mutex mutex; + mutable std::mutex mutex; bool is_running = false; // TODO(serxa): add metrics for number of jobs in every state diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 4a86938817c..68de24460bb 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -399,5 +399,34 @@ TEST(AsyncLoader, TestConcurrency) for (int i = 0; i < concurrency; i++) tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); t.loader.wait(); + ASSERT_EQ(executing, 0); + } +} + +TEST(AsyncLoader, TestOverload) +{ + AsyncLoaderTest t(3); + t.loader.start(); + + size_t max_threads = t.loader.getMaxThreads(); + std::atomic executing{0}; + + for (int concurrency = 4; concurrency <= 8; concurrency++) + { + auto job_func = [&] (const LoadJob &) + { + executing++; + t.randomSleepUs(100, 200, 100); + ASSERT_LE(executing, max_threads); + executing--; + }; + + t.loader.stop(); + std::vector tasks; + for (int i = 0; i < concurrency; i++) + tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); + t.loader.start(); + t.loader.wait(); + ASSERT_EQ(executing, 0); } } From a83fc81ed12759589a846a0a317989e3ea5deb3c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 13:14:31 +0000 Subject: [PATCH 014/628] test deps are completed + refactoring --- src/Common/AsyncLoader.h | 6 ++- src/Common/tests/gtest_async_loader.cpp | 64 +++++++++++++------------ 2 files changed, 37 insertions(+), 33 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 9467a705c10..96ea6165d34 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -105,7 +105,9 @@ private: finished.notify_all(); } - std::function func; + // TODO(serxa): add callback/status for cancel? + + std::function func; mutable std::mutex mutex; mutable std::condition_variable finished; @@ -576,7 +578,7 @@ private: try { ALLOW_ALLOCATIONS_IN_SCOPE; - job->func(*job); + job->func(job); exception_from_job = {}; } catch (...) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 68de24460bb..d913f8f6362 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -93,9 +93,9 @@ TEST(AsyncLoader, Smoke) std::atomic jobs_done{0}; std::atomic low_priority_jobs_done{0}; - auto job_func = [&] (const LoadJob & self) { + auto job_func = [&] (const LoadJobPtr & self) { jobs_done++; - if (self.priority == low_priority) + if (self->priority == low_priority) low_priority_jobs_done++; }; @@ -134,7 +134,7 @@ TEST(AsyncLoader, CycleDetection) { AsyncLoaderTest t; - auto job_func = [&] (const LoadJob &) {}; + auto job_func = [&] (const LoadJobPtr &) {}; LoadJobPtr cycle_breaker; // To avoid memleak we introduce with a cycle @@ -178,7 +178,7 @@ TEST(AsyncLoader, CancelPendingJob) { AsyncLoaderTest t; - auto job_func = [&] (const LoadJob &) {}; + auto job_func = [&] (const LoadJobPtr &) {}; auto job = makeLoadJob({}, "job", job_func); auto task = t.loader.schedule({ job }); @@ -201,7 +201,7 @@ TEST(AsyncLoader, CancelPendingTask) { AsyncLoaderTest t; - auto job_func = [&] (const LoadJob &) {}; + auto job_func = [&] (const LoadJobPtr &) {}; auto job1 = makeLoadJob({}, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); @@ -238,7 +238,7 @@ TEST(AsyncLoader, CancelPendingDependency) { AsyncLoaderTest t; - auto job_func = [&] (const LoadJob &) {}; + auto job_func = [&] (const LoadJobPtr &) {}; auto job1 = makeLoadJob({}, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); @@ -278,7 +278,7 @@ TEST(AsyncLoader, CancelExecutingJob) std::barrier sync(2); - auto job_func = [&] (const LoadJob &) + auto job_func = [&] (const LoadJobPtr &) { sync.arrive_and_wait(); // (A) sync with main thread sync.arrive_and_wait(); // (B) wait for waiter @@ -309,19 +309,19 @@ TEST(AsyncLoader, CancelExecutingTask) t.loader.start(); std::barrier sync(2); - auto blocker_job_func = [&] (const LoadJob &) + auto blocker_job_func = [&] (const LoadJobPtr &) { sync.arrive_and_wait(); // (A) sync with main thread sync.arrive_and_wait(); // (B) wait for waiter // signals (C) }; - auto job_to_cancel_func = [&] (const LoadJob &) + auto job_to_cancel_func = [&] (const LoadJobPtr &) { FAIL(); // this job should be canceled }; - auto job_to_succeed_func = [&] (const LoadJob &) + auto job_to_succeed_func = [&] (const LoadJobPtr &) { }; @@ -358,25 +358,6 @@ TEST(AsyncLoader, CancelExecutingTask) } } -TEST(AsyncLoader, RandomTasks) -{ - AsyncLoaderTest t(16); - t.loader.start(); - - auto job_func = [&] (const LoadJob &) - { - t.randomSleepUs(100, 500, 5); - }; - - std::vector tasks; - for (int i = 0; i < 512; i++) - { - int job_count = t.randomInt(1, 32); - tasks.push_back(t.loader.schedule(t.randomJobSet(job_count, 5, job_func))); - t.randomSleepUs(100, 900, 20); // avg=100us - } -} - TEST(AsyncLoader, TestConcurrency) { AsyncLoaderTest t(10); @@ -387,7 +368,7 @@ TEST(AsyncLoader, TestConcurrency) std::barrier sync(concurrency); std::atomic executing{0}; - auto job_func = [&] (const LoadJob &) + auto job_func = [&] (const LoadJobPtr &) { executing++; ASSERT_LE(executing, concurrency); @@ -413,7 +394,7 @@ TEST(AsyncLoader, TestOverload) for (int concurrency = 4; concurrency <= 8; concurrency++) { - auto job_func = [&] (const LoadJob &) + auto job_func = [&] (const LoadJobPtr &) { executing++; t.randomSleepUs(100, 200, 100); @@ -430,3 +411,24 @@ TEST(AsyncLoader, TestOverload) ASSERT_EQ(executing, 0); } } + +TEST(AsyncLoader, RandomTasks) +{ + AsyncLoaderTest t(16); + t.loader.start(); + + auto job_func = [&] (const LoadJobPtr & self) + { + for (const auto & dep : self->dependencies) + ASSERT_EQ(dep->status(), LoadStatus::SUCCESS); + t.randomSleepUs(100, 500, 5); + }; + + std::vector tasks; + for (int i = 0; i < 512; i++) + { + int job_count = t.randomInt(1, 32); + tasks.push_back(t.loader.schedule(t.randomJobSet(job_count, 5, job_func))); + t.randomSleepUs(100, 900, 20); // avg=100us + } +} From 78d5a4d88b44cffd828ff0bf8bf8126971f4840d Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 14:00:54 +0000 Subject: [PATCH 015/628] add LoadStatus::CANCELED --- src/Common/AsyncLoader.h | 96 ++++++++++++++----------- src/Common/tests/gtest_async_loader.cpp | 24 +++---- 2 files changed, 67 insertions(+), 53 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 96ea6165d34..6af1b79bec8 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -36,9 +36,10 @@ namespace ErrorCodes enum class LoadStatus { - PENDING, // Load is not finished yet - SUCCESS, // Load was successful - FAILED // Load failed or canceled + PENDING, // Load job is not started yet + OK, // Load job executed and was successful + FAILED, // Load job executed and failed + CANCELED // Load job is not going to be executed due to removal or dependency failure }; class LoadJob : private boost::noncopyable @@ -54,14 +55,14 @@ public: LoadStatus status() const { std::unique_lock lock{mutex}; - return !is_finished ? LoadStatus::PENDING : (exception ? LoadStatus::FAILED : LoadStatus::SUCCESS); + return load_status; } void wait() const { std::unique_lock lock{mutex}; waiters++; - finished.wait(lock, [this] { return is_finished; }); + finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); waiters--; if (exception) std::rethrow_exception(exception); @@ -71,7 +72,7 @@ public: { std::unique_lock lock{mutex}; waiters++; - finished.wait(lock, [this] { return is_finished; }); + finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); waiters--; } @@ -88,18 +89,27 @@ public: private: friend class AsyncLoader; - void setSuccess() + void ok() { std::unique_lock lock{mutex}; - is_finished = true; + load_status = LoadStatus::OK; if (waiters > 0) finished.notify_all(); } - void setFailure(const std::exception_ptr & ptr) + void failed(const std::exception_ptr & ptr) { std::unique_lock lock{mutex}; - is_finished = true; + load_status = LoadStatus::FAILED; + exception = ptr; + if (waiters > 0) + finished.notify_all(); + } + + void canceled(const std::exception_ptr & ptr) + { + std::unique_lock lock{mutex}; + load_status = LoadStatus::CANCELED; exception = ptr; if (waiters > 0) finished.notify_all(); @@ -112,7 +122,7 @@ private: mutable std::mutex mutex; mutable std::condition_variable finished; mutable size_t waiters = 0; - bool is_finished = false; + LoadStatus load_status{LoadStatus::PENDING}; std::exception_ptr exception; }; @@ -148,6 +158,12 @@ private: UInt64 ready_seqno = 0; // zero means that job is not in ready queue LoadJobSet dependent_jobs; + // Three independent states of a non-finished jobs + bool is_blocked() const { return dependencies_left > 0; } + bool is_ready() const { return dependencies_left == 0 && ready_seqno > 0; } + bool is_executing() const { return dependencies_left == 0 && ready_seqno == 0; } + + // Get key of a ready job ReadyKey key() const { return {priority, ready_seqno}; @@ -318,8 +334,8 @@ public: prioritize(dep, priority, lock); } - // Place jobs w/o dependencies to ready queue - if (info.dependencies_left == 0) + // Enqueue non-blocked jobs (w/o dependencies) to ready queue + if (!info.is_blocked()) enqueue(info, job, lock); } @@ -346,14 +362,22 @@ public: { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { - if (info->second.dependencies_left > 0) // Job is not ready yet - canceled(job, lock); - else if (info->second.ready_seqno) // Job is enqueued in ready queue + if (info->second.is_executing()) + continue; // Skip executing jobs on the first pass + if (info->second.is_ready()) { ready_queue.erase(info->second.key()); info->second.ready_seqno = 0; - canceled(job, lock); } + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr( + Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' canceled", + job->name)); + }); + markNotOk(job, e, LoadStatus::CANCELED, lock); } } // On the second pass wait for executing jobs to finish @@ -362,8 +386,7 @@ public: if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { // Job is currently executing - chassert(info->second.dependencies_left == 0); - chassert(info->second.ready_seqno == 0); + chassert(info->second.is_executing()); lock.unlock(); job->waitNoThrow(); // Wait for job to finish lock.lock(); @@ -417,23 +440,10 @@ private: return {}; } - void canceled(const LoadJobPtr & job, std::unique_lock & lock) - { - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' canceled", - job->name)); - }); - failed(job, e, lock); - } - - void loaded(const LoadJobPtr & job, std::unique_lock & lock) + void markOk(const LoadJobPtr & job, std::unique_lock & lock) { // Notify waiters - job->setSuccess(); + job->ok(); // Update dependent jobs and enqueue if ready chassert(scheduled_jobs.contains(job)); // Job was pending @@ -441,17 +451,21 @@ private: { chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending Info & dep_info = scheduled_jobs[dep]; - if (--dep_info.dependencies_left == 0) + dep_info.dependencies_left--; + if (!dep_info.is_blocked()) enqueue(dep_info, dep, lock); } finish(job, lock); } - void failed(const LoadJobPtr & job, std::exception_ptr exception_from_job, std::unique_lock & lock) + void markNotOk(const LoadJobPtr & job, std::exception_ptr exception_from_job, LoadStatus status, std::unique_lock & lock) { // Notify waiters - job->setFailure(exception_from_job); + if (status == LoadStatus::FAILED) + job->failed(exception_from_job); + else if (status == LoadStatus::CANCELED) + job->canceled(exception_from_job); // Recurse into all dependent jobs chassert(scheduled_jobs.contains(job)); // Job was pending @@ -469,7 +483,7 @@ private: dep->name, getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); }); - failed(dep, e, lock); + markNotOk(dep, e, LoadStatus::CANCELED, lock); } // Clean dependency graph edges @@ -518,7 +532,7 @@ private: void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) { - chassert(info.dependencies_left == 0); + chassert(!info.is_blocked()); chassert(info.ready_seqno == 0); info.ready_seqno = ++last_ready_seqno; NOEXCEPT_SCOPE({ @@ -555,9 +569,9 @@ private: // Handle just executed job if (exception_from_job) - failed(job, exception_from_job, lock); + markNotOk(job, exception_from_job, LoadStatus::FAILED, lock); else if (job) - loaded(job, lock); + markOk(job, lock); if (!is_running) return; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index d913f8f6362..d7706311fa4 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -120,8 +120,8 @@ TEST(AsyncLoader, Smoke) waiter_thread.join(); - ASSERT_EQ(job1->status(), LoadStatus::SUCCESS); - ASSERT_EQ(job2->status(), LoadStatus::SUCCESS); + ASSERT_EQ(job1->status(), LoadStatus::OK); + ASSERT_EQ(job2->status(), LoadStatus::OK); } ASSERT_EQ(jobs_done, 5); @@ -185,7 +185,7 @@ TEST(AsyncLoader, CancelPendingJob) task.remove(); // this cancels pending the job (async loader was not started to execute it) - ASSERT_EQ(job->status(), LoadStatus::FAILED); + ASSERT_EQ(job->status(), LoadStatus::CANCELED); try { job->wait(); @@ -209,8 +209,8 @@ TEST(AsyncLoader, CancelPendingTask) task.remove(); // this cancels both jobs (async loader was not started to execute it) - ASSERT_EQ(job1->status(), LoadStatus::FAILED); - ASSERT_EQ(job2->status(), LoadStatus::FAILED); + ASSERT_EQ(job1->status(), LoadStatus::CANCELED); + ASSERT_EQ(job2->status(), LoadStatus::CANCELED); try { @@ -247,8 +247,8 @@ TEST(AsyncLoader, CancelPendingDependency) task1.remove(); // this cancels both jobs, due to dependency (async loader was not started to execute it) - ASSERT_EQ(job1->status(), LoadStatus::FAILED); - ASSERT_EQ(job2->status(), LoadStatus::FAILED); + ASSERT_EQ(job1->status(), LoadStatus::CANCELED); + ASSERT_EQ(job2->status(), LoadStatus::CANCELED); try { @@ -299,7 +299,7 @@ TEST(AsyncLoader, CancelExecutingJob) sync.arrive_and_wait(); // (B) sync with job canceler.join(); - ASSERT_EQ(job->status(), LoadStatus::SUCCESS); + ASSERT_EQ(job->status(), LoadStatus::OK); job->wait(); } @@ -348,12 +348,12 @@ TEST(AsyncLoader, CancelExecutingTask) canceler.join(); t.loader.wait(); - ASSERT_EQ(blocker_job->status(), LoadStatus::SUCCESS); - ASSERT_EQ(job_to_succeed->status(), LoadStatus::SUCCESS); + ASSERT_EQ(blocker_job->status(), LoadStatus::OK); + ASSERT_EQ(job_to_succeed->status(), LoadStatus::OK); for (const auto & job : task1_jobs) { if (job != blocker_job) - ASSERT_EQ(job->status(), LoadStatus::FAILED); + ASSERT_EQ(job->status(), LoadStatus::CANCELED); } } } @@ -420,7 +420,7 @@ TEST(AsyncLoader, RandomTasks) auto job_func = [&] (const LoadJobPtr & self) { for (const auto & dep : self->dependencies) - ASSERT_EQ(dep->status(), LoadStatus::SUCCESS); + ASSERT_EQ(dep->status(), LoadStatus::OK); t.randomSleepUs(100, 500, 5); }; From 8584f7f9e480d490ebff6fdd1d065e79cb2b4d92 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 17:16:30 +0000 Subject: [PATCH 016/628] fix scheduling of jobs with dependencies on finished jobs --- src/Common/AsyncLoader.h | 211 +++++++++++++++--------- src/Common/ErrorCodes.cpp | 1 - src/Common/tests/gtest_async_loader.cpp | 86 +++++++++- 3 files changed, 211 insertions(+), 87 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 6af1b79bec8..84b709f14a1 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -31,7 +31,6 @@ namespace ErrorCodes extern const int ASYNC_LOAD_SCHEDULE_FAILED; extern const int ASYNC_LOAD_FAILED; extern const int ASYNC_LOAD_CANCELED; - extern const int ASYNC_LOAD_DEPENDENCY_FAILED; } enum class LoadStatus @@ -58,14 +57,20 @@ public: return load_status; } + std::exception_ptr exception() const + { + std::unique_lock lock{mutex}; + return load_exception; + } + void wait() const { std::unique_lock lock{mutex}; waiters++; finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); waiters--; - if (exception) - std::rethrow_exception(exception); + if (load_exception) + std::rethrow_exception(load_exception); } void waitNoThrow() const @@ -101,7 +106,7 @@ private: { std::unique_lock lock{mutex}; load_status = LoadStatus::FAILED; - exception = ptr; + load_exception = ptr; if (waiters > 0) finished.notify_all(); } @@ -110,7 +115,7 @@ private: { std::unique_lock lock{mutex}; load_status = LoadStatus::CANCELED; - exception = ptr; + load_exception = ptr; if (waiters > 0) finished.notify_all(); } @@ -123,7 +128,7 @@ private: mutable std::condition_variable finished; mutable size_t waiters = 0; LoadStatus load_status{LoadStatus::PENDING}; - std::exception_ptr exception; + std::exception_ptr load_exception; }; template @@ -241,8 +246,9 @@ public: LoadJobSet jobs; }; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_) - : max_threads(max_threads_) + AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_) + : log_failures(log_failures_) + , max_threads(max_threads_) , pool(metric_threads, metric_active_threads, max_threads) {} @@ -310,7 +316,7 @@ public: job->priority.store(priority); // Set user-facing priority } - // Process incoming dependencies + // Process dependencies on scheduled pending jobs for (const auto & job : jobs) { Info & info = scheduled_jobs.find(job)->second; @@ -324,14 +330,10 @@ public: dep_info->second.dependent_jobs.insert(job); }); info.dependencies_left++; - } - else - { - // TODO(serxa): check status: (1) pending: it is wrong - throw? (2) success: good - no dep. (3) failed: propagate failure! - } - // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion - prioritize(dep, priority, lock); + // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion + prioritize(dep, priority, lock); + } } // Enqueue non-blocked jobs (w/o dependencies) to ready queue @@ -339,6 +341,57 @@ public: enqueue(info, job, lock); } + // Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any). + for (const auto & job : jobs) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + for (const auto & dep : job->dependencies) + { + if (scheduled_jobs.contains(dep)) + continue; // Skip dependencies on scheduled pending jobs (already processed) + LoadStatus dep_status = dep->status(); + if (dep_status == LoadStatus::OK) + continue; // Dependency on already successfully finished job -- it's okay. + + if (dep_status == LoadStatus::PENDING) + { + // Dependency on not scheduled pending job -- it's bad. + // Probably, there is an error in `jobs` set: not all jobs were passed to `schedule()` call. + // We are not going to run any dependent job, so cancel them all. + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> Load job '{}': not scheduled pending load job (it must be also scheduled), all dependent load jobs are canceled", + job->name, + dep->name)); + }); + finish(lock, job, LoadStatus::CANCELED, e); + break; // This job is now finished, stop its dependencies processing + } + if (dep_status == LoadStatus::FAILED || dep_status == LoadStatus::CANCELED) + { + // Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs. + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> {}", + job->name, + getExceptionMessage(dep->exception(), /* with_stack_trace = */ false))); + }); + finish(lock, job, LoadStatus::CANCELED, e); + break; // This job is now finished, stop its dependencies processing + } + } + } + else + { + // Job was already canceled on previous iteration of this cycle -- skip + } + } + return Task(this, std::move(jobs)); } @@ -364,20 +417,12 @@ public: { if (info->second.is_executing()) continue; // Skip executing jobs on the first pass - if (info->second.is_ready()) - { - ready_queue.erase(info->second.key()); - info->second.ready_seqno = 0; - } std::exception_ptr e; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' canceled", - job->name)); + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); }); - markNotOk(job, e, LoadStatus::CANCELED, lock); + finish(lock, job, LoadStatus::CANCELED, e); } } // On the second pass wait for executing jobs to finish @@ -440,63 +485,64 @@ private: return {}; } - void markOk(const LoadJobPtr & job, std::unique_lock & lock) + void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}) { - // Notify waiters - job->ok(); - - // Update dependent jobs and enqueue if ready - chassert(scheduled_jobs.contains(job)); // Job was pending - for (const auto & dep : scheduled_jobs[job].dependent_jobs) + if (status == LoadStatus::OK) { - chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending - Info & dep_info = scheduled_jobs[dep]; - dep_info.dependencies_left--; - if (!dep_info.is_blocked()) - enqueue(dep_info, dep, lock); + // Notify waiters + job->ok(); + + // Update dependent jobs and enqueue if ready + chassert(scheduled_jobs.contains(job)); // Job was pending + for (const auto & dep : scheduled_jobs[job].dependent_jobs) + { + chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending + Info & dep_info = scheduled_jobs[dep]; + dep_info.dependencies_left--; + if (!dep_info.is_blocked()) + enqueue(dep_info, dep, lock); + } } - - finish(job, lock); - } - - void markNotOk(const LoadJobPtr & job, std::exception_ptr exception_from_job, LoadStatus status, std::unique_lock & lock) - { - // Notify waiters - if (status == LoadStatus::FAILED) - job->failed(exception_from_job); - else if (status == LoadStatus::CANCELED) - job->canceled(exception_from_job); - - // Recurse into all dependent jobs - chassert(scheduled_jobs.contains(job)); // Job was pending - Info & info = scheduled_jobs[job]; - LoadJobSet dependent; - dependent.swap(info.dependent_jobs); // To avoid container modification during recursion - for (const auto & dep : dependent) + else { - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED, - "Load job '{}' -> {}", - dep->name, - getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); - }); - markNotOk(dep, e, LoadStatus::CANCELED, lock); - } + // Notify waiters + if (status == LoadStatus::FAILED) + job->failed(exception_from_job); + else if (status == LoadStatus::CANCELED) + job->canceled(exception_from_job); - // Clean dependency graph edges - for (const auto & dep : job->dependencies) - if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) - dep_info->second.dependent_jobs.erase(job); + chassert(scheduled_jobs.contains(job)); // Job was pending + Info & info = scheduled_jobs[job]; + if (info.is_ready()) + { + ready_queue.erase(info.key()); + info.ready_seqno = 0; + } + + // Recurse into all dependent jobs + LoadJobSet dependent; + dependent.swap(info.dependent_jobs); // To avoid container modification during recursion + for (const auto & dep : dependent) + { + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr( + Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> {}", + dep->name, + getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); + }); + finish(lock, dep, LoadStatus::CANCELED, e); + } + + // Clean dependency graph edges + for (const auto & dep : job->dependencies) + if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) + dep_info->second.dependent_jobs.erase(job); + } // Job became finished - finish(job, lock); - } - - void finish(const LoadJobPtr & job, std::unique_lock &) - { scheduled_jobs.erase(job); NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; @@ -569,9 +615,9 @@ private: // Handle just executed job if (exception_from_job) - markNotOk(job, exception_from_job, LoadStatus::FAILED, lock); + finish(lock, job, LoadStatus::FAILED, exception_from_job); else if (job) - markOk(job, lock); + finish(lock, job, LoadStatus::OK); if (!is_running) return; @@ -589,17 +635,18 @@ private: scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue } + ALLOW_ALLOCATIONS_IN_SCOPE; + try { - ALLOW_ALLOCATIONS_IN_SCOPE; job->func(job); exception_from_job = {}; } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; + if (log_failures) + tryLogCurrentException(__PRETTY_FUNCTION__); exception_from_job = std::make_exception_ptr( Exception(ErrorCodes::ASYNC_LOAD_FAILED, "Load job '{}' failed: {}", @@ -610,6 +657,8 @@ private: } } + const bool log_failures; + mutable std::mutex mutex; bool is_running = false; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0d203fc799c..64d07ccc9cf 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -653,7 +653,6 @@ M(692, ASYNC_LOAD_SCHEDULE_FAILED) \ M(693, ASYNC_LOAD_FAILED) \ M(694, ASYNC_LOAD_CANCELED) \ - M(695, ASYNC_LOAD_DEPENDENCY_FAILED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index d7706311fa4..5be715db2f6 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -28,7 +29,6 @@ namespace DB::ErrorCodes extern const int ASYNC_LOAD_SCHEDULE_FAILED; extern const int ASYNC_LOAD_FAILED; extern const int ASYNC_LOAD_CANCELED; - extern const int ASYNC_LOAD_DEPENDENCY_FAILED; } struct AsyncLoaderTest @@ -39,7 +39,7 @@ struct AsyncLoaderTest pcg64 rng{randomSeed()}; explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads) + : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads, /* log_failures = */ false) {} template @@ -229,8 +229,7 @@ TEST(AsyncLoader, CancelPendingTask) } catch (Exception & e) { - // Result depend on non-deterministic cancel order - ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED || e.code() == ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED); + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED); } } @@ -267,7 +266,7 @@ TEST(AsyncLoader, CancelPendingDependency) } catch (Exception & e) { - ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_DEPENDENCY_FAILED); + ASSERT_TRUE(e.code() == ErrorCodes::ASYNC_LOAD_CANCELED); } } @@ -358,6 +357,83 @@ TEST(AsyncLoader, CancelExecutingTask) } } +TEST(AsyncLoader, JobFailure) +{ + AsyncLoaderTest t; + t.loader.start(); + + std::string_view error_message = "test job failure"; + + auto job_func = [&] (const LoadJobPtr &) { + throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message); + }; + + auto job = makeLoadJob({}, "job", job_func); + auto task = t.loader.schedule({ job }); + + t.loader.wait(); + + ASSERT_EQ(job->status(), LoadStatus::FAILED); + try + { + job->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_FAILED); + ASSERT_TRUE(e.message().find(error_message) != String::npos); + } +} + +TEST(AsyncLoader, ScheduleJobWithFailedDependencies) +{ + AsyncLoaderTest t; + t.loader.start(); + + std::string_view error_message = "test job failure"; + + auto failed_job_func = [&] (const LoadJobPtr &) { + throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message); + }; + + auto failed_job = makeLoadJob({}, "failed_job", failed_job_func); + auto failed_task = t.loader.schedule({ failed_job }); + + t.loader.wait(); + + auto job_func = [&] (const LoadJobPtr &) {}; + + auto job1 = makeLoadJob({ failed_job }, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task = t.loader.schedule({ job1, job2 }); + + t.loader.wait(); + + ASSERT_EQ(job1->status(), LoadStatus::CANCELED); + ASSERT_EQ(job2->status(), LoadStatus::CANCELED); + try + { + job1->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_CANCELED); + ASSERT_TRUE(e.message().find(error_message) != String::npos); + } + try + { + job2->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_CANCELED); + ASSERT_TRUE(e.message().find(error_message) != String::npos); + } +} + TEST(AsyncLoader, TestConcurrency) { AsyncLoaderTest t(10); From 3dfe950ae3678590b4d651229d6857c49ee08ac0 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 17:27:47 +0000 Subject: [PATCH 017/628] add ScheduleJobWithCanceledDependencies test --- src/Common/tests/gtest_async_loader.cpp | 40 +++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 5be715db2f6..e713e0be562 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -434,6 +434,46 @@ TEST(AsyncLoader, ScheduleJobWithFailedDependencies) } } +TEST(AsyncLoader, ScheduleJobWithCanceledDependencies) +{ + AsyncLoaderTest t; + + auto canceled_job_func = [&] (const LoadJobPtr &) {}; + auto canceled_job = makeLoadJob({}, "canceled_job", canceled_job_func); + auto canceled_task = t.loader.schedule({ canceled_job }); + canceled_task.remove(); + + t.loader.start(); + + auto job_func = [&] (const LoadJobPtr &) {}; + auto job1 = makeLoadJob({ canceled_job }, "job1", job_func); + auto job2 = makeLoadJob({ job1 }, "job2", job_func); + auto task = t.loader.schedule({ job1, job2 }); + + t.loader.wait(); + + ASSERT_EQ(job1->status(), LoadStatus::CANCELED); + ASSERT_EQ(job2->status(), LoadStatus::CANCELED); + try + { + job1->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_CANCELED); + } + try + { + job2->wait(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::ASYNC_LOAD_CANCELED); + } +} + TEST(AsyncLoader, TestConcurrency) { AsyncLoaderTest t(10); From 42ad5f3df65075f768522e318a999a741249e1fa Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 19:29:24 +0000 Subject: [PATCH 018/628] fix worker shutdown --- src/Common/AsyncLoader.h | 5 +---- src/Common/tests/gtest_async_loader.cpp | 2 -- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 84b709f14a1..98f2e4153d4 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -619,10 +619,7 @@ private: else if (job) finish(lock, job, LoadStatus::OK); - if (!is_running) - return; - - if (ready_queue.empty()) + if (!is_running || ready_queue.empty()) { workers--; return; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index e713e0be562..8a1d9f8b2de 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -14,10 +14,8 @@ #include #include - using namespace DB; - namespace CurrentMetrics { extern const Metric TablesLoaderThreads; From 35daa18c6e9c5aacb2d4f1be2fae08d61774e99b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Apr 2023 20:49:56 +0000 Subject: [PATCH 019/628] add random test with dependent tasks --- src/Common/AsyncLoader.h | 6 +++ src/Common/tests/gtest_async_loader.cpp | 59 ++++++++++++++++++++++++- 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 98f2e4153d4..6d19ed05463 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -449,6 +449,12 @@ public: return pool.getMaxThreads(); } + size_t getScheduledJobCount() const + { + std::unique_lock lock{mutex}; + return scheduled_jobs.size(); + } + private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) { diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 8a1d9f8b2de..208d5590f3e 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -71,6 +71,25 @@ struct AsyncLoaderTest return {jobs.begin(), jobs.end()}; } + template + LoadJobSet randomJobSet(int job_count, int dep_probability_percent, std::vector external_deps, JobFunc job_func, std::string_view name_prefix = "job") + { + std::vector jobs; + for (int j = 0; j < job_count; j++) + { + LoadJobSet deps; + for (int d = 0; d < j; d++) + { + if (randomInt(0, 99) < dep_probability_percent) + deps.insert(jobs[d]); + } + if (randomInt(0, 99) < dep_probability_percent) + deps.insert(external_deps[randomInt(0, external_deps.size() - 1)]); + jobs.push_back(makeLoadJob(std::move(deps), fmt::format("{}{}", name_prefix, j), job_func)); + } + return {jobs.begin(), jobs.end()}; + } + template LoadJobSet chainJobSet(int job_count, JobFunc job_func, std::string_view name_prefix = "job") { @@ -526,7 +545,7 @@ TEST(AsyncLoader, TestOverload) } } -TEST(AsyncLoader, RandomTasks) +TEST(AsyncLoader, RandomIndependentTasks) { AsyncLoaderTest t(16); t.loader.start(); @@ -546,3 +565,41 @@ TEST(AsyncLoader, RandomTasks) t.randomSleepUs(100, 900, 20); // avg=100us } } + +TEST(AsyncLoader, RandomDependentTasks) +{ + AsyncLoaderTest t(16); + t.loader.start(); + + std::mutex mutex; + std::condition_variable cv; + std::vector tasks; + std::vector all_jobs; + + auto job_func = [&] (const LoadJobPtr & self) + { + for (const auto & dep : self->dependencies) + ASSERT_EQ(dep->status(), LoadStatus::OK); + cv.notify_one(); + }; + + std::unique_lock lock{mutex}; + + int tasks_left = 1000; + while (tasks_left-- > 0) + { + cv.wait(lock, [&] { return t.loader.getScheduledJobCount() < 100; }); + + // Add one new task + int job_count = t.randomInt(1, 32); + LoadJobSet jobs = t.randomJobSet(job_count, 5, all_jobs, job_func); + all_jobs.insert(all_jobs.end(), jobs.begin(), jobs.end()); + tasks.push_back(t.loader.schedule(std::move(jobs))); + + // Cancel random old task + if (tasks.size() > 100) + tasks.erase(tasks.begin() + t.randomInt(0, tasks.size() - 1)); + } + + t.loader.wait(); +} From 8835245954f142ad32d7e4830521c477f9d3af1e Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 15 Apr 2023 14:22:01 +0000 Subject: [PATCH 020/628] make performance-inefficient-vector-operation,-warnings-as-errors happy --- src/Common/tests/gtest_async_loader.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 208d5590f3e..ad1cb206b94 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -58,6 +58,7 @@ struct AsyncLoaderTest LoadJobSet randomJobSet(int job_count, int dep_probability_percent, JobFunc job_func, std::string_view name_prefix = "job") { std::vector jobs; + jobs.reserve(job_count); for (int j = 0; j < job_count; j++) { LoadJobSet deps; @@ -75,6 +76,7 @@ struct AsyncLoaderTest LoadJobSet randomJobSet(int job_count, int dep_probability_percent, std::vector external_deps, JobFunc job_func, std::string_view name_prefix = "job") { std::vector jobs; + jobs.reserve(job_count); for (int j = 0; j < job_count; j++) { LoadJobSet deps; @@ -94,6 +96,7 @@ struct AsyncLoaderTest LoadJobSet chainJobSet(int job_count, JobFunc job_func, std::string_view name_prefix = "job") { std::vector jobs; + jobs.reserve(job_count); jobs.push_back(makeLoadJob({}, fmt::format("{}{}", name_prefix, 0), job_func)); for (int j = 1; j < job_count; j++) jobs.push_back(makeLoadJob({ jobs[j - 1] }, fmt::format("{}{}", name_prefix, j), job_func)); @@ -158,6 +161,7 @@ TEST(AsyncLoader, CycleDetection) try { std::vector jobs; + jobs.reserve(16); jobs.push_back(makeLoadJob({}, "job0", job_func)); jobs.push_back(makeLoadJob({ jobs[0] }, "job1", job_func)); jobs.push_back(makeLoadJob({ jobs[0], jobs[1] }, "job2", job_func)); @@ -344,6 +348,7 @@ TEST(AsyncLoader, CancelExecutingTask) // Make several iterations to catch the race (if any) for (int iteration = 0; iteration < 10; iteration++) { std::vector task1_jobs; + task1_jobs.reserve(256); auto blocker_job = makeLoadJob({}, "blocker_job", blocker_job_func); task1_jobs.push_back(blocker_job); for (int i = 0; i < 100; i++) @@ -510,6 +515,7 @@ TEST(AsyncLoader, TestConcurrency) }; std::vector tasks; + tasks.reserve(concurrency); for (int i = 0; i < concurrency; i++) tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); t.loader.wait(); @@ -537,6 +543,7 @@ TEST(AsyncLoader, TestOverload) t.loader.stop(); std::vector tasks; + tasks.reserve(concurrency); for (int i = 0; i < concurrency; i++) tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); t.loader.start(); @@ -558,6 +565,7 @@ TEST(AsyncLoader, RandomIndependentTasks) }; std::vector tasks; + tasks.reserve(512); for (int i = 0; i < 512; i++) { int job_count = t.randomInt(1, 32); @@ -586,6 +594,7 @@ TEST(AsyncLoader, RandomDependentTasks) std::unique_lock lock{mutex}; int tasks_left = 1000; + tasks.reserve(tasks_left); while (tasks_left-- > 0) { cv.wait(lock, [&] { return t.loader.getScheduledJobCount() < 100; }); From d34aeaba4162da4184417ae0fa8c3df2b2b6b56f Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 15 Apr 2023 19:48:10 +0000 Subject: [PATCH 021/628] add test for static priorities --- src/Common/AsyncLoader.h | 26 +++++++++++------- src/Common/tests/gtest_async_loader.cpp | 35 +++++++++++++++++++++++++ 2 files changed, 52 insertions(+), 9 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 6d19ed05463..5fff5ee89c1 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -144,23 +144,31 @@ private: // Key of a pending job in ready queue struct ReadyKey { - ssize_t priority; - UInt64 ready_seqno; + ssize_t priority; // Ascending order + ssize_t initial_priority; // Ascending order + UInt64 ready_seqno; // Descending order bool operator<(const ReadyKey & rhs) const { - if (priority == rhs.priority) - return ready_seqno < rhs.ready_seqno; - return priority > rhs.priority; + if (priority > rhs.priority) + return true; + if (priority < rhs.priority) + return false; + if (initial_priority > rhs.initial_priority) + return true; + if (initial_priority < rhs.initial_priority) + return false; + return ready_seqno < rhs.ready_seqno; } }; // Scheduling information for a pending job struct Info { - ssize_t priority = 0; + ssize_t initial_priority = 0; // Initial priority passed into schedule() + ssize_t priority = 0; // Elevated priority, due to priority inheritance or prioritize() size_t dependencies_left = 0; - UInt64 ready_seqno = 0; // zero means that job is not in ready queue + UInt64 ready_seqno = 0; // Zero means that job is not in ready queue LoadJobSet dependent_jobs; // Three independent states of a non-finished jobs @@ -171,7 +179,7 @@ private: // Get key of a ready job ReadyKey key() const { - return {priority, ready_seqno}; + return {.priority = priority, .initial_priority = initial_priority, .ready_seqno = ready_seqno}; } }; @@ -311,7 +319,7 @@ public: { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - scheduled_jobs.emplace(job, Info{.priority = priority}); + scheduled_jobs.emplace(job, Info{.initial_priority = priority, .priority = priority}); }); job->priority.store(priority); // Set user-facing priority } diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index ad1cb206b94..a72cc0f994b 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -552,6 +552,41 @@ TEST(AsyncLoader, TestOverload) } } +TEST(AsyncLoader, StaticPriorities) +{ + AsyncLoaderTest t(1); + + std::string schedule; + + auto job_func = [&] (const LoadJobPtr & self) + { + schedule += fmt::format("{}{}", self->name, self->priority); + }; + + std::vector jobs; + jobs.push_back(makeLoadJob({}, "A", job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, "B", job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, "C", job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, "D", job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, "E", job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, "G", job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, "H", job_func)); // 7 + auto task = t.loader.schedule({ jobs[0] }, 0); + task.merge(t.loader.schedule({ jobs[1] }, 3)); + task.merge(t.loader.schedule({ jobs[2] }, 4)); + task.merge(t.loader.schedule({ jobs[3] }, 1)); + task.merge(t.loader.schedule({ jobs[4] }, 2)); + task.merge(t.loader.schedule({ jobs[5] }, 0)); + task.merge(t.loader.schedule({ jobs[6] }, 0)); + task.merge(t.loader.schedule({ jobs[7] }, 9)); + + t.loader.start(); + t.loader.wait(); + + ASSERT_EQ(schedule, "A9E9D9F9G9H9C4B3"); +} + TEST(AsyncLoader, RandomIndependentTasks) { AsyncLoaderTest t(16); From 05c63bc60852e234644bbac04b4b36e9dbbb453d Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 08:35:59 +0000 Subject: [PATCH 022/628] fix test --- src/Common/tests/gtest_async_loader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index a72cc0f994b..755f856d340 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -73,7 +73,7 @@ struct AsyncLoaderTest } template - LoadJobSet randomJobSet(int job_count, int dep_probability_percent, std::vector external_deps, JobFunc job_func, std::string_view name_prefix = "job") + LoadJobSet randomJobSet(int job_count, int dep_probability_percent, const std::vector & external_deps, JobFunc job_func, std::string_view name_prefix = "job") { std::vector jobs; jobs.reserve(job_count); @@ -85,7 +85,7 @@ struct AsyncLoaderTest if (randomInt(0, 99) < dep_probability_percent) deps.insert(jobs[d]); } - if (randomInt(0, 99) < dep_probability_percent) + if (!external_deps.empty() && randomInt(0, 99) < dep_probability_percent) deps.insert(external_deps[randomInt(0, external_deps.size() - 1)]); jobs.push_back(makeLoadJob(std::move(deps), fmt::format("{}{}", name_prefix, j), job_func)); } From 2b99aee503fb3a012a3e9d63a00692bd64a2ea05 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 11:04:59 +0000 Subject: [PATCH 023/628] allow to change max number of workers in runtime --- src/Common/AsyncLoader.h | 22 ++++++++--- src/Common/tests/gtest_async_loader.cpp | 49 +++++++++++++++++++++++++ 2 files changed, 66 insertions(+), 5 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 5fff5ee89c1..d639c1031cd 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -120,8 +120,6 @@ private: finished.notify_all(); } - // TODO(serxa): add callback/status for cancel? - std::function func; mutable std::mutex mutex; @@ -243,6 +241,7 @@ public: } // Do not track jobs in this task + // WARNING: Jobs will never be removed() and are going to be stored as finished_jobs until ~AsyncLoader() void detach() { loader = nullptr; @@ -451,10 +450,23 @@ public: finished_jobs.erase(job); } + void setMaxThreads(size_t value) + { + std::unique_lock lock{mutex}; + pool.setMaxThreads(value); + pool.setMaxFreeThreads(value); + pool.setQueueSize(value); + max_threads = value; + if (!is_running) + return; + for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) + spawn(lock); + } + size_t getMaxThreads() const { std::unique_lock lock{mutex}; - return pool.getMaxThreads(); + return max_threads; } size_t getScheduledJobCount() const @@ -600,7 +612,7 @@ private: ready_queue.emplace(info.key(), job); }); - if (is_running && workers < max_threads) // TODO(serxa): Can we make max_thread changeable in runtime? + if (is_running && workers < max_threads) spawn(lock); } @@ -633,7 +645,7 @@ private: else if (job) finish(lock, job, LoadStatus::OK); - if (!is_running || ready_queue.empty()) + if (!is_running || ready_queue.empty() || workers > max_threads) { workers--; return; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 755f856d340..52cc3436595 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -647,3 +647,52 @@ TEST(AsyncLoader, RandomDependentTasks) t.loader.wait(); } + +TEST(AsyncLoader, SetMaxThreads) +{ + AsyncLoaderTest t(1); + + std::atomic sync_index{0}; + std::atomic executing{0}; + int max_threads_values[] = {1, 2, 3, 4, 5, 4, 3, 2, 1, 5, 10, 5, 1, 20, 1}; + std::vector>> syncs; + syncs.reserve(std::size(max_threads_values)); + for (int max_threads : max_threads_values) + syncs.push_back(std::make_unique>(max_threads + 1)); + + + auto job_func = [&] (const LoadJobPtr &) + { + int idx = sync_index; + if (idx < syncs.size()) + { + executing++; + syncs[idx]->arrive_and_wait(); // (A) + executing--; + syncs[idx]->arrive_and_wait(); // (B) + } + }; + + // Generate enough independent jobs + for (int i = 0; i < 1000; i++) + t.loader.schedule({makeLoadJob({}, "job", job_func)}).detach(); + + t.loader.start(); + while (sync_index < syncs.size()) + { + // Wait for `max_threads` jobs to start executing + int idx = sync_index; + while (executing.load() != max_threads_values[idx]) + { + ASSERT_LE(executing, max_threads_values[idx]); + std::this_thread::yield(); + } + + // Allow all jobs to finish + syncs[idx]->arrive_and_wait(); // (A) + sync_index++; + if (sync_index < syncs.size()) + t.loader.setMaxThreads(max_threads_values[sync_index]); + syncs[idx]->arrive_and_wait(); // (B) this sync point is required to allow `executing` value to go back down to zero after we change number of workers + } +} From 1d590fd29c4daf75599eb047e1e0fd08e73a4d91 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 12:07:46 +0000 Subject: [PATCH 024/628] add a good comment --- src/Common/AsyncLoader.h | 42 +++++++++++++++++++++++++++++++++++++--- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d639c1031cd..9e54b3ae75f 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -135,7 +135,45 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && return std::make_shared(std::move(dependencies), name, std::forward(func)); } -// TODO(serxa): write good comment +// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. +// Basic usage example: +// auto job_func = [&] (const LoadJobPtr & self) { +// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority); +// }; +// auto job1 = makeLoadJob({}, "job1", job_func); +// auto job2 = makeLoadJob({ job1 }, "job2", job_func); +// auto job3 = makeLoadJob({ job1 }, "job3", job_func); +// auto task = async_loader.schedule({ job1, job2, job3 }, /* priority = */ 0); +// Here we have created and scheduled a task consisting of two jobs. Job1 has no dependencies and is run first. +// Job2 and job3 depends on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: +// async_loader->prioritize(job3, 1); // higher priority jobs are run first +// job3->wait(); // blocks until job completion or cancelation and rethrow an exception (if any) +// +// AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: +// 1) Job is constructed with PENDING status. +// 2) Job is scheduled and placed into a task. Scheduled job may be ready (i.e. have all its dependencies finished) or blocked. +// 3a) When all dependencies are successfully executed, job became ready. Ready job is enqueued into the ready queue. +// 3b) If at least one of job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). +// On cancelation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. Job status is changed to CANCELED. +// Exception is rethrown by any existing or new `wait()` call. Job is moved to the set of finished jobs. +// 4) Scheduled pending ready job starts execution by a worker. Job is dequeuedCallback `job_func` is called. +// Status of an executing job is PENDING. And it is still considered as scheduled job by AsyncLoader. +// Note that `job_func` of a CANCELED job is never executed. +// 5a) On successful execution job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. +// 5b) Any exception thrown out of `job_func` is wrapped into ASYNC_LOAD_FAILED exception and save inside LoadJob. +// Job status is changed to FAILED. All dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. +// 6) Job is no longer considered as scheduled and is instead moved to finished jobs set. This is required for introspection of finished jobs. +// 7) Task object containing this job is destructed or `remove()` is explicitly called. Job is removed from the finished job set. +// 8) Job is destructed. +// +// Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated +// (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited); +// (b) or job was explicitly prioritized by `prioritize(job, higher_priority)` call (this also leads to a priority inheritance for all the dependencies). +// Note that to avoid priority inversion `job_func` should use `self->priority` to schedule new jobs in AsyncLoader or any other pool. +// Value stored in load job priority field is atomic and can be increased even during job execution. +// +// When task is scheduled. It can contain dependencies on previously scheduled jobs. These jobs can have any status. +// The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all dependent jobs are immediately canceled. class AsyncLoader : private boost::noncopyable { private: @@ -685,8 +723,6 @@ private: mutable std::mutex mutex; bool is_running = false; - // TODO(serxa): add metrics for number of jobs in every state - // Full set of scheduled pending jobs along with scheduling info std::unordered_map scheduled_jobs; From 4636a45a04c75fee78c5347ab31c0409e0e70838 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 12:13:23 +0000 Subject: [PATCH 025/628] fix --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 9e54b3ae75f..aca35b599f4 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -172,7 +172,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // Note that to avoid priority inversion `job_func` should use `self->priority` to schedule new jobs in AsyncLoader or any other pool. // Value stored in load job priority field is atomic and can be increased even during job execution. // -// When task is scheduled. It can contain dependencies on previously scheduled jobs. These jobs can have any status. +// When task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status. // The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all dependent jobs are immediately canceled. class AsyncLoader : private boost::noncopyable { From feb690d55f554622954cef1535ed29e43d148be6 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sun, 16 Apr 2023 15:09:44 +0200 Subject: [PATCH 026/628] Update src/Common/AsyncLoader.h --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index aca35b599f4..1423b7f83c7 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -154,7 +154,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // 2) Job is scheduled and placed into a task. Scheduled job may be ready (i.e. have all its dependencies finished) or blocked. // 3a) When all dependencies are successfully executed, job became ready. Ready job is enqueued into the ready queue. // 3b) If at least one of job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). -// On cancelation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. Job status is changed to CANCELED. +// On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. Job status is changed to CANCELED. // Exception is rethrown by any existing or new `wait()` call. Job is moved to the set of finished jobs. // 4) Scheduled pending ready job starts execution by a worker. Job is dequeuedCallback `job_func` is called. // Status of an executing job is PENDING. And it is still considered as scheduled job by AsyncLoader. From eab312e0b45939f2348ffef2d16e28fc4826771a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Sun, 16 Apr 2023 15:09:55 +0200 Subject: [PATCH 027/628] Update src/Common/AsyncLoader.h --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 1423b7f83c7..34de4fe3c8b 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -147,7 +147,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // Here we have created and scheduled a task consisting of two jobs. Job1 has no dependencies and is run first. // Job2 and job3 depends on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: // async_loader->prioritize(job3, 1); // higher priority jobs are run first -// job3->wait(); // blocks until job completion or cancelation and rethrow an exception (if any) +// job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) // // AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: // 1) Job is constructed with PENDING status. From b55d5a869df6c2c027c72246395f35cd609d6aec Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 18:17:11 +0000 Subject: [PATCH 028/628] split AsyncLoader into .h and .cpp files + more docs --- src/Common/AsyncLoader.cpp | 560 +++++++++++++++++++++ src/Common/AsyncLoader.h | 632 +++--------------------- src/Common/tests/gtest_async_loader.cpp | 4 +- 3 files changed, 641 insertions(+), 555 deletions(-) create mode 100644 src/Common/AsyncLoader.cpp diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp new file mode 100644 index 00000000000..09dd4f129ae --- /dev/null +++ b/src/Common/AsyncLoader.cpp @@ -0,0 +1,560 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ASYNC_LOAD_SCHEDULE_FAILED; + extern const int ASYNC_LOAD_FAILED; + extern const int ASYNC_LOAD_CANCELED; +} + + +LoadStatus LoadJob::status() const +{ + std::unique_lock lock{mutex}; + return load_status; +} + +std::exception_ptr LoadJob::exception() const +{ + std::unique_lock lock{mutex}; + return load_exception; +} + +ssize_t LoadJob::priority() const +{ + return load_priority; +} + +void LoadJob::wait() const +{ + std::unique_lock lock{mutex}; + waiters++; + finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); + waiters--; + if (load_exception) + std::rethrow_exception(load_exception); +} + +void LoadJob::waitNoThrow() const noexcept +{ + std::unique_lock lock{mutex}; + waiters++; + finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); + waiters--; +} + +size_t LoadJob::waiters_count() const +{ + std::unique_lock lock{mutex}; + return waiters; +} + +void LoadJob::ok() +{ + std::unique_lock lock{mutex}; + load_status = LoadStatus::OK; + if (waiters > 0) + finished.notify_all(); +} + +void LoadJob::failed(const std::exception_ptr & ptr) +{ + std::unique_lock lock{mutex}; + load_status = LoadStatus::FAILED; + load_exception = ptr; + if (waiters > 0) + finished.notify_all(); +} + +void LoadJob::canceled(const std::exception_ptr & ptr) +{ + std::unique_lock lock{mutex}; + load_status = LoadStatus::CANCELED; + load_exception = ptr; + if (waiters > 0) + finished.notify_all(); +} + + +AsyncLoader::Task::Task() + : loader(nullptr) +{} + +AsyncLoader::Task::Task(AsyncLoader * loader_, LoadJobSet && jobs_) + : loader(loader_) + , jobs(std::move(jobs_)) +{} + +AsyncLoader::Task::Task(AsyncLoader::Task && o) noexcept + : loader(std::exchange(o.loader, nullptr)) + , jobs(std::move(o.jobs)) +{} + +AsyncLoader::Task::~Task() +{ + remove(); +} + +AsyncLoader::Task & AsyncLoader::Task::operator=(AsyncLoader::Task && o) noexcept +{ + loader = std::exchange(o.loader, nullptr); + jobs = std::move(o.jobs); + return *this; +} + +void AsyncLoader::Task::merge(AsyncLoader::Task && o) +{ + if (!loader) + { + *this = std::move(o); + } + else + { + chassert(loader == o.loader); + jobs.merge(o.jobs); + o.loader = nullptr; + } +} + +void AsyncLoader::Task::remove() +{ + if (loader) + { + loader->remove(jobs); + detach(); + } +} + +void AsyncLoader::Task::detach() +{ + loader = nullptr; + jobs.clear(); +} + +AsyncLoader::AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_) + : log_failures(log_failures_) + , max_threads(max_threads_) + , pool(metric_threads, metric_active_threads, max_threads) +{} + +AsyncLoader::~AsyncLoader() +{ + stop(); +} + +void AsyncLoader::start() +{ + std::unique_lock lock{mutex}; + is_running = true; + for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) + spawn(lock); +} + +void AsyncLoader::wait() +{ + pool.wait(); +} + +void AsyncLoader::stop() +{ + { + std::unique_lock lock{mutex}; + is_running = false; + // NOTE: there is no need to notify because workers never wait + } + pool.wait(); +} + +AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) +{ + std::unique_lock lock{mutex}; + + // Sanity checks + for (const auto & job : jobs) + { + if (job->status() != LoadStatus::PENDING) + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Trying to schedule already finished load job '{}'", job->name); + if (scheduled_jobs.contains(job)) + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job '{}' has been already scheduled", job->name); + } + + // Ensure scheduled_jobs graph will have no cycles. The only way to get a cycle is to add a cycle, assuming old jobs cannot reference new ones. + checkCycle(jobs, lock); + + // We do not want any exception to be throws after this point, because the following code is not exception-safe + DENY_ALLOCATIONS_IN_SCOPE; + + // Schedule all incoming jobs + for (const auto & job : jobs) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + scheduled_jobs.emplace(job, Info{.initial_priority = priority, .priority = priority}); + }); + job->load_priority.store(priority); // Set user-facing priority + } + + // Process dependencies on scheduled pending jobs + for (const auto & job : jobs) + { + Info & info = scheduled_jobs.find(job)->second; + for (const auto & dep : job->dependencies) + { + // Register every dependency on scheduled job with back-link to dependent job + if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + dep_info->second.dependent_jobs.insert(job); + }); + info.dependencies_left++; + + // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion + prioritize(dep, priority, lock); + } + } + + // Enqueue non-blocked jobs (w/o dependencies) to ready queue + if (!info.is_blocked()) + enqueue(info, job, lock); + } + + // Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any). + for (const auto & job : jobs) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + for (const auto & dep : job->dependencies) + { + if (scheduled_jobs.contains(dep)) + continue; // Skip dependencies on scheduled pending jobs (already processed) + LoadStatus dep_status = dep->status(); + if (dep_status == LoadStatus::OK) + continue; // Dependency on already successfully finished job -- it's okay. + + if (dep_status == LoadStatus::PENDING) + { + // Dependency on not scheduled pending job -- it's bad. + // Probably, there is an error in `jobs` set: not all jobs were passed to `schedule()` call. + // We are not going to run any dependent job, so cancel them all. + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> Load job '{}': not scheduled pending load job (it must be also scheduled), all dependent load jobs are canceled", + job->name, + dep->name)); + }); + finish(lock, job, LoadStatus::CANCELED, e); + break; // This job is now finished, stop its dependencies processing + } + if (dep_status == LoadStatus::FAILED || dep_status == LoadStatus::CANCELED) + { + // Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs. + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> {}", + job->name, + getExceptionMessage(dep->exception(), /* with_stack_trace = */ false))); + }); + finish(lock, job, LoadStatus::CANCELED, e); + break; // This job is now finished, stop its dependencies processing + } + } + } + else + { + // Job was already canceled on previous iteration of this cycle -- skip + } + } + + return Task(this, std::move(jobs)); +} + +void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority) +{ + DENY_ALLOCATIONS_IN_SCOPE; + std::unique_lock lock{mutex}; + prioritize(job, new_priority, lock); +} + +void AsyncLoader::remove(const LoadJobSet & jobs) +{ + DENY_ALLOCATIONS_IN_SCOPE; + std::unique_lock lock{mutex}; + // On the first pass: + // - cancel all not executing jobs to avoid races + // - do not wait executing jobs (otherwise, on unlock a worker could start executing a dependent job, that should be canceled) + for (const auto & job : jobs) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + if (info->second.is_executing()) + continue; // Skip executing jobs on the first pass + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); + }); + finish(lock, job, LoadStatus::CANCELED, e); + } + } + // On the second pass wait for executing jobs to finish + for (const auto & job : jobs) + { + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + // Job is currently executing + chassert(info->second.is_executing()); + lock.unlock(); + job->waitNoThrow(); // Wait for job to finish + lock.lock(); + } + } + // On the third pass all jobs are finished - remove them all + // It is better to do it under one lock to avoid exposing intermediate states + for (const auto & job : jobs) + finished_jobs.erase(job); +} + +void AsyncLoader::setMaxThreads(size_t value) +{ + std::unique_lock lock{mutex}; + pool.setMaxThreads(value); + pool.setMaxFreeThreads(value); + pool.setQueueSize(value); + max_threads = value; + if (!is_running) + return; + for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) + spawn(lock); +} + +size_t AsyncLoader::getMaxThreads() const +{ + std::unique_lock lock{mutex}; + return max_threads; +} + +size_t AsyncLoader::getScheduledJobCount() const +{ + std::unique_lock lock{mutex}; + return scheduled_jobs.size(); +} + +void AsyncLoader::checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) +{ + LoadJobSet left = jobs; + LoadJobSet visited; + visited.reserve(left.size()); + while (!left.empty()) + { + LoadJobPtr job = *left.begin(); + checkCycleImpl(job, left, visited, lock); + } +} + +String AsyncLoader::checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock) +{ + if (!left.contains(job)) + return {}; // Do not consider external dependencies and already processed jobs + if (auto [_, inserted] = visited.insert(job); !inserted) + { + visited.erase(job); // Mark where cycle ends + return job->name; + } + for (const auto & dep : job->dependencies) + { + if (auto chain = checkCycleImpl(dep, left, visited, lock); !chain.empty()) + { + if (!visited.contains(job)) // Check for cycle end + throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job dependency cycle detected: {} -> {}", job->name, chain); + else + return fmt::format("{} -> {}", job->name, chain); // chain is not a cycle yet -- continue building + } + } + left.erase(job); + return {}; +} + +void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job) +{ + if (status == LoadStatus::OK) + { + // Notify waiters + job->ok(); + + // Update dependent jobs and enqueue if ready + chassert(scheduled_jobs.contains(job)); // Job was pending + for (const auto & dep : scheduled_jobs[job].dependent_jobs) + { + chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending + Info & dep_info = scheduled_jobs[dep]; + dep_info.dependencies_left--; + if (!dep_info.is_blocked()) + enqueue(dep_info, dep, lock); + } + } + else + { + // Notify waiters + if (status == LoadStatus::FAILED) + job->failed(exception_from_job); + else if (status == LoadStatus::CANCELED) + job->canceled(exception_from_job); + + chassert(scheduled_jobs.contains(job)); // Job was pending + Info & info = scheduled_jobs[job]; + if (info.is_ready()) + { + ready_queue.erase(info.key()); + info.ready_seqno = 0; + } + + // Recurse into all dependent jobs + LoadJobSet dependent; + dependent.swap(info.dependent_jobs); // To avoid container modification during recursion + for (const auto & dep : dependent) + { + std::exception_ptr e; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + e = std::make_exception_ptr( + Exception(ErrorCodes::ASYNC_LOAD_CANCELED, + "Load job '{}' -> {}", + dep->name, + getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); + }); + finish(lock, dep, LoadStatus::CANCELED, e); + } + + // Clean dependency graph edges + for (const auto & dep : job->dependencies) + if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) + dep_info->second.dependent_jobs.erase(job); + } + + // Job became finished + scheduled_jobs.erase(job); + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + finished_jobs.insert(job); + }); +} + +void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) +{ + if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) + { + if (info->second.priority >= new_priority) + return; // Never lower priority + + // Update priority and push job forward through ready queue if needed + if (info->second.ready_seqno) + ready_queue.erase(info->second.key()); + info->second.priority = new_priority; + job->load_priority.store(new_priority); // Set user-facing priority (may affect executing jobs) + if (info->second.ready_seqno) + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + ready_queue.emplace(info->second.key(), job); + }); + } + + // Recurse into dependencies + for (const auto & dep : job->dependencies) + prioritize(dep, new_priority, lock); + } +} + +void AsyncLoader::enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) +{ + chassert(!info.is_blocked()); + chassert(info.ready_seqno == 0); + info.ready_seqno = ++last_ready_seqno; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + ready_queue.emplace(info.key(), job); + }); + + if (is_running && workers < max_threads) + spawn(lock); +} + +void AsyncLoader::spawn(std::unique_lock &) +{ + workers++; + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + pool.scheduleOrThrowOnError([this] { worker(); }); + }); +} + +void AsyncLoader::worker() +{ + DENY_ALLOCATIONS_IN_SCOPE; + + LoadJobPtr job; + std::exception_ptr exception_from_job; + while (true) + { + // This is inside the loop to also reset previous thread names set inside the jobs + setThreadName("AsyncLoader"); + + { + std::unique_lock lock{mutex}; + + // Handle just executed job + if (exception_from_job) + finish(lock, job, LoadStatus::FAILED, exception_from_job); + else if (job) + finish(lock, job, LoadStatus::OK); + + if (!is_running || ready_queue.empty() || workers > max_threads) + { + workers--; + return; + } + + // Take next job to be executed from the ready queue + auto it = ready_queue.begin(); + job = it->second; + ready_queue.erase(it); + scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue + } + + ALLOW_ALLOCATIONS_IN_SCOPE; + + try + { + job->func(job); + exception_from_job = {}; + } + catch (...) + { + NOEXCEPT_SCOPE({ + if (log_failures) + tryLogCurrentException(__PRETTY_FUNCTION__); + exception_from_job = std::make_exception_ptr( + Exception(ErrorCodes::ASYNC_LOAD_FAILED, + "Load job '{}' failed: {}", + job->name, + getCurrentExceptionMessage(/* with_stacktrace = */ true))); + }); + } + } +} + +} diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 34de4fe3c8b..bf35b5de736 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -8,14 +8,9 @@ #include #include #include -#include #include #include #include -#include -#include -#include -#include namespace DB { @@ -26,21 +21,18 @@ using LoadJobSet = std::unordered_set; class LoadTask; class AsyncLoader; -namespace ErrorCodes -{ - extern const int ASYNC_LOAD_SCHEDULE_FAILED; - extern const int ASYNC_LOAD_FAILED; - extern const int ASYNC_LOAD_CANCELED; -} - +// Execution status of a load job. enum class LoadStatus { - PENDING, // Load job is not started yet - OK, // Load job executed and was successful - FAILED, // Load job executed and failed - CANCELED // Load job is not going to be executed due to removal or dependency failure + PENDING, // Load job is not started yet. + OK, // Load job executed and was successful. + FAILED, // Load job executed and failed. + CANCELED // Load job is not going to be executed due to removal or dependency failure. }; +// Smallest indivisible part of a loading process. Load job can have multiple dependencies, thus jobs constitute a direct acyclic graph (DAG). +// Job encapsulates a function to be executed by `AsyncLoader` as soon as job functions of all dependencies are successfully executed. +// Job can be waited for by an arbitrary number of threads. See `AsyncLoader` class description for more details. class LoadJob : private boost::noncopyable { public: @@ -51,76 +43,35 @@ public: , func(std::forward(func_)) {} - LoadStatus status() const - { - std::unique_lock lock{mutex}; - return load_status; - } + // Current job status. + LoadStatus status() const; + std::exception_ptr exception() const; - std::exception_ptr exception() const - { - std::unique_lock lock{mutex}; - return load_exception; - } + // Returns current value of a priority of the job. May differ from initial priority passed to `AsyncLoader:::schedule()` call. + ssize_t priority() const; - void wait() const - { - std::unique_lock lock{mutex}; - waiters++; - finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); - waiters--; - if (load_exception) - std::rethrow_exception(load_exception); - } + // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. + // Throws if job is FAILED or CANCELED. Returns or throws immediately on non-pending job. + void wait() const; - void waitNoThrow() const - { - std::unique_lock lock{mutex}; - waiters++; - finished.wait(lock, [this] { return load_status != LoadStatus::PENDING; }); - waiters--; - } + // Wait for a job to reach any non PENDING status. + void waitNoThrow() const noexcept; - size_t waiters_count() const - { - std::unique_lock lock{mutex}; - return waiters; - } + // Returns number of threads blocked by `wait()` or `waitNoThrow()` calls. + size_t waiters_count() const; - const LoadJobSet dependencies; // jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard + const LoadJobSet dependencies; // Jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard const String name; - std::atomic priority{0}; private: friend class AsyncLoader; - void ok() - { - std::unique_lock lock{mutex}; - load_status = LoadStatus::OK; - if (waiters > 0) - finished.notify_all(); - } - - void failed(const std::exception_ptr & ptr) - { - std::unique_lock lock{mutex}; - load_status = LoadStatus::FAILED; - load_exception = ptr; - if (waiters > 0) - finished.notify_all(); - } - - void canceled(const std::exception_ptr & ptr) - { - std::unique_lock lock{mutex}; - load_status = LoadStatus::CANCELED; - load_exception = ptr; - if (waiters > 0) - finished.notify_all(); - } + void ok(); + void failed(const std::exception_ptr & ptr); + void canceled(const std::exception_ptr & ptr); std::function func; + std::atomic load_priority{0}; mutable std::mutex mutex; mutable std::condition_variable finished; @@ -177,7 +128,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && class AsyncLoader : private boost::noncopyable { private: - // Key of a pending job in ready queue + // Key of a pending job in the ready queue. struct ReadyKey { ssize_t priority; // Ascending order @@ -198,14 +149,14 @@ private: } }; - // Scheduling information for a pending job + // Scheduling information for a pending job. struct Info { - ssize_t initial_priority = 0; // Initial priority passed into schedule() - ssize_t priority = 0; // Elevated priority, due to priority inheritance or prioritize() - size_t dependencies_left = 0; - UInt64 ready_seqno = 0; // Zero means that job is not in ready queue - LoadJobSet dependent_jobs; + ssize_t initial_priority = 0; // Initial priority passed into schedule(). + ssize_t priority = 0; // Elevated priority, due to priority inheritance or prioritize(). + size_t dependencies_left = 0; // Current number of dependencies on pending jobs. + UInt64 ready_seqno = 0; // Zero means that job is not in ready queue. + LoadJobSet dependent_jobs; // Set of jobs dependent on this job. // Three independent states of a non-finished jobs bool is_blocked() const { return dependencies_left > 0; } @@ -222,521 +173,96 @@ private: public: using Metric = CurrentMetrics::Metric; - // Helper class that removes all not started jobs in destructor and wait all executing jobs to finish + // Helper class that removes all not started and finished jobs in destructor and waits for all the executing jobs to finish. class Task { public: - Task() - : loader(nullptr) - {} - - Task(AsyncLoader * loader_, LoadJobSet && jobs_) - : loader(loader_) - , jobs(std::move(jobs_)) - {} - + Task(); + Task(AsyncLoader * loader_, LoadJobSet && jobs_); Task(const Task & o) = delete; + Task(Task && o) noexcept; Task & operator=(const Task & o) = delete; + ~Task(); + Task & operator=(Task && o) noexcept; - Task(Task && o) noexcept - : loader(std::exchange(o.loader, nullptr)) - , jobs(std::move(o.jobs)) - {} + // Merge all jobs from other task into this task. Useful for merging jobs with different priorities into one task. + void merge(Task && o); - Task & operator=(Task && o) noexcept - { - loader = std::exchange(o.loader, nullptr); - jobs = std::move(o.jobs); - return *this; - } + // Remove all jobs from AsyncLoader. + void remove(); - void merge(Task && o) - { - if (!loader) - { - *this = std::move(o); - } - else - { - chassert(loader == o.loader); - jobs.merge(o.jobs); - o.loader = nullptr; - } - } - - ~Task() - { - remove(); - } - - void remove() - { - if (loader) - { - loader->remove(jobs); - detach(); - } - } - - // Do not track jobs in this task - // WARNING: Jobs will never be removed() and are going to be stored as finished_jobs until ~AsyncLoader() - void detach() - { - loader = nullptr; - jobs.clear(); - } + // Do not track jobs in this task. + // WARNING: Jobs will never be removed() and are going to be stored as finished_jobs until ~AsyncLoader(). + void detach(); private: AsyncLoader * loader; LoadJobSet jobs; }; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_) - : log_failures(log_failures_) - , max_threads(max_threads_) - , pool(metric_threads, metric_active_threads, max_threads) - {} + AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_); - // WARNING: all Task instances returned by `schedule()` should be destructed before AsyncLoader - ~AsyncLoader() - { - stop(); - } + // WARNING: all Task instances returned by `schedule()` should be destructed before AsyncLoader. + ~AsyncLoader(); - // Start workers to execute scheduled load jobs - void start() - { - std::unique_lock lock{mutex}; - is_running = true; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); - } + // Start workers to execute scheduled load jobs. + void start(); // Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs. - void wait() - { - pool.wait(); - } + void wait(); // Wait for currently executing jobs to finish, but do not run any other pending jobs. // Not finished jobs are left in pending state: // - they can be resumed by calling start() again; // - or canceled using ~Task() or remove() later. - void stop() - { - { - std::unique_lock lock{mutex}; - is_running = false; - // NOTE: there is no need to notify because workers never wait - } - pool.wait(); - } + void stop(); - [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0) - { - std::unique_lock lock{mutex}; - - // Sanity checks - for (const auto & job : jobs) - { - if (job->status() != LoadStatus::PENDING) - throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Trying to schedule already finished load job '{}'", job->name); - if (scheduled_jobs.contains(job)) - throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job '{}' has been already scheduled", job->name); - } - - // Ensure scheduled_jobs graph will have no cycles. The only way to get a cycle is to add a cycle, assuming old jobs cannot reference new ones. - checkCycle(jobs, lock); - - // We do not want any exception to be throws after this point, because the following code is not exception-safe - DENY_ALLOCATIONS_IN_SCOPE; - - // Schedule all incoming jobs - for (const auto & job : jobs) - { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - scheduled_jobs.emplace(job, Info{.initial_priority = priority, .priority = priority}); - }); - job->priority.store(priority); // Set user-facing priority - } - - // Process dependencies on scheduled pending jobs - for (const auto & job : jobs) - { - Info & info = scheduled_jobs.find(job)->second; - for (const auto & dep : job->dependencies) - { - // Register every dependency on scheduled job with back-link to dependent job - if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) - { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - dep_info->second.dependent_jobs.insert(job); - }); - info.dependencies_left++; - - // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion - prioritize(dep, priority, lock); - } - } - - // Enqueue non-blocked jobs (w/o dependencies) to ready queue - if (!info.is_blocked()) - enqueue(info, job, lock); - } - - // Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any). - for (const auto & job : jobs) - { - if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) - { - for (const auto & dep : job->dependencies) - { - if (scheduled_jobs.contains(dep)) - continue; // Skip dependencies on scheduled pending jobs (already processed) - LoadStatus dep_status = dep->status(); - if (dep_status == LoadStatus::OK) - continue; // Dependency on already successfully finished job -- it's okay. - - if (dep_status == LoadStatus::PENDING) - { - // Dependency on not scheduled pending job -- it's bad. - // Probably, there is an error in `jobs` set: not all jobs were passed to `schedule()` call. - // We are not going to run any dependent job, so cancel them all. - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' -> Load job '{}': not scheduled pending load job (it must be also scheduled), all dependent load jobs are canceled", - job->name, - dep->name)); - }); - finish(lock, job, LoadStatus::CANCELED, e); - break; // This job is now finished, stop its dependencies processing - } - if (dep_status == LoadStatus::FAILED || dep_status == LoadStatus::CANCELED) - { - // Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs. - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' -> {}", - job->name, - getExceptionMessage(dep->exception(), /* with_stack_trace = */ false))); - }); - finish(lock, job, LoadStatus::CANCELED, e); - break; // This job is now finished, stop its dependencies processing - } - } - } - else - { - // Job was already canceled on previous iteration of this cycle -- skip - } - } - - return Task(this, std::move(jobs)); - } + // Schedule all `jobs` with given `priority` and return task containing these jobs. + // Higher priority jobs (with greater `priority` value) are executed earlier. + // All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job + // never wait for (blocked by) lower priority jobs (no priority inversion). + [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0); // Increase priority of a job and all its dependencies recursively - void prioritize(const LoadJobPtr & job, ssize_t new_priority) - { - DENY_ALLOCATIONS_IN_SCOPE; - std::unique_lock lock{mutex}; - prioritize(job, new_priority, lock); - } + void prioritize(const LoadJobPtr & job, ssize_t new_priority); - // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them - void remove(const LoadJobSet & jobs) - { - DENY_ALLOCATIONS_IN_SCOPE; - std::unique_lock lock{mutex}; - // On the first pass: - // - cancel all not executing jobs to avoid races - // - do not wait executing jobs (otherwise, on unlock a worker could start executing a dependent job, that should be canceled) - for (const auto & job : jobs) - { - if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) - { - if (info->second.is_executing()) - continue; // Skip executing jobs on the first pass - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); - }); - finish(lock, job, LoadStatus::CANCELED, e); - } - } - // On the second pass wait for executing jobs to finish - for (const auto & job : jobs) - { - if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) - { - // Job is currently executing - chassert(info->second.is_executing()); - lock.unlock(); - job->waitNoThrow(); // Wait for job to finish - lock.lock(); - } - } - // On the third pass all jobs are finished - remove them all - // It is better to do it under one lock to avoid exposing intermediate states - for (const auto & job : jobs) - finished_jobs.erase(job); - } + // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them. + void remove(const LoadJobSet & jobs); - void setMaxThreads(size_t value) - { - std::unique_lock lock{mutex}; - pool.setMaxThreads(value); - pool.setMaxFreeThreads(value); - pool.setQueueSize(value); - max_threads = value; - if (!is_running) - return; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); - } + // Increase or decrease maximum number of simultaneously executing jobs. + void setMaxThreads(size_t value); - size_t getMaxThreads() const - { - std::unique_lock lock{mutex}; - return max_threads; - } - - size_t getScheduledJobCount() const - { - std::unique_lock lock{mutex}; - return scheduled_jobs.size(); - } + size_t getMaxThreads() const; + size_t getScheduledJobCount() const; private: - void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) - { - LoadJobSet left = jobs; - LoadJobSet visited; - visited.reserve(left.size()); - while (!left.empty()) - { - LoadJobPtr job = *left.begin(); - checkCycleImpl(job, left, visited, lock); - } - } + void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); + String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); + void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}); + void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock); + void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock); + void spawn(std::unique_lock &); + void worker(); - String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock) - { - if (!left.contains(job)) - return {}; // Do not consider external dependencies and already processed jobs - if (auto [_, inserted] = visited.insert(job); !inserted) - { - visited.erase(job); // Mark where cycle ends - return job->name; - } - for (const auto & dep : job->dependencies) - { - if (auto chain = checkCycleImpl(dep, left, visited, lock); !chain.empty()) - { - if (!visited.contains(job)) // Check for cycle end - throw Exception(ErrorCodes::ASYNC_LOAD_SCHEDULE_FAILED, "Load job dependency cycle detected: {} -> {}", job->name, chain); - else - return fmt::format("{} -> {}", job->name, chain); // chain is not a cycle yet -- continue building - } - } - left.erase(job); - return {}; - } + const bool log_failures; // Worker should log all exceptions caught from job functions. - void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}) - { - if (status == LoadStatus::OK) - { - // Notify waiters - job->ok(); - - // Update dependent jobs and enqueue if ready - chassert(scheduled_jobs.contains(job)); // Job was pending - for (const auto & dep : scheduled_jobs[job].dependent_jobs) - { - chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending - Info & dep_info = scheduled_jobs[dep]; - dep_info.dependencies_left--; - if (!dep_info.is_blocked()) - enqueue(dep_info, dep, lock); - } - } - else - { - // Notify waiters - if (status == LoadStatus::FAILED) - job->failed(exception_from_job); - else if (status == LoadStatus::CANCELED) - job->canceled(exception_from_job); - - chassert(scheduled_jobs.contains(job)); // Job was pending - Info & info = scheduled_jobs[job]; - if (info.is_ready()) - { - ready_queue.erase(info.key()); - info.ready_seqno = 0; - } - - // Recurse into all dependent jobs - LoadJobSet dependent; - dependent.swap(info.dependent_jobs); // To avoid container modification during recursion - for (const auto & dep : dependent) - { - std::exception_ptr e; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - e = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_CANCELED, - "Load job '{}' -> {}", - dep->name, - getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); - }); - finish(lock, dep, LoadStatus::CANCELED, e); - } - - // Clean dependency graph edges - for (const auto & dep : job->dependencies) - if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) - dep_info->second.dependent_jobs.erase(job); - } - - // Job became finished - scheduled_jobs.erase(job); - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - finished_jobs.insert(job); - }); - } - - void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) - { - if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) - { - if (info->second.priority >= new_priority) - return; // Never lower priority - - // Update priority and push job forward through ready queue if needed - if (info->second.ready_seqno) - ready_queue.erase(info->second.key()); - info->second.priority = new_priority; - job->priority.store(new_priority); // Set user-facing priority (may affect executing jobs) - if (info->second.ready_seqno) - { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info->second.key(), job); - }); - } - - // Recurse into dependencies - for (const auto & dep : job->dependencies) - prioritize(dep, new_priority, lock); - } - } - - void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) - { - chassert(!info.is_blocked()); - chassert(info.ready_seqno == 0); - info.ready_seqno = ++last_ready_seqno; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info.key(), job); - }); - - if (is_running && workers < max_threads) - spawn(lock); - } - - void spawn(std::unique_lock &) - { - workers++; - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - pool.scheduleOrThrowOnError([this] { worker(); }); - }); - } - - void worker() - { - DENY_ALLOCATIONS_IN_SCOPE; - - LoadJobPtr job; - std::exception_ptr exception_from_job; - while (true) - { - /// This is inside the loop to also reset previous thread names set inside the jobs - setThreadName("AsyncLoader"); - - { - std::unique_lock lock{mutex}; - - // Handle just executed job - if (exception_from_job) - finish(lock, job, LoadStatus::FAILED, exception_from_job); - else if (job) - finish(lock, job, LoadStatus::OK); - - if (!is_running || ready_queue.empty() || workers > max_threads) - { - workers--; - return; - } - - // Take next job to be executed from the ready queue - auto it = ready_queue.begin(); - job = it->second; - ready_queue.erase(it); - scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue - } - - ALLOW_ALLOCATIONS_IN_SCOPE; - - try - { - job->func(job); - exception_from_job = {}; - } - catch (...) - { - NOEXCEPT_SCOPE({ - if (log_failures) - tryLogCurrentException(__PRETTY_FUNCTION__); - exception_from_job = std::make_exception_ptr( - Exception(ErrorCodes::ASYNC_LOAD_FAILED, - "Load job '{}' failed: {}", - job->name, - getCurrentExceptionMessage(/* with_stacktrace = */ true))); - }); - } - } - } - - const bool log_failures; - - mutable std::mutex mutex; + mutable std::mutex mutex; // Guards all the fields below. bool is_running = false; - // Full set of scheduled pending jobs along with scheduling info + // Full set of scheduled pending jobs along with scheduling info. std::unordered_map scheduled_jobs; - // Subset of scheduled pending jobs with resolved dependencies (waiting for a thread to be executed) - // Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities + // Subset of scheduled pending non-blocked jobs (waiting for a worker to be executed). + // Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities. std::map ready_queue; - // Set of finished jobs (for introspection only, until job is removed) + // Set of finished jobs (for introspection only, until jobs are removed). LoadJobSet finished_jobs; - // Increasing counter for ReadyKey assignment (to preserve FIFO order of jobs with equal priority) + // Increasing counter for `ReadyKey` assignment (to preserve FIFO order of the jobs with equal priorities). UInt64 last_ready_seqno = 0; - // For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs + // For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs. size_t max_threads; size_t workers = 0; ThreadPool pool; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 52cc3436595..e6baa87c182 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -115,7 +115,7 @@ TEST(AsyncLoader, Smoke) auto job_func = [&] (const LoadJobPtr & self) { jobs_done++; - if (self->priority == low_priority) + if (self->priority() == low_priority) low_priority_jobs_done++; }; @@ -560,7 +560,7 @@ TEST(AsyncLoader, StaticPriorities) auto job_func = [&] (const LoadJobPtr & self) { - schedule += fmt::format("{}{}", self->name, self->priority); + schedule += fmt::format("{}{}", self->name, self->priority()); }; std::vector jobs; From 883f5c4fd975e7602e74dff6139156a59947457e Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 18:18:50 +0000 Subject: [PATCH 029/628] fix comment --- src/Common/AsyncLoader.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index bf35b5de736..4b05dd40bf1 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -89,7 +89,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. // Basic usage example: // auto job_func = [&] (const LoadJobPtr & self) { -// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority); +// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority()); // }; // auto job1 = makeLoadJob({}, "job1", job_func); // auto job2 = makeLoadJob({ job1 }, "job2", job_func); @@ -120,7 +120,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated // (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited); // (b) or job was explicitly prioritized by `prioritize(job, higher_priority)` call (this also leads to a priority inheritance for all the dependencies). -// Note that to avoid priority inversion `job_func` should use `self->priority` to schedule new jobs in AsyncLoader or any other pool. +// Note that to avoid priority inversion `job_func` should use `self->priority()` to schedule new jobs in AsyncLoader or any other pool. // Value stored in load job priority field is atomic and can be increased even during job execution. // // When task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status. From 241c4766d3120c6113b4357f458f6ab6b87e9463 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 18:40:24 +0000 Subject: [PATCH 030/628] fix double finish() during cancellation of a job with multiple paths --- src/Common/AsyncLoader.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 09dd4f129ae..8dbfe27e5cc 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -427,6 +427,8 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & dependent.swap(info.dependent_jobs); // To avoid container modification during recursion for (const auto & dep : dependent) { + if (!scheduled_jobs.contains(dep)) + continue; // Job has already been canceled std::exception_ptr e; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; @@ -439,7 +441,7 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & finish(lock, dep, LoadStatus::CANCELED, e); } - // Clean dependency graph edges + // Clean dependency graph edges pointing to canceled jobs for (const auto & dep : job->dependencies) if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end()) dep_info->second.dependent_jobs.erase(job); From e7dcf91037c62f12afd5853dc9965219a2524284 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 19:28:14 +0000 Subject: [PATCH 031/628] add test for dynamic prioritization --- src/Common/AsyncLoader.cpp | 2 + src/Common/tests/gtest_async_loader.cpp | 56 +++++++++++++++++++++++++ 2 files changed, 58 insertions(+) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 8dbfe27e5cc..0734741f469 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -284,6 +284,8 @@ AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority) { + if (!job) + return; DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; prioritize(job, new_priority, lock); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index e6baa87c182..21a0ba1f41e 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -587,6 +587,62 @@ TEST(AsyncLoader, StaticPriorities) ASSERT_EQ(schedule, "A9E9D9F9G9H9C4B3"); } +TEST(AsyncLoader, DynamicPriorities) +{ + AsyncLoaderTest t(1); + + for (bool prioritize : {false, true}) + { + std::string schedule; + + LoadJobPtr job_to_prioritize; + + auto job_func = [&] (const LoadJobPtr & self) + { + if (prioritize && self->name == "C") + t.loader.prioritize(job_to_prioritize, 9); // dynamic prioritization + schedule += fmt::format("{}{}", self->name, self->priority()); + }; + + // Job DAG with initial priorities. During execution of C4, job G0 priority is increased to G9, postponing B3 job executing. + // A0 -+-> B3 + // | + // `-> C4 + // | + // `-> D1 -. + // | +-> F0 --> G0 --> H0 + // `-> E2 -' + std::vector jobs; + jobs.push_back(makeLoadJob({}, "A", job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, "B", job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, "C", job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, "D", job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, "E", job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, "G", job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, "H", job_func)); // 7 + auto task = t.loader.schedule({ jobs[0] }, 0); + task.merge(t.loader.schedule({ jobs[1] }, 3)); + task.merge(t.loader.schedule({ jobs[2] }, 4)); + task.merge(t.loader.schedule({ jobs[3] }, 1)); + task.merge(t.loader.schedule({ jobs[4] }, 2)); + task.merge(t.loader.schedule({ jobs[5] }, 0)); + task.merge(t.loader.schedule({ jobs[6] }, 0)); + task.merge(t.loader.schedule({ jobs[7] }, 0)); + + job_to_prioritize = jobs[6]; + + t.loader.start(); + t.loader.wait(); + t.loader.stop(); + + if (prioritize) + ASSERT_EQ(schedule, "A4C4E9D9F9G9B3H0"); + else + ASSERT_EQ(schedule, "A4C4B3E2D1F0G0H0"); + } +} + TEST(AsyncLoader, RandomIndependentTasks) { AsyncLoaderTest t(16); From 28f11299aa7721831635f0b91b3ff9d3bf4124ac Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 16 Apr 2023 21:15:44 +0000 Subject: [PATCH 032/628] fix tidy build --- src/Common/AsyncLoader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 0734741f469..8da41a03d3d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -266,7 +266,7 @@ AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' -> {}", job->name, - getExceptionMessage(dep->exception(), /* with_stack_trace = */ false))); + getExceptionMessage(dep->exception(), /* with_stacktrace = */ false))); }); finish(lock, job, LoadStatus::CANCELED, e); break; // This job is now finished, stop its dependencies processing @@ -438,7 +438,7 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' -> {}", dep->name, - getExceptionMessage(exception_from_job, /* with_stack_trace = */ false))); + getExceptionMessage(exception_from_job, /* with_stacktrace = */ false))); }); finish(lock, dep, LoadStatus::CANCELED, e); } From d08fc909cda280f89087690aac37ea65ed87976e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 17 Apr 2023 09:17:51 +0000 Subject: [PATCH 033/628] fix data race in test --- src/Common/tests/gtest_async_loader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 21a0ba1f41e..d6039162994 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -751,4 +751,5 @@ TEST(AsyncLoader, SetMaxThreads) t.loader.setMaxThreads(max_threads_values[sync_index]); syncs[idx]->arrive_and_wait(); // (B) this sync point is required to allow `executing` value to go back down to zero after we change number of workers } + t.loader.wait(); } From b07b2439468a01fbc1ed13867ed51d0e7c62de9f Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 17 Apr 2023 09:47:16 +0000 Subject: [PATCH 034/628] fix typos --- src/Common/AsyncLoader.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 4b05dd40bf1..383d4b5ef24 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -95,9 +95,9 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && // auto job2 = makeLoadJob({ job1 }, "job2", job_func); // auto job3 = makeLoadJob({ job1 }, "job3", job_func); // auto task = async_loader.schedule({ job1, job2, job3 }, /* priority = */ 0); -// Here we have created and scheduled a task consisting of two jobs. Job1 has no dependencies and is run first. -// Job2 and job3 depends on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: -// async_loader->prioritize(job3, 1); // higher priority jobs are run first +// Here we have created and scheduled a task consisting of three jobs. Job1 has no dependencies and is run first. +// Job2 and job3 depend on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: +// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first // job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) // // AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: From 6edd7a72fd17eaef0f4197c6fb08636be2127500 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 17 Apr 2023 10:03:54 +0000 Subject: [PATCH 035/628] fix more comments --- src/Common/AsyncLoader.h | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 383d4b5ef24..d3de883f757 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -158,7 +158,7 @@ private: UInt64 ready_seqno = 0; // Zero means that job is not in ready queue. LoadJobSet dependent_jobs; // Set of jobs dependent on this job. - // Three independent states of a non-finished jobs + // Three independent states of a non-finished job. bool is_blocked() const { return dependencies_left > 0; } bool is_ready() const { return dependencies_left == 0 && ready_seqno > 0; } bool is_executing() const { return dependencies_left == 0 && ready_seqno == 0; } @@ -188,11 +188,11 @@ public: // Merge all jobs from other task into this task. Useful for merging jobs with different priorities into one task. void merge(Task && o); - // Remove all jobs from AsyncLoader. + // Remove all jobs of this task from AsyncLoader. void remove(); // Do not track jobs in this task. - // WARNING: Jobs will never be removed() and are going to be stored as finished_jobs until ~AsyncLoader(). + // WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader(). void detach(); private: @@ -213,17 +213,19 @@ public: // Wait for currently executing jobs to finish, but do not run any other pending jobs. // Not finished jobs are left in pending state: - // - they can be resumed by calling start() again; + // - they can be executed by calling start() again; // - or canceled using ~Task() or remove() later. void stop(); - // Schedule all `jobs` with given `priority` and return task containing these jobs. + // Schedule all `jobs` with given `priority` and return a task containing these jobs. // Higher priority jobs (with greater `priority` value) are executed earlier. // All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job // never wait for (blocked by) lower priority jobs (no priority inversion). + // Returned task destructor ensures that all the `jobs` are finished (OK, FAILED or CANCELED) + // and are removed from AsyncLoader, so it is thread-safe to destroy them. [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0); - // Increase priority of a job and all its dependencies recursively + // Increase priority of a job and all its dependencies recursively. void prioritize(const LoadJobPtr & job, ssize_t new_priority); // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them. From 8ae2f727754a867f498ea2527ac8a39d1dfe875d Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 17 Apr 2023 10:11:59 +0000 Subject: [PATCH 036/628] better job shutdown --- src/Common/AsyncLoader.cpp | 12 ++++++++---- src/Common/AsyncLoader.h | 1 + 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 8da41a03d3d..d382100a203 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -62,8 +62,7 @@ void LoadJob::ok() { std::unique_lock lock{mutex}; load_status = LoadStatus::OK; - if (waiters > 0) - finished.notify_all(); + finish(); } void LoadJob::failed(const std::exception_ptr & ptr) @@ -71,8 +70,7 @@ void LoadJob::failed(const std::exception_ptr & ptr) std::unique_lock lock{mutex}; load_status = LoadStatus::FAILED; load_exception = ptr; - if (waiters > 0) - finished.notify_all(); + finish(); } void LoadJob::canceled(const std::exception_ptr & ptr) @@ -80,6 +78,12 @@ void LoadJob::canceled(const std::exception_ptr & ptr) std::unique_lock lock{mutex}; load_status = LoadStatus::CANCELED; load_exception = ptr; + finish(); +} + +void LoadJob::finish() +{ + func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` and `task->wait()` return if (waiters > 0) finished.notify_all(); } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d3de883f757..2e2c605cfe6 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -69,6 +69,7 @@ private: void ok(); void failed(const std::exception_ptr & ptr); void canceled(const std::exception_ptr & ptr); + void finish(); std::function func; std::atomic load_priority{0}; From cf7a341489e2dee44b982a2fb741283799f9b0d9 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 17 Apr 2023 10:16:01 +0000 Subject: [PATCH 037/628] typo --- src/Common/AsyncLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index d382100a203..7fe88ce7212 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -83,7 +83,7 @@ void LoadJob::canceled(const std::exception_ptr & ptr) void LoadJob::finish() { - func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` and `task->wait()` return + func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` and `LoadJob::wait()` return if (waiters > 0) finished.notify_all(); } From d6a83ba6866a79c35cf756c6259be59449fb5e75 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 19 Apr 2023 13:35:58 +0200 Subject: [PATCH 038/628] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 2e2c605cfe6..5f87601e753 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -37,9 +37,9 @@ class LoadJob : private boost::noncopyable { public: template - LoadJob(LoadJobSet && dependencies_, const String & name_, Func && func_) + LoadJob(LoadJobSet && dependencies_, String name_, Func && func_) : dependencies(std::move(dependencies_)) - , name(name_) + , name(std::move(name_)) , func(std::forward(func_)) {} From f3c68fbeb55ab10a65391104c3ce2b49301eb3ca Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 19 Apr 2023 11:46:56 +0000 Subject: [PATCH 039/628] review fixes --- src/Common/AsyncLoader.cpp | 2 +- src/Common/AsyncLoader.h | 2 +- src/Common/tests/gtest_async_loader.cpp | 9 +++++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 7fe88ce7212..1d428b3601f 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -52,7 +52,7 @@ void LoadJob::waitNoThrow() const noexcept waiters--; } -size_t LoadJob::waiters_count() const +size_t LoadJob::waitersCount() const { std::unique_lock lock{mutex}; return waiters; diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 5f87601e753..e393fbde7d6 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -58,7 +58,7 @@ public: void waitNoThrow() const noexcept; // Returns number of threads blocked by `wait()` or `waitNoThrow()` calls. - size_t waiters_count() const; + size_t waitersCount() const; const LoadJobSet dependencies; // Jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard const String name; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index d6039162994..c05dbb751ca 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -313,7 +314,7 @@ TEST(AsyncLoader, CancelExecutingJob) { task.remove(); // waits for (C) }); - while (job->waiters_count() == 0) + while (job->waitersCount() == 0) std::this_thread::yield(); ASSERT_EQ(job->status(), LoadStatus::PENDING); sync.arrive_and_wait(); // (B) sync with job @@ -362,7 +363,7 @@ TEST(AsyncLoader, CancelExecutingTask) { task1.remove(); // waits for (C) }); - while (blocker_job->waiters_count() == 0) + while (blocker_job->waitersCount() == 0) std::this_thread::yield(); ASSERT_EQ(blocker_job->status(), LoadStatus::PENDING); sync.arrive_and_wait(); // (B) sync with job @@ -384,10 +385,10 @@ TEST(AsyncLoader, JobFailure) AsyncLoaderTest t; t.loader.start(); - std::string_view error_message = "test job failure"; + std::string error_message = "test job failure"; auto job_func = [&] (const LoadJobPtr &) { - throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message); + throw std::runtime_error(error_message); }; auto job = makeLoadJob({}, "job", job_func); From f74864f7950273d90351c89992e8d2fa15968af6 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 19 Apr 2023 13:48:38 +0200 Subject: [PATCH 040/628] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index e393fbde7d6..6f49b9b2d72 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -82,9 +82,9 @@ private: }; template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, const String & name, Func && func) +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func) { - return std::make_shared(std::move(dependencies), name, std::forward(func)); + return std::make_shared(std::move(dependencies), std::move(name), std::forward(func)); } // `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. From 198b99215864cc0bb1d98031dc737063eea82a98 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 22 Apr 2023 17:28:53 +0000 Subject: [PATCH 041/628] improve AsyncLoader interface --- src/Common/AsyncLoader.cpp | 85 +++++++-------- src/Common/AsyncLoader.h | 132 +++++++++++++++--------- src/Common/tests/gtest_async_loader.cpp | 127 +++++++++++------------ 3 files changed, 184 insertions(+), 160 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 1d428b3601f..804b8ea71dc 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -88,59 +88,40 @@ void LoadJob::finish() finished.notify_all(); } - -AsyncLoader::Task::Task() - : loader(nullptr) -{} - -AsyncLoader::Task::Task(AsyncLoader * loader_, LoadJobSet && jobs_) +LoadTask::LoadTask(AsyncLoader & loader_, LoadJobSet && jobs_, LoadJobSet && goal_jobs_) : loader(loader_) , jobs(std::move(jobs_)) + , goal_jobs(std::move(goal_jobs_)) {} -AsyncLoader::Task::Task(AsyncLoader::Task && o) noexcept - : loader(std::exchange(o.loader, nullptr)) - , jobs(std::move(o.jobs)) -{} - -AsyncLoader::Task::~Task() +LoadTask::~LoadTask() { remove(); } -AsyncLoader::Task & AsyncLoader::Task::operator=(AsyncLoader::Task && o) noexcept +void LoadTask::merge(const LoadTaskPtr & task) { - loader = std::exchange(o.loader, nullptr); - jobs = std::move(o.jobs); - return *this; + chassert(&loader == &task->loader); + jobs.merge(task->jobs); + goal_jobs.merge(task->goal_jobs); } -void AsyncLoader::Task::merge(AsyncLoader::Task && o) +void LoadTask::schedule() { - if (!loader) + loader.schedule(*this); +} + +void LoadTask::remove() +{ + if (!jobs.empty()) { - *this = std::move(o); - } - else - { - chassert(loader == o.loader); - jobs.merge(o.jobs); - o.loader = nullptr; + loader.remove(jobs); + jobs.clear(); } } -void AsyncLoader::Task::remove() +void LoadTask::detach() { - if (loader) - { - loader->remove(jobs); - detach(); - } -} - -void AsyncLoader::Task::detach() -{ - loader = nullptr; jobs.clear(); } @@ -178,7 +159,30 @@ void AsyncLoader::stop() pool.wait(); } -AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) +void AsyncLoader::schedule(LoadTask & task) +{ + chassert(this == &task.loader); + scheduleImpl(task.jobs); +} + +void AsyncLoader::schedule(const LoadTaskPtr & task) +{ + chassert(this == &task->loader); + scheduleImpl(task->jobs); +} + +void AsyncLoader::schedule(const std::vector & tasks) +{ + LoadJobSet all_jobs; + for (const auto & task : tasks) + { + chassert(this == &task->loader); + all_jobs.insert(task->jobs.begin(), task->jobs.end()); + } + scheduleImpl(all_jobs); +} + +void AsyncLoader::scheduleImpl(const LoadJobSet & jobs) { std::unique_lock lock{mutex}; @@ -202,9 +206,8 @@ AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - scheduled_jobs.emplace(job, Info{.initial_priority = priority, .priority = priority}); + scheduled_jobs.emplace(job, Info{.initial_priority = job->load_priority, .priority = job->load_priority}); }); - job->load_priority.store(priority); // Set user-facing priority } // Process dependencies on scheduled pending jobs @@ -223,7 +226,7 @@ AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) info.dependencies_left++; // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion - prioritize(dep, priority, lock); + prioritize(dep, info.priority, lock); } } @@ -282,8 +285,6 @@ AsyncLoader::Task AsyncLoader::schedule(LoadJobSet && jobs, ssize_t priority) // Job was already canceled on previous iteration of this cycle -- skip } } - - return Task(this, std::move(jobs)); } void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 6f49b9b2d72..abeb456cacb 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -19,6 +20,7 @@ class LoadJob; using LoadJobPtr = std::shared_ptr; using LoadJobSet = std::unordered_set; class LoadTask; +using LoadTaskPtr = std::shared_ptr; class AsyncLoader; // Execution status of a load job. @@ -37,17 +39,23 @@ class LoadJob : private boost::noncopyable { public: template - LoadJob(LoadJobSet && dependencies_, String name_, Func && func_) + LoadJob(LoadJobSet && dependencies_, String name_, ssize_t priority_, Func && func_) : dependencies(std::move(dependencies_)) , name(std::move(name_)) , func(std::forward(func_)) + , load_priority(priority_) + {} + + template + LoadJob(LoadJobSet && dependencies_, String name_, Func && func_) + : LoadJob(std::move(dependencies_), std::move(name_), 0, std::forward(func_)) {} // Current job status. LoadStatus status() const; std::exception_ptr exception() const; - // Returns current value of a priority of the job. May differ from initial priority passed to `AsyncLoader:::schedule()` call. + // Returns current value of a priority of the job. May differ from initial priority. ssize_t priority() const; // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. @@ -72,7 +80,7 @@ private: void finish(); std::function func; - std::atomic load_priority{0}; + std::atomic load_priority; mutable std::mutex mutex; mutable std::condition_variable finished; @@ -87,6 +95,49 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func) return std::make_shared(std::move(dependencies), std::move(name), std::forward(func)); } +template +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, ssize_t priority, Func && func) +{ + return std::make_shared(std::move(dependencies), std::move(name), priority, std::forward(func)); +} + +// Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set). +class LoadTask : private boost::noncopyable +{ +public: + LoadTask(AsyncLoader & loader_, LoadJobSet && jobs_, LoadJobSet && goals_ = {}); + ~LoadTask(); + + // Merge all jobs from other task into this task. + void merge(const LoadTaskPtr & o); + + // Schedule all jobs with AsyncLoader. + void schedule(); + + // Remove all jobs of this task from AsyncLoader. + void remove(); + + // Do not track jobs in this task. + // WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader(). + void detach(); + + // Return the final jobs in this tasks. This jobs subset should be used as `dependencies` for dependent jobs or tasks. + const LoadJobSet & goals() const { return goal_jobs.empty() ? jobs : goal_jobs; } + +private: + friend class AsyncLoader; + + AsyncLoader & loader; + LoadJobSet jobs; + LoadJobSet goal_jobs; +}; + +inline LoadTaskPtr makeLoadTask(AsyncLoader & loader, LoadJobSet && jobs, LoadJobSet && goals = {}) +{ + return std::make_shared(loader, std::move(jobs), std::move(goals)); +} + + // `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. // Basic usage example: // auto job_func = [&] (const LoadJobPtr & self) { @@ -95,28 +146,29 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func) // auto job1 = makeLoadJob({}, "job1", job_func); // auto job2 = makeLoadJob({ job1 }, "job2", job_func); // auto job3 = makeLoadJob({ job1 }, "job3", job_func); -// auto task = async_loader.schedule({ job1, job2, job3 }, /* priority = */ 0); +// auto task = makeLoadTask(async_loader, { job1, job2, job3 }); +// task.schedule(); // Here we have created and scheduled a task consisting of three jobs. Job1 has no dependencies and is run first. // Job2 and job3 depend on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: -// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first +// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first, default priority is zero. // job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) // // AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: -// 1) Job is constructed with PENDING status. -// 2) Job is scheduled and placed into a task. Scheduled job may be ready (i.e. have all its dependencies finished) or blocked. -// 3a) When all dependencies are successfully executed, job became ready. Ready job is enqueued into the ready queue. -// 3b) If at least one of job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). -// On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. Job status is changed to CANCELED. -// Exception is rethrown by any existing or new `wait()` call. Job is moved to the set of finished jobs. -// 4) Scheduled pending ready job starts execution by a worker. Job is dequeuedCallback `job_func` is called. -// Status of an executing job is PENDING. And it is still considered as scheduled job by AsyncLoader. +// 1) Job is constructed with PENDING status and initial priority. The job is placed into a task. +// 2) The task is scheduled with all its jobs. A scheduled job may be ready (i.e. have all its dependencies finished) or blocked. +// 3a) When all dependencies are successfully executed, the job became ready. A ready job is enqueued into the ready queue. +// 3b) If at least one of the job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). +// On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. The job status is changed to CANCELED. +// Exception is rethrown by any existing or new `wait()` call. The job is moved to the set of the finished jobs. +// 4) The scheduled pending ready job starts execution by a worker. The job is dequeuedCallback `job_func` is called. +// Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader. // Note that `job_func` of a CANCELED job is never executed. -// 5a) On successful execution job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. -// 5b) Any exception thrown out of `job_func` is wrapped into ASYNC_LOAD_FAILED exception and save inside LoadJob. -// Job status is changed to FAILED. All dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. -// 6) Job is no longer considered as scheduled and is instead moved to finished jobs set. This is required for introspection of finished jobs. -// 7) Task object containing this job is destructed or `remove()` is explicitly called. Job is removed from the finished job set. -// 8) Job is destructed. +// 5a) On successful execution the job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. +// 5b) Any exception thrown out of `job_func` is wrapped into an ASYNC_LOAD_FAILED exception and save inside LoadJob. +// The job status is changed to FAILED. All the dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. +// 6) The job is no longer considered as scheduled and is instead moved to the finished jobs set. This is just for introspection of the finished jobs. +// 7) The task containing this job is destructed or `remove()` is explicitly called. The job is removed from the finished job set. +// 8) The job is destructed. // // Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated // (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited); @@ -125,7 +177,7 @@ LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func) // Value stored in load job priority field is atomic and can be increased even during job execution. // // When task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status. -// The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all dependent jobs are immediately canceled. +// The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all the dependent jobs are immediately canceled. class AsyncLoader : private boost::noncopyable { private: @@ -174,36 +226,9 @@ private: public: using Metric = CurrentMetrics::Metric; - // Helper class that removes all not started and finished jobs in destructor and waits for all the executing jobs to finish. - class Task - { - public: - Task(); - Task(AsyncLoader * loader_, LoadJobSet && jobs_); - Task(const Task & o) = delete; - Task(Task && o) noexcept; - Task & operator=(const Task & o) = delete; - ~Task(); - Task & operator=(Task && o) noexcept; - - // Merge all jobs from other task into this task. Useful for merging jobs with different priorities into one task. - void merge(Task && o); - - // Remove all jobs of this task from AsyncLoader. - void remove(); - - // Do not track jobs in this task. - // WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader(). - void detach(); - - private: - AsyncLoader * loader; - LoadJobSet jobs; - }; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_); - // WARNING: all Task instances returned by `schedule()` should be destructed before AsyncLoader. + // WARNING: all LoadTask instances returned by `schedule()` should be destructed before AsyncLoader. ~AsyncLoader(); // Start workers to execute scheduled load jobs. @@ -218,13 +243,17 @@ public: // - or canceled using ~Task() or remove() later. void stop(); - // Schedule all `jobs` with given `priority` and return a task containing these jobs. + // Schedule all jobs of given task. // Higher priority jobs (with greater `priority` value) are executed earlier. // All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job // never wait for (blocked by) lower priority jobs (no priority inversion). - // Returned task destructor ensures that all the `jobs` are finished (OK, FAILED or CANCELED) + // Task destructor ensures that all the `jobs` are finished (OK, FAILED or CANCELED) // and are removed from AsyncLoader, so it is thread-safe to destroy them. - [[nodiscard]] Task schedule(LoadJobSet && jobs, ssize_t priority = 0); + void schedule(LoadTask & task); + void schedule(const LoadTaskPtr & task); + + // Schedule all tasks atomically. To ensure only highest priority jobs among all tasks are run first. + void schedule(const std::vector & tasks); // Increase priority of a job and all its dependencies recursively. void prioritize(const LoadJobPtr & job, ssize_t new_priority); @@ -242,6 +271,7 @@ private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}); + void scheduleImpl(const LoadJobSet & jobs); void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock); void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock); void spawn(std::unique_lock &); diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index c05dbb751ca..7bd80c28284 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -19,8 +19,8 @@ using namespace DB; namespace CurrentMetrics { - extern const Metric TablesLoaderThreads; - extern const Metric TablesLoaderThreadsActive; + extern const Metric AsyncLoaderThreads; + extern const Metric AsyncLoaderThreadsActive; } namespace DB::ErrorCodes @@ -38,7 +38,7 @@ struct AsyncLoaderTest pcg64 rng{randomSeed()}; explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads, /* log_failures = */ false) + : loader(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, max_threads, /* log_failures = */ false) {} template @@ -103,6 +103,13 @@ struct AsyncLoaderTest jobs.push_back(makeLoadJob({ jobs[j - 1] }, fmt::format("{}{}", name_prefix, j), job_func)); return {jobs.begin(), jobs.end()}; } + + LoadTaskPtr schedule(LoadJobSet && jobs) + { + LoadTaskPtr task = makeLoadTask(loader, std::move(jobs)); + task->schedule(); + return task; + } }; TEST(AsyncLoader, Smoke) @@ -123,13 +130,13 @@ TEST(AsyncLoader, Smoke) { auto job1 = makeLoadJob({}, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task1 = t.loader.schedule({ job1, job2 }); + auto task1 = t.schedule({ job1, job2 }); auto job3 = makeLoadJob({ job2 }, "job3", job_func); auto job4 = makeLoadJob({ job2 }, "job4", job_func); - auto task2 = t.loader.schedule({ job3, job4 }); - auto job5 = makeLoadJob({ job3, job4 }, "job5", job_func); - task2.merge(t.loader.schedule({ job5 }, low_priority)); + auto task2 = t.schedule({ job3, job4 }); + auto job5 = makeLoadJob({ job3, job4 }, "job5", low_priority, job_func); + task2->merge(t.schedule({ job5 })); std::thread waiter_thread([=] { job5->wait(); }); @@ -183,7 +190,7 @@ TEST(AsyncLoader, CycleDetection) jobs.push_back(makeLoadJob({}, "job9", job_func)); jobs.push_back(makeLoadJob({ jobs[9] }, "job10", job_func)); - auto task1 = t.loader.schedule({ jobs.begin(), jobs.end()}); + auto task1 = t.schedule({ jobs.begin(), jobs.end()}); FAIL(); } catch (Exception & e) @@ -203,9 +210,9 @@ TEST(AsyncLoader, CancelPendingJob) auto job_func = [&] (const LoadJobPtr &) {}; auto job = makeLoadJob({}, "job", job_func); - auto task = t.loader.schedule({ job }); + auto task = t.schedule({ job }); - task.remove(); // this cancels pending the job (async loader was not started to execute it) + task->remove(); // this cancels pending the job (async loader was not started to execute it) ASSERT_EQ(job->status(), LoadStatus::CANCELED); try @@ -227,9 +234,9 @@ TEST(AsyncLoader, CancelPendingTask) auto job1 = makeLoadJob({}, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task = t.loader.schedule({ job1, job2 }); + auto task = t.schedule({ job1, job2 }); - task.remove(); // this cancels both jobs (async loader was not started to execute it) + task->remove(); // this cancels both jobs (async loader was not started to execute it) ASSERT_EQ(job1->status(), LoadStatus::CANCELED); ASSERT_EQ(job2->status(), LoadStatus::CANCELED); @@ -263,10 +270,10 @@ TEST(AsyncLoader, CancelPendingDependency) auto job1 = makeLoadJob({}, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task1 = t.loader.schedule({ job1 }); - auto task2 = t.loader.schedule({ job2 }); + auto task1 = t.schedule({ job1 }); + auto task2 = t.schedule({ job2 }); - task1.remove(); // this cancels both jobs, due to dependency (async loader was not started to execute it) + task1->remove(); // this cancels both jobs, due to dependency (async loader was not started to execute it) ASSERT_EQ(job1->status(), LoadStatus::CANCELED); ASSERT_EQ(job2->status(), LoadStatus::CANCELED); @@ -307,12 +314,12 @@ TEST(AsyncLoader, CancelExecutingJob) }; auto job = makeLoadJob({}, "job", job_func); - auto task = t.loader.schedule({ job }); + auto task = t.schedule({ job }); sync.arrive_and_wait(); // (A) wait for job to start executing std::thread canceler([&] { - task.remove(); // waits for (C) + task->remove(); // waits for (C) }); while (job->waitersCount() == 0) std::this_thread::yield(); @@ -354,14 +361,14 @@ TEST(AsyncLoader, CancelExecutingTask) task1_jobs.push_back(blocker_job); for (int i = 0; i < 100; i++) task1_jobs.push_back(makeLoadJob({ blocker_job }, "job_to_cancel", job_to_cancel_func)); - auto task1 = t.loader.schedule({ task1_jobs.begin(), task1_jobs.end() }); + auto task1 = t.schedule({ task1_jobs.begin(), task1_jobs.end() }); auto job_to_succeed = makeLoadJob({ blocker_job }, "job_to_succeed", job_to_succeed_func); - auto task2 = t.loader.schedule({ job_to_succeed }); + auto task2 = t.schedule({ job_to_succeed }); sync.arrive_and_wait(); // (A) wait for job to start executing std::thread canceler([&] { - task1.remove(); // waits for (C) + task1->remove(); // waits for (C) }); while (blocker_job->waitersCount() == 0) std::this_thread::yield(); @@ -392,7 +399,7 @@ TEST(AsyncLoader, JobFailure) }; auto job = makeLoadJob({}, "job", job_func); - auto task = t.loader.schedule({ job }); + auto task = t.schedule({ job }); t.loader.wait(); @@ -421,7 +428,7 @@ TEST(AsyncLoader, ScheduleJobWithFailedDependencies) }; auto failed_job = makeLoadJob({}, "failed_job", failed_job_func); - auto failed_task = t.loader.schedule({ failed_job }); + auto failed_task = t.schedule({ failed_job }); t.loader.wait(); @@ -429,7 +436,7 @@ TEST(AsyncLoader, ScheduleJobWithFailedDependencies) auto job1 = makeLoadJob({ failed_job }, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task = t.loader.schedule({ job1, job2 }); + auto task = t.schedule({ job1, job2 }); t.loader.wait(); @@ -463,15 +470,15 @@ TEST(AsyncLoader, ScheduleJobWithCanceledDependencies) auto canceled_job_func = [&] (const LoadJobPtr &) {}; auto canceled_job = makeLoadJob({}, "canceled_job", canceled_job_func); - auto canceled_task = t.loader.schedule({ canceled_job }); - canceled_task.remove(); + auto canceled_task = t.schedule({ canceled_job }); + canceled_task->remove(); t.loader.start(); auto job_func = [&] (const LoadJobPtr &) {}; auto job1 = makeLoadJob({ canceled_job }, "job1", job_func); auto job2 = makeLoadJob({ job1 }, "job2", job_func); - auto task = t.loader.schedule({ job1, job2 }); + auto task = t.schedule({ job1, job2 }); t.loader.wait(); @@ -515,10 +522,10 @@ TEST(AsyncLoader, TestConcurrency) executing--; }; - std::vector tasks; + std::vector tasks; tasks.reserve(concurrency); for (int i = 0; i < concurrency; i++) - tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); + tasks.push_back(t.schedule(t.chainJobSet(5, job_func))); t.loader.wait(); ASSERT_EQ(executing, 0); } @@ -543,10 +550,10 @@ TEST(AsyncLoader, TestOverload) }; t.loader.stop(); - std::vector tasks; + std::vector tasks; tasks.reserve(concurrency); for (int i = 0; i < concurrency; i++) - tasks.push_back(t.loader.schedule(t.chainJobSet(5, job_func))); + tasks.push_back(t.schedule(t.chainJobSet(5, job_func))); t.loader.start(); t.loader.wait(); ASSERT_EQ(executing, 0); @@ -565,22 +572,15 @@ TEST(AsyncLoader, StaticPriorities) }; std::vector jobs; - jobs.push_back(makeLoadJob({}, "A", job_func)); // 0 - jobs.push_back(makeLoadJob({ jobs[0] }, "B", job_func)); // 1 - jobs.push_back(makeLoadJob({ jobs[0] }, "C", job_func)); // 2 - jobs.push_back(makeLoadJob({ jobs[0] }, "D", job_func)); // 3 - jobs.push_back(makeLoadJob({ jobs[0] }, "E", job_func)); // 4 - jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", job_func)); // 5 - jobs.push_back(makeLoadJob({ jobs[5] }, "G", job_func)); // 6 - jobs.push_back(makeLoadJob({ jobs[6] }, "H", job_func)); // 7 - auto task = t.loader.schedule({ jobs[0] }, 0); - task.merge(t.loader.schedule({ jobs[1] }, 3)); - task.merge(t.loader.schedule({ jobs[2] }, 4)); - task.merge(t.loader.schedule({ jobs[3] }, 1)); - task.merge(t.loader.schedule({ jobs[4] }, 2)); - task.merge(t.loader.schedule({ jobs[5] }, 0)); - task.merge(t.loader.schedule({ jobs[6] }, 0)); - task.merge(t.loader.schedule({ jobs[7] }, 9)); + jobs.push_back(makeLoadJob({}, "A", 0, job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, "B", 3, job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, "C", 4, job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, "D", 1, job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, "E", 2, job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", 0, job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, "G", 0, job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, "H", 9, job_func)); // 7 + auto task = t.schedule({ jobs.begin(), jobs.end() }); t.loader.start(); t.loader.wait(); @@ -614,22 +614,15 @@ TEST(AsyncLoader, DynamicPriorities) // | +-> F0 --> G0 --> H0 // `-> E2 -' std::vector jobs; - jobs.push_back(makeLoadJob({}, "A", job_func)); // 0 - jobs.push_back(makeLoadJob({ jobs[0] }, "B", job_func)); // 1 - jobs.push_back(makeLoadJob({ jobs[0] }, "C", job_func)); // 2 - jobs.push_back(makeLoadJob({ jobs[0] }, "D", job_func)); // 3 - jobs.push_back(makeLoadJob({ jobs[0] }, "E", job_func)); // 4 - jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", job_func)); // 5 - jobs.push_back(makeLoadJob({ jobs[5] }, "G", job_func)); // 6 - jobs.push_back(makeLoadJob({ jobs[6] }, "H", job_func)); // 7 - auto task = t.loader.schedule({ jobs[0] }, 0); - task.merge(t.loader.schedule({ jobs[1] }, 3)); - task.merge(t.loader.schedule({ jobs[2] }, 4)); - task.merge(t.loader.schedule({ jobs[3] }, 1)); - task.merge(t.loader.schedule({ jobs[4] }, 2)); - task.merge(t.loader.schedule({ jobs[5] }, 0)); - task.merge(t.loader.schedule({ jobs[6] }, 0)); - task.merge(t.loader.schedule({ jobs[7] }, 0)); + jobs.push_back(makeLoadJob({}, "A", 0, job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, "B", 3, job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, "C", 4, job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, "D", 1, job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, "E", 2, job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", 0, job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, "G", 0, job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, "H", 0, job_func)); // 7 + auto task = t.schedule({ jobs.begin(), jobs.end() }); job_to_prioritize = jobs[6]; @@ -656,12 +649,12 @@ TEST(AsyncLoader, RandomIndependentTasks) t.randomSleepUs(100, 500, 5); }; - std::vector tasks; + std::vector tasks; tasks.reserve(512); for (int i = 0; i < 512; i++) { int job_count = t.randomInt(1, 32); - tasks.push_back(t.loader.schedule(t.randomJobSet(job_count, 5, job_func))); + tasks.push_back(t.schedule(t.randomJobSet(job_count, 5, job_func))); t.randomSleepUs(100, 900, 20); // avg=100us } } @@ -673,7 +666,7 @@ TEST(AsyncLoader, RandomDependentTasks) std::mutex mutex; std::condition_variable cv; - std::vector tasks; + std::vector tasks; std::vector all_jobs; auto job_func = [&] (const LoadJobPtr & self) @@ -695,7 +688,7 @@ TEST(AsyncLoader, RandomDependentTasks) int job_count = t.randomInt(1, 32); LoadJobSet jobs = t.randomJobSet(job_count, 5, all_jobs, job_func); all_jobs.insert(all_jobs.end(), jobs.begin(), jobs.end()); - tasks.push_back(t.loader.schedule(std::move(jobs))); + tasks.push_back(t.schedule(std::move(jobs))); // Cancel random old task if (tasks.size() > 100) @@ -732,7 +725,7 @@ TEST(AsyncLoader, SetMaxThreads) // Generate enough independent jobs for (int i = 0; i < 1000; i++) - t.loader.schedule({makeLoadJob({}, "job", job_func)}).detach(); + t.schedule({makeLoadJob({}, "job", job_func)})->detach(); t.loader.start(); while (sync_index < syncs.size()) From 3546699c9529909402faab5d8f592f7baa480582 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 26 Apr 2023 18:25:39 +0000 Subject: [PATCH 042/628] wip async tables loading --- src/Common/AsyncLoader.h | 3 +- src/Common/CurrentMetrics.cpp | 4 +- src/Core/ServerSettings.h | 1 + src/Databases/IDatabase.h | 61 ++++++++++++- src/Databases/TablesDependencyGraph.cpp | 2 +- src/Databases/TablesLoader.cpp | 114 ++++++++++++++++++++++-- src/Databases/TablesLoader.h | 15 +++- src/Interpreters/Context.cpp | 33 +++++++ src/Interpreters/Context.h | 4 + 9 files changed, 219 insertions(+), 18 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index abeb456cacb..fe6683da29b 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -21,6 +21,7 @@ using LoadJobPtr = std::shared_ptr; using LoadJobSet = std::unordered_set; class LoadTask; using LoadTaskPtr = std::shared_ptr; +using LoadTaskPtrs = std::vector; class AsyncLoader; // Execution status of a load job. @@ -253,7 +254,7 @@ public: void schedule(const LoadTaskPtr & task); // Schedule all tasks atomically. To ensure only highest priority jobs among all tasks are run first. - void schedule(const std::vector & tasks); + void schedule(const LoadTaskPtrs & tasks); // Increase priority of a job and all its dependencies recursively. void prioritize(const LoadJobPtr & job, ssize_t new_priority); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 0802941bff6..e3eea93a25b 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -96,8 +96,8 @@ M(DiskObjectStorageAsyncThreadsActive, "Number of threads in the async thread pool for DiskObjectStorage running a task.") \ M(StorageHiveThreads, "Number of threads in the StorageHive thread pool.") \ M(StorageHiveThreadsActive, "Number of threads in the StorageHive thread pool running a task.") \ - M(TablesLoaderThreads, "Number of threads in the tables loader thread pool.") \ - M(TablesLoaderThreadsActive, "Number of threads in the tables loader thread pool running a task.") \ + M(AsyncLoaderThreads, "Number of threads in the async loader thread pool.") \ + M(AsyncLoaderThreadsActive, "Number of threads in the async loader thread pool running a task.") \ M(DatabaseOrdinaryThreads, "Number of threads in the Ordinary database thread pool.") \ M(DatabaseOrdinaryThreadsActive, "Number of threads in the Ordinary database thread pool running a task.") \ M(DatabaseOnDiskThreads, "Number of threads in the DatabaseOnDisk thread pool.") \ diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index aabc89cc6d7..cbc94940f1c 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -74,6 +74,7 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ + M(UInt64, async_loader_pool_size, 16, "The maximum number of threads that will be used for async loading of tables after server start.", 0) \ DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 53a2f372814..beb19519e17 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -139,6 +140,20 @@ public: { } + // TODO(serxa): do we really need loadStoredObjectsAsync()? better to always do it sync. + /// Create a task to load existing tables after specified dependencies `load_after` using `async_loader`. + /// The returned task is also stored inside the database for cancellation on destruction. + /// You can call it only once, right after the object is created. + virtual LoadTaskPtr loadStoredObjectsAsync( + AsyncLoader & /*async_loader*/, + LoadJobSet /*load_after*/, + ContextMutablePtr /*context*/, + LoadingStrictnessLevel /*mode*/, + bool /* skip_startup_tables */) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); + } + virtual bool supportsLoadingInTopologicalOrder() const { return false; } virtual void beforeLoadingMetadata( @@ -151,14 +166,58 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - virtual void loadTableFromMetadata(ContextMutablePtr /*local_context*/, const String & /*file_path*/, const QualifiedTableName & /*name*/, const ASTPtr & /*ast*/, + virtual void loadTableFromMetadata( + ContextMutablePtr /*local_context*/, + const String & /*file_path*/, + const QualifiedTableName & /*name*/, + const ASTPtr & /*ast*/, LoadingStrictnessLevel /*mode*/) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + /// Create a task to load table `name` after specified dependencies `startup_after` using `async_loader`. + /// `load_after` must contain the tasks returned by `loadTableFromMetadataAsync()` for dependent tables (see TablesLoader). + /// The returned task is also stored inside the database for cancellation on destruction. + virtual LoadTaskPtr loadTableFromMetadataAsync( + AsyncLoader & /*async_loader*/, + LoadJobSet /*load_after*/, + ContextMutablePtr /*local_context*/, + const String & /*file_path*/, + const QualifiedTableName & /*name*/, + const ASTPtr & /*ast*/, + LoadingStrictnessLevel /*mode*/) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); + } + + /// Start all tables and the database itself virtual void startupTables(ThreadPool & /*thread_pool*/, LoadingStrictnessLevel /*mode*/) {} + /// Create a task to startup table `name` after specified dependencies `startup_after` using `async_loader`. + /// `startup_after` must contain the task returned by `loadTableFromMetadataAsync()` for this table (see TablesLoader). + /// The returned task is also stored inside the database for cancellation on destruction. + virtual LoadTaskPtr startupTableAsync( + AsyncLoader & /*async_loader*/, + LoadJobSet /*startup_after*/, + const QualifiedTableName & /*name*/, + LoadingStrictnessLevel /*mode*/) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); // TODO: should return empty task instead because startupTables default implementation do no throw? + } + + /// Create a task to startup database after specified dependencies `startup_after` using `async_loader`. + /// `startup_after` must contain all the tasks returned by `startupTableAsync()` for every table (see TablesLoader). + /// The returned task is also stored inside the database for cancellation on destruction. + virtual LoadTaskPtr startupDatabaseAsync( + AsyncLoader & /*async_loader*/, + LoadJobSet /*startup_after*/, + const QualifiedTableName & /*name*/, + LoadingStrictnessLevel /*mode*/) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); // TODO: should return empty task instead because startupTables default implementation do no throw? + } + /// Check the existence of the table in memory (attached). virtual bool isTableExist(const String & name, ContextPtr context) const = 0; diff --git a/src/Databases/TablesDependencyGraph.cpp b/src/Databases/TablesDependencyGraph.cpp index 4ea9e646ab9..cf6bf3e2120 100644 --- a/src/Databases/TablesDependencyGraph.cpp +++ b/src/Databases/TablesDependencyGraph.cpp @@ -698,7 +698,7 @@ std::vector TablesDependencyGraph::getTablesSortedByDependency() cons return res; } - +// TODO(serxa): get rid of getTablesSortedByDependencyForParallel() std::vector> TablesDependencyGraph::getTablesSortedByDependencyForParallel() const { std::vector> res; diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index ec0fdd85eec..ef904c70e13 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -11,10 +11,11 @@ #include #include + namespace CurrentMetrics { - extern const Metric TablesLoaderThreads; - extern const Metric TablesLoaderThreadsActive; + extern const Metric AsyncLoaderThreads; + extern const Metric AsyncLoaderThreadsActive; } namespace DB @@ -44,15 +45,108 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database , referential_dependencies("ReferentialDeps") , loading_dependencies("LoadingDeps") , all_loading_dependencies("LoadingDeps") - , pool(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive) + , async_loader(global_context->getAsyncLoader()) + , pool(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive) { metadata.default_database = global_context->getCurrentDatabase(); log = &Poco::Logger::get("TablesLoader"); } +void TablesLoader::createTasks(LoadJobSet load_after) +{ + bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading"); + + /// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first. + for (auto & database : databases) + { + if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder()) + databases_to_load.push_back(database.first); + else + load_tables.push_back(database.second->loadStoredObjectsAsync(async_loader, load_after, global_context, strictness_mode, /* skip_startup_tables */ true)); + } + + if (databases_to_load.empty()) + return; + + /// Read and parse metadata from Ordinary, Atomic, Materialized*, Replicated, etc databases. Build dependency graph. + for (auto & database_name : databases_to_load) + { + databases[database_name]->beforeLoadingMetadata(global_context, strictness_mode); + bool is_startup = LoadingStrictnessLevel::FORCE_ATTACH <= strictness_mode; + databases[database_name]->loadTablesMetadata(global_context, metadata, is_startup); + } + + LOG_INFO(log, "Parsed metadata of {} tables in {} databases in {} sec", + metadata.parsed_tables.size(), databases_to_load.size(), stopwatch.elapsedSeconds()); + + stopwatch.restart(); + + LoadJobSet load_databases_without_dependencies; + for (const auto task : load_tables) + load_databases_without_dependencies.insert(task->goals().begin(), task->goals().end()); + if (load_databases_without_dependencies.empty()) + load_databases_without_dependencies = std::move(load_after); + + buildDependencyGraph(); + + /// Update existing info (it's important for ATTACH DATABASE) + DatabaseCatalog::instance().addDependencies(referential_dependencies, loading_dependencies); + + /// Remove tables that do not exist + removeUnresolvableDependencies(); + + + std::unordered_map load_table; /// table uuid -> load task + std::unordered_map startup_database; /// database name -> all its tables startup tasks + for (const auto & table_id : all_loading_dependencies.getTablesSortedByDependency()) + { + /// Make set of jobs to load before this table + LoadJobSet load_before; + for (StorageID dependency_id : all_loading_dependencies.getDependencies(table_id)) + { + const auto & goals = load_table[dependency_id.uuid]->goals(); + load_before.insert(goals.begin(), goals.end()); + } + if (load_before.empty()) + load_before = load_databases_without_dependencies; + + // Make load table task + auto table_name = table_id.getQualifiedName(); + const auto & path_and_query = metadata.parsed_tables[table_name]; + auto load_task = databases[table_name.database]->loadTableFromMetadataAsync(async_loader, load_before, load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); + load_table[table_id.uuid] = load_task; + load_tables.push_back(load_task); + + // Make startup table task + auto startup_task = databases[table_name.database]->startupTableAsync(async_loader, load_task->goals(), table_name, strictness_mode); + startup_database[table_name.database] = startup_task; + startup_tables.push_back(startup_task); + + // TODO(serxa): we should report progress, a job should be attached to task.goals() here to report it. But what task should contain that job is unclear yet + // logAboutProgress(log, ++tables_processed, total_tables, stopwatch); + } + + // TODO(serxa): make startup database tasks + // for (auto [database_name, startup_tables] : startup_database) + // { + // } +} + +LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) +{ + createTasks(load_after); + return load_tables; +} + +LoadTaskPtr TablesLoader::startupTablesAsync() +{ + return startup_tables; +} + void TablesLoader::loadTables() { + // TODO(serxa): rewrite using loadTablesAsync() bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading"); /// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first. @@ -88,12 +182,13 @@ void TablesLoader::loadTables() /// Remove tables that do not exist removeUnresolvableDependencies(); - loadTablesInTopologicalOrder(pool); + loadTablesInTopologicalOrder(); } void TablesLoader::startupTables() { + // TODO(serxa): rewrite using startupTablesAsync() /// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading. for (auto & database : databases) database.second->startupTables(pool, strictness_mode); @@ -177,24 +272,25 @@ void TablesLoader::removeUnresolvableDependencies() } -void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool_) +void TablesLoader::loadTablesInTopologicalOrder() { /// Compatibility setting which should be enabled by default on attach /// Otherwise server will be unable to start for some old-format of IPv6/IPv4 types of columns ContextMutablePtr load_context = Context::createCopy(global_context); load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); + // TODO(serxa): remove the following code. Return waitable job or job set instead of sync wait. /// Load tables in parallel. auto tables_to_load = all_loading_dependencies.getTablesSortedByDependencyForParallel(); for (size_t level = 0; level != tables_to_load.size(); ++level) { - startLoadingTables(pool_, load_context, tables_to_load[level], level); - pool_.wait(); + startLoadingTables(load_context, tables_to_load[level], level); + pool.wait(); } } -void TablesLoader::startLoadingTables(ThreadPool & pool_, ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level) +void TablesLoader::startLoadingTables(ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level) { size_t total_tables = metadata.parsed_tables.size(); @@ -202,7 +298,7 @@ void TablesLoader::startLoadingTables(ThreadPool & pool_, ContextMutablePtr load for (const auto & table_id : tables_to_load) { - pool_.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]() + pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]() { const auto & path_and_query = metadata.parsed_tables[table_name]; databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index eb07351bd7f..e7bb408e282 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include namespace Poco { @@ -57,10 +57,15 @@ public: TablesLoader(ContextMutablePtr global_context_, Databases databases_, LoadingStrictnessLevel strictness_mode_); TablesLoader() = delete; + LoadTaskPtr loadTablesAsync(LoadJobSet load_after = {}); + LoadTaskPtr startupTablesAsync(); void loadTables(); void startupTables(); private: + LoadTaskPtrs load_tables; // Tasks to load all tables + LoadTaskPtrs startup_tables; // Tasks to startup all tables after loading + ContextMutablePtr global_context; Databases databases; LoadingStrictnessLevel strictness_mode; @@ -74,12 +79,14 @@ private: std::atomic tables_processed{0}; AtomicStopwatch stopwatch; - ThreadPool pool; + AsyncLoader & async_loader; + ThreadPool pool; // TODO(serxa): get rid of it + void createTasks(LoadJobSet load_after); void buildDependencyGraph(); void removeUnresolvableDependencies(); - void loadTablesInTopologicalOrder(ThreadPool & pool); - void startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level); + void loadTablesInTopologicalOrder(); + void startLoadingTables(ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level); }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 73d78e84198..0344f4accb7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -152,6 +154,8 @@ namespace CurrentMetrics extern const Metric IOPrefetchThreadsActive; extern const Metric IOWriterThreads; extern const Metric IOWriterThreadsActive; + extern const Metric AsyncLoaderThreads; + extern const Metric AsyncLoaderThreadsActive; } namespace DB @@ -228,6 +232,9 @@ struct ContextSharedPart : boost::noncopyable /// Initialized once during server startup. TemporaryDataOnDiskScopePtr root_temp_data_on_disk; + mutable LoadJobPtr server_start_job; + mutable std::unique_ptr async_loader; /// Thread pool for asynchronous initialization of arbitrary DAG of `LoadJob`s (used for tables loading) + mutable std::unique_ptr embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization. mutable std::unique_ptr external_dictionaries_loader; @@ -1913,6 +1920,32 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries() return getEmbeddedDictionariesImpl(false); } +size_t Context::getAsyncLoaderPoolSize() const +{ + auto lock = getLock(); + // We use all CPU cores before server is started, due to: + // 1) no other work to do - no reason to waste CPU resources; + // 2) backward-compatibility - we do not want to change behaviour in the case every table is loaded synchronously + // before server start (port listen). + // After server is started incoming queries can compete for resources with loading of the rest of the tables. + // Thus it can be advantageous to lower number of threads after start using server setting `async_loader_pool_size`. + // TODO(serxa): set async_loader max threads during server_start_job + return shared->server_start_job && shared->server_start_job->status() == LoadStatus::OK ? + shared->server_settings.async_loader_pool_size : getNumberOfPhysicalCPUCores(); +} + +AsyncLoader & Context::getAsyncLoader() const +{ + auto lock = getLock(); + if (!shared->async_loader) + shared->async_loader = std::make_unique( + CurrentMetrics::AsyncLoaderThreads, + CurrentMetrics::AsyncLoaderThreadsActive, + getAsyncLoaderPoolSize(), + /* log_failures = */ true); + return *shared->async_loader; +} + const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 95611e99d51..cba1d76ede1 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -706,6 +707,9 @@ public: /// Returns the current constraints (can return null). std::shared_ptr getSettingsConstraintsAndCurrentProfiles() const; + size_t getAsyncLoaderPoolSize() const; + AsyncLoader & getAsyncLoader() const; + const ExternalDictionariesLoader & getExternalDictionariesLoader() const; ExternalDictionariesLoader & getExternalDictionariesLoader(); ExternalDictionariesLoader & getExternalDictionariesLoaderUnlocked(); From 86122409e212aa2af8be8ed6256fe8db6e3b9994 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 26 Apr 2023 18:27:23 +0000 Subject: [PATCH 043/628] more work --- .../server-configuration-parameters/settings.md | 11 +++++++++++ programs/server/Server.cpp | 2 ++ programs/server/config.xml | 1 + 3 files changed, 14 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 7c97d0ab640..9cdb8777e50 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1166,6 +1166,17 @@ Possible values: Default value: 128. +## async_loader_pool_size {#async_loader_pool_size} + +Sets the number of threads performing asynchronous load jobs after server startup. Note that before server startup all available CPUs are used for load jobs. + +Possible values: + +- Any positive integer. + +Default value: 16. + + ## merge_tree {#server_configuration_parameters-merge_tree} Fine tuning for tables in the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index cdbd1b21ff2..3f60ee88af1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1299,6 +1299,8 @@ try global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); + global_context->getAsyncLoader().setMaxThreads(global_context->getAsyncLoaderPoolSize()); + if (config->has("resources")) { global_context->getResourceManager()->updateConfiguration(*config); diff --git a/programs/server/config.xml b/programs/server/config.xml index cfcd2ff93e0..59ecf9e51c2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -346,6 +346,7 @@ 128 16 16 + 16 --> + diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 5786d4f9e63..db5dd057eb9 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -140,20 +140,6 @@ public: { } - // TODO(serxa): do we really need loadStoredObjectsAsync()? better to always do it sync. - /// Create a task to load existing tables after specified dependencies `load_after` using `async_loader`. - /// The returned task is also stored inside the database for cancellation on destruction. - /// You can call it only once, right after the object is created. - virtual LoadTaskPtr loadStoredObjectsAsync( - AsyncLoader & /*async_loader*/, - LoadJobSet /*load_after*/, - ContextMutablePtr /*context*/, - LoadingStrictnessLevel /*mode*/, - bool /* skip_startup_tables */) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); - } - virtual bool supportsLoadingInTopologicalOrder() const { return false; } virtual void beforeLoadingMetadata( diff --git a/src/Databases/TablesDependencyGraph.cpp b/src/Databases/TablesDependencyGraph.cpp index cf6bf3e2120..16404c6870f 100644 --- a/src/Databases/TablesDependencyGraph.cpp +++ b/src/Databases/TablesDependencyGraph.cpp @@ -698,22 +698,6 @@ std::vector TablesDependencyGraph::getTablesSortedByDependency() cons return res; } -// TODO(serxa): get rid of getTablesSortedByDependencyForParallel() -std::vector> TablesDependencyGraph::getTablesSortedByDependencyForParallel() const -{ - std::vector> res; - std::optional last_level; - for (const auto * node : getNodesSortedByLevel()) - { - if (node->level != last_level) - res.emplace_back(); - auto & table_ids = res.back(); - table_ids.emplace_back(node->storage_id); - last_level = node->level; - } - return res; -} - void TablesDependencyGraph::log() const { diff --git a/src/Databases/TablesDependencyGraph.h b/src/Databases/TablesDependencyGraph.h index e5be59d1ee9..e71d5ecc5fc 100644 --- a/src/Databases/TablesDependencyGraph.h +++ b/src/Databases/TablesDependencyGraph.h @@ -107,9 +107,6 @@ public: /// tables which depend on the tables which depend on the tables without dependencies, and so on. std::vector getTablesSortedByDependency() const; - /// The same as getTablesSortedByDependency() but make a list for parallel processing. - std::vector> getTablesSortedByDependencyForParallel() const; - /// Outputs information about this graph as a bunch of logging messages. void log() const; diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 0855c89b088..adc735d2137 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -40,21 +40,22 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database log = &Poco::Logger::get("TablesLoader"); } -void TablesLoader::createTasks(LoadJobSet load_after) +LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) { bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading"); /// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first. + /// Note that this loading is NOT async because it should be fast and it cannot have any dependencies for (auto & database : databases) { if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder()) databases_to_load.push_back(database.first); else - load_tables.push_back(database.second->loadStoredObjectsAsync(async_loader, load_after, global_context, strictness_mode, /* skip_startup_tables */ true)); + database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true); } if (databases_to_load.empty()) - return; + return load_tasks; /// Read and parse metadata from Ordinary, Atomic, Materialized*, Replicated, etc databases. Build dependency graph. for (auto & database_name : databases_to_load) @@ -69,12 +70,6 @@ void TablesLoader::createTasks(LoadJobSet load_after) stopwatch.restart(); - LoadJobSet load_databases_without_dependencies; - for (const auto task : load_tables) - load_databases_without_dependencies.insert(task->goals().begin(), task->goals().end()); - if (load_databases_without_dependencies.empty()) - load_databases_without_dependencies = std::move(load_after); - buildDependencyGraph(); /// Update existing info (it's important for ATTACH DATABASE) @@ -88,112 +83,65 @@ void TablesLoader::createTasks(LoadJobSet load_after) ContextMutablePtr load_context = Context::createCopy(global_context); load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); - std::unordered_map load_table; /// table uuid -> load task - std::unordered_map startup_database; /// database name -> all its tables startup tasks for (const auto & table_id : all_loading_dependencies.getTablesSortedByDependency()) { - /// Make set of jobs to load before this table - LoadJobSet load_dependencies; + /// Gather tasks to load before this table + LoadTaskPtrs load_dependency_tasks; for (StorageID dependency_id : all_loading_dependencies.getDependencies(table_id)) - { - const auto & goals = load_table[dependency_id.uuid]->goals(); - load_dependencies.insert(goals.begin(), goals.end()); - } - if (load_dependencies.empty()) - load_dependencies = load_databases_without_dependencies; + load_dependency_tasks.push_back(load_table[dependency_id.uuid]); // Make load table task auto table_name = table_id.getQualifiedName(); const auto & path_and_query = metadata.parsed_tables[table_name]; - auto load_task = databases[table_name.database]->loadTableFromMetadataAsync(async_loader, load_dependencies, load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); - load_table[table_id.uuid] = load_task; - load_tables.push_back(load_task); + auto task = databases[table_name.database]->loadTableFromMetadataAsync( + async_loader, + load_dependency_tasks.empty() ? load_after : getGoals(load_dependency_tasks), + load_context, + path_and_query.path, + table_name, + path_and_query.ast, + strictness_mode); + load_table[table_id.uuid] = task; + load_tasks.push_back(task); + } + return load_tasks; +} + +LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) +{ + std::unordered_map startup_database; /// database name -> all its tables startup tasks + for (const auto & table_id : all_loading_dependencies.getTablesSortedByDependency()) + { // Make startup table task - auto startup_task = databases[table_name.database]->startupTableAsync(async_loader, load_task->goals(), table_name, strictness_mode); - startup_database[table_name.database].push_back(startup_task); - startup_tables.push_back(startup_task); + auto table_name = table_id.getQualifiedName(); + auto task = databases[table_name.database]->startupTableAsync( + async_loader, + joinJobs(load_table[table_id.uuid]->goals(), startup_after), + table_name, + strictness_mode); + startup_database[table_name.database].push_back(task); + startup_tasks.push_back(task); } /// Make startup database tasks for (auto [database_name, startup_table_tasks] : startup_database) { - LoadJobSet startup_after; - for (const auto & startup_task : startup_table_tasks) - startup_after.insert(startup_task->goals().begin(), startup_task->goals().end()); - auto startup_database_task = databases[database_name]->startupDatabaseAsync(async_loader, startup_after, strictness_mode); - startup_databases.push_back(startup_database_task); + auto task = databases[database_name]->startupDatabaseAsync( + async_loader, + getGoals(startup_table_tasks), + strictness_mode); + startup_tasks.push_back(task); } - // Schedule all tasks in right order - async_loader.schedule(load_tables); - async_loader.schedule(startup_tables); - async_loader.schedule(startup_databases); + return startup_tasks; } -LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) +LoadJobSet TablesLoader::goals() { - createTasks(load_after); - return load_tables; + return getGoals(startup_tasks); } -LoadTaskPtrs TablesLoader::startupTablesAsync() -{ - return startup_tables; -} - - -void TablesLoader::loadTables() -{ - // TODO(serxa): rewrite using loadTablesAsync() - bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading"); - - /// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first. - for (auto & database : databases) - { - if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder()) - databases_to_load.push_back(database.first); - else - database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true); - } - - if (databases_to_load.empty()) - return; - - /// Read and parse metadata from Ordinary, Atomic, Materialized*, Replicated, etc databases. Build dependency graph. - for (auto & database_name : databases_to_load) - { - databases[database_name]->beforeLoadingMetadata(global_context, strictness_mode); - bool is_startup = LoadingStrictnessLevel::FORCE_ATTACH <= strictness_mode; - databases[database_name]->loadTablesMetadata(global_context, metadata, is_startup); - } - - LOG_INFO(log, "Parsed metadata of {} tables in {} databases in {} sec", - metadata.parsed_tables.size(), databases_to_load.size(), stopwatch.elapsedSeconds()); - - stopwatch.restart(); - - buildDependencyGraph(); - - /// Update existing info (it's important for ATTACH DATABASE) - DatabaseCatalog::instance().addDependencies(referential_dependencies, loading_dependencies); - - /// Remove tables that do not exist - removeUnresolvableDependencies(); - - loadTablesInTopologicalOrder(); -} - - -void TablesLoader::startupTables() -{ - // TODO(serxa): rewrite using startupTablesAsync() - /// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading. - for (auto & database : databases) - database.second->startupTables(pool, strictness_mode); -} - - void TablesLoader::buildDependencyGraph() { for (const auto & [table_name, table_metadata] : metadata.parsed_tables) @@ -270,40 +218,4 @@ void TablesLoader::removeUnresolvableDependencies() all_loading_dependencies.checkNoCyclicDependencies(); } - -void TablesLoader::loadTablesInTopologicalOrder() -{ - /// Compatibility setting which should be enabled by default on attach - /// Otherwise server will be unable to start for some old-format of IPv6/IPv4 types of columns - ContextMutablePtr load_context = Context::createCopy(global_context); - load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); - - // TODO(serxa): remove the following code. Return waitable job or job set instead of sync wait. - /// Load tables in parallel. - auto tables_to_load = all_loading_dependencies.getTablesSortedByDependencyForParallel(); - - for (size_t level = 0; level != tables_to_load.size(); ++level) - { - startLoadingTables(load_context, tables_to_load[level], level); - pool.wait(); - } -} - -void TablesLoader::startLoadingTables(ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level) -{ - size_t total_tables = metadata.parsed_tables.size(); - - LOG_INFO(log, "Loading {} tables with dependency level {}", tables_to_load.size(), level); - - for (const auto & table_id : tables_to_load) - { - pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]() - { - const auto & path_and_query = metadata.parsed_tables[table_name]; - databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); - logAboutProgress(log, ++tables_processed, total_tables, stopwatch); - }); - } -} - } diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 931e615ebb8..e35ff3ca30c 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -54,10 +54,15 @@ public: TablesLoader(ContextMutablePtr global_context_, Databases databases_, LoadingStrictnessLevel strictness_mode_); TablesLoader() = delete; - LoadTaskPtrs loadTablesAsync(LoadJobSet load_after = {}); - LoadTaskPtrs startupTablesAsync(); - void loadTables(); - void startupTables(); + // Create tasks for async loading of all tables in `databases` after specified jobs `load_after` + [[nodiscard]] LoadTaskPtrs loadTablesAsync(LoadJobSet load_after = {}); + + // Create tasks for async startup of all tables in `databases` after specified jobs `startup_after` + // Note that for every table startup an extra dependency on that table loading will be added along with `startup_after` + [[nodiscard]] LoadTaskPtrs startupTablesAsync(LoadJobSet startup_after = {}); + + // Set of goal jobs for the whole loading process. Useful for scheduling more tasks after table loading + LoadJobSet goals(); private: ContextMutablePtr global_context; @@ -74,13 +79,12 @@ private: AtomicStopwatch stopwatch; AsyncLoader & async_loader; - LoadTaskPtrs load_tables; // Tasks to load all tables - LoadTaskPtrs startup_tables; // Tasks to startup all tables after loading - LoadTaskPtrs startup_databases; // Tasks to startup all databases after tables startup + std::unordered_map load_table; /// table uuid -> load task + LoadTaskPtrs load_tasks; // Tasks to load all tables + LoadTaskPtrs startup_tasks; // Tasks to startup all tables and databases after loading ThreadPool pool; // TODO(serxa): get rid of it - void createTasks(LoadJobSet load_after); void buildDependencyGraph(); void removeUnresolvableDependencies(); void loadTablesInTopologicalOrder(); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 77179ad2f32..9ec0836864e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -311,11 +311,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (!load_database_without_tables) { - /// We use global context here, because storages lifetime is bigger than query context lifetime TablesLoader loader{getContext()->getGlobalContext(), {{database_name, database}}, mode}; - loader.loadTables(); - loader.startupTables(); + scheduleAndWaitLoad(loader.loadTablesAsync()); + scheduleAndWaitLoad(loader.startupTablesAsync()); } } catch (...) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 83af2684322..4e7d3d182f9 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -158,7 +158,7 @@ static void checkIncompleteOrdinaryToAtomicConversion(ContextPtr context, const } } -void loadMetadata(ContextMutablePtr context, const String & default_database_name) +LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); @@ -230,20 +230,30 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam auto mode = getLoadingStrictnessLevel(/* attach */ true, /* force_attach */ true, has_force_restore_data_flag); TablesLoader loader{context, std::move(loaded_databases), mode}; - loader.loadTables(); - loader.startupTables(); + auto load_tasks = loader.loadTablesAsync(); + auto startup_tasks = loader.startupTablesAsync(); + + // First, load all tables + scheduleLoad(load_tasks); + waitLoad(load_tasks); // TODO(serxa): only wait for tables that must be loaded before server start + + // Then, startup all tables + scheduleLoad(startup_tasks); + waitLoad(startup_tasks); // TODO(serxa): only wait for tables that must be started before server start if (has_force_restore_data_flag) { try { - fs::remove(force_restore_data_flag_file); + fs::remove(force_restore_data_flag_file); // TODO(serxa): when we should remove it with async loading? should we disable async loading with restore? } catch (...) { tryLogCurrentException("Load metadata", "Can't remove force restore file to enable data sanity checks"); } } + + return joinTasks(load_tasks, startup_tasks); } static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & database_name, const String & default_engine) @@ -425,11 +435,11 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons {database_name, DatabaseCatalog::instance().getDatabase(database_name)}, }; TablesLoader loader{context, databases, LoadingStrictnessLevel::FORCE_RESTORE}; - loader.loadTables(); + scheduleAndWaitLoad(loader.loadTablesAsync()); /// Startup tables if they were started before conversion and detach/attach if (tables_started) - loader.startupTables(); + scheduleAndWaitLoad(loader.startupTablesAsync()); } catch (Exception & e) { @@ -450,7 +460,7 @@ void maybeConvertSystemDatabase(ContextMutablePtr context) maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, /* tables_started */ false); } -void convertDatabasesEnginesIfNeed(ContextMutablePtr context) +void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context) { auto convert_flag_path = fs::path(context->getFlagsPath()) / "convert_ordinary_to_atomic"; if (!fs::exists(convert_flag_path)) @@ -459,6 +469,9 @@ void convertDatabasesEnginesIfNeed(ContextMutablePtr context) LOG_INFO(&Poco::Logger::get("loadMetadata"), "Found convert_ordinary_to_atomic file in flags directory, " "will try to convert all Ordinary databases to Atomic"); + // Wait for all table to be loaded and started + waitLoad(load_metadata); + for (const auto & [name, _] : DatabaseCatalog::instance().getDatabases()) if (name != DatabaseCatalog::SYSTEM_DATABASE) maybeConvertOrdinaryDatabaseToAtomic(context, name, /* tables_started */ true); @@ -486,7 +499,7 @@ void loadMetadataSystem(ContextMutablePtr context) {DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, DatabaseCatalog::instance().getDatabase(DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)}, }; TablesLoader loader{context, databases, LoadingStrictnessLevel::FORCE_RESTORE}; - loader.loadTables(); + scheduleAndWaitLoad(loader.loadTablesAsync()); /// Will startup tables in system database after all databases are loaded. } diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 3553011fe4d..7ded7faf7c7 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -1,7 +1,7 @@ #pragma once #include - +#include namespace DB { @@ -12,7 +12,7 @@ void loadMetadataSystem(ContextMutablePtr context); /// Load tables from databases and add them to context. Database 'system' and 'information_schema' is ignored. /// Use separate function to load system tables. -void loadMetadata(ContextMutablePtr context, const String & default_database_name = {}); +[[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}); /// Background operations in system tables may slowdown loading of the rest tables, /// so we startup system tables after all databases are loaded. @@ -22,6 +22,7 @@ void startupSystemTables(); void maybeConvertSystemDatabase(ContextMutablePtr context); /// Converts all databases (except system) from Ordinary to Atomic if convert_ordinary_to_atomic flag exists -void convertDatabasesEnginesIfNeed(ContextMutablePtr context); +/// Waits for `load_metadata` task before convertions +void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context); } From 9e115e142ecd69e6b2c078cf74e965b06e1a5f9e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 1 May 2023 12:42:42 +0000 Subject: [PATCH 052/628] fix typo --- src/Interpreters/loadMetadata.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 7ded7faf7c7..9a0963c2d97 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -22,7 +22,7 @@ void startupSystemTables(); void maybeConvertSystemDatabase(ContextMutablePtr context); /// Converts all databases (except system) from Ordinary to Atomic if convert_ordinary_to_atomic flag exists -/// Waits for `load_metadata` task before convertions +/// Waits for `load_metadata` task before conversions void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context); } From 85d66984b2dfda635aa4c912c0310795fcdf385a Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 1 May 2023 12:50:07 +0000 Subject: [PATCH 053/628] cleanup --- programs/server/config.d/move_test.xml | 27 -------------------------- 1 file changed, 27 deletions(-) delete mode 100644 programs/server/config.d/move_test.xml diff --git a/programs/server/config.d/move_test.xml b/programs/server/config.d/move_test.xml deleted file mode 100644 index 330fea6a1a4..00000000000 --- a/programs/server/config.d/move_test.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - From c5181019217d9af1c58ecced38c5d0d75a5b644e Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 2 May 2023 08:44:03 +0000 Subject: [PATCH 054/628] improve helper functions --- src/Common/AsyncLoader.h | 38 ++++++++++++++++--------- src/Common/tests/gtest_async_loader.cpp | 34 +++++++++++----------- 2 files changed, 42 insertions(+), 30 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 80ec50e3fc3..311a0e89359 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -45,19 +45,14 @@ enum class LoadStatus class LoadJob : private boost::noncopyable { public: - template - LoadJob(LoadJobSet && dependencies_, String name_, ssize_t priority_, Func && func_) - : dependencies(std::move(dependencies_)) + template + LoadJob(LoadJobSetType && dependencies_, String name_, Func && func_, ssize_t priority_ = 0) + : dependencies(std::forward(dependencies_)) , name(std::move(name_)) , func(std::forward(func_)) , load_priority(priority_) {} - template - LoadJob(LoadJobSet && dependencies_, String name_, Func && func_) - : LoadJob(std::move(dependencies_), std::move(name_), 0, std::forward(func_)) - {} - // Current job status. LoadStatus status() const; std::exception_ptr exception() const; @@ -103,16 +98,33 @@ private: UInt64 finished_ns = 0; }; -template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func) +struct EmptyJobFunc +{ + void operator()(const LoadJobPtr &) {} +}; + +template +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func = EmptyJobFunc()) { return std::make_shared(std::move(dependencies), std::move(name), std::forward(func)); } -template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, ssize_t priority, Func && func) +template +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(std::move(dependencies), std::move(name), priority, std::forward(func)); + return std::make_shared(dependencies, std::move(name), std::forward(func)); +} + +template +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +{ + return std::make_shared(std::move(dependencies), std::move(name), std::forward(func), priority); +} + +template +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +{ + return std::make_shared(dependencies, std::move(name), std::forward(func), priority); } // Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set). diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 557c8481fca..2d9a6502046 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -135,7 +135,7 @@ TEST(AsyncLoader, Smoke) auto job3 = makeLoadJob({ job2 }, "job3", job_func); auto job4 = makeLoadJob({ job2 }, "job4", job_func); auto task2 = t.schedule({ job3, job4 }); - auto job5 = makeLoadJob({ job3, job4 }, "job5", low_priority, job_func); + auto job5 = makeLoadJob({ job3, job4 }, low_priority, "job5", job_func); task2->merge(t.schedule({ job5 })); std::thread waiter_thread([=] { job5->wait(); }); @@ -572,14 +572,14 @@ TEST(AsyncLoader, StaticPriorities) }; std::vector jobs; - jobs.push_back(makeLoadJob({}, "A", 0, job_func)); // 0 - jobs.push_back(makeLoadJob({ jobs[0] }, "B", 3, job_func)); // 1 - jobs.push_back(makeLoadJob({ jobs[0] }, "C", 4, job_func)); // 2 - jobs.push_back(makeLoadJob({ jobs[0] }, "D", 1, job_func)); // 3 - jobs.push_back(makeLoadJob({ jobs[0] }, "E", 2, job_func)); // 4 - jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", 0, job_func)); // 5 - jobs.push_back(makeLoadJob({ jobs[5] }, "G", 0, job_func)); // 6 - jobs.push_back(makeLoadJob({ jobs[6] }, "H", 9, job_func)); // 7 + jobs.push_back(makeLoadJob({}, 0, "A", job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, 3, "B", job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, 4, "C", job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, 1, "D", job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, 2, "E", job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, 0, "F", job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, 0, "G", job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, 9, "H", job_func)); // 7 auto task = t.schedule({ jobs.begin(), jobs.end() }); t.loader.start(); @@ -614,14 +614,14 @@ TEST(AsyncLoader, DynamicPriorities) // | +-> F0 --> G0 --> H0 // `-> E2 -' std::vector jobs; - jobs.push_back(makeLoadJob({}, "A", 0, job_func)); // 0 - jobs.push_back(makeLoadJob({ jobs[0] }, "B", 3, job_func)); // 1 - jobs.push_back(makeLoadJob({ jobs[0] }, "C", 4, job_func)); // 2 - jobs.push_back(makeLoadJob({ jobs[0] }, "D", 1, job_func)); // 3 - jobs.push_back(makeLoadJob({ jobs[0] }, "E", 2, job_func)); // 4 - jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, "F", 0, job_func)); // 5 - jobs.push_back(makeLoadJob({ jobs[5] }, "G", 0, job_func)); // 6 - jobs.push_back(makeLoadJob({ jobs[6] }, "H", 0, job_func)); // 7 + jobs.push_back(makeLoadJob({}, 0, "A", job_func)); // 0 + jobs.push_back(makeLoadJob({ jobs[0] }, 3, "B", job_func)); // 1 + jobs.push_back(makeLoadJob({ jobs[0] }, 4, "C", job_func)); // 2 + jobs.push_back(makeLoadJob({ jobs[0] }, 1, "D", job_func)); // 3 + jobs.push_back(makeLoadJob({ jobs[0] }, 2, "E", job_func)); // 4 + jobs.push_back(makeLoadJob({ jobs[3], jobs[4] }, 0, "F", job_func)); // 5 + jobs.push_back(makeLoadJob({ jobs[5] }, 0, "G", job_func)); // 6 + jobs.push_back(makeLoadJob({ jobs[6] }, 0, "H", job_func)); // 7 auto task = t.schedule({ jobs.begin(), jobs.end() }); job_to_prioritize = jobs[6]; From 6eda6e858508bc26e20bdbfab48c24afd70cbb5a Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 2 May 2023 09:18:50 +0000 Subject: [PATCH 055/628] implement async loading interface in database engines --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Common/CurrentMetrics.cpp | 2 - src/Databases/DatabaseAtomic.cpp | 40 ++-- src/Databases/DatabaseAtomic.h | 6 +- src/Databases/DatabaseOrdinary.cpp | 188 ++++++++---------- src/Databases/DatabaseOrdinary.h | 31 ++- src/Databases/DatabaseReplicated.cpp | 45 +++-- src/Databases/DatabaseReplicated.h | 8 +- src/Databases/IDatabase.h | 12 +- .../MySQL/DatabaseMaterializedMySQL.cpp | 21 +- .../MySQL/DatabaseMaterializedMySQL.h | 4 +- src/Databases/MySQL/DatabaseMySQL.cpp | 1 - .../DatabaseMaterializedPostgreSQL.cpp | 15 +- .../DatabaseMaterializedPostgreSQL.h | 4 +- src/Databases/TablesLoader.cpp | 2 - src/Databases/TablesLoader.h | 15 +- src/Interpreters/loadMetadata.cpp | 14 +- src/Interpreters/loadMetadata.h | 2 +- 19 files changed, 214 insertions(+), 200 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 664a611eb2b..0c2d000743c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -664,7 +664,7 @@ void LocalServer::processConfig() attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); - startupSystemTables(); + startupSystemTables(global_context); if (!config().has("only-system-tables")) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f041d98adc0..1f3545ec7b6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1627,7 +1627,7 @@ try load_metadata = loadMetadata(global_context, default_database); /// If we need to convert database engines, disable async tables loading convertDatabasesEnginesIfNeed(load_metadata, global_context); - startupSystemTables(); + startupSystemTables(global_context); database_catalog.startupBackgroundCleanup(); /// After loading validate that default database exists database_catalog.assertDatabaseExists(default_database); diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index e3eea93a25b..1dd4dc47fad 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -118,8 +118,6 @@ M(MergeTreeBackgroundExecutorThreadsActive, "Number of threads in the MergeTreeBackgroundExecutor thread pool running a task.") \ M(AsynchronousInsertThreads, "Number of threads in the AsynchronousInsert thread pool.") \ M(AsynchronousInsertThreadsActive, "Number of threads in the AsynchronousInsert thread pool running a task.") \ - M(StartupSystemTablesThreads, "Number of threads in the StartupSystemTables thread pool.") \ - M(StartupSystemTablesThreadsActive, "Number of threads in the StartupSystemTables thread pool running a task.") \ M(AggregatorThreads, "Number of threads in the Aggregator thread pool.") \ M(AggregatorThreadsActive, "Number of threads in the Aggregator thread pool running a task.") \ M(DDLWorkerThreads, "Number of threads in the DDLWorker thread pool for ON CLUSTER queries.") \ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 7e20b6f6535..81a6781f9c4 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -441,29 +441,29 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin } } -void DatabaseAtomic::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +LoadTaskPtr DatabaseAtomic::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { - beforeLoadingMetadata(local_context, mode); - DatabaseOrdinary::loadStoredObjects(local_context, mode, skip_startup_tables); -} + std::scoped_lock lock{mutex}; + auto job = makeLoadJob( + DatabaseOrdinary::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + DATABASE_STARTUP_PRIORITY, + fmt::format("startup Atomic database {}", database_name), + [this, mode] (const LoadJobPtr &) + { + if (mode < LoadingStrictnessLevel::FORCE_RESTORE) + return; -void DatabaseAtomic::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) -{ - DatabaseOrdinary::startupTables(thread_pool, mode); + NameToPathMap table_names; + { + std::lock_guard lock2{mutex}; + table_names = table_name_to_path; + } - if (mode < LoadingStrictnessLevel::FORCE_RESTORE) - return; - - NameToPathMap table_names; - { - std::lock_guard lock{mutex}; - table_names = table_name_to_path; - } - - fs::create_directories(path_to_table_symlinks); - for (const auto & table : table_names) - tryCreateSymlink(table.first, table.second, true); + fs::create_directories(path_to_table_symlinks); + for (const auto & table : table_names) + tryCreateSymlink(table.first, table.second, true); + }); + return startup_atomic_database_task = makeLoadTask(async_loader, {job}); } void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path, bool if_data_path_exist) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index cb275812098..8a1aa92ef4e 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -48,11 +48,9 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; - void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; - void startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) override; + LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleanup) override; @@ -87,6 +85,8 @@ protected: String path_to_table_symlinks; String path_to_metadata_symlink; const UUID db_uuid; + + LoadTaskPtr startup_atomic_database_task TSA_GUARDED_BY(mutex); }; } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 0db16f80656..c328f5b963c 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -88,82 +88,10 @@ DatabaseOrdinary::DatabaseOrdinary( { } -void DatabaseOrdinary::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel, bool) { - /** Tables load faster if they are loaded in sorted (by name) order. - * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, - * which does not correspond to order tables creation and does not correspond to order of their location on disk. - */ - - ParsedTablesMetadata metadata; - bool force_attach = LoadingStrictnessLevel::FORCE_ATTACH <= mode; - loadTablesMetadata(local_context, metadata, force_attach); - - size_t total_tables = metadata.parsed_tables.size() - metadata.total_dictionaries; - - AtomicStopwatch watch; - std::atomic dictionaries_processed{0}; - std::atomic tables_processed{0}; - - ThreadPool pool(CurrentMetrics::DatabaseOrdinaryThreads, CurrentMetrics::DatabaseOrdinaryThreadsActive); - - /// We must attach dictionaries before attaching tables - /// because while we're attaching tables we may need to have some dictionaries attached - /// (for example, dictionaries can be used in the default expressions for some tables). - /// On the other hand we can attach any dictionary (even sourced from ClickHouse table) - /// without having any tables attached. It is so because attaching of a dictionary means - /// loading of its config only, it doesn't involve loading the dictionary itself. - - /// Attach dictionaries. - for (const auto & name_with_path_and_query : metadata.parsed_tables) - { - const auto & name = name_with_path_and_query.first; - const auto & path = name_with_path_and_query.second.path; - const auto & ast = name_with_path_and_query.second.ast; - const auto & create_query = ast->as(); - - if (create_query.is_dictionary) - { - pool.scheduleOrThrowOnError([&]() - { - loadTableFromMetadata(local_context, path, name, ast, mode); - - /// Messages, so that it's not boring to wait for the server to load for a long time. - logAboutProgress(log, ++dictionaries_processed, metadata.total_dictionaries, watch); - }); - } - } - - pool.wait(); - - /// Attach tables. - for (const auto & name_with_path_and_query : metadata.parsed_tables) - { - const auto & name = name_with_path_and_query.first; - const auto & path = name_with_path_and_query.second.path; - const auto & ast = name_with_path_and_query.second.ast; - const auto & create_query = ast->as(); - - if (!create_query.is_dictionary) - { - pool.scheduleOrThrowOnError([&]() - { - loadTableFromMetadata(local_context, path, name, ast, mode); - - /// Messages, so that it's not boring to wait for the server to load for a long time. - logAboutProgress(log, ++tables_processed, total_tables, watch); - }); - } - } - - pool.wait(); - - if (!skip_startup_tables) - { - /// After all tables was basically initialized, startup them. - startupTables(pool, mode); - } + // Because it supportsLoadingInTopologicalOrder, we don't need this loading method. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTablesMetadata & metadata, bool is_startup) @@ -236,7 +164,11 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables TSA_SUPPRESS_WARNING_FOR_READ(database_name), tables_in_database, dictionaries_in_database); } -void DatabaseOrdinary::loadTableFromMetadata(ContextMutablePtr local_context, const String & file_path, const QualifiedTableName & name, const ASTPtr & ast, +void DatabaseOrdinary::loadTableFromMetadata( + ContextMutablePtr local_context, + const String & file_path, + const QualifiedTableName & name, + const ASTPtr & ast, LoadingStrictnessLevel mode) { assert(name.database == TSA_SUPPRESS_WARNING_FOR_READ(database_name)); @@ -247,43 +179,89 @@ void DatabaseOrdinary::loadTableFromMetadata(ContextMutablePtr local_context, co create_query, *this, name.database, - file_path, LoadingStrictnessLevel::FORCE_RESTORE <= mode); + file_path, + LoadingStrictnessLevel::FORCE_RESTORE <= mode); } -void DatabaseOrdinary::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel /*mode*/) +LoadTaskPtr DatabaseOrdinary::loadTableFromMetadataAsync( + AsyncLoader & async_loader, + LoadJobSet load_after, + ContextMutablePtr local_context, + const String & file_path, + const QualifiedTableName & name, + const ASTPtr & ast, + LoadingStrictnessLevel mode) +{ + std::scoped_lock lock(mutex); + auto job = makeLoadJob( + std::move(load_after), + TABLE_LOAD_PRIORITY, + fmt::format("load table {}", name.getFullName()), + [this, local_context, file_path, name, ast, mode] (const LoadJobPtr &) + { + loadTableFromMetadata(local_context, file_path, name, ast, mode); + }); + + return load_table[name.table] = makeLoadTask(async_loader, {job}); +} + +void DatabaseOrdinary::startupTablesAndDatabase(AsyncLoader & async_loader, LoadingStrictnessLevel mode) { LOG_INFO(log, "Starting up tables."); + LoadTaskPtrs tasks; + for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) + tasks.push_back(startupTableAsync( + async_loader, + {}, + QualifiedTableName{.database = getDatabaseName(), .table = table.first}, + mode)); + scheduleAndWaitLoad(tasks); + scheduleAndWaitLoad(startupDatabaseAsync(async_loader, {}, mode)); +} - /// NOTE No concurrent writes are possible during database loading - const size_t total_tables = TSA_SUPPRESS_WARNING_FOR_READ(tables).size(); - if (!total_tables) - return; +LoadTaskPtr DatabaseOrdinary::startupTableAsync( + AsyncLoader & async_loader, + LoadJobSet startup_after, + const QualifiedTableName & name, + LoadingStrictnessLevel /*mode*/) +{ + std::scoped_lock lock(mutex); - AtomicStopwatch watch; - std::atomic tables_processed{0}; - - auto startup_one_table = [&](const StoragePtr & table) + /// Initialize progress indication on the first call + if (total_tables_to_startup == 0) { - /// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on) - /// until startup finished. - auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef().lock_acquire_timeout); - table->startup(); - logAboutProgress(log, ++tables_processed, total_tables, watch); - }; - - - try - { - for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) - thread_pool.scheduleOrThrowOnError([&]() { startup_one_table(table.second); }); + total_tables_to_startup = tables.size(); + startup_watch.restart(); } - catch (...) - { - /// We have to wait for jobs to finish here, because job function has reference to variables on the stack of current thread. - thread_pool.wait(); - throw; - } - thread_pool.wait(); + + auto job = makeLoadJob( + std::move(startup_after), + TABLE_STARTUP_PRIORITY, + fmt::format("startup table {}", name.getFullName()), + [this, table = getTableUnlocked(name.table)] (const LoadJobPtr &) + { + /// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on) + /// until startup finished. + auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef().lock_acquire_timeout); + table->startup(); + logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); + }); + + return startup_table[name.table] = makeLoadTask(async_loader, {job}); +} + +LoadTaskPtr DatabaseOrdinary::startupDatabaseAsync( + AsyncLoader & async_loader, + LoadJobSet startup_after, + LoadingStrictnessLevel /*mode*/) +{ + std::scoped_lock lock(mutex); + // NOTE: this task is empty, but it is required for correct dependency handling (startup should be done after tables loading) + auto job = makeLoadJob( + std::move(startup_after), + DATABASE_STARTUP_PRIORITY, + fmt::format("startup Ordinary database {}", database_name)); + return makeLoadTask(async_loader, {job}); } void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index f9aa3214ef5..d4e9d40cbcf 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -27,10 +27,31 @@ public: void loadTablesMetadata(ContextPtr context, ParsedTablesMetadata & metadata, bool is_startup) override; - void loadTableFromMetadata(ContextMutablePtr local_context, const String & file_path, const QualifiedTableName & name, const ASTPtr & ast, + void loadTableFromMetadata( + ContextMutablePtr local_context, + const String & file_path, + const QualifiedTableName & name, + const ASTPtr & ast, LoadingStrictnessLevel mode) override; - void startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) override; + LoadTaskPtr loadTableFromMetadataAsync( + AsyncLoader & async_loader, + LoadJobSet load_after, + ContextMutablePtr local_context, + const String & file_path, + const QualifiedTableName & name, + const ASTPtr & ast, + LoadingStrictnessLevel mode) override; + + void startupTablesAndDatabase(AsyncLoader & async_loader, LoadingStrictnessLevel mode) override; + + LoadTaskPtr startupTableAsync( + AsyncLoader & async_loader, + LoadJobSet startup_after, + const QualifiedTableName & name, + LoadingStrictnessLevel mode) override; + + LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; void alterTable( ContextPtr context, @@ -48,6 +69,12 @@ protected: ContextPtr query_context); Strings permanently_detached_tables; + + std::unordered_map load_table TSA_GUARDED_BY(mutex); + std::unordered_map startup_table TSA_GUARDED_BY(mutex); + std::atomic total_tables_to_startup{0}; + std::atomic tables_started{0}; + AtomicStopwatch startup_watch; }; } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 5f5cd2667cb..297d3ef3b08 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -448,41 +448,42 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt createEmptyLogEntry(current_zookeeper); } -void DatabaseReplicated::beforeLoadingMetadata(ContextMutablePtr /*context*/, LoadingStrictnessLevel mode) +void DatabaseReplicated::beforeLoadingMetadata(ContextMutablePtr context_, LoadingStrictnessLevel mode) { + DatabaseAtomic::beforeLoadingMetadata(context_, mode); tryConnectToZooKeeperAndInitDatabase(mode); } -void DatabaseReplicated::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) -{ - beforeLoadingMetadata(local_context, mode); - DatabaseAtomic::loadStoredObjects(local_context, mode, skip_startup_tables); -} - UInt64 DatabaseReplicated::getMetadataHash(const String & table_name) const { return DB::getMetadataHash(table_name, readMetadataFile(table_name)); } -void DatabaseReplicated::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) +LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { - DatabaseAtomic::startupTables(thread_pool, mode); + std::scoped_lock lock{mutex}; + auto job = makeLoadJob( + DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + DATABASE_STARTUP_PRIORITY, + fmt::format("startup Replicated database {}", database_name), + [this] (const LoadJobPtr &) + { + /// TSA: No concurrent writes are possible during loading + UInt64 digest = 0; + for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) + digest += getMetadataHash(table.first); - /// TSA: No concurrent writes are possible during loading - UInt64 digest = 0; - for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) - digest += getMetadataHash(table.first); + LOG_DEBUG(log, "Calculated metadata digest of {} tables: {}", TSA_SUPPRESS_WARNING_FOR_READ(tables).size(), digest); + chassert(!TSA_SUPPRESS_WARNING_FOR_READ(tables_metadata_digest)); + TSA_SUPPRESS_WARNING_FOR_WRITE(tables_metadata_digest) = digest; - LOG_DEBUG(log, "Calculated metadata digest of {} tables: {}", TSA_SUPPRESS_WARNING_FOR_READ(tables).size(), digest); - chassert(!TSA_SUPPRESS_WARNING_FOR_READ(tables_metadata_digest)); - TSA_SUPPRESS_WARNING_FOR_WRITE(tables_metadata_digest) = digest; + if (is_probably_dropped) + return; - if (is_probably_dropped) - return; - - ddl_worker = std::make_unique(this, getContext()); - ddl_worker->startup(); + ddl_worker = std::make_unique(this, getContext()); + ddl_worker->startup(); + }); + return startup_replicated_database_task = makeLoadTask(async_loader, {job}); } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 6a897f7322a..45f772f0d81 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -64,11 +64,9 @@ public: void drop(ContextPtr /*context*/) override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void beforeLoadingMetadata(ContextMutablePtr context_, LoadingStrictnessLevel mode) override; - void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; - - void startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) override; + LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; void shutdown() override; @@ -142,6 +140,8 @@ private: UInt64 tables_metadata_digest TSA_GUARDED_BY(metadata_mutex); mutable ClusterPtr cluster; + + LoadTaskPtr startup_replicated_database_task TSA_GUARDED_BY(mutex); }; } diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index db5dd057eb9..75a5f0c03e5 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -42,6 +42,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static constexpr auto TABLE_LOAD_PRIORITY = -1; /// Initial priority for table loading jobs +static constexpr auto TABLE_STARTUP_PRIORITY = -2; /// Initial priority for table startup jobs +static constexpr auto DATABASE_STARTUP_PRIORITY = -2; /// Initial priority for table startup jobs +static constexpr auto TABLE_WAIT_PRIORITY = 0; /// Prioritize load jobs that block queries + class IDatabaseTablesIterator { public: @@ -178,12 +183,11 @@ public: } /// Start all tables and the database itself - virtual void startupTables(ThreadPool & /*thread_pool*/, LoadingStrictnessLevel /*mode*/) {} + virtual void startupTablesAndDatabase(AsyncLoader & /*async_loader*/, LoadingStrictnessLevel /*mode*/) {} /// Create a task to startup table `name` after specified dependencies `startup_after` using `async_loader`. - /// `startup_after` must contain the task returned by `loadTableFromMetadataAsync()` for this table (see TablesLoader). /// The returned task is also stored inside the database for cancellation on destruction. - virtual LoadTaskPtr startupTableAsync( + [[nodiscard]] virtual LoadTaskPtr startupTableAsync( AsyncLoader & /*async_loader*/, LoadJobSet /*startup_after*/, const QualifiedTableName & /*name*/, @@ -195,7 +199,7 @@ public: /// Create a task to startup database after specified dependencies `startup_after` using `async_loader`. /// `startup_after` must contain all the tasks returned by `startupTableAsync()` for every table (see TablesLoader). /// The returned task is also stored inside the database for cancellation on destruction. - virtual LoadTaskPtr startupDatabaseAsync( + [[nodiscard]] virtual LoadTaskPtr startupDatabaseAsync( AsyncLoader & /*async_loader*/, LoadJobSet /*startup_after*/, LoadingStrictnessLevel /*mode*/) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 653c2dc27b6..6315c3a10e9 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -63,15 +63,22 @@ void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exceptio exception = exception_; } -void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) +LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { - DatabaseAtomic::startupTables(thread_pool, mode); + std::scoped_lock lock{mutex}; + auto job = makeLoadJob( + DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + DATABASE_STARTUP_PRIORITY, + fmt::format("startup MaterializedMySQL database {}", database_name), + [this, mode] (const LoadJobPtr &) + { + if (mode < LoadingStrictnessLevel::FORCE_ATTACH) + materialize_thread.assertMySQLAvailable(); - if (mode < LoadingStrictnessLevel::FORCE_ATTACH) - materialize_thread.assertMySQLAvailable(); - - materialize_thread.startSynchronization(); - started_up = true; + materialize_thread.startSynchronization(); + started_up = true; + }); + return startup_mysql_database_task = makeLoadTask(async_loader, {job}); } void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 3698abf5542..960fb909c9d 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -45,10 +45,12 @@ protected: std::atomic_bool started_up{false}; + LoadTaskPtr startup_mysql_database_task; + public: String getEngineName() const override { return "MaterializedMySQL"; } - void startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) override; + LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 9bd1dd65ab3..afab39db94a 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -404,7 +404,6 @@ String DatabaseMySQL::getMetadataPath() const void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) { - std::lock_guard lock{mutex}; fs::directory_iterator iter(getMetadataPath()); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 04807bb3daf..8022e14814d 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -125,13 +125,20 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() } -void DatabaseMaterializedPostgreSQL::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) +LoadTaskPtr DatabaseMaterializedPostgreSQL::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { - DatabaseAtomic::startupTables(thread_pool, mode); - startup_task->activateAndSchedule(); + std::scoped_lock lock{mutex}; + auto job = makeLoadJob( + DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + DATABASE_STARTUP_PRIORITY, + fmt::format("startup MaterializedMySQL database {}", database_name), + [this] (const LoadJobPtr &) + { + startup_task->activateAndSchedule(); + }); + return startup_postgresql_database_task = makeLoadTask(async_loader, {job}); } - void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) { std::lock_guard lock(handler_mutex); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index edbef281da4..6f8006bfcb9 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -40,7 +40,7 @@ public: String getMetadataPath() const override { return metadata_path; } - void startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) override; + LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; @@ -91,6 +91,8 @@ private: BackgroundSchedulePool::TaskHolder startup_task; bool shutdown_called = false; + + LoadTaskPtr startup_postgresql_database_task; }; } diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index adc735d2137..e2258c3b104 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -34,7 +33,6 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database , loading_dependencies("LoadingDeps") , all_loading_dependencies("LoadingDeps") , async_loader(global_context->getAsyncLoader()) - , pool(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive) { metadata.default_database = global_context->getCurrentDatabase(); log = &Poco::Logger::get("TablesLoader"); diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index e35ff3ca30c..7e2f5130a02 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -54,14 +54,15 @@ public: TablesLoader(ContextMutablePtr global_context_, Databases databases_, LoadingStrictnessLevel strictness_mode_); TablesLoader() = delete; - // Create tasks for async loading of all tables in `databases` after specified jobs `load_after` + /// Create tasks for async loading of all tables in `databases` after specified jobs `load_after`. [[nodiscard]] LoadTaskPtrs loadTablesAsync(LoadJobSet load_after = {}); - // Create tasks for async startup of all tables in `databases` after specified jobs `startup_after` - // Note that for every table startup an extra dependency on that table loading will be added along with `startup_after` + /// Create tasks for async startup of all tables in `databases` after specified jobs `startup_after`. + /// Note that for every table startup an extra dependency on that table loading will be added along with `startup_after`. + /// Must be called only after `loadTablesAsync()`. [[nodiscard]] LoadTaskPtrs startupTablesAsync(LoadJobSet startup_after = {}); - // Set of goal jobs for the whole loading process. Useful for scheduling more tasks after table loading + /// Set of goal jobs for the whole loading process. Useful for scheduling more tasks after table loading LoadJobSet goals(); private: @@ -80,10 +81,8 @@ private: AsyncLoader & async_loader; std::unordered_map load_table; /// table uuid -> load task - LoadTaskPtrs load_tasks; // Tasks to load all tables - LoadTaskPtrs startup_tasks; // Tasks to startup all tables and databases after loading - - ThreadPool pool; // TODO(serxa): get rid of it + LoadTaskPtrs load_tasks; /// Tasks to load all tables + LoadTaskPtrs startup_tasks; /// Tasks to startup all tables and databases after loading void buildDependencyGraph(); void removeUnresolvableDependencies(); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 4e7d3d182f9..1eaa9f34527 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -28,12 +28,6 @@ namespace fs = std::filesystem; -namespace CurrentMetrics -{ - extern const Metric StartupSystemTablesThreads; - extern const Metric StartupSystemTablesThreadsActive; -} - namespace DB { @@ -384,8 +378,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons if (!tables_started) { /// It's not quite correct to run DDL queries while database is not started up. - ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive); - DatabaseCatalog::instance().getSystemDatabase()->startupTables(pool, LoadingStrictnessLevel::FORCE_RESTORE); + startupSystemTables(context); // NOTE: tables_started can be false only for system tables } auto local_context = Context::createCopy(context); @@ -480,10 +473,9 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu fs::remove(convert_flag_path); } -void startupSystemTables() +void startupSystemTables(ContextMutablePtr context) { - ThreadPool pool(CurrentMetrics::StartupSystemTablesThreads, CurrentMetrics::StartupSystemTablesThreadsActive); - DatabaseCatalog::instance().getSystemDatabase()->startupTables(pool, LoadingStrictnessLevel::FORCE_RESTORE); + DatabaseCatalog::instance().getSystemDatabase()->startupTablesAndDatabase(context->getAsyncLoader(), LoadingStrictnessLevel::FORCE_RESTORE); } void loadMetadataSystem(ContextMutablePtr context) diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 9a0963c2d97..ca4c7ab0119 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -16,7 +16,7 @@ void loadMetadataSystem(ContextMutablePtr context); /// Background operations in system tables may slowdown loading of the rest tables, /// so we startup system tables after all databases are loaded. -void startupSystemTables(); +void startupSystemTables(ContextMutablePtr context); /// Converts `system` database from Ordinary to Atomic (if needed) void maybeConvertSystemDatabase(ContextMutablePtr context); From 53dbdab0f7191ae0bdc5811e74c768aec407ad9f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 3 May 2023 17:13:40 +0000 Subject: [PATCH 056/628] add AsyncLoader introspection --- src/Common/AsyncLoader.cpp | 41 +++++++++++++++++++++++++++++++++----- src/Common/AsyncLoader.h | 31 ++++++++++++++++++++++++---- 2 files changed, 63 insertions(+), 9 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index e6bd6eb12d4..dccba3e77c1 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -96,19 +96,25 @@ void LoadJob::canceled(const std::exception_ptr & ptr) void LoadJob::finish() { func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` and `LoadJob::wait()` return - finished_ns = clock_gettime_ns(); + finish_time = std::chrono::system_clock::now(); if (waiters > 0) finished.notify_all(); } void LoadJob::scheduled() { - scheduled_ns = clock_gettime_ns(); + schedule_time = std::chrono::system_clock::now(); +} + +void LoadJob::enqueued() +{ + if (enqueue_time.load() == TimePoint{}) // Do not rewrite in case of requeue + enqueue_time = std::chrono::system_clock::now(); } void LoadJob::execute(const LoadJobPtr & self) { - started_ns = clock_gettime_ns(); + start_time = std::chrono::system_clock::now(); func(self); } @@ -217,7 +223,7 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) // Restart watches after idle period if (scheduled_jobs.empty()) { - busy_period_started_ns = clock_gettime_ns(); + busy_period_start_time = std::chrono::system_clock::now(); stopwatch.restart(); old_jobs = finished_jobs.size(); } @@ -367,7 +373,7 @@ void AsyncLoader::remove(const LoadJobSet & jobs) for (const auto & job : jobs) { size_t erased = finished_jobs.erase(job); - if (old_jobs >= erased && job->finished_ns && job->finished_ns < busy_period_started_ns) + if (old_jobs >= erased && job->finishTime() != LoadJob::TimePoint{} && job->finishTime() < busy_period_start_time) old_jobs -= erased; } } @@ -397,6 +403,29 @@ size_t AsyncLoader::getScheduledJobCount() const return scheduled_jobs.size(); } +std::vector AsyncLoader::getJobStates() const +{ + std::unique_lock lock{mutex}; + std::multimap states; + for (const auto & [job, info] : scheduled_jobs) + states.emplace(job->name, JobState{ + .job = job, + .dependencies_left = info.dependencies_left, + .is_executing = info.is_executing(), + .is_blocked = info.is_blocked(), + .is_ready = info.is_ready(), + .initial_priority = info.initial_priority, + .ready_seqno = last_ready_seqno + }); + for (const auto & job : finished_jobs) + states.emplace(job->name, JobState{.job = job}); + lock.unlock(); + std::vector result; + for (auto && [_, state] : states) + result.emplace_back(std::move(state)); + return result; +} + void AsyncLoader::checkCycle(const LoadJobSet & jobs, std::unique_lock & lock) { LoadJobSet left = jobs; @@ -536,6 +565,8 @@ void AsyncLoader::enqueue(Info & info, const LoadJobPtr & job, std::unique_lock< ready_queue.emplace(info.key(), job); }); + job->enqueued(); + if (is_running && workers < max_threads) spawn(lock); } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 311a0e89359..1637b39f01a 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -70,6 +70,13 @@ public: // Returns number of threads blocked by `wait()` or `waitNoThrow()` calls. size_t waitersCount() const; + // Introspection + using TimePoint = std::chrono::system_clock::time_point; + TimePoint scheduleTime() const { return schedule_time; } + TimePoint enqueueTime() const { return enqueue_time; } + TimePoint startTime() const { return start_time; } + TimePoint finishTime() const { return finish_time; } + const LoadJobSet dependencies; // Jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard const String name; @@ -82,6 +89,7 @@ private: void finish(); void scheduled(); + void enqueued(); void execute(const LoadJobPtr & self); std::function func; @@ -93,9 +101,10 @@ private: LoadStatus load_status{LoadStatus::PENDING}; std::exception_ptr load_exception; - UInt64 scheduled_ns = 0; - UInt64 started_ns = 0; - UInt64 finished_ns = 0; + std::atomic schedule_time{TimePoint{}}; + std::atomic enqueue_time{TimePoint{}}; + std::atomic start_time{TimePoint{}}; + std::atomic finish_time{TimePoint{}}; }; struct EmptyJobFunc @@ -373,6 +382,20 @@ public: size_t getMaxThreads() const; size_t getScheduledJobCount() const; + // Helper class for introspection + struct JobState { + LoadJobPtr job; + size_t dependencies_left = 0; + bool is_executing = false; + bool is_blocked = false; + bool is_ready = false; + std::optional initial_priority; + std::optional ready_seqno; + }; + + // For introspection and debug only, see `system.async_loader` table + std::vector getJobStates() const; + private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); @@ -387,7 +410,7 @@ private: // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. Poco::Logger * log; - UInt64 busy_period_started_ns; + std::chrono::system_clock::time_point busy_period_start_time; AtomicStopwatch stopwatch; size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) From d834ffd59a7aa4b5c9cc77abb831e5cf66f8924b Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 3 May 2023 17:15:34 +0000 Subject: [PATCH 057/628] add system.async_loader table --- .../System/StorageSystemAsyncLoader.cpp | 135 ++++++++++++++++++ .../System/StorageSystemAsyncLoader.h | 26 ++++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 163 insertions(+) create mode 100644 src/Storages/System/StorageSystemAsyncLoader.cpp create mode 100644 src/Storages/System/StorageSystemAsyncLoader.h diff --git a/src/Storages/System/StorageSystemAsyncLoader.cpp b/src/Storages/System/StorageSystemAsyncLoader.cpp new file mode 100644 index 00000000000..1535657c5ec --- /dev/null +++ b/src/Storages/System/StorageSystemAsyncLoader.cpp @@ -0,0 +1,135 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using TimePoint = LoadJob::TimePoint; +using TimeDuration = LoadJob::TimePoint::duration; + +static constexpr auto TIME_SCALE = 6; + +namespace +{ + template + DataTypeEnum8::Values getTypeEnumValues() + { + DataTypeEnum8::Values enum_values; + for (auto value : magic_enum::enum_values()) + enum_values.emplace_back(magic_enum::enum_name(value), magic_enum::enum_integer(value)); + return enum_values; + } + + auto timeInMicroseconds(const TimePoint & time_point) + { + auto time_us = std::chrono::duration_cast(time_point.time_since_epoch()).count(); + DecimalUtils::DecimalComponents components{time_us / 1'000'000, time_us % 1'000'000}; + return DecimalField(DecimalUtils::decimalFromComponents(components, TIME_SCALE), TIME_SCALE); + } + + Field optionalTimeInMicroseconds(const TimePoint & time_point) + { + if (time_point == TimePoint{}) + return {}; + else + return timeInMicroseconds(time_point); + } +} + +NamesAndTypesList StorageSystemAsyncLoader::getNamesAndTypes() +{ + return { + { "job", std::make_shared() }, + { "dependencies", std::make_shared(std::make_shared()) }, + { "dependencies_left", std::make_shared() }, + { "status", std::make_shared(getTypeEnumValues()) }, + { "is_executing", std::make_shared() }, + { "is_blocked", std::make_shared() }, + { "is_ready", std::make_shared() }, + { "elapsed", std::make_shared()}, + { "priority", std::make_shared() }, + { "initial_priority", std::make_shared(std::make_shared()) }, + { "ready_seqno", std::make_shared(std::make_shared()) }, + { "waiters", std::make_shared() }, + { "exception", std::make_shared(std::make_shared()) }, + { "schedule_time", std::make_shared(TIME_SCALE) }, + { "enqueue_time", std::make_shared(std::make_shared(TIME_SCALE)) }, + { "start_time", std::make_shared(std::make_shared(TIME_SCALE)) }, + { "finish_time", std::make_shared(std::make_shared(TIME_SCALE)) }, + }; +} + +void StorageSystemAsyncLoader::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + TimePoint now = std::chrono::system_clock::now(); + + for (const auto & state : context->getAsyncLoader().getJobStates()) + { + + Array dependencies; + dependencies.reserve(state.job->dependencies.size()); + for (const auto & dep : state.job->dependencies) + dependencies.emplace_back(dep->name); + + TimePoint started = state.job->startTime(); + TimePoint finished = state.job->finishTime(); + TimePoint last = finished != TimePoint{} ? finished : now; + TimeDuration elapsed = started != TimePoint{} ? last - started : TimeDuration{0}; + double elapsed_sec = std::chrono::duration_cast(elapsed).count() * 1e-9; + + Field initial_priority; + if (state.initial_priority) + initial_priority = *state.initial_priority; + + Field ready_seqno; + if (state.ready_seqno) + ready_seqno = *state.ready_seqno; + + Field exception; + if (state.job->exception()) + { + try + { + std::rethrow_exception(state.job->exception()); + } + catch (Exception & e) + { + exception = e.message(); + } + catch (...) // just in case + { + exception = String("unknown error"); + } + } + + size_t i = 0; + res_columns[i++]->insert(state.job->name); + res_columns[i++]->insert(dependencies); + res_columns[i++]->insert(state.dependencies_left); + res_columns[i++]->insert(static_cast(state.job->status())); + res_columns[i++]->insert(state.is_executing); + res_columns[i++]->insert(state.is_blocked); + res_columns[i++]->insert(state.is_ready); + res_columns[i++]->insert(elapsed_sec); + res_columns[i++]->insert(state.job->priority()); + res_columns[i++]->insert(initial_priority); + res_columns[i++]->insert(ready_seqno); + res_columns[i++]->insert(state.job->waitersCount()); + res_columns[i++]->insert(exception); + res_columns[i++]->insert(timeInMicroseconds(state.job->scheduleTime())); + res_columns[i++]->insert(optionalTimeInMicroseconds(state.job->enqueueTime())); + res_columns[i++]->insert(optionalTimeInMicroseconds(state.job->startTime())); + res_columns[i++]->insert(optionalTimeInMicroseconds(state.job->finishTime())); + } +} + +} diff --git a/src/Storages/System/StorageSystemAsyncLoader.h b/src/Storages/System/StorageSystemAsyncLoader.h new file mode 100644 index 00000000000..b6e51107b1e --- /dev/null +++ b/src/Storages/System/StorageSystemAsyncLoader.h @@ -0,0 +1,26 @@ +#pragma once + + +#include + + +namespace DB +{ + +class Context; + +/// system.async_loader table. Takes data from context.getAsyncLoader() +class StorageSystemAsyncLoader final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemReplicatedFetches"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index d6982ba30d5..0df722774b7 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -184,6 +185,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "remote_data_paths"); attach(context, system_database, "certificates"); attach(context, system_database, "named_collections"); + attach(context, system_database, "async_loader"); if (has_zookeeper) attach(context, system_database, "zookeeper"); From defdd2a35434cf0af49d0833d33d98460700feb0 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 3 May 2023 15:41:12 +0200 Subject: [PATCH 058/628] Update src/Common/AsyncLoader.cpp Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index dccba3e77c1..69cc145d8c6 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -24,7 +24,7 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic { if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) { - LOG_INFO(log, "{}%", processed * 100.0 / total); + LOG_INFO(log, "Processed: {}%", processed * 100.0 / total); watch.restart(); } } From 8edb31823c26cf9191a4cf3f20ef7d84b4af6054 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 3 May 2023 15:42:18 +0200 Subject: [PATCH 059/628] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 1637b39f01a..7352ca20493 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -230,13 +230,11 @@ inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) inline LoadJobSet joinJobs(const LoadJobSet & jobs1, const LoadJobSet & jobs2) { - if (jobs1.empty()) - return jobs2; - if (jobs2.empty()) - return jobs1; LoadJobSet result; - result.insert(jobs1.begin(), jobs1.end()); - result.insert(jobs2.begin(), jobs2.end()); + if (!jobs1.empty()) + result.insert(jobs1.begin(), jobs1.end()); + if (!jobs2.empty()) + result.insert(jobs2.begin(), jobs2.end()); return result; } From 55e530fb19f1e48b111ad1b563aa7155e9a6acfc Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 3 May 2023 15:43:31 +0200 Subject: [PATCH 060/628] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 7352ca20493..39bd1ffe5b7 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -273,7 +273,7 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // 3b) If at least one of the job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). // On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. The job status is changed to CANCELED. // Exception is rethrown by any existing or new `wait()` call. The job is moved to the set of the finished jobs. -// 4) The scheduled pending ready job starts execution by a worker. The job is dequeuedCallback `job_func` is called. +// 4) The scheduled pending ready job starts execution by a worker. The job is dequeued. Callback `job_func` is called. // Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader. // Note that `job_func` of a CANCELED job is never executed. // 5a) On successful execution the job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. From 935b094187cbb2cac119da783f032cd18b6145f8 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 3 May 2023 15:43:52 +0200 Subject: [PATCH 061/628] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 39bd1ffe5b7..2e67aca2545 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -277,7 +277,7 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & // Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader. // Note that `job_func` of a CANCELED job is never executed. // 5a) On successful execution the job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. -// 5b) Any exception thrown out of `job_func` is wrapped into an ASYNC_LOAD_FAILED exception and save inside LoadJob. +// 5b) Any exception thrown out of `job_func` is wrapped into an ASYNC_LOAD_FAILED exception and saved inside LoadJob. // The job status is changed to FAILED. All the dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. // 6) The job is no longer considered as scheduled and is instead moved to the finished jobs set. This is just for introspection of the finished jobs. // 7) The task containing this job is destructed or `remove()` is explicitly called. The job is removed from the finished job set. From 93a4f7ffe4dafc75c6af2b29389f9e0851265d6c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 3 May 2023 18:43:29 +0000 Subject: [PATCH 062/628] do not pollute unittests output with progress --- src/Common/AsyncLoader.cpp | 6 ++++-- src/Common/AsyncLoader.h | 3 ++- src/Common/tests/gtest_async_loader.cpp | 2 +- src/Interpreters/Context.cpp | 3 ++- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 69cc145d8c6..6c19b1910eb 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -156,8 +156,9 @@ void LoadTask::detach() jobs.clear(); } -AsyncLoader::AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_) +AsyncLoader::AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_) : log_failures(log_failures_) + , log_progress(log_progress_) , log(&Poco::Logger::get("AsyncLoader")) , max_threads(max_threads_) , pool(metric_threads, metric_active_threads, max_threads) @@ -525,7 +526,8 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; finished_jobs.insert(job); - logAboutProgress(log, finished_jobs.size() - old_jobs, finished_jobs.size() + scheduled_jobs.size() - old_jobs, stopwatch); + if (log_progress) + logAboutProgress(log, finished_jobs.size() - old_jobs, finished_jobs.size() + scheduled_jobs.size() - old_jobs, stopwatch); }); } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 2e67aca2545..f1c4d3cf0da 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -339,7 +339,7 @@ private: public: using Metric = CurrentMetrics::Metric; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_); + AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_); // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); @@ -407,6 +407,7 @@ private: // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. + const bool log_progress; // Periodically log total progress Poco::Logger * log; std::chrono::system_clock::time_point busy_period_start_time; AtomicStopwatch stopwatch; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 2d9a6502046..ee52dfd96b4 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -38,7 +38,7 @@ struct AsyncLoaderTest pcg64 rng{randomSeed()}; explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, max_threads, /* log_failures = */ false) + : loader(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, max_threads, /* log_failures = */ false, /* log_progress = */ false) {} template diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 281f599d33d..84013e922eb 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1947,7 +1947,8 @@ AsyncLoader & Context::getAsyncLoader() const CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, getAsyncLoaderPoolSize(), - /* log_failures = */ true); + /* log_failures = */ true, + /* log_progress = */ true); return *shared->async_loader; } From 8539882a769376c980adc13285cc4ac049f9f6a5 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 3 May 2023 18:57:55 +0000 Subject: [PATCH 063/628] fix style --- src/Common/AsyncLoader.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index f1c4d3cf0da..5c16efab16f 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -381,7 +381,8 @@ public: size_t getScheduledJobCount() const; // Helper class for introspection - struct JobState { + struct JobState + { LoadJobPtr job; size_t dependencies_left = 0; bool is_executing = false; From 3327c0fa1e0490531ab9db8baf44c048bcce353a Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 3 May 2023 19:00:58 +0000 Subject: [PATCH 064/628] fix style, get rid of DatabaseOrdinaryThreads* metrics --- src/Common/CurrentMetrics.cpp | 2 -- src/Databases/DatabaseOrdinary.cpp | 6 ------ src/Databases/TablesLoader.cpp | 6 ------ 3 files changed, 14 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 1dd4dc47fad..e471eefbcb0 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -98,8 +98,6 @@ M(StorageHiveThreadsActive, "Number of threads in the StorageHive thread pool running a task.") \ M(AsyncLoaderThreads, "Number of threads in the async loader thread pool.") \ M(AsyncLoaderThreadsActive, "Number of threads in the async loader thread pool running a task.") \ - M(DatabaseOrdinaryThreads, "Number of threads in the Ordinary database thread pool.") \ - M(DatabaseOrdinaryThreadsActive, "Number of threads in the Ordinary database thread pool running a task.") \ M(DatabaseOnDiskThreads, "Number of threads in the DatabaseOnDisk thread pool.") \ M(DatabaseOnDiskThreadsActive, "Number of threads in the DatabaseOnDisk thread pool running a task.") \ M(DatabaseCatalogThreads, "Number of threads in the DatabaseCatalog thread pool.") \ diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index c328f5b963c..cc716e7c58a 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -29,12 +29,6 @@ namespace fs = std::filesystem; -namespace CurrentMetrics -{ - extern const Metric DatabaseOrdinaryThreads; - extern const Metric DatabaseOrdinaryThreadsActive; -} - namespace DB { diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index e2258c3b104..c825bc406c2 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -11,12 +11,6 @@ #include -namespace CurrentMetrics -{ - extern const Metric AsyncLoaderThreads; - extern const Metric AsyncLoaderThreadsActive; -} - namespace DB { From 6bbc061ba08e7c3d472435127c69212141e96dae Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 10 May 2023 19:56:48 +0000 Subject: [PATCH 065/628] start loader, fix helpers, fix deadlock --- programs/server/Server.cpp | 2 ++ src/Common/AsyncLoader.h | 8 ++++---- src/Databases/DatabaseAtomic.cpp | 3 ++- src/Databases/DatabaseReplicated.cpp | 3 ++- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 3 ++- .../PostgreSQL/DatabaseMaterializedPostgreSQL.cpp | 3 ++- src/Databases/TablesLoader.h | 1 + src/Interpreters/Context.h | 2 +- 8 files changed, 16 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b8f4e92406a..3f1e7ea9fa3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -779,6 +779,8 @@ try 0, // We don't need any threads one all the parts will be loaded server_settings.outdated_part_loading_thread_pool_queue_size); + global_context->getAsyncLoader().start(); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 5c16efab16f..9f842cc5907 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -185,7 +185,7 @@ inline void scheduleLoad(const LoadTaskPtrs & tasks) } template -inline void scheduleLoad(Args && ... args) +inline void scheduleLoadAll(Args && ... args) { (scheduleLoad(std::forward(args)), ...); } @@ -208,7 +208,7 @@ inline void waitLoad(const LoadTaskPtrs & tasks) } template -inline void waitLoad(Args && ... args) +inline void waitLoadAll(Args && ... args) { (waitLoad(std::forward(args)), ...); } @@ -216,8 +216,8 @@ inline void waitLoad(Args && ... args) template inline void scheduleAndWaitLoad(Args && ... args) { - scheduleLoad(std::forward(args)...); - waitLoad(std::forward(args)...); + scheduleLoadAll(std::forward(args)...); + waitLoadAll(std::forward(args)...); } inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 81a6781f9c4..c1b5f4d0067 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -443,9 +443,10 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin LoadTaskPtr DatabaseAtomic::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { + auto base = DatabaseOrdinary::startupDatabaseAsync(async_loader, std::move(startup_after), mode); std::scoped_lock lock{mutex}; auto job = makeLoadJob( - DatabaseOrdinary::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + base->goals(), DATABASE_STARTUP_PRIORITY, fmt::format("startup Atomic database {}", database_name), [this, mode] (const LoadJobPtr &) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 8de7afab18a..da9cb3cb783 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -502,9 +502,10 @@ UInt64 DatabaseReplicated::getMetadataHash(const String & table_name) const LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { + auto base = DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode); std::scoped_lock lock{mutex}; auto job = makeLoadJob( - DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + base->goals(), DATABASE_STARTUP_PRIORITY, fmt::format("startup Replicated database {}", database_name), [this] (const LoadJobPtr &) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 6315c3a10e9..1c0d1fd3c05 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -65,9 +65,10 @@ void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exceptio LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { + auto base = DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode); std::scoped_lock lock{mutex}; auto job = makeLoadJob( - DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + base->goals(), DATABASE_STARTUP_PRIORITY, fmt::format("startup MaterializedMySQL database {}", database_name), [this, mode] (const LoadJobPtr &) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 8022e14814d..15986bd04b6 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -127,9 +127,10 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() LoadTaskPtr DatabaseMaterializedPostgreSQL::startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) { + auto base = DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode); std::scoped_lock lock{mutex}; auto job = makeLoadJob( - DatabaseAtomic::startupDatabaseAsync(async_loader, std::move(startup_after), mode)->goals(), + base->goals(), DATABASE_STARTUP_PRIORITY, fmt::format("startup MaterializedMySQL database {}", database_name), [this] (const LoadJobPtr &) diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 7e2f5130a02..52e230b9511 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -12,6 +12,7 @@ #include #include + namespace Poco { class Logger; // NOLINT(cppcoreguidelines-virtual-class-destructor) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e57b88b6157..f720bd4c0fc 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -130,6 +129,7 @@ class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; template class MergeTreeBackgroundExecutor; +class AsyncLoader; /// Scheduling policy can be changed using `background_merges_mutations_scheduling_policy` config option. /// By default concurrent merges are scheduled using "round_robin" to ensure fair and starvation-free operation. From a7f514e66eb8f124757ec651ab648233cd56ea0c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 11 May 2023 09:09:32 +0000 Subject: [PATCH 066/628] fix startup of database w/o tables --- src/Common/AsyncLoader.h | 8 ++++++++ src/Databases/TablesLoader.cpp | 11 +++-------- src/Databases/TablesLoader.h | 3 --- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 9f842cc5907..b5725dd601c 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -228,6 +228,14 @@ inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) return result; } +inline LoadJobSet getGoalsOr(const LoadTaskPtrs & tasks, const LoadJobSet & alternative) +{ + LoadJobSet result; + for (const auto & task : tasks) + result.insert(task->goals().begin(), task->goals().end()); + return result.empty() ? alternative : result; +} + inline LoadJobSet joinJobs(const LoadJobSet & jobs1, const LoadJobSet & jobs2) { LoadJobSet result; diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index c825bc406c2..9682609435c 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -87,7 +87,7 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) const auto & path_and_query = metadata.parsed_tables[table_name]; auto task = databases[table_name.database]->loadTableFromMetadataAsync( async_loader, - load_dependency_tasks.empty() ? load_after : getGoals(load_dependency_tasks), + getGoalsOr(load_dependency_tasks, load_after), load_context, path_and_query.path, table_name, @@ -117,11 +117,11 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) } /// Make startup database tasks - for (auto [database_name, startup_table_tasks] : startup_database) + for (auto & database_name : databases_to_load) { auto task = databases[database_name]->startupDatabaseAsync( async_loader, - getGoals(startup_table_tasks), + getGoalsOr(startup_database[database_name], startup_after), strictness_mode); startup_tasks.push_back(task); } @@ -129,11 +129,6 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) return startup_tasks; } -LoadJobSet TablesLoader::goals() -{ - return getGoals(startup_tasks); -} - void TablesLoader::buildDependencyGraph() { for (const auto & [table_name, table_metadata] : metadata.parsed_tables) diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 52e230b9511..321fa2393b9 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -63,9 +63,6 @@ public: /// Must be called only after `loadTablesAsync()`. [[nodiscard]] LoadTaskPtrs startupTablesAsync(LoadJobSet startup_after = {}); - /// Set of goal jobs for the whole loading process. Useful for scheduling more tasks after table loading - LoadJobSet goals(); - private: ContextMutablePtr global_context; Databases databases; From 529a7023911b494ab2b57c3051b429a6d209baf7 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 11 May 2023 13:23:53 +0000 Subject: [PATCH 067/628] fix LocalServer fast-tests --- programs/server/Server.cpp | 2 -- src/Common/AsyncLoader.h | 4 ++-- src/Common/tests/gtest_async_loader.cpp | 4 +++- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 3f1e7ea9fa3..b8f4e92406a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -779,8 +779,6 @@ try 0, // We don't need any threads one all the parts will be loaded server_settings.outdated_part_loading_thread_pool_queue_size); - global_context->getAsyncLoader().start(); - /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index b5725dd601c..cc46ff41397 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -352,7 +352,7 @@ public: // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); - // Start workers to execute scheduled load jobs. + // Start workers to execute scheduled load jobs. Note that AsyncLoader is constructed as already started. void start(); // Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs. @@ -423,7 +423,7 @@ private: size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) mutable std::mutex mutex; // Guards all the fields below. - bool is_running = false; + bool is_running = true; // Full set of scheduled pending jobs along with scheduling info. std::unordered_map scheduled_jobs; diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index ee52dfd96b4..97a789056a4 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -39,7 +39,9 @@ struct AsyncLoaderTest explicit AsyncLoaderTest(size_t max_threads = 1) : loader(CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, max_threads, /* log_failures = */ false, /* log_progress = */ false) - {} + { + loader.stop(); // All tests call `start()` manually to better control ordering + } template T randomInt(T from, T to) From f0921f2335fe941cf4bb64da2b5748982b8fecd1 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 12:02:45 +0000 Subject: [PATCH 068/628] temporarily disable JobFailure test --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 97a789056a4..3cdada3b726 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -389,7 +389,7 @@ TEST(AsyncLoader, CancelExecutingTask) } } -TEST(AsyncLoader, JobFailure) +TEST(AsyncLoader, DISABLEDJobFailure) { AsyncLoaderTest t; t.loader.start(); From 3c1ae12c85b376c910fed9284a2e2e323ee1ce34 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 14:51:33 +0000 Subject: [PATCH 069/628] fix disable test --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 3cdada3b726..67b11566c62 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -389,7 +389,7 @@ TEST(AsyncLoader, CancelExecutingTask) } } -TEST(AsyncLoader, DISABLEDJobFailure) +TEST(AsyncLoader, DISABLED_JobFailure) { AsyncLoaderTest t; t.loader.start(); From 4d31051bb9148825cf3bc31d702578b648b6ed1c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 17:54:51 +0000 Subject: [PATCH 070/628] add async_load_databases server setting --- .../settings.md | 15 +++++ programs/server/Server.cpp | 6 +- programs/server/config.xml | 5 ++ src/Core/ServerSettings.h | 1 + src/Interpreters/Context.cpp | 1 + src/Interpreters/loadMetadata.cpp | 58 +++++++++++-------- src/Interpreters/loadMetadata.h | 2 +- 7 files changed, 59 insertions(+), 29 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 58b2dba8ec7..107ae1c024e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1192,6 +1192,21 @@ Possible values: Default value: 128. +## async_load_databases {#async_load_databases} + +Asynchronous loading of databases and tables. + +If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after ClickHouse server start up. Loading is done by AsyncLoader (see `system.async_loader` table and `async_loader_pool_size` server setting). Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. + +If `false`, all databases are loaded when the server starts. + +The default is `false`. + +**Example** + +``` xml +true +``` ## async_loader_pool_size {#async_loader_pool_size} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b8f4e92406a..64705497398 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1626,9 +1626,6 @@ try LOG_INFO(log, "Loading metadata from {}", path_str); - /// Tasks for loading and starting up all databases except system - LoadTaskPtrs load_metadata; - try { auto & database_catalog = DatabaseCatalog::instance(); @@ -1649,7 +1646,7 @@ try database_catalog.loadMarkedAsDroppedTables(); database_catalog.createBackgroundTasks(); /// Then, load remaining databases (some of them maybe be loaded asynchronously) - load_metadata = loadMetadata(global_context, default_database); + auto load_metadata = loadMetadata(global_context, default_database, server_settings.async_load_databases); /// If we need to convert database engines, disable async tables loading convertDatabasesEnginesIfNeed(load_metadata, global_context); startupSystemTables(global_context); @@ -1664,6 +1661,7 @@ try tryLogCurrentException(log, "Caught exception while loading metadata"); throw; } + LOG_DEBUG(log, "Loaded metadata."); /// Init trace collector only after trace_log system table was created diff --git a/programs/server/config.xml b/programs/server/config.xml index 2b4e515f78c..1f10a7d345b 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -349,6 +349,11 @@ 16 --> + + false + 0.9 diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 8411a6923dd..d349f9d8d03 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -79,6 +79,7 @@ namespace DB M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ M(UInt64, async_loader_pool_size, 16, "The maximum number of threads that will be used for async loading of tables after server start.", 0) \ + M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 3b2301bf4aa..01e4ce6bff4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1945,6 +1945,7 @@ size_t Context::getAsyncLoaderPoolSize() const // After server is started incoming queries can compete for resources with loading of the rest of the tables. // Thus it can be advantageous to lower number of threads after start using server setting `async_loader_pool_size`. // TODO(serxa): set async_loader max threads during server_start_job + // TODO(serxa): we need to add `turboMode()` if there are waiting queries. But how to create more workers only for foreground work? return shared->server_start_job && shared->server_start_job->status() == LoadStatus::OK ? shared->server_settings.async_loader_pool_size : getNumberOfPhysicalCPUCores(); } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 1eaa9f34527..e744d56ec02 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -152,19 +152,29 @@ static void checkIncompleteOrdinaryToAtomicConversion(ContextPtr context, const } } -LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name) +LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name, bool async_load_databases) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); String path = context->getPath() + "metadata"; - /** There may exist 'force_restore_data' file, that means, - * skip safety threshold on difference of data parts while initializing tables. - * This file is deleted after successful loading of tables. - * (flag is "one-shot") - */ + /// There may exist 'force_restore_data' file, which means skip safety threshold + /// on difference of data parts while initializing tables. + /// This file is immediately deleted i.e. "one-shot". auto force_restore_data_flag_file = fs::path(context->getFlagsPath()) / "force_restore_data"; bool has_force_restore_data_flag = fs::exists(force_restore_data_flag_file); + if (has_force_restore_data_flag) + { + try + { + fs::remove(force_restore_data_flag_file); + } + catch (...) + { + tryLogCurrentException("Load metadata", "Can't remove force restore file to enable data sanity checks"); + } + } + /// Loop over databases. std::map databases; @@ -227,27 +237,27 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data auto load_tasks = loader.loadTablesAsync(); auto startup_tasks = loader.startupTablesAsync(); - // First, load all tables - scheduleLoad(load_tasks); - waitLoad(load_tasks); // TODO(serxa): only wait for tables that must be loaded before server start + if (!async_load_databases) { + // First, load all tables + scheduleLoad(load_tasks); + waitLoad(load_tasks); - // Then, startup all tables - scheduleLoad(startup_tasks); - waitLoad(startup_tasks); // TODO(serxa): only wait for tables that must be started before server start + // Then, startup all tables. This is done to postpone merges and mutations + // Note that with async loader it would be a total barrier, which is unacceptable for the purpose of waiting. + scheduleLoad(startup_tasks); + waitLoad(startup_tasks); + return {}; + } else { + // Schedule all the jobs. + // Note that to achieve behaviour similar to synchronous case (postponing of merges) we use priorities. + // All startup jobs have lower priorities than load jobs. + // So _almost_ all tables will finish loading before the first table startup it there are no queries. + // Query waiting for a table boost its priority to finish table startup faster than load of the other tables. + scheduleLoadAll(load_tasks, startup_tasks); - if (has_force_restore_data_flag) - { - try - { - fs::remove(force_restore_data_flag_file); // TODO(serxa): when we should remove it with async loading? should we disable async loading with restore? - } - catch (...) - { - tryLogCurrentException("Load metadata", "Can't remove force restore file to enable data sanity checks"); - } + // Do NOT wait, just return tasks for continuation or later wait. + return joinTasks(load_tasks, startup_tasks); } - - return joinTasks(load_tasks, startup_tasks); } static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & database_name, const String & default_engine) diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index ca4c7ab0119..92403f53c04 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -12,7 +12,7 @@ void loadMetadataSystem(ContextMutablePtr context); /// Load tables from databases and add them to context. Database 'system' and 'information_schema' is ignored. /// Use separate function to load system tables. -[[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}); +[[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}, bool async_load_databases = false); /// Background operations in system tables may slowdown loading of the rest tables, /// so we startup system tables after all databases are loaded. From 576a27a62dfb80468c413fc2690ba547931a4f5b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 18:15:35 +0000 Subject: [PATCH 071/628] fix --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index d349f9d8d03..43cb51de557 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -79,7 +79,7 @@ namespace DB M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ M(UInt64, async_loader_pool_size, 16, "The maximum number of threads that will be used for async loading of tables after server start.", 0) \ - M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) + M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) From e1761a0e30b63f0dad52a219461f4c4e9b6395a1 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 18:26:03 +0000 Subject: [PATCH 072/628] fix metrics docs --- docs/en/operations/system-tables/metrics.md | 32 ++++++--------------- 1 file changed, 8 insertions(+), 24 deletions(-) diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..946c1b2459f 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -44,6 +44,14 @@ Number of threads in the Aggregator thread pool. Number of threads in the Aggregator thread pool running a task. +### AsyncLoaderThreads + +Number of threads in the async loader thread pool. + +### AsyncLoaderThreadsActive + +Number of threads in the async loader thread pool running a task. + ### AsyncInsertCacheSize Number of async insert hash id in cache @@ -196,14 +204,6 @@ Number of threads in the DatabaseOnDisk thread pool. Number of threads in the DatabaseOnDisk thread pool running a task. -### DatabaseOrdinaryThreads - -Number of threads in the Ordinary database thread pool. - -### DatabaseOrdinaryThreadsActive - -Number of threads in the Ordinary database thread pool running a task. - ### DelayedInserts Number of INSERT queries that are throttled due to high number of active data parts for partition in a MergeTree table. @@ -624,14 +624,6 @@ Number of connections that are sending data for external tables to remote server Number of connections that are sending data for scalars to remote servers. -### StartupSystemTablesThreads - -Number of threads in the StartupSystemTables thread pool. - -### StartupSystemTablesThreadsActive - -Number of threads in the StartupSystemTables thread pool running a task. - ### StorageBufferBytes Number of bytes in buffers of Buffer tables @@ -676,14 +668,6 @@ Number of threads in the system.replicas thread pool running a task. Number of connections to TCP server (clients with native interface), also included server-server distributed query connections -### TablesLoaderThreads - -Number of threads in the tables loader thread pool. - -### TablesLoaderThreadsActive - -Number of threads in the tables loader thread pool running a task. - ### TablesToDropQueueSize Number of dropped tables, that are waiting for background data removal. From 9bf52924159ef9b5127e19f145e49b7b7c696793 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 19:49:47 +0000 Subject: [PATCH 073/628] refact --- programs/local/LocalServer.cpp | 4 ++-- programs/server/Server.cpp | 6 +++--- src/Databases/IDatabase.h | 2 +- src/Databases/TablesLoader.cpp | 17 ++++++++++------- src/Databases/TablesLoader.h | 2 -- src/Interpreters/loadMetadata.cpp | 28 ++++++++++++++-------------- src/Interpreters/loadMetadata.h | 2 +- 7 files changed, 31 insertions(+), 30 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b9083464b6d..50f881c2833 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -665,11 +665,11 @@ void LocalServer::processConfig() status.emplace(fs::path(path) / "status", StatusFile::write_full_info); LOG_DEBUG(log, "Loading metadata from {}", path); - loadMetadataSystem(global_context); + auto startup_system_tasks = loadMetadataSystem(global_context); attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); - startupSystemTables(global_context); + scheduleAndWaitLoad(startup_system_tasks); if (!config().has("only-system-tables")) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1c9a0e8e7bf..ca5a4bece1b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1631,8 +1631,8 @@ try auto & database_catalog = DatabaseCatalog::instance(); /// We load temporary database first, because projections need it. database_catalog.initializeAndLoadTemporaryDatabase(); - loadMetadataSystem(global_context); - maybeConvertSystemDatabase(global_context); + auto system_startup_tasks = loadMetadataSystem(global_context); + maybeConvertSystemDatabase(global_context, system_startup_tasks); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); @@ -1649,7 +1649,7 @@ try auto load_metadata = loadMetadata(global_context, default_database, server_settings.async_load_databases); /// If we need to convert database engines, disable async tables loading convertDatabasesEnginesIfNeed(load_metadata, global_context); - startupSystemTables(global_context); + scheduleAndWaitLoad(system_startup_tasks); database_catalog.startupBackgroundCleanup(); /// After loading validate that default database exists database_catalog.assertDatabaseExists(default_database); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 75a5f0c03e5..002a393428c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -182,7 +182,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - /// Start all tables and the database itself + /// TODO(serxa): remove this method. Start all tables and the database itself virtual void startupTablesAndDatabase(AsyncLoader & /*async_loader*/, LoadingStrictnessLevel /*mode*/) {} /// Create a task to startup table `name` after specified dependencies `startup_after` using `async_loader`. diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 9682609435c..70f45cf7d22 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -47,7 +47,9 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) } if (databases_to_load.empty()) - return load_tasks; + return {}; + + LoadTaskPtrs result; /// Read and parse metadata from Ordinary, Atomic, Materialized*, Replicated, etc databases. Build dependency graph. for (auto & database_name : databases_to_load) @@ -94,15 +96,17 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) path_and_query.ast, strictness_mode); load_table[table_id.uuid] = task; - load_tasks.push_back(task); + result.push_back(task); } - return load_tasks; + return result; } LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) { + LoadTaskPtrs result; std::unordered_map startup_database; /// database name -> all its tables startup tasks + for (const auto & table_id : all_loading_dependencies.getTablesSortedByDependency()) { // Make startup table task @@ -113,7 +117,7 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) table_name, strictness_mode); startup_database[table_name.database].push_back(task); - startup_tasks.push_back(task); + result.push_back(task); } /// Make startup database tasks @@ -123,10 +127,10 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) async_loader, getGoalsOr(startup_database[database_name], startup_after), strictness_mode); - startup_tasks.push_back(task); + result.push_back(task); } - return startup_tasks; + return result; } void TablesLoader::buildDependencyGraph() @@ -151,7 +155,6 @@ void TablesLoader::buildDependencyGraph() all_loading_dependencies.log(); } - void TablesLoader::removeUnresolvableDependencies() { auto need_exclude_dependency = [this](const StorageID & table_id) diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 321fa2393b9..fee16f605c4 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -79,8 +79,6 @@ private: AsyncLoader & async_loader; std::unordered_map load_table; /// table uuid -> load task - LoadTaskPtrs load_tasks; /// Tasks to load all tables - LoadTaskPtrs startup_tasks; /// Tasks to startup all tables and databases after loading void buildDependencyGraph(); void removeUnresolvableDependencies(); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index e744d56ec02..5239dbaf448 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -358,7 +358,7 @@ static void convertOrdinaryDatabaseToAtomic(Poco::Logger * log, ContextMutablePt /// Converts database with Ordinary engine to Atomic. Does nothing if database is not Ordinary. /// Can be called only during server startup when there are no queries from users. -static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const String & database_name, bool tables_started) +static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, const String & database_name, LoadTaskPtrs * startup_tasks = nullptr) { Poco::Logger * log = &Poco::Logger::get("loadMetadata"); @@ -385,10 +385,11 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons try { - if (!tables_started) + if (startup_tasks) // NOTE: only for system database { /// It's not quite correct to run DDL queries while database is not started up. - startupSystemTables(context); // NOTE: tables_started can be false only for system tables + scheduleAndWaitLoad(*startup_tasks); + startup_tasks->clear(); } auto local_context = Context::createCopy(context); @@ -441,8 +442,10 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons scheduleAndWaitLoad(loader.loadTablesAsync()); /// Startup tables if they were started before conversion and detach/attach - if (tables_started) - scheduleAndWaitLoad(loader.startupTablesAsync()); + if (startup_tasks) // NOTE: only for system database + *startup_tasks = loader.startupTablesAsync(); // We have loaded old database(s), replace tasks to startup new database + else + scheduleAndWaitLoad(loader.startupTablesAsync()); // An old database was already loaded, so we should load new one as well } catch (Exception & e) { @@ -454,13 +457,13 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons } } -void maybeConvertSystemDatabase(ContextMutablePtr context) +void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system_startup_tasks) { /// TODO remove this check, convert system database unconditionally if (context->getSettingsRef().allow_deprecated_database_ordinary) return; - maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, /* tables_started */ false); + maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, *system_startup_tasks); } void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context) @@ -477,18 +480,13 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu for (const auto & [name, _] : DatabaseCatalog::instance().getDatabases()) if (name != DatabaseCatalog::SYSTEM_DATABASE) - maybeConvertOrdinaryDatabaseToAtomic(context, name, /* tables_started */ true); + maybeConvertOrdinaryDatabaseToAtomic(context, name); LOG_INFO(&Poco::Logger::get("loadMetadata"), "Conversion finished, removing convert_ordinary_to_atomic flag"); fs::remove(convert_flag_path); } -void startupSystemTables(ContextMutablePtr context) -{ - DatabaseCatalog::instance().getSystemDatabase()->startupTablesAndDatabase(context->getAsyncLoader(), LoadingStrictnessLevel::FORCE_RESTORE); -} - -void loadMetadataSystem(ContextMutablePtr context) +LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context) { loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic"); loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA, "Memory"); @@ -502,7 +500,9 @@ void loadMetadataSystem(ContextMutablePtr context) }; TablesLoader loader{context, databases, LoadingStrictnessLevel::FORCE_RESTORE}; scheduleAndWaitLoad(loader.loadTablesAsync()); + /// Will startup tables in system database after all databases are loaded. + return loader.startupTablesAsync(); } } diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 92403f53c04..5fd45e56498 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -19,7 +19,7 @@ void loadMetadataSystem(ContextMutablePtr context); void startupSystemTables(ContextMutablePtr context); /// Converts `system` database from Ordinary to Atomic (if needed) -void maybeConvertSystemDatabase(ContextMutablePtr context); +void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system_startup_tasks); /// Converts all databases (except system) from Ordinary to Atomic if convert_ordinary_to_atomic flag exists /// Waits for `load_metadata` task before conversions From 646bc6b4432f4ac0c939c1d9389681cef8db4bcd Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 12 May 2023 20:27:41 +0000 Subject: [PATCH 074/628] fix --- src/Databases/DatabaseOrdinary.cpp | 14 -------------- src/Databases/DatabaseOrdinary.h | 2 -- src/Databases/IDatabase.h | 3 --- src/Interpreters/loadMetadata.cpp | 8 +++----- src/Interpreters/loadMetadata.h | 15 ++++++++------- 5 files changed, 11 insertions(+), 31 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index cc716e7c58a..4905b83dd72 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -199,20 +199,6 @@ LoadTaskPtr DatabaseOrdinary::loadTableFromMetadataAsync( return load_table[name.table] = makeLoadTask(async_loader, {job}); } -void DatabaseOrdinary::startupTablesAndDatabase(AsyncLoader & async_loader, LoadingStrictnessLevel mode) -{ - LOG_INFO(log, "Starting up tables."); - LoadTaskPtrs tasks; - for (const auto & table : TSA_SUPPRESS_WARNING_FOR_READ(tables)) - tasks.push_back(startupTableAsync( - async_loader, - {}, - QualifiedTableName{.database = getDatabaseName(), .table = table.first}, - mode)); - scheduleAndWaitLoad(tasks); - scheduleAndWaitLoad(startupDatabaseAsync(async_loader, {}, mode)); -} - LoadTaskPtr DatabaseOrdinary::startupTableAsync( AsyncLoader & async_loader, LoadJobSet startup_after, diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index d4e9d40cbcf..3972d71e171 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -43,8 +43,6 @@ public: const ASTPtr & ast, LoadingStrictnessLevel mode) override; - void startupTablesAndDatabase(AsyncLoader & async_loader, LoadingStrictnessLevel mode) override; - LoadTaskPtr startupTableAsync( AsyncLoader & async_loader, LoadJobSet startup_after, diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 002a393428c..f334b01eff0 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -182,9 +182,6 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - /// TODO(serxa): remove this method. Start all tables and the database itself - virtual void startupTablesAndDatabase(AsyncLoader & /*async_loader*/, LoadingStrictnessLevel /*mode*/) {} - /// Create a task to startup table `name` after specified dependencies `startup_after` using `async_loader`. /// The returned task is also stored inside the database for cancellation on destruction. [[nodiscard]] virtual LoadTaskPtr startupTableAsync( diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 5239dbaf448..6fc7dc8c543 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -239,13 +239,11 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data if (!async_load_databases) { // First, load all tables - scheduleLoad(load_tasks); - waitLoad(load_tasks); + scheduleAndWaitLoad(load_tasks); // Then, startup all tables. This is done to postpone merges and mutations // Note that with async loader it would be a total barrier, which is unacceptable for the purpose of waiting. - scheduleLoad(startup_tasks); - waitLoad(startup_tasks); + scheduleAndWaitLoad(startup_tasks); return {}; } else { // Schedule all the jobs. @@ -463,7 +461,7 @@ void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system if (context->getSettingsRef().allow_deprecated_database_ordinary) return; - maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, *system_startup_tasks); + maybeConvertOrdinaryDatabaseToAtomic(context, DatabaseCatalog::SYSTEM_DATABASE, &system_startup_tasks); } void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMutablePtr context) diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index 5fd45e56498..b0d97d53de3 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -8,15 +8,16 @@ namespace DB /// Load tables from system database. Only real tables like query_log, part_log. /// You should first load system database, then attach system tables that you need into it, then load other databases. -void loadMetadataSystem(ContextMutablePtr context); - -/// Load tables from databases and add them to context. Database 'system' and 'information_schema' is ignored. -/// Use separate function to load system tables. -[[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}, bool async_load_databases = false); - +/// It returns tasks to startup system tables. /// Background operations in system tables may slowdown loading of the rest tables, /// so we startup system tables after all databases are loaded. -void startupSystemTables(ContextMutablePtr context); +[[nodiscard]] LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context); + +/// Load tables from databases and add them to context. Databases 'system' and 'information_schema' are ignored. +/// Use separate function to load system tables. +/// If `async_load_databases = true` returns tasks for asynchronous load and startup of all tables +/// Note that returned tasks are already scheduled. +[[nodiscard]] LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_database_name = {}, bool async_load_databases = false); /// Converts `system` database from Ordinary to Atomic (if needed) void maybeConvertSystemDatabase(ContextMutablePtr context, LoadTaskPtrs & system_startup_tasks); From e14acdf68371a57e35f0827fc201b79290b08c27 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 14 May 2023 13:16:56 +0000 Subject: [PATCH 075/628] fix style --- src/Interpreters/loadMetadata.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 6fc7dc8c543..a591a7fa19f 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -237,7 +237,8 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data auto load_tasks = loader.loadTablesAsync(); auto startup_tasks = loader.startupTablesAsync(); - if (!async_load_databases) { + if (!async_load_databases) + { // First, load all tables scheduleAndWaitLoad(load_tasks); @@ -245,7 +246,9 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data // Note that with async loader it would be a total barrier, which is unacceptable for the purpose of waiting. scheduleAndWaitLoad(startup_tasks); return {}; - } else { + } + else + { // Schedule all the jobs. // Note that to achieve behaviour similar to synchronous case (postponing of merges) we use priorities. // All startup jobs have lower priorities than load jobs. From ec7f4fba27d9c1a211043a891584372ddd953f74 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 14 May 2023 13:27:53 +0000 Subject: [PATCH 076/628] fix conflicts after merge --- src/Common/AsyncLoader.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 070955c3666..cc46ff41397 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -186,7 +186,6 @@ inline void scheduleLoad(const LoadTaskPtrs & tasks) template inline void scheduleLoadAll(Args && ... args) -inline void scheduleLoad(Args && ... args) { (scheduleLoad(std::forward(args)), ...); } @@ -210,7 +209,6 @@ inline void waitLoad(const LoadTaskPtrs & tasks) template inline void waitLoadAll(Args && ... args) -inline void waitLoad(Args && ... args) { (waitLoad(std::forward(args)), ...); } From 4127599a0e3d077b34649cab7d00337bfa850b9c Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 22 May 2023 17:02:54 +0000 Subject: [PATCH 077/628] fix `UNKNOWN_TABLE` problem during async startup --- src/Databases/DatabaseOrdinary.cpp | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 4905b83dd72..734be6bd73b 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -218,13 +218,19 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync( std::move(startup_after), TABLE_STARTUP_PRIORITY, fmt::format("startup table {}", name.getFullName()), - [this, table = getTableUnlocked(name.table)] (const LoadJobPtr &) + [this, name] (const LoadJobPtr &) { - /// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on) - /// until startup finished. - auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef().lock_acquire_timeout); - table->startup(); - logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); + if (auto table = DatabaseOrdinary::tryGetTable(name.table, {})) + { + /// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on) + /// until startup finished. + auto table_lock_holder = table->lockForShare(RWLockImpl::NO_QUERY, getContext()->getSettingsRef().lock_acquire_timeout); + table->startup(); + logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); + } + else + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist during startup", + backQuote(name.database), backQuote(name.table)); }); return startup_table[name.table] = makeLoadTask(async_loader, {job}); From edbc9ba1040080a64e169fd5de6ccfae40e6fbab Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 May 2023 13:43:23 +0000 Subject: [PATCH 078/628] fix style --- src/Databases/DatabaseOrdinary.cpp | 1 + src/Databases/IDatabase.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 734be6bd73b..3cda9220c53 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -35,6 +35,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; } static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index f334b01eff0..11f6bf5e77c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -44,7 +44,7 @@ namespace ErrorCodes static constexpr auto TABLE_LOAD_PRIORITY = -1; /// Initial priority for table loading jobs static constexpr auto TABLE_STARTUP_PRIORITY = -2; /// Initial priority for table startup jobs -static constexpr auto DATABASE_STARTUP_PRIORITY = -2; /// Initial priority for table startup jobs +static constexpr auto DATABASE_STARTUP_PRIORITY = -2; /// Initial priority for database startup jobs static constexpr auto TABLE_WAIT_PRIORITY = 0; /// Prioritize load jobs that block queries class IDatabaseTablesIterator From a9db6bd830bf4d5a74c845401f5a68c3873d6156 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 May 2023 17:04:40 +0000 Subject: [PATCH 079/628] refactoring --- src/Databases/DatabaseOrdinary.cpp | 59 ++++++++++-------------------- src/Databases/TablesLoader.cpp | 2 +- 2 files changed, 20 insertions(+), 41 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 3cda9220c53..747de58bec1 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -40,38 +40,6 @@ namespace ErrorCodes static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; -namespace -{ - void tryAttachTable( - ContextMutablePtr context, - const ASTCreateQuery & query, - DatabaseOrdinary & database, - const String & database_name, - const String & metadata_path, - bool force_restore) - { - try - { - auto [table_name, table] = createTableFromAST( - query, - database_name, - database.getTableDataPath(query), - context, - force_restore); - - database.attachTable(context, table_name, table, database.getTableDataPath(query)); - } - catch (Exception & e) - { - e.addMessage( - "Cannot attach table " + backQuote(database_name) + "." + backQuote(query.getTable()) + " from metadata file " + metadata_path - + " from query " + serializeAST(query)); - throw; - } - } -} - - DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, ContextPtr context_) : DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_) { @@ -167,15 +135,26 @@ void DatabaseOrdinary::loadTableFromMetadata( LoadingStrictnessLevel mode) { assert(name.database == TSA_SUPPRESS_WARNING_FOR_READ(database_name)); - const auto & create_query = ast->as(); + const auto & query = ast->as(); - tryAttachTable( - local_context, - create_query, - *this, - name.database, - file_path, - LoadingStrictnessLevel::FORCE_RESTORE <= mode); + try + { + auto [table_name, table] = createTableFromAST( + query, + name.database, + getTableDataPath(query), + local_context, + LoadingStrictnessLevel::FORCE_RESTORE <= mode); + + attachTable(local_context, table_name, table, getTableDataPath(query)); + } + catch (Exception & e) + { + e.addMessage( + "Cannot attach table " + backQuote(name.database) + "." + backQuote(query.getTable()) + " from metadata file " + file_path + + " from query " + serializeAST(query)); + throw; + } } LoadTaskPtr DatabaseOrdinary::loadTableFromMetadataAsync( diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 70f45cf7d22..d4c18056df3 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -36,7 +36,7 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) { bool need_resolve_dependencies = !global_context->getConfigRef().has("ignore_table_dependencies_on_metadata_loading"); - /// Load all Lazy, MySQl, PostgreSQL, SQLite, etc databases first. + /// Load all Lazy, MySQL, PostgreSQL, SQLite, etc databases first. /// Note that this loading is NOT async because it should be fast and it cannot have any dependencies for (auto & database : databases) { From 87a16d852f85a22368cab77ed6100bd0f838fd9a Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 23 May 2023 17:07:30 +0000 Subject: [PATCH 080/628] fix tidy build --- src/Databases/TablesLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index d4c18056df3..798aaab5ed4 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -81,7 +81,7 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) { /// Gather tasks to load before this table LoadTaskPtrs load_dependency_tasks; - for (StorageID dependency_id : all_loading_dependencies.getDependencies(table_id)) + for (const StorageID & dependency_id : all_loading_dependencies.getDependencies(table_id)) load_dependency_tasks.push_back(load_table[dependency_id.uuid]); // Make load table task From e1045f76ded15d2edb27d7a2489ab2597c7bb2c3 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 31 May 2023 16:23:41 +0000 Subject: [PATCH 081/628] rework async loader initialization and reconfiguration for bg/fg pools --- .../settings.md | 25 +++++++--- programs/server/Server.cpp | 5 +- programs/server/config.xml | 3 +- src/Common/AsyncLoader.h | 4 +- src/Common/AsyncLoaderPool.h | 13 ++++++ src/Common/CurrentMetrics.cpp | 6 ++- src/Common/tests/gtest_async_loader.cpp | 8 ++-- src/Core/ServerSettings.h | 3 +- src/Databases/DatabaseOrdinary.cpp | 23 +++++++++- src/Databases/DatabaseOrdinary.h | 4 ++ .../DatabaseMaterializedPostgreSQL.cpp | 6 +-- src/Interpreters/Context.cpp | 46 ++++++++++--------- src/Interpreters/Context.h | 1 - 13 files changed, 103 insertions(+), 44 deletions(-) create mode 100644 src/Common/AsyncLoaderPool.h diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 86d68344281..fea602a2c46 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -80,7 +80,7 @@ Default: 2 ## background_merges_mutations_scheduling_policy -The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. +The policy on how to perform a scheduling for background merges and mutations. Possible values are: `round_robin` and `shortest_task_first`. ## background_merges_mutations_scheduling_policy @@ -521,7 +521,7 @@ Both the cache for `local_disk`, and temporary data will be stored in `/tiny_loc Type: String -Default: +Default: ## thread_pool_queue_size @@ -578,7 +578,7 @@ When `/disk1` is full, temporary data will be stored on `/disk2`. ``` Type: String -Default: +Default: ## uncompressed_cache_policy @@ -1556,7 +1556,7 @@ Default value: `0.5`. Asynchronous loading of databases and tables. -If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after ClickHouse server start up. Loading is done by AsyncLoader (see `system.async_loader` table and `async_loader_pool_size` server setting). Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. +If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. Loading is done by AsyncLoader (see `system.async_loader` table, `async_loader_background_pool_size` and `async_loader_foreground_pool_size` server settings). Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. If `false`, all databases are loaded when the server starts. @@ -1568,13 +1568,26 @@ The default is `false`. true ``` -## async_loader_pool_size {#async_loader_pool_size} +## async_loader_foreground_pool_size {#async_loader_foreground_pool_size} -Sets the number of threads performing asynchronous load jobs after server startup. Note that before server startup all available CPUs are used for load jobs. +Sets the number of threads performing asynchronous load jobs in foreground pool. The foreground pool is used for loading table synchronously before server start listening on a port and for loading tables that are waited for. Foreground pool has higher priority than background pool. It means that no job starts in background pool while there are jobs running in foreground pool. Possible values: - Any positive integer. +- Zero. Use all available CPUs. + +Default value: 0. + + +## async_loader_background_pool_size {#async_loader_background_pool_size} + +Sets the number of threads performing asynchronous load jobs in background pool. The background pool is used for loading tables asynchronously after server start in case there are no queries waiting for the table. It could be beneficial to keep low number of threads in background pool if there are a lot of tables. It will reserve CPU resources for concurrent query execution. + +Possible values: + +- Any positive integer. +- Zero. Use all available CPUs. Default value: 16. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1588a7c1e42..71349d6e4c2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1226,7 +1226,10 @@ try global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); - global_context->getAsyncLoader().setMaxThreads(global_context->getAsyncLoaderPoolSize()); + auto fg_pool_size = server_settings_.async_loader_foreground_pool_size; + auto bg_pool_size = server_settings_.async_loader_background_pool_size; + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPool::Foreground, fg_pool_size ? fg_pool_size : getNumberOfPhysicalCPUCores()); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPool::Background, bg_pool_size ? bg_pool_size : getNumberOfPhysicalCPUCores()); if (config->has("resources")) { diff --git a/programs/server/config.xml b/programs/server/config.xml index 8bcfbab0226..9f41dcdac64 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -346,7 +346,8 @@ 128 16 16 - 16 + 64 + 16 --> + 4 + + + + + + id + + + a + String + XX + + + + diff --git a/tests/integration/test_async_load_databases/configs/dictionaries/dep_y.xml b/tests/integration/test_async_load_databases/configs/dictionaries/dep_y.xml new file mode 100644 index 00000000000..0156a1dd019 --- /dev/null +++ b/tests/integration/test_async_load_databases/configs/dictionaries/dep_y.xml @@ -0,0 +1,40 @@ + + + dep_y + + + localhost + 9000 + default + + test + elements
+
+ + + 4 + + + + + + id + + + b + Int32 + -1 + + + c + Float64 + -2 + + + a + String + YY + + +
+
diff --git a/tests/integration/test_async_load_databases/configs/dictionaries/dep_z.xml b/tests/integration/test_async_load_databases/configs/dictionaries/dep_z.xml new file mode 100644 index 00000000000..8e254d769ea --- /dev/null +++ b/tests/integration/test_async_load_databases/configs/dictionaries/dep_z.xml @@ -0,0 +1,36 @@ + + + dep_z + + + localhost + 9000 + default + + dict + dep_y
+ SELECT intDiv(count(), 4) from dict.dep_y +
+ + + 4 + + + + + + id + + + b + Int32 + -3 + + + a + String + ZZ + + +
+
diff --git a/tests/integration/test_async_load_databases/configs/dictionaries/node.xml b/tests/integration/test_async_load_databases/configs/dictionaries/node.xml new file mode 100644 index 00000000000..2cd957a3720 --- /dev/null +++ b/tests/integration/test_async_load_databases/configs/dictionaries/node.xml @@ -0,0 +1,25 @@ + + + node + + + localhost + 9000 + default + + system + select dummy, toString(dummy) from system.one + + + 0 + + + key + + name + String + + + + + diff --git a/tests/integration/test_async_load_databases/configs/users.xml b/tests/integration/test_async_load_databases/configs/users.xml new file mode 100644 index 00000000000..4555a2ed494 --- /dev/null +++ b/tests/integration/test_async_load_databases/configs/users.xml @@ -0,0 +1,22 @@ + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py new file mode 100644 index 00000000000..b881c76fcdf --- /dev/null +++ b/tests/integration/test_async_load_databases/test.py @@ -0,0 +1,168 @@ +import pytest +import random +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +DICTIONARY_FILES = [ + "configs/dictionaries/dep_x.xml", + "configs/dictionaries/dep_y.xml", + "configs/dictionaries/dep_z.xml", + "configs/dictionaries/node.xml", +] + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", main_configs=["configs/config.xml"], dictionaries=DICTIONARY_FILES, stay_alive=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + instance.query( + """ + CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; + CREATE DATABASE IF NOT EXISTS test; + DROP TABLE IF EXISTS test.elements; + CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log; + INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); + """ + ) + + yield cluster + + finally: + cluster.shutdown() + + +def get_status(dictionary_name): + return instance.query( + "SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'" + ).rstrip("\n") + + +def test_dict_get_data(started_cluster): + query = instance.query + + # dictionaries_lazy_load == false, so these dictionary are not loaded. + assert get_status("dep_x") == "NOT_LOADED" + assert get_status("dep_y") == "NOT_LOADED" + assert get_status("dep_z") == "NOT_LOADED" + + # Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'. + # So they all should be loaded at once. + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n" + assert get_status("dep_x") == "LOADED" + assert get_status("dep_y") == "LOADED" + assert get_status("dep_z") == "LOADED" + + # Other dictionaries should work too. + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n" + + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n" + + # Update the source table. + query("INSERT INTO test.elements VALUES (3, 'fire', 30, 8)") + + # Wait for dictionaries to be reloaded. + assert_eq_with_retry( + instance, + "SELECT dictHas('dep_x', toUInt64(3))", + "1", + sleep_time=2, + retry_count=10, + ) + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n" + + # dep_z (and hence dep_x) are updated only when there `intDiv(count(), 4)` is changed, now `count()==4`, + # so dep_x and dep_z are not going to be updated after the following INSERT. + query("INSERT INTO test.elements VALUES (4, 'ether', 404, 0.001)") + assert_eq_with_retry( + instance, + "SELECT dictHas('dep_y', toUInt64(4))", + "1", + sleep_time=2, + retry_count=10, + ) + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "XX\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ZZ\n" + + +def dependent_tables_assert(): + res = instance.query("select database || '.' || name from system.tables") + assert "system.join" in res + assert "default.src" in res + assert "dict.dep_y" in res + assert "lazy.log" in res + assert "test.d" in res + assert "default.join" in res + assert "a.t" in res + + +def test_dependent_tables(started_cluster): + query = instance.query + query("create database lazy engine=Lazy(10)") + query("create database a") + query("create table lazy.src (n int, m int) engine=Log") + query( + "create dictionary a.d (n int default 0, m int default 42) primary key n " + "source(clickhouse(host 'localhost' port tcpPort() user 'default' table 'src' password '' db 'lazy'))" + "lifetime(min 1 max 10) layout(flat())" + ) + query("create table system.join (n int, m int) engine=Join(any, left, n)") + query("insert into system.join values (1, 1)") + query( + "create table src (n int, m default joinGet('system.join', 'm', 1::int)," + "t default dictGetOrNull('a.d', 'm', toUInt64(3))," + "k default dictGet('a.d', 'm', toUInt64(4))) engine=MergeTree order by n" + ) + query( + "create dictionary test.d (n int default 0, m int default 42) primary key n " + "source(clickhouse(host 'localhost' port tcpPort() user 'default' table 'src' password '' db 'default'))" + "lifetime(min 1 max 10) layout(flat())" + ) + query( + "create table join (n int, m default dictGet('a.d', 'm', toUInt64(3))," + "k default dictGet('test.d', 'm', toUInt64(0))) engine=Join(any, left, n)" + ) + query( + "create table lazy.log (n default dictGet(test.d, 'm', toUInt64(0))) engine=Log" + ) + query( + "create table a.t (n default joinGet('system.join', 'm', 1::int)," + "m default dictGet('test.d', 'm', toUInt64(3))," + "k default joinGet(join, 'm', 1::int)) engine=MergeTree order by n" + ) + + dependent_tables_assert() + instance.restart_clickhouse() + dependent_tables_assert() + query("drop table a.t") + query("drop table lazy.log") + query("drop table join") + query("drop dictionary test.d") + query("drop table src") + query("drop table system.join") + query("drop database a") + query("drop database lazy") + +def test_multiple_tables(started_cluster): + query = instance.query + tables_count = 20 + for i in range(tables_count): + query(f"create table test.table_{i} (n UInt64, s String) engine=MergeTree order by n as select number, randomString(100) from numbers(100)") + + instance.restart_clickhouse() + + order = [i for i in range(tables_count)] + random.shuffle(order) + for i in order: + assert query(f"select count() from test.table_{i}") == "100\n" From a0cf23ccd22d37c6f3fcd0c59824f90f85944b21 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 4 Aug 2023 17:21:55 +0000 Subject: [PATCH 126/628] Automatic style fix --- tests/integration/test_async_load_databases/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index b881c76fcdf..050b529a227 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -12,7 +12,10 @@ DICTIONARY_FILES = [ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( - "instance", main_configs=["configs/config.xml"], dictionaries=DICTIONARY_FILES, stay_alive=True + "instance", + main_configs=["configs/config.xml"], + dictionaries=DICTIONARY_FILES, + stay_alive=True, ) @@ -154,11 +157,14 @@ def test_dependent_tables(started_cluster): query("drop database a") query("drop database lazy") + def test_multiple_tables(started_cluster): query = instance.query tables_count = 20 for i in range(tables_count): - query(f"create table test.table_{i} (n UInt64, s String) engine=MergeTree order by n as select number, randomString(100) from numbers(100)") + query( + f"create table test.table_{i} (n UInt64, s String) engine=MergeTree order by n as select number, randomString(100) from numbers(100)" + ) instance.restart_clickhouse() From eecd9a0c932ae8b0447719e1d20ab9f62344d948 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Aug 2023 09:23:24 +0000 Subject: [PATCH 127/628] fix style --- tests/integration/test_async_load_databases/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_async_load_databases/__init__.py diff --git a/tests/integration/test_async_load_databases/__init__.py b/tests/integration/test_async_load_databases/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From aac5b0386a7b179b14b39929887ee2f6a9968646 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 1 Sep 2023 20:40:38 +0200 Subject: [PATCH 128/628] Update src/Storages/System/StorageSystemAsyncLoader.h --- src/Storages/System/StorageSystemAsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemAsyncLoader.h b/src/Storages/System/StorageSystemAsyncLoader.h index b6e51107b1e..1a466904df7 100644 --- a/src/Storages/System/StorageSystemAsyncLoader.h +++ b/src/Storages/System/StorageSystemAsyncLoader.h @@ -13,7 +13,7 @@ class Context; class StorageSystemAsyncLoader final : public IStorageSystemOneBlock { public: - std::string getName() const override { return "SystemReplicatedFetches"; } + std::string getName() const override { return "SystemAsyncLoader"; } static NamesAndTypesList getNamesAndTypes(); From a2f8f6bf29e11d910af4a5fa5268dac6006c4614 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 7 Sep 2023 13:21:43 +0000 Subject: [PATCH 129/628] working on review comments --- .../settings.md | 6 +-- .../operations/system-tables/async_loader.md | 54 +++++++++++++++++++ programs/server/Server.cpp | 8 ++- src/Common/AsyncLoader.cpp | 21 ++++---- src/Common/AsyncLoader.h | 23 ++++---- src/Core/ServerSettings.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/TablesLoader.cpp | 4 +- src/Interpreters/Context.cpp | 8 ++- src/Interpreters/loadMetadata.cpp | 20 +++---- .../System/StorageSystemAsyncLoader.cpp | 12 ++--- 11 files changed, 103 insertions(+), 57 deletions(-) create mode 100644 docs/en/operations/system-tables/async_loader.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 9a9aa590db8..00c282ae911 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1665,7 +1665,7 @@ Possible values: - Any positive integer. - Zero. Use all available CPUs. -Default value: 16. +Default value: 0. ## merge_tree {#server_configuration_parameters-merge_tree} @@ -1987,7 +1987,7 @@ If the table does not exist, ClickHouse will create it. If the structure of the 7500 1048576 8192 - 524288 + 524288 false ``` @@ -2402,7 +2402,7 @@ This section contains the following parameters: * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname. * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. - + **Example configuration** ``` xml diff --git a/docs/en/operations/system-tables/async_loader.md b/docs/en/operations/system-tables/async_loader.md new file mode 100644 index 00000000000..4e8651a6d3e --- /dev/null +++ b/docs/en/operations/system-tables/async_loader.md @@ -0,0 +1,54 @@ +--- +slug: /en/operations/system-tables/async_loader +--- +# async_loader + +Contains information and status for recent asynchronous jobs (e.g. for tables loading). The table contains a row for every job. There is a tool for visualizing information from this table `utils/async_loader_graph`. + +Example: + +``` sql +SELECT * +FROM system.async_loader +FORMAT Vertical +LIMIT 1 +``` + +``` text +``` + +Columns: + +- `job` (`String`) - Job name (may be not unique). +- `job_id` (`UInt64`) - Unique ID of the job. +- `dependencies` (`Array(UInt64)`) - List of IDs of jobs that should be done before this job. +- `dependencies_left` (`UInt64`) - Current number of dependencies left to be done. +- `status` (`Enum`) - Current load status of a job: + `PENDING`: Load job is not started yet. + `OK`: Load job executed and was successful. + `FAILED`: Load job executed and failed. + `CANCELED`: Load job is not going to be executed due to removal or dependency failure. + +A pending job might be in one of the following states: +- `is_executing` (`UInt8`) - The job is currently being executed by a worker. +- `is_blocked` (`UInt8`) - The job waits for its dependencies to be done. +- `is_ready` (`UInt8`) - The job is ready to be executed and waits for a worker. +- `elapsed` (`Float64`) - Seconds elapsed since start of execution. Zero if job is not started. Total execution time if job finished. + +Every job has a pool associated with it and is started in this pool. Each pool has a constant priority and a mutable maximum number of workers. Higher priority (lower `priority` value) jobs are run first. No job with lower priority is started while there is at least one higher priority job ready or executing. Job priority can be elevated (but cannot be lowered) by prioritizing it. For example jobs for a table loading and startup will be prioritized if incoming query required this table. It is possible prioritize a job during its execution, but job is not moved from its `execution_pool` to newly assigned `pool`. The job uses `pool` for creating new jobs to avoid priority inversion. Already started jobs are not preempted by higher priority jobs and always run to completion after start. +- `pool_id` (`UInt64`) - ID of a pool currently assigned to the job. +- `pool` (`String`) - Name of `pool_id` pool. +- `priority` (`Int64`) - Priority of `pool_id` pool. +- `execution_pool_id` (`UInt64`) - ID of a pool the job is executed in. Equals initially assigned pool before execution starts. +- `execution_pool` (`String`) - Name of `execution_pool_id` pool. +- `execution_priority` (`Int64`) - Priority of `execution_pool_id` pool. + +- `ready_seqno` (`Nullable(UInt64)`) - Not null for ready jobs. Worker pulls the next job to be executed from a ready queue of its pool. If there are multiple ready jobs, then job with the lowest value of `ready_seqno` is picked. +- `waiters` (`UInt64`) - The number of threads waiting on this job. +- `exception` (`Nullable(String)`) - Not null for failed and canceled jobs. Holds error message raised during query execution or error leading to cancelling of this job along with dependency failure chain of job names. + +Time instants during job lifetime: +- `schedule_time` (`DateTime64`) - Time when job was created and scheduled to be executed (usually with all its dependencies). +- `enqueue_time` (`Nullable(DateTime64)`) - Time when job became ready and was enqueued into a ready queue of it's pool. Null if the job is not ready yet. +- `start_time` (`Nullable(DateTime64)`) - Time when worker dequeues the job from ready queue and start its execution. Null if the job is not started yet. +- `finish_time` (`Nullable(DateTime64)`) - Time when job execution is finished. Null if the job is not finished yet. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 850d4c6b75a..5b1070586ab 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1344,11 +1344,9 @@ try global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); - auto fg_pool_size = server_settings_.async_loader_foreground_pool_size; - auto bg_pool_size = server_settings_.async_loader_background_pool_size; - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::Foreground, fg_pool_size ? fg_pool_size : getNumberOfPhysicalCPUCores()); - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundLoad, bg_pool_size ? bg_pool_size : getNumberOfPhysicalCPUCores()); - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundStartup, bg_pool_size ? bg_pool_size : getNumberOfPhysicalCPUCores()); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::Foreground, server_settings_.async_loader_foreground_pool_size); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundLoad, server_settings_.async_loader_background_pool_size); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundStartup, server_settings_.async_loader_background_pool_size); getIOThreadPool().reloadConfiguration( server_settings.max_io_thread_pool_size, diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 15b23d64ee7..7962637ddfd 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -172,7 +173,7 @@ AsyncLoader::AsyncLoader(std::vector pool_initializers, bool lo /* max_free_threads = */ 0, // We do not require free threads /* queue_size = */0), // Unlimited queue to avoid blocking during worker spawning .ready_queue = {}, - .max_threads = init.max_threads + .max_threads = init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores() }); } @@ -227,7 +228,6 @@ void AsyncLoader::schedule(const LoadTaskPtr & task) void AsyncLoader::schedule(const LoadTaskPtrs & tasks) { - // TODO(serxa): optimize it to avoid creating new all_jobs - looks like unnecessary allocations LoadJobSet all_jobs; for (const auto & task : tasks) { @@ -420,6 +420,8 @@ void AsyncLoader::remove(const LoadJobSet & jobs) void AsyncLoader::setMaxThreads(size_t pool, size_t value) { + if (value == 0) + value = getNumberOfPhysicalCPUCores(); std::unique_lock lock{mutex}; auto & p = pools[pool]; // Note that underlying `ThreadPool` always has unlimited `queue_size` and `max_threads`. @@ -633,7 +635,7 @@ void AsyncLoader::enqueue(Info & info, const LoadJobPtr & job, std::unique_lock< // Keep track of currently executing load jobs to be able to: // 1) Detect "wait dependent" deadlocks -- throw LOGICAL_ERROR // (when job A function waits for job B that depends on job A) -// 2) Resolve "wait not scheduled" deadlocks -- implicitly schedule job with all its dependencies +// 2) Detect "wait not scheduled" deadlocks -- throw LOGICAL_ERROR // (thread T is waiting on an assigned job A, but job A is not yet scheduled) // 3) Resolve "priority inversion" deadlocks -- apply priority inheritance // (when high-priority job A function waits for a lower-priority job B, and B never starts due to its priority) @@ -646,13 +648,13 @@ size_t currentPoolOr(size_t pool) return current_load_job ? current_load_job->executionPool() : pool; } -bool DetectWaitDependentDeadlock(const LoadJobPtr & waited) +bool detectWaitDependentDeadlock(const LoadJobPtr & waited) { if (waited.get() == current_load_job) return true; for (const auto & dep : waited->dependencies) { - if (DetectWaitDependentDeadlock(dep)) + if (detectWaitDependentDeadlock(dep)) return true; } return false; @@ -662,17 +664,12 @@ void AsyncLoader::wait(std::unique_lock & job_lock, const LoadJobPtr { // Ensure job we are going to wait was scheduled to avoid "wait not scheduled" deadlocks if (job->job_id == 0) - { - job_lock.unlock(); // Avoid reverse locking order - schedule(LoadJobSet{job}); - job_lock.lock(); - chassert(job->job_id != 0); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Load job '{}' waits for not scheduled load job '{}'", current_load_job->name, job->name); // Deadlock detection and resolution if (current_load_job && job->load_status == LoadStatus::PENDING) { - if (DetectWaitDependentDeadlock(job)) + if (detectWaitDependentDeadlock(job)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Load job '{}' waits for dependent load job '{}'", current_load_job->name, job->name); auto worker_pool = current_load_job->executionPool(); diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 831b68756ee..d90a21cbb6f 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -21,6 +21,16 @@ namespace Poco { class Logger; } namespace DB { +// TERMINOLOGY: +// Job (`LoadJob`) - The smallest part of loading process, executed by worker. Job can depend on the other jobs. Jobs are grouped in tasks. +// Task (`LoadTask`) - Owning holder of a set of jobs. Should be held during the whole job lifetime. Cancels all jobs on destruction. +// Goal jobs (goals) - a subset of "final" jobs of a task (usually no job in task depend on a goal job). +// By default all jobs in task are included in goal jobs. +// Goals should used if you need to create a job that depends on a task (to avoid placing all jobs of the task in dependencies). +// Pool (worker pool) - A set of workers with specific priority. Every job is assigned to a pool. Job can change its pool dynamically. +// Priority (pool priority) - Constant integer value showing relative priority of a pool. Lower value means higher priority. +// AsyncLoader - scheduling system responsible for job dependency tracking and worker management respecting pool priorities. + class LoadJob; using LoadJobPtr = std::shared_ptr; using LoadJobSet = std::unordered_set; @@ -43,6 +53,7 @@ enum class LoadStatus // Smallest indivisible part of a loading process. Load job can have multiple dependencies, thus jobs constitute a direct acyclic graph (DAG). // Job encapsulates a function to be executed by `AsyncLoader` as soon as job functions of all dependencies are successfully executed. // Job can be waited for by an arbitrary number of threads. See `AsyncLoader` class description for more details. +// WARNING: jobs are usually held with ownership by tasks (see `LoadTask`). You are encouraged to add jobs into a tasks as soon as the are created. class LoadJob : private boost::noncopyable { public: @@ -283,7 +294,7 @@ public: String name; Metric metric_threads; Metric metric_active_threads; - size_t max_threads; + size_t max_threads; // Zero means use all CPU cores Priority priority; }; @@ -469,15 +480,7 @@ inline void waitLoad(size_t pool_id, const LoadTaskPtrs & tasks) waitLoad(tasks); } -inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) -{ - LoadJobSet result; - for (const auto & task : tasks) - result.insert(task->goals().begin(), task->goals().end()); - return result; -} - -inline LoadJobSet getGoalsOr(const LoadTaskPtrs & tasks, const LoadJobSet & alternative) +inline LoadJobSet getGoals(const LoadTaskPtrs & tasks, const LoadJobSet & alternative = {}) { LoadJobSet result; for (const auto & task : tasks) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 9a8cee76036..7a0a7cfd93c 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -92,7 +92,7 @@ namespace DB M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ M(UInt64, async_loader_foreground_pool_size, 0, "The maximum number of threads that will be used for foreground (that is being waited for by a query) async loading of tables. Also used for synchronous loading of tables before the server start. Zero means use all CPUs.", 0) \ - M(UInt64, async_loader_background_pool_size, 16, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \ + M(UInt64, async_loader_background_pool_size, 0, "The maximum number of threads that will be used for background async loading of tables. Zero means use all CPUs.", 0) \ M(Bool, async_load_databases, false, "Enable asynchronous loading of databases and tables to speedup server startup. Queries to not yet loaded entity will be blocked until load is finished.", 0) \ M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ \ diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index c348dc7b980..d99e77a4fb8 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -213,7 +213,7 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync( logAboutProgress(log, ++tables_started, total_tables_to_startup, startup_watch); } else - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist during startup", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {}.{} doesn't exist during startup", backQuote(name.database), backQuote(name.table)); }); diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index da8ae7ebdc0..a3fbfbced6c 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -89,7 +89,7 @@ LoadTaskPtrs TablesLoader::loadTablesAsync(LoadJobSet load_after) const auto & path_and_query = metadata.parsed_tables[table_name]; auto task = databases[table_name.database]->loadTableFromMetadataAsync( async_loader, - getGoalsOr(load_dependency_tasks, load_after), + getGoals(load_dependency_tasks, load_after), load_context, path_and_query.path, table_name, @@ -125,7 +125,7 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) { auto task = databases[database_name]->startupDatabaseAsync( async_loader, - getGoalsOr(startup_database[database_name], startup_after), + getGoals(startup_database[database_name], startup_after), strictness_mode); result.push_back(task); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 57fd0285104..a1756444031 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2085,8 +2085,6 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries() AsyncLoader & Context::getAsyncLoader() const { auto lock = getLock(); - size_t fg_max_threads = shared->server_settings.async_loader_foreground_pool_size; - size_t bg_max_threads = shared->server_settings.async_loader_background_pool_size; if (!shared->async_loader) shared->async_loader = std::make_unique(std::vector{ // IMPORTANT: Pool declaration order should match the order in `AsyncLoaderPoolId.h` to get the indices right. @@ -2094,21 +2092,21 @@ AsyncLoader & Context::getAsyncLoader() const "FgLoad", CurrentMetrics::AsyncLoaderForegroundThreads, CurrentMetrics::AsyncLoaderForegroundThreadsActive, - fg_max_threads ? fg_max_threads : getNumberOfPhysicalCPUCores(), + shared->server_settings.async_loader_foreground_pool_size, Priority{0} }, { // AsyncLoaderPoolId::BackgroundLoad "BgLoad", CurrentMetrics::AsyncLoaderBackgroundThreads, CurrentMetrics::AsyncLoaderBackgroundThreadsActive, - bg_max_threads ? bg_max_threads : getNumberOfPhysicalCPUCores(), + shared->server_settings.async_loader_background_pool_size, Priority{1} }, { // AsyncLoaderPoolId::BackgroundStartup "BgStartup", CurrentMetrics::AsyncLoaderBackgroundThreads, CurrentMetrics::AsyncLoaderBackgroundThreadsActive, - bg_max_threads ? bg_max_threads : getNumberOfPhysicalCPUCores(), + shared->server_settings.async_loader_background_pool_size, Priority{2} } }, diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index df97f8a4d5f..cf6af7befc7 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -238,16 +238,7 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data auto load_tasks = loader.loadTablesAsync(); auto startup_tasks = loader.startupTablesAsync(); - if (!async_load_databases) - { - LOG_INFO(log, "Start synchronous loading of databases"); - - // Note that wait implicitly calls schedule - waitLoad(AsyncLoaderPoolId::Foreground, load_tasks); // First prioritize, schedule and wait all the load table tasks - waitLoad(AsyncLoaderPoolId::Foreground, startup_tasks); // Only then prioritize, schedule and wait all the startup tasks - return {}; - } - else + if (async_load_databases) { LOG_INFO(log, "Start asynchronous loading of databases"); @@ -263,6 +254,15 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data // Do NOT wait, just return tasks for continuation or later wait. return joinTasks(load_tasks, startup_tasks); } + else + { + LOG_INFO(log, "Start synchronous loading of databases"); + + // Note that wait implicitly calls schedule + waitLoad(AsyncLoaderPoolId::Foreground, load_tasks); // First prioritize, schedule and wait all the load table tasks + waitLoad(AsyncLoaderPoolId::Foreground, startup_tasks); // Only then prioritize, schedule and wait all the startup tasks + return {}; + } } static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & database_name, const String & default_engine) diff --git a/src/Storages/System/StorageSystemAsyncLoader.cpp b/src/Storages/System/StorageSystemAsyncLoader.cpp index cf20432e09f..aa182e9a4f3 100644 --- a/src/Storages/System/StorageSystemAsyncLoader.cpp +++ b/src/Storages/System/StorageSystemAsyncLoader.cpp @@ -57,10 +57,10 @@ NamesAndTypesList StorageSystemAsyncLoader::getNamesAndTypes() { "is_blocked", std::make_shared() }, { "is_ready", std::make_shared() }, { "elapsed", std::make_shared()}, - { "pool_id", std::make_shared() }, + { "pool_id", std::make_shared() }, { "pool", std::make_shared() }, { "priority", std::make_shared() }, - { "execution_pool_id", std::make_shared() }, + { "execution_pool_id", std::make_shared() }, { "execution_pool", std::make_shared() }, { "execution_priority", std::make_shared() }, { "ready_seqno", std::make_shared(std::make_shared()) }, @@ -103,13 +103,9 @@ void StorageSystemAsyncLoader::fillData(MutableColumns & res_columns, ContextPtr { std::rethrow_exception(state.job->exception()); } - catch (Exception & e) + catch (...) { - exception = e.message(); - } - catch (...) // just in case - { - exception = String("unknown error"); + exception = getCurrentExceptionMessage(false); } } From 28e65883aa7899e898bb473cabd65d15c86dafe9 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 13 Sep 2023 12:22:46 +0000 Subject: [PATCH 130/628] rename settings --- .../server-configuration-parameters/settings.md | 8 ++++---- programs/server/Server.cpp | 6 +++--- programs/server/config.xml | 4 ++-- src/Core/ServerSettings.h | 4 ++-- src/Interpreters/Context.cpp | 6 +++--- 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 00c282ae911..5365e4fe926 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1632,7 +1632,7 @@ Default value: `0.5`. Asynchronous loading of databases and tables. -If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.async_loader` table, `async_loader_background_pool_size` and `async_loader_foreground_pool_size` server settings. Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. +If `true` all non-system databases with `Ordinary`, `Atomic` and `Replicated` engine will be loaded asynchronously after the ClickHouse server start up. See `system.async_loader` table, `tables_loader_background_pool_size` and `tables_loader_foreground_pool_size` server settings. Any query that tries to access a table, that is not yet loaded, will wait for exactly this table to be started up. If load job fails, query will rethrow an error (instead of shutting down the whole server in case of `async_load_databases = false`). The table that is waited for by at least one query will be loaded with higher priority. DDL queries on a database will wait for exactly that database to be started up. If `false`, all databases are loaded when the server starts. @@ -1644,9 +1644,9 @@ The default is `false`. true ``` -## async_loader_foreground_pool_size {#async_loader_foreground_pool_size} +## tables_loader_foreground_pool_size {#tables_loader_foreground_pool_size} -Sets the number of threads performing asynchronous load jobs in foreground pool. The foreground pool is used for loading table synchronously before server start listening on a port and for loading tables that are waited for. Foreground pool has higher priority than background pool. It means that no job starts in background pool while there are jobs running in foreground pool. +Sets the number of threads performing load jobs in foreground pool. The foreground pool is used for loading table synchronously before server start listening on a port and for loading tables that are waited for. Foreground pool has higher priority than background pool. It means that no job starts in background pool while there are jobs running in foreground pool. Possible values: @@ -1656,7 +1656,7 @@ Possible values: Default value: 0. -## async_loader_background_pool_size {#async_loader_background_pool_size} +## tables_loader_background_pool_size {#tables_loader_background_pool_size} Sets the number of threads performing asynchronous load jobs in background pool. The background pool is used for loading tables asynchronously after server start in case there are no queries waiting for the table. It could be beneficial to keep low number of threads in background pool if there are a lot of tables. It will reserve CPU resources for concurrent query execution. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5b1070586ab..143d0101073 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1344,9 +1344,9 @@ try global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::Foreground, server_settings_.async_loader_foreground_pool_size); - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundLoad, server_settings_.async_loader_background_pool_size); - global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundStartup, server_settings_.async_loader_background_pool_size); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::Foreground, server_settings_.tables_loader_foreground_pool_size); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundLoad, server_settings_.tables_loader_background_pool_size); + global_context->getAsyncLoader().setMaxThreads(AsyncLoaderPoolId::BackgroundStartup, server_settings_.tables_loader_background_pool_size); getIOThreadPool().reloadConfiguration( server_settings.max_io_thread_pool_size, diff --git a/programs/server/config.xml b/programs/server/config.xml index 977a3161eb4..bfb7db6f4cb 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -353,8 +353,8 @@ 128 16 16 - 64 - 16 + 64 + 16 --> + 100 + 8192 + 524288 + false + + diff --git a/tests/integration/test_system_logs_hostname/test_replicated.py b/tests/integration/test_system_logs_hostname/test_replicated.py new file mode 100644 index 00000000000..b9db6b03673 --- /dev/null +++ b/tests/integration/test_system_logs_hostname/test_replicated.py @@ -0,0 +1,71 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +def fill_nodes(nodes, shard): + for node in nodes: + node.query( + """ + CREATE DATABASE test; + + CREATE TABLE test.test_table(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date); + """.format( + shard=shard, replica=node.name + ) + ) + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) +node2 = cluster.add_instance( + "node2", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) +node3 = cluster.add_instance( + "node3", with_zookeeper=True, main_configs=["configs/replicated_servers.xml"] +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + fill_nodes([node1, node2, node3], 1) + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + + +def test_truncate_database_replicated(start_cluster): + node1.query("SELECT 1", query_id="query_node1") + node2.query("SELECT 1", query_id="query_node2") + node3.query("SELECT 1", query_id="query_node3") + node1.query("SYSTEM FLUSH LOGS") + node2.query("SYSTEM FLUSH LOGS") + node3.query("SYSTEM FLUSH LOGS") + assert ( + node1.query( + "SELECT hostname from system.query_log where query_id='query_node1' LIMIT 1" + ) + == "node1\n" + ) + assert ( + node2.query( + "SELECT hostname from system.query_log where query_id='query_node2' LIMIT 1" + ) + == "node2\n" + ) + assert ( + node3.query( + "SELECT hostname from system.query_log where query_id='query_node3' LIMIT 1" + ) + == "node3\n" + ) From f21e8ec5d6ccfd766cfa29a760c9b7649ed7703a Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Fri, 27 Oct 2023 23:01:44 -0700 Subject: [PATCH 164/628] add another test --- .../02905_system_logs_hostname.reference | 1 + .../02905_system_logs_hostname.sql | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02905_system_logs_hostname.reference create mode 100644 tests/queries/0_stateless/02905_system_logs_hostname.sql diff --git a/tests/queries/0_stateless/02905_system_logs_hostname.reference b/tests/queries/0_stateless/02905_system_logs_hostname.reference new file mode 100644 index 00000000000..ea10927888b --- /dev/null +++ b/tests/queries/0_stateless/02905_system_logs_hostname.reference @@ -0,0 +1 @@ +test hostname in system log tables diff --git a/tests/queries/0_stateless/02905_system_logs_hostname.sql b/tests/queries/0_stateless/02905_system_logs_hostname.sql new file mode 100644 index 00000000000..aa7a73a5edc --- /dev/null +++ b/tests/queries/0_stateless/02905_system_logs_hostname.sql @@ -0,0 +1,26 @@ +SELECT 'test hostname in system log tables'; + +set log_query_threads=1; +set log_queries_min_type='QUERY_FINISH'; +set log_queries=1; +select '02095_system_logs_hostname' from system.one format Null; +set log_queries=0; +set log_query_threads=0; + +system flush logs; + +select hostname +from system.query_log +where + query like 'select \'02095_system_logs_hostname%' + and current_database = currentDatabase() + and event_date >= yesterday() LIMIT 1 FORMAT Null; + + +select hostName(), hostname +from system.query_thread_log +where + query like 'select \'02095_system_logs_hostname%' + and current_database = currentDatabase() + and event_date >= yesterday() LIMIT 1 FORMAT Null; + From a19e5fe0db94ebf7688c9e75560ce1ef556b2b26 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 28 Oct 2023 16:55:25 +0200 Subject: [PATCH 165/628] increase timeout for sqllogic-test --- docker/test/sqllogic/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index dc1d037a779..48457a99de3 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -36,7 +36,7 @@ RUN git clone --recursive ${odbc_repo} \ && odbcinst -i -s -l -f /clickhouse-odbc/packaging/odbc.ini.sample ENV TZ=Europe/Amsterdam -ENV MAX_RUN_TIME=900 +ENV MAX_RUN_TIME=9000 RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ARG sqllogic_test_repo="https://github.com/gregrahn/sqllogictest.git" From 9eb498b48d4238158fda98884e6e5d225467fa03 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 10:50:05 +0000 Subject: [PATCH 166/628] Fix tests for HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 730b694ef87..e692e900a18 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -391,6 +391,9 @@ namespace private: std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) + return std::nullopt; + auto & schema_cache = StorageHDFS::getSchemaCache(getContext()); for (const auto & path_with_info : paths_with_info_) { From 7689961a8f1e12bdf6050bc90dc393c59ba3f4f1 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 19:44:02 +0000 Subject: [PATCH 167/628] Try to fix test with azuer --- .../test_storage_azure_blob_storage/test.py | 336 ++++++++++++------ 1 file changed, 230 insertions(+), 106 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 909ff490481..a90879c3a00 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -29,7 +29,6 @@ def cluster(): with_azurite=True, ) cluster.start() - yield cluster finally: cluster.shutdown() @@ -50,6 +49,7 @@ def azure_query( "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "DB::Exception: Azure::Core::Http::TransportException: Fail to get a new connection" "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", @@ -69,19 +69,29 @@ def azure_query( continue -def get_azure_file_content(filename): +def get_azure_file_content(filename, port): container_name = "cont" - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" - blob_service_client = BlobServiceClient.from_connection_string(connection_string) + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string( + str(connection_string) + ) container_client = blob_service_client.get_container_client(container_name) blob_client = container_client.get_blob_client(filename) download_stream = blob_client.download_blob() return download_stream.readall().decode("utf-8") -def put_azure_file_content(filename, data): +def put_azure_file_content(filename, port, data): container_name = "cont" - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) blob_service_client = BlobServiceClient.from_connection_string(connection_string) try: container_client = blob_service_client.create_container(container_name) @@ -94,8 +104,13 @@ def put_azure_file_content(filename, data): @pytest.fixture(autouse=True, scope="function") -def delete_all_files(): - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" +def delete_all_files(cluster): + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) blob_service_client = BlobServiceClient.from_connection_string(connection_string) containers = blob_service_client.list_containers() for container in containers: @@ -115,7 +130,8 @@ def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')", + f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}'," + f"'cont', 'test_create_connection_string', 'CSV')", ) @@ -123,57 +139,67 @@ def test_create_table_account_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + f"CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f"'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) def test_simple_write_account_string(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + f"CREATE TABLE test_simple_write (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") - print(get_azure_file_content("test_simple_write.csv")) - assert get_azure_file_content("test_simple_write.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write.csv", port)) + assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' def test_simple_write_connection_string(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", + f"CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', " + f"'cont', 'test_simple_write_c.csv', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") - print(get_azure_file_content("test_simple_write_c.csv")) - assert get_azure_file_content("test_simple_write_c.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_c.csv", port)) + assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' def test_simple_write_named_collection_1(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", + f"CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']}')", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" ) - print(get_azure_file_content("test_simple_write_named.csv")) - assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named.csv", port)) + assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1") def test_simple_write_named_collection_2(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", + f"CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" ) - print(get_azure_file_content("test_simple_write_named_2.csv")) - assert get_azure_file_content("test_simple_write_named_2.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named_2.csv", port)) + assert get_azure_file_content("test_simple_write_named_2.csv", port) == '1,"a"\n' def test_partition_by(cluster): @@ -182,16 +208,19 @@ def test_partition_by(cluster): partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') " + f"PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_write VALUES {values}") - assert "1,2,3\n" == get_azure_file_content("test_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_3.csv", port) + assert "3,2,1\n" == get_azure_file_content("test_1.csv", port) + assert "78,43,45\n" == get_azure_file_content("test_45.csv", port) def test_partition_by_string_column(cluster): @@ -200,15 +229,18 @@ def test_partition_by_string_column(cluster): partition_by = "col_str" values = "(1, 'foo/bar'), (3, 'йцук'), (78, '你好')" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') " + f"PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_string_write VALUES {values}") - assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv") - assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv") - assert '78,"你好"\n' == get_azure_file_content("test_你好.csv") + assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv", port) + assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv", port) + assert '78,"你好"\n' == get_azure_file_content("test_你好.csv", port) def test_partition_by_const_column(cluster): @@ -218,46 +250,54 @@ def test_partition_by_const_column(cluster): partition_by = "'88'" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')" + f" PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") - assert values_csv == get_azure_file_content("test_88.csv") + assert values_csv == get_azure_file_content("test_88.csv", port) def test_truncate(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_truncate (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')", + f"CREATE TABLE test_truncate (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_truncate.csv', format='CSV')", ) azure_query(node, "INSERT INTO test_truncate VALUES (1, 'a')") - assert get_azure_file_content("test_truncate.csv") == '1,"a"\n' + assert get_azure_file_content("test_truncate.csv", port) == '1,"a"\n' azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): - print(get_azure_file_content("test_truncate.csv")) + print(get_azure_file_content("test_truncate.csv", port)) def test_simple_read_write(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_read_write.csv', format='CSV')", + f"CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_simple_read_write.csv', " + f"format='CSV')", ) azure_query(node, "INSERT INTO test_simple_read_write VALUES (1, 'a')") - assert get_azure_file_content("test_simple_read_write.csv") == '1,"a"\n' + assert get_azure_file_content("test_simple_read_write.csv", port) == '1,"a"\n' print(azure_query(node, "SELECT * FROM test_simple_read_write")) assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n" def test_create_new_files_on_insert(cluster): node = cluster.instances["node"] - azure_query( node, - f"create table test_multiple_inserts(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')", + f"create table test_multiple_inserts(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_parquet', format='Parquet')", ) azure_query(node, "truncate table test_multiple_inserts") azure_query( @@ -281,10 +321,10 @@ def test_create_new_files_on_insert(cluster): def test_overwrite(cluster): node = cluster.instances["node"] - azure_query( node, - f"create table test_overwrite(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')", + f"create table test_overwrite(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_parquet_overwrite', format='Parquet')", ) azure_query(node, "truncate table test_overwrite") @@ -308,7 +348,8 @@ def test_insert_with_path_with_globs(cluster): node = cluster.instances["node"] azure_query( node, - f"create table test_insert_globs(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')", + f"create table test_insert_globs(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_insert_with_globs*', format='Parquet')", ) node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" @@ -331,7 +372,8 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{path}', format='CSV')", ) query = f"insert into test_put_{i}_{j} VALUES {values}" @@ -339,7 +381,8 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", + f"CREATE TABLE test_glob_select ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", ) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select" assert azure_query(node, query).splitlines() == [ @@ -363,7 +406,8 @@ def test_azure_glob_scheherazade(cluster): unique_num = random.randint(1, 10000) azure_query( node, - f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{path}', format='CSV')", ) query = ( f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" @@ -382,7 +426,8 @@ def test_azure_glob_scheherazade(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')", + f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='night_*/tale.csv', format='CSV')", ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] @@ -394,6 +439,7 @@ def test_azure_glob_scheherazade(cluster): ) def test_storage_azure_get_gzip(cluster, extension, method): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] filename = f"test_get_gzip.{extension}" name = f"test_get_gzip_{extension}" data = [ @@ -420,14 +466,13 @@ def test_storage_azure_get_gzip(cluster, extension, method): compressed = gzip.GzipFile(fileobj=buf, mode="wb") compressed.write(("\n".join(data)).encode()) compressed.close() - put_azure_file_content(filename, buf.getvalue()) + put_azure_file_content(filename, port, buf.getvalue()) azure_query( node, - f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = AzureBlobStorage( - azure_conf2, container='cont', blob_path ='{filename}', - format='CSV', - compression='{method}')""", + f"CREATE TABLE {name} (name String, id UInt32) ENGINE = AzureBlobStorage( azure_conf2," + f" storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path ='{filename}'," + f"format='CSV', compression='{method}')", ) assert azure_query(node, f"SELECT sum(id) FROM {name}").splitlines() == ["565"] @@ -439,7 +484,9 @@ def test_schema_inference_no_globs(cluster): table_format = "column1 UInt32, column2 String, column3 UInt32" azure_query( node, - f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", + f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", ) query = f"insert into test_schema_inference_src SELECT number, toString(number), number * number FROM numbers(1000)" @@ -447,7 +494,8 @@ def test_schema_inference_no_globs(cluster): azure_query( node, - f"CREATE TABLE test_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')", + f"CREATE TABLE test_select_inference Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_schema_inference_no_globs.csv')", ) print(node.query("SHOW CREATE TABLE test_select_inference")) @@ -474,7 +522,9 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')", + f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSVWithNames')", ) query = f"insert into test_schema_{i}_{j} VALUES {values}" @@ -482,7 +532,8 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", + f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) @@ -497,36 +548,47 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String')" + f" VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_tf.csv")) - assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_tf.csv", port)) + assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', " + f"'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) + print(get_azure_file_content("test_simple_write_connection_tf.csv", port)) + assert ( + get_azure_file_content("test_simple_write_connection_tf.csv", port) == '1,"a"\n' ) - print(get_azure_file_content("test_simple_write_connection_tf.csv")) - assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf1) VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']}') VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_named.csv")) - assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named.csv", port)) + assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' azure_query( node, - "CREATE TABLE drop_table (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", + f"CREATE TABLE drop_table (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']};')", ) azure_query( @@ -537,13 +599,14 @@ def test_simple_write_named_collection_1_table_function(cluster): def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] - + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_named_2_tf.csv")) - assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) + assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' def test_put_get_with_globs_tf(cluster): @@ -562,9 +625,14 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + query = ( + f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + ) assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -576,10 +644,18 @@ def test_schema_inference_no_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" - query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" + query = ( + f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') " + f"SELECT number, toString(number), number * number FROM numbers(1000)" + ) azure_query(node, query) - query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" + query = ( + f"select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='test_schema_inference_no_globs_tf.csv')" + ) assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] @@ -600,10 +676,17 @@ def test_schema_inference_from_globs_tf(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + query = ( + f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + ) azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + query = ( + f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + ) assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -617,15 +700,18 @@ def test_partition_by_tf(cluster): partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_partition_tf_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) + assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv", port) + assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv", port) def test_filter_using_file(cluster): @@ -637,45 +723,64 @@ def test_filter_using_file(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " + f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" + query = ( + f"select count(*) from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', 'test_partition_tf_*.csv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " + f"'{table_format}') WHERE _file='test_partition_tf_3.csv'" + ) assert azure_query(node, query) == "1\n" def test_read_subcolumns(cluster): node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", ) res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "2\tcont/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "0\tcont/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" @@ -683,15 +788,18 @@ def test_read_subcolumns(cluster): def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] - query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" + query = ( + f"select * from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont_not_exists', 'test_table.csv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" + ) expected_err_msg = "container does not exist" assert expected_err_msg in azure_query(node, query, expect_error="true") def test_function_signatures(cluster): node = cluster.instances["node"] - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" - storage_account_url = "http://azurite1:10000/devstoreaccount1" + connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] account_name = "devstoreaccount1" account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( @@ -745,7 +853,8 @@ def check_profile_event_for_query(instance, file, profile_event, amount): query_pattern = f"azureBlobStorage%{file}".replace("'", "\\'") res = int( instance.query( - f"select ProfileEvents['{profile_event}'] from system.query_log where query like '%{query_pattern}%' and query not like '%ProfileEvents%' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" + f"select ProfileEvents['{profile_event}'] from system.query_log where query like '%{query_pattern}%' and query not like '%ProfileEvents%' " + f"and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" ) ) @@ -804,15 +913,16 @@ def check_cache(instance, expected_files): def test_schema_inference_cache(cluster): node = cluster.instances["node"] - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" - storage_account_url = "http://azurite1:10000/devstoreaccount1" + connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] account_name = "devstoreaccount1" account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" node.query("system drop schema cache") azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') select * from numbers(100)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100)", ) time.sleep(1) @@ -826,7 +936,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -836,7 +947,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -849,7 +961,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache2.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache2.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -895,7 +1008,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache3.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache3.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -919,7 +1033,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -943,7 +1058,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') select * from numbers(200) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') " + f"select * from numbers(200) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -958,7 +1074,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.csv', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.csv', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -991,7 +1108,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache.parquet', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache.parquet', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -1007,23 +1125,29 @@ def test_schema_inference_cache(cluster): def test_filtering_by_file_or_path(cluster): node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter1.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter1.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter2.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter2.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter3.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_filter3.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", ) node.query( - f"select count() from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter*.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') where _file = 'test_filter1.tsv'" + f"select count() from azureBlobStorage('{storage_account_url}', 'cont', 'test_filter*.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') " + f"where _file = 'test_filter1.tsv'" ) node.query("SYSTEM FLUSH LOGS") From 1d58cedce5b2666b011482e8ca599a3c6a095e2d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 31 Oct 2023 22:45:42 +0100 Subject: [PATCH 168/628] better --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 4 +--- src/IO/SeekableReadBuffer.h | 3 ++- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index dc3802589d0..4737ce1c79e 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -162,14 +162,12 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm } ReadSettings local_read_settings{settings}; + local_read_settings.local_fs_prefetch = false; if (local_read_settings.local_fs_method != LocalFSReadMethod::pread_threadpool) local_read_settings.local_fs_method = LocalFSReadMethod::pread; if (use_external_buffer) - { local_read_settings.local_fs_buffer_size = 0; - local_read_settings.local_fs_prefetch = false; - } cache_file_reader = createReadBufferFromFileBase( path, diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 2917d9d142e..3bce65cadb0 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -92,7 +92,8 @@ public: /// Sometimes however seek is basically free because underlying read buffer wasn't yet initialised (or re-initialised after reset). virtual bool seekIsCheap() { return false; } - /// If content is cached it makes sense to read it synchronously on the current thread, otherwise it makes sense to read on thread pool. + /// For tables that have an external storage (like S3) as their main storage we'd like to distinguish whether we're reading from this storage or from a local cache. + /// It allows to reuse all the optimisations done for reading from local tables when reading from cache. virtual bool contentIsCached() { return false; } }; From 9f954ece81ae0e9fbd2cfee186623a7be3fc2800 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 2 Nov 2023 14:55:53 +0000 Subject: [PATCH 169/628] fix --- src/Databases/DatabaseOrdinary.cpp | 3 +-- src/Databases/DatabasesCommon.cpp | 15 ++++++++++----- src/Databases/DatabasesCommon.h | 1 + 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index be6354174a5..dae523741f0 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -202,8 +202,7 @@ LoadTaskPtr DatabaseOrdinary::startupTableAsync( fmt::format("startup table {}", name.getFullName()), [this, name] (AsyncLoader &, const LoadJobPtr &) { - // `DatabaseOnDisk::tryGetTable()` is used to avoid waiting for a startup - if (auto table = DatabaseOnDisk::tryGetTable(name.table, {})) + if (auto table = tryGetTableNoWait(name.table)) { /// Since startup() method can use physical paths on disk we don't allow any exclusive actions (rename, drop so on) /// until startup finished. diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 1d8900e5943..851cf0be800 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -200,11 +200,7 @@ bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, ContextP StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, ContextPtr) const { waitTableStarted(table_name); - std::lock_guard lock(mutex); - auto it = tables.find(table_name); - if (it != tables.end()) - return it->second; - return {}; + return tryGetTableNoWait(table_name); } DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const @@ -376,4 +372,13 @@ void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & cre interpreter.execute(); } +StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_name) const +{ + std::lock_guard lock(mutex); + auto it = tables.find(table_name); + if (it != tables.end()) + return it->second; + return {}; +} + } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index c5842d7dac3..fc67596d3de 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -52,6 +52,7 @@ protected: void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex); StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); + StoragePtr tryGetTableNoWait(const String & table_name) const; }; } From 940449dd59ea79229d9322e922372bc204aa5d28 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Nov 2023 15:09:53 +0000 Subject: [PATCH 170/628] Try to fix azure test --- .../test_storage_azure_blob_storage/test.py | 118 +++++++++--------- 1 file changed, 60 insertions(+), 58 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index a90879c3a00..3b4a5bf571b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -49,7 +49,6 @@ def azure_query( "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", - "DB::Exception: Azure::Core::Http::TransportException: Fail to get a new connection" "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", @@ -911,6 +910,66 @@ def check_cache(instance, expected_files): ) +def test_union_schema_inference_mode(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + ) + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + ) + + node.query("system drop schema cache for azure") + + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" + + result = node.query( + "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + ) + assert ( + result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" + "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" + ) + result = azure_query( + node, + f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union' format TSV", + ) + assert result == "1\t\\N\n" "\\N\t2\n" + node.query(f"system drop schema cache for hdfs") + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "b\tNullable(Int64)\n" + + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + ) + + error = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + expect_error="true", + ) + assert "Cannot extract table structure" in error + + def test_schema_inference_cache(cluster): node = cluster.instances["node"] connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] @@ -1157,60 +1216,3 @@ def test_filtering_by_file_or_path(cluster): ) assert int(result) == 1 - - -def test_union_schema_inference_mode(cluster): - node = cluster.instances["node"] - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference1.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 1 as a", - ) - - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 2 as b", - ) - - node.query("system drop schema cache for azure") - - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" - - result = node.query( - "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" - ) - assert ( - result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" - "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" - ) - result = azure_query( - node, - "select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "1\t\\N\n" "\\N\t2\n" - node.query(f"system drop schema cache for hdfs") - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "b\tNullable(Int64)\n" - - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference3.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', TSV) select 'Error'", - ) - - error = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - expect_error="true", - ) - assert "Cannot extract table structure" in error From 1fe344ca45a82d3b6b88f9187e137777e62a1f3c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 2 Nov 2023 16:14:10 +0000 Subject: [PATCH 171/628] add AsyncLoaderWaitMicroseconds profile event --- src/Common/AsyncLoader.cpp | 10 ++++++++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 12 insertions(+) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 7962637ddfd..9813b9ff1d6 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -11,6 +11,14 @@ #include #include #include +#include +#include + + +namespace ProfileEvents +{ + extern const Event AsyncLoaderWaitMicroseconds; +} namespace DB { @@ -693,9 +701,11 @@ void AsyncLoader::wait(std::unique_lock & job_lock, const LoadJobPtr } } + Stopwatch watch; job->waiters++; job->finished.wait(job_lock, [&] { return job->load_status != LoadStatus::PENDING; }); job->waiters--; + ProfileEvents::increment(ProfileEvents::AsyncLoaderWaitMicroseconds, watch.elapsedMicroseconds()); } void AsyncLoader::workerIsSuspendedByWait(size_t pool_id, const LoadJobPtr & job) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f4d7242f70a..0d7f6eb1aea 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -555,6 +555,8 @@ The server successfully detected this situation and will download merged part fr \ M(ConnectionPoolIsFullMicroseconds, "Total time spent waiting for a slot in connection pool.") \ \ + M(AsyncLoaderWaitMicroseconds, "Total time a query was waiting for async loader jobs.") \ + \ M(LogTest, "Number of log messages with level Test") \ M(LogTrace, "Number of log messages with level Trace") \ M(LogDebug, "Number of log messages with level Debug") \ From c1c390348ebee7a8c0d30a4b558784931125a3ad Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 2 Nov 2023 17:07:21 +0000 Subject: [PATCH 172/628] `DatabaseCatalog::getTable()` should also wait --- src/Interpreters/DatabaseCatalog.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2e82ff464d2..e1d8d0d4bf6 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -349,6 +349,9 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return {}; } + /// Wait for table to be started because we are going to return StoragePtr + db_and_table.first->waitTableStarted(table_id.getTableName()); + #if USE_LIBPQXX if (!context_->isInternalQuery() && (db_and_table.first->getEngineName() == "MaterializedPostgreSQL")) { From 036af7f103a1257dc743bbec4692b8bd84522b15 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 3 Nov 2023 15:14:37 +0100 Subject: [PATCH 173/628] add test --- .../00180_read_from_cache.reference | 1 + .../1_stateful/00180_read_from_cache.sh | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/1_stateful/00180_read_from_cache.reference create mode 100755 tests/queries/1_stateful/00180_read_from_cache.sh diff --git a/tests/queries/1_stateful/00180_read_from_cache.reference b/tests/queries/1_stateful/00180_read_from_cache.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/1_stateful/00180_read_from_cache.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/1_stateful/00180_read_from_cache.sh b/tests/queries/1_stateful/00180_read_from_cache.sh new file mode 100755 index 00000000000..a9332009015 --- /dev/null +++ b/tests/queries/1_stateful/00180_read_from_cache.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# Tags: no-parallel, no-random-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +# Warm up the cache +$CLICKHOUSE_CLIENT -q "SELECT * FROM test.hits_s3 WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test.hits_s3 WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" + +query_id=02906_read_from_cache_$RANDOM +$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM test.hits_s3 WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" + +$CLICKHOUSE_CLIENT -nq " + SYSTEM FLUSH LOGS; + SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'] FROM system.query_log WHERE query_id = '$query_id' AND type = 'QueryFinish' +" From d0b93d90c7ec903c3f97c7fa42df9891852d1757 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 3 Nov 2023 15:31:32 +0000 Subject: [PATCH 174/628] wait for db to be started iff it was async-started --- src/Databases/DatabaseAtomic.cpp | 4 ++-- src/Databases/DatabaseOrdinary.cpp | 4 ++-- src/Databases/DatabaseReplicated.cpp | 4 ++-- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 4 ++-- src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ecdd144c96a..a70e7e6e05e 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -472,8 +472,8 @@ LoadTaskPtr DatabaseAtomic::startupDatabaseAsync(AsyncLoader & async_loader, Loa void DatabaseAtomic::waitDatabaseStarted() const { - assert(startup_atomic_database_task); - waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_atomic_database_task); + if (startup_atomic_database_task) + waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_atomic_database_task); } void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path, bool if_data_path_exist) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index dae523741f0..ba5f6896926 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -248,8 +248,8 @@ void DatabaseOrdinary::waitTableStarted(const String & name) const void DatabaseOrdinary::waitDatabaseStarted() const { /// Prioritize load and startup of all tables and database itself and wait for them synchronously - assert(startup_database_task); - waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_database_task); + if (startup_database_task) + waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_database_task); } DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3ce0f7917b2..a11404f1e1a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -573,8 +573,8 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, void DatabaseReplicated::waitDatabaseStarted() const { - assert(startup_replicated_database_task); - waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_replicated_database_task); + if (startup_replicated_database_task) + waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_replicated_database_task); } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 981b309755f..841eb53db4c 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -85,8 +85,8 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_ void DatabaseMaterializedMySQL::waitDatabaseStarted() const { - assert(startup_mysql_database_task); - waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_mysql_database_task); + if (startup_mysql_database_task) + waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_mysql_database_task); } void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index bfd19b9ce7a..0f500c632a7 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -155,8 +155,8 @@ LoadTaskPtr DatabaseMaterializedPostgreSQL::startupDatabaseAsync(AsyncLoader & a void DatabaseMaterializedPostgreSQL::waitDatabaseStarted() const { - assert(startup_postgresql_database_task); - waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_postgresql_database_task); + if (startup_postgresql_database_task) + waitLoad(currentPoolOr(AsyncLoaderPoolId::Foreground), startup_postgresql_database_task); } void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) From 6345d94d2bde4de879e27b29920f680f5314db65 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 3 Nov 2023 16:27:04 +0000 Subject: [PATCH 175/628] Fix test --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3b4a5bf571b..6c72dece5d8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -942,7 +942,7 @@ def test_union_schema_inference_mode(cluster): ) result = azure_query( node, - f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union' format TSV", + f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') order by tuple(*) settings schema_inference_mode='union' format TSV", ) assert result == "1\t\\N\n" "\\N\t2\n" node.query(f"system drop schema cache for hdfs") From 7e220182eb2b32856f6a566e36b3252a794708a4 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 3 Nov 2023 16:41:32 +0000 Subject: [PATCH 176/628] better --- src/Databases/TablesLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index a3fbfbced6c..f1b5c4377fe 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -107,7 +107,7 @@ LoadTaskPtrs TablesLoader::startupTablesAsync(LoadJobSet startup_after) LoadTaskPtrs result; std::unordered_map startup_database; /// database name -> all its tables startup tasks - for (const auto & table_id : all_loading_dependencies.getTablesSortedByDependency()) + for (const auto & table_id : all_loading_dependencies.getTables()) { // Make startup table task auto table_name = table_id.getQualifiedName(); From 77dc2e63b1990212a3216d8d1ed8b11a25335f09 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 3 Nov 2023 17:15:51 +0000 Subject: [PATCH 177/628] randomize async_load_databases in stress tests --- docker/test/stress/run.sh | 6 ++++++ programs/server/config.xml | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index b5092fd40df..afc1a386a48 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -191,6 +191,12 @@ sudo cat /etc/clickhouse-server/config.d/logger_trace.xml \ > /etc/clickhouse-server/config.d/logger_trace.xml.tmp mv /etc/clickhouse-server/config.d/logger_trace.xml.tmp /etc/clickhouse-server/config.d/logger_trace.xml +# Randomize async_load_databases +if [ $(( $(date +%-d) % 2 )) -eq 1 ]; then + sudo echo "true" \ + > /etc/clickhouse-server/config.d/enable_async_load_databases.xml +fi + start stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ diff --git a/programs/server/config.xml b/programs/server/config.xml index 775776336c9..184eade7816 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -372,7 +372,7 @@ - false + From 70468320560aa76c86e2069a054a52dfb1568dbb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 3 Nov 2023 22:50:03 +0100 Subject: [PATCH 178/628] change signatures --- src/Disks/IO/AsynchronousBoundedReadBuffer.cpp | 6 +++--- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 4 ++-- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 8 ++++---- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 4 ++-- src/IO/SeekableReadBuffer.h | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index d52748b04bf..1952d8ae253 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -311,7 +311,7 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) if (read_until_position && new_pos > *read_until_position) { - if (!impl->seekIsCheap()) + if (!impl->isSeekCheap()) ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); file_offset_of_buffer_end = new_pos = *read_until_position; /// read_until_position is a non-included boundary. impl->seek(file_offset_of_buffer_end, SEEK_SET); @@ -322,7 +322,7 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. * Note: we read in range [file_offset_of_buffer_end, read_until_position). */ - if (!impl->seekIsCheap() && file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position + if (!impl->isSeekCheap() && file_offset_of_buffer_end && read_until_position && new_pos < *read_until_position && new_pos > file_offset_of_buffer_end && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) { ProfileEvents::increment(ProfileEvents::RemoteFSLazySeeks); @@ -330,7 +330,7 @@ off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) } else { - if (!impl->seekIsCheap()) + if (!impl->isSeekCheap()) ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); file_offset_of_buffer_end = new_pos; impl->seek(file_offset_of_buffer_end, SEEK_SET); diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 021678b890a..7aa478a870a 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -60,9 +60,9 @@ public: REMOTE_FS_READ_AND_PUT_IN_CACHE, }; - bool seekIsCheap() override; + bool isSeekCheap() override; - bool contentIsCached() override; + bool isContentCached(size_t offset) override; private: using ImplementationBufferPtr = std::shared_ptr; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index cc6c385907d..90a1bd9d240 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -265,16 +265,16 @@ ReadBufferFromRemoteFSGather::~ReadBufferFromRemoteFSGather() appendUncachedReadInfo(); } -bool ReadBufferFromRemoteFSGather::seekIsCheap() +bool ReadBufferFromRemoteFSGather::isSeekCheap() { - return !current_buf || current_buf->seekIsCheap(); + return !current_buf || current_buf->isSeekCheap(); } -bool ReadBufferFromRemoteFSGather::contentIsCached() +bool ReadBufferFromRemoteFSGather::isContentCached(size_t offset) { if (!current_buf) initialize(); - return current_buf && current_buf->contentIsCached(); + return current_buf && current_buf->isContentCached(offset); } } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 97c978bad6a..09568752009 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -48,9 +48,9 @@ public: off_t getPosition() override { return file_offset_of_buffer_end - available(); } - bool seekIsCheap() override; + bool isSeekCheap() override; - bool contentIsCached() override; + bool isContentCached(size_t offset) override; private: SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 3bce65cadb0..0d09489f488 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -90,11 +90,11 @@ public: /// We do some tricks to avoid seek cost. E.g we read more data and than ignore it (see remote_read_min_bytes_for_seek). /// Sometimes however seek is basically free because underlying read buffer wasn't yet initialised (or re-initialised after reset). - virtual bool seekIsCheap() { return false; } + virtual bool isSeekCheap() { return false; } /// For tables that have an external storage (like S3) as their main storage we'd like to distinguish whether we're reading from this storage or from a local cache. /// It allows to reuse all the optimisations done for reading from local tables when reading from cache. - virtual bool contentIsCached() { return false; } + virtual bool isContentCached([[maybe_unused]] size_t offset) { return false; } }; From 5040a3b6d3bca5bdd16c93bf1e367b54f2b242b3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 3 Nov 2023 22:52:12 +0100 Subject: [PATCH 179/628] do seek before calling isContentCached() --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 18 +++++++++-- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 30 +++++++++++++------ src/Disks/IO/ThreadPoolRemoteFSReader.h | 2 ++ src/Interpreters/Cache/FileSegment.h | 2 ++ 4 files changed, 41 insertions(+), 11 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 4737ce1c79e..bce345e9066 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -697,6 +697,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() { + if (file_segments->empty()) + return false; + auto & file_segment = file_segments->front(); const auto & current_read_range = file_segment.range(); auto current_state = file_segment.state(); @@ -1069,6 +1072,9 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() first_offset, file_segments->toString()); + if (file_offset_of_buffer_end > current_read_range.right) + completeFileSegmentAndGetNext(); + /// Release buffer a little bit earlier. if (read_until_position == file_offset_of_buffer_end) implementation_buffer.reset(); @@ -1232,16 +1238,24 @@ String CachedOnDiskReadBufferFromFile::getInfoForLog() current_file_segment_info); } -bool CachedOnDiskReadBufferFromFile::seekIsCheap() +bool CachedOnDiskReadBufferFromFile::isSeekCheap() { return !initialized || read_type == ReadType::CACHED; } -bool CachedOnDiskReadBufferFromFile::contentIsCached() +bool CachedOnDiskReadBufferFromFile::isContentCached(size_t offset) { if (!initialized) initialize(file_offset_of_buffer_end, getTotalSizeToRead()); + if (!file_segments->front().range().contains(offset) || offset != file_offset_of_buffer_end) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Assumption is wrong: offset={}, file_offset_of_buffer_end={}, file_segments->front()={}", + offset, + file_offset_of_buffer_end, + file_segments->front().range().toString()); + return canStartFromCache(file_offset_of_buffer_end, file_segments->front()); } } diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 71f234d33d3..524a21c5e93 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -11,7 +11,6 @@ #include #include #include -#include "Parsers/IAST.h" #include "config.h" #include @@ -69,22 +68,31 @@ ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queu std::future ThreadPoolRemoteFSReader::submit(Request request) { auto * fd = assert_cast(request.descriptor.get()); - if (fd->getReader().contentIsCached()) + auto & reader = fd->getReader(); + + /// `seek` have to be done before checking `isContentCached`, and `set` have to be done prior to `seek` + reader.set(request.buf, request.size); + reader.seek(request.offset, SEEK_SET); + + if (reader.isContentCached(request.offset)) { std::promise promise; std::future future = promise.get_future(); - promise.set_value(execute(request)); + promise.set_value(execute(request, /*seek_performed=*/true)); return future; } ProfileEventTimeIncrement elapsed(ProfileEvents::ThreadpoolReaderSubmit); - return scheduleFromThreadPool([request, this]() -> Result { return execute(request); }, - *pool, - "VFSRead", - request.priority); + return scheduleFromThreadPool( + [request, this]() -> Result { return execute(request, /*seek_performed=*/true); }, *pool, "VFSRead", request.priority); } IAsynchronousReader::Result ThreadPoolRemoteFSReader::execute(Request request) +{ + return execute(request, /*seek_performed=*/false); +} + +IAsynchronousReader::Result ThreadPoolRemoteFSReader::execute(Request request, bool seek_performed) { CurrentMetrics::Increment metric_increment{CurrentMetrics::RemoteRead}; @@ -96,8 +104,12 @@ IAsynchronousReader::Result ThreadPoolRemoteFSReader::execute(Request request) auto watch = std::make_unique(CLOCK_REALTIME); - reader.set(request.buf, request.size); - reader.seek(request.offset, SEEK_SET); + if (!seek_performed) + { + reader.set(request.buf, request.size); + reader.seek(request.offset, SEEK_SET); + } + if (request.ignore) { ProfileEvents::increment(ProfileEvents::AsynchronousReaderIgnoredBytes, request.ignore); diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h index 192a12370e3..cd2bf223f33 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.h +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.h @@ -21,6 +21,8 @@ public: void wait() override; private: + IAsynchronousReader::Result execute(Request request, bool seek_performed); + std::unique_ptr pool; }; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 607fcaeb9c6..8e0c47fe680 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -135,6 +135,8 @@ public: size_t size() const { return right - left + 1; } + bool contains(size_t point) const { return left <= point && point <= right; } + String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } }; From 6548ce2eb52200383907e2dc6d411664efaf8df7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 3 Nov 2023 23:15:38 +0100 Subject: [PATCH 180/628] fix style --- tests/queries/1_stateful/00180_read_from_cache.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00180_read_from_cache.sh b/tests/queries/1_stateful/00180_read_from_cache.sh index a9332009015..846f74a1a66 100755 --- a/tests/queries/1_stateful/00180_read_from_cache.sh +++ b/tests/queries/1_stateful/00180_read_from_cache.sh @@ -16,5 +16,5 @@ $CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM test.hits_s3 WHERE U $CLICKHOUSE_CLIENT -nq " SYSTEM FLUSH LOGS; - SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'] FROM system.query_log WHERE query_id = '$query_id' AND type = 'QueryFinish' + SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'] FROM system.query_log WHERE query_id = '$query_id' AND type = 'QueryFinish' AND event_date >= yesterday() AND current_database = currentDatabase() " From 28128d3f7872694b0d1b12f470dee289c4231a6f Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 6 Nov 2023 12:23:09 +0000 Subject: [PATCH 181/628] fix defaults for new pools in server config.xml --- programs/server/config.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 184eade7816..2f018009fe4 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -365,8 +365,8 @@ 128 16 16 - 64 - 16 + 0 + 0 --> From c5fabcc4727d6137f0f54d58d91a38aba9e03d8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:50:50 +0100 Subject: [PATCH 234/628] Fix tests --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e02e8695135..9a5af77d57c 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -581,10 +581,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, context); + bool allow_suspicious_ttl = args.attach || args.getLocalContext()->getSettingsRef().allow_suspicious_ttl_expressions; + if (args.storage_def->ttl_table) { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - args.storage_def->ttl_table->ptr(), metadata.columns, context, metadata.primary_key, args.attach); + args.storage_def->ttl_table->ptr(), metadata.columns, context, metadata.primary_key, allow_suspicious_ttl); } if (args.query.columns_list && args.query.columns_list->indices) @@ -602,7 +604,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, context, metadata.primary_key, args.attach); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, context, metadata.primary_key, allow_suspicious_ttl); metadata.column_ttls_by_name[name] = new_ttl_entry; } From 0f8d9515ec637bd346baf65d10c1f6b883d4c37c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 17:53:53 +0100 Subject: [PATCH 235/628] Fix style check --- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index dfc3686683d..02ef91c6aa2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { size_t getRatio(size_t total, double ratio) From 81a50703c3e73f64660790a6686450121366e5b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 23:39:40 +0100 Subject: [PATCH 236/628] Fix tests --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bfd1693bfb2..13503dbf444 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2884,7 +2884,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!settings.allow_non_metadata_alters) { - auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, getContext()); + auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, local_context); if (!mutation_commands.empty()) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, @@ -2893,7 +2893,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context queryToString(mutation_commands.ast())); } - commands.apply(new_metadata, getContext()); + commands.apply(new_metadata, local_context); if (commands.hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, @@ -3133,7 +3133,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!columns_to_check_conversion.empty()) { auto old_header = old_metadata.getSampleBlock(); - performRequiredConversions(old_header, columns_to_check_conversion, getContext()); + performRequiredConversions(old_header, columns_to_check_conversion, local_context); } if (old_metadata.hasSettingsChanges()) @@ -3165,7 +3165,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } if (setting_name == "storage_policy") - checkStoragePolicy(getContext()->getStoragePolicy(new_value.safeGet())); + checkStoragePolicy(local_context->getStoragePolicy(new_value.safeGet())); } /// Check if it is safe to reset the settings From 86d14a07e127adfdd2bab53d420dce65a36521a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 02:12:08 +0100 Subject: [PATCH 237/628] Fix test --- tests/queries/0_stateless/01070_mutations_with_dependencies.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index 566bb16b10c..813ebf3f5a7 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -34,6 +34,8 @@ select count() from ttl where s = 'b'; drop table ttl; -- check only that it doesn't throw exceptions. +SET allow_suspicious_ttl_expressions = 1; + create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default'; alter table ttl materialize ttl; drop table ttl; From 8fac59b18e15a9d2854a08ffb2b485ff81a2902e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 12:58:18 +0100 Subject: [PATCH 238/628] Fix cache load on server startup with slru policy --- src/Interpreters/Cache/FileCache.cpp | 9 ++---- src/Interpreters/Cache/IFileCachePriority.h | 10 +++++- .../Cache/LRUFileCachePriority.cpp | 3 +- src/Interpreters/Cache/LRUFileCachePriority.h | 10 ++++-- .../Cache/SLRUFileCachePriority.cpp | 31 +++++++++++++++++-- .../Cache/SLRUFileCachePriority.h | 9 +++++- 6 files changed, 58 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0c818e10c9c..f1b3d24dbc1 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1001,9 +1001,6 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) const auto key = Key::fromKeyString(key_directory.filename().string()); auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true); - const size_t size_limit = main_priority->getSizeLimit(); - const size_t elements_limit = main_priority->getElementsLimit(); - for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it) { auto offset_with_suffix = offset_it->path().filename().string(); @@ -1044,13 +1041,13 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) bool limits_satisfied; IFileCachePriority::IteratorPtr cache_it; + { auto lock = lockCache(); - limits_satisfied = (size_limit == 0 || main_priority->getSize(lock) + size <= size_limit) - && (elements_limit == 0 || main_priority->getElementsCount(lock) + 1 <= elements_limit); + limits_satisfied = main_priority->canFit(size, lock); if (limits_satisfied) - cache_it = main_priority->add(key_metadata, offset, size, lock); + cache_it = main_priority->add(key_metadata, offset, size, lock, /* is_startup */true); /// TODO: we can get rid of this lockCache() if we first load everything in parallel /// without any mutual lock between loading threads, and only after do removeOverflow(). diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index fe5de21ec48..44fb450658b 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -59,7 +59,15 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - virtual IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; + /// Throws exception if there is not enought size to fit it. + virtual IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) = 0; + + virtual bool canFit(size_t size, const CacheGuard::Lock &) const = 0; virtual void shuffle(const CacheGuard::Lock &) = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index f9b0ddfce15..cb4da79af59 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -30,7 +30,8 @@ IFileCachePriority::IteratorPtr LRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, - const CacheGuard::Lock & lock) + const CacheGuard::Lock & lock, + bool /* is_startup */) { return std::make_shared(add(Entry(key_metadata->key, offset, size, key_metadata), lock)); } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 289968602ca..31a5c527ca8 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -25,7 +25,14 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; } - IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + bool canFit(size_t size, const CacheGuard::Lock &) const override; + + IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) override; bool collectCandidatesForEviction( size_t size, @@ -53,7 +60,6 @@ private: /// because of invalidated entries. std::atomic current_elements_num = 0; - bool canFit(size_t size, const CacheGuard::Lock &) const; bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; LRUQueue::iterator remove(LRUQueue::iterator it, const CacheGuard::Lock &); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 02ef91c6aa2..f2c96c4df24 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -41,14 +41,39 @@ size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) co return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock); } +bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const +{ + return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock); +} + IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, - const CacheGuard::Lock & lock) + const CacheGuard::Lock & lock, + bool is_startup) { - auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); - return std::make_shared(this, std::move(lru_iterator), false); + if (is_startup) + { + /// If it is server startup, we put entries in any queue it will fit in, + /// but with preference for probationary queue, + /// because we do not know the distribution between queues after server restart. + if (probationary_queue.canFit(size, lock)) + { + auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), false); + } + else + { + auto lru_iterator = protected_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), true); + } + } + else + { + auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), false); + } } bool SLRUFileCachePriority::collectCandidatesForEviction( diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 9dad6c15fee..7753f6f23b2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -24,7 +24,14 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override; - IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + bool canFit(size_t size, const CacheGuard::Lock &) const override; + + IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) override; bool collectCandidatesForEviction( size_t size, From 5c5cbc9ef507256a13f78aa8e57ea1981b0453ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 15:46:45 +0100 Subject: [PATCH 239/628] Fix typo --- src/Interpreters/Cache/IFileCachePriority.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 44fb450658b..e7a05b0e46d 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -59,7 +59,7 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - /// Throws exception if there is not enought size to fit it. + /// Throws exception if there is not enough size to fit it. virtual IteratorPtr add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, From 411660c267a5c378069d209067a225801bd9b3fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 17:35:56 +0100 Subject: [PATCH 240/628] Fix integration tests --- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_ttl_move/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 30669feb6b3..2484ef1bdf1 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1834,7 +1834,7 @@ def _insert_merge_execute( ORDER BY tuple() PARTITION BY tuple() TTL now()-1 TO VOLUME 'external' - SETTINGS storage_policy='{policy}' + SETTINGS storage_policy='{policy}', allow_suspicious_ttl_expressions = 1 """.format( name=name, policy=policy ) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index c1c076277bb..3faa20506ed 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -297,7 +297,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): # Second expression is preferred because d1 > now()-3600. node1.query( - """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external'""".format( + """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external' SETTINGS allow_suspicious_ttl_expressions = 1""".format( name=name ) ) From 791c01d666b50efa7090da9e5651cd3cc76b9e26 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 18:34:34 +0100 Subject: [PATCH 241/628] Fix case with database replicated --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5c41c0b0829..e80156563d8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -662,7 +662,7 @@ class IColumn; M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result, one of: 'none', 'throw', 'null_status_on_timeout', 'never_throw'", 0) \ - M(UInt64, distributed_ddl_entry_format_version, 5, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ + M(UInt64, distributed_ddl_entry_format_version, 6, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ From 0f7b5ebd68d44f1789a9186082a9e8fdfa906b34 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 18:42:56 +0100 Subject: [PATCH 242/628] Fix clang-tidy --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 +- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index cb4da79af59..a596d041941 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IFileCachePriority::IteratorPtr LRUFileCachePriority::add( +IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, size_t size, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index f2c96c4df24..8f7f5e4aa32 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -46,7 +46,7 @@ bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) c return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock); } -IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( +IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, size_t size, From 6937c0de19e50708445e1a818c3560a92eb2b5d3 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 25 Nov 2023 15:42:37 +0000 Subject: [PATCH 243/628] wait for database to load before shutdown --- src/Databases/DatabaseOnDisk.cpp | 7 +++++++ src/Databases/DatabaseOnDisk.h | 2 ++ 2 files changed, 9 insertions(+) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a9652e2ea1d..e342da5ef75 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -163,6 +163,13 @@ DatabaseOnDisk::DatabaseOnDisk( } +void DatabaseOnDisk::shutdown() +{ + waitDatabaseStarted(); + DatabaseWithOwnTablesBase::shutdown(); +} + + void DatabaseOnDisk::createTable( ContextPtr local_context, const String & table_name, diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 0db6a94b86d..59c2c27068e 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -32,6 +32,8 @@ class DatabaseOnDisk : public DatabaseWithOwnTablesBase public: DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, ContextPtr context); + void shutdown() override; + void createTable( ContextPtr context, const String & table_name, From 34945227ac94f9df011962b4450f127c6556fbf2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 25 Nov 2023 20:51:57 +0100 Subject: [PATCH 244/628] address comments --- src/Parsers/ExpressionListParsers.cpp | 2 +- tests/sqllogic/test_parser.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 5f8b932045a..d2852300e70 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2604,7 +2604,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos } } - /// ignore all heading plus + /// ignore all leading plus while (pos->type == TokenType::Plus) { ++pos; diff --git a/tests/sqllogic/test_parser.py b/tests/sqllogic/test_parser.py index 378c2794560..cb1144d7dd9 100755 --- a/tests/sqllogic/test_parser.py +++ b/tests/sqllogic/test_parser.py @@ -167,7 +167,7 @@ class FileBlockBase: "NULLABLE", expressions=[cast.to] ) new_sql = ast.sql("clickhouse") - logger.info("convert from %s to %s", sql, new_sql) + # logger.info("convert from %s to %s", sql, new_sql) return new_sql return sql From 1790626406ae707c551fccce6ceb166d81d360bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 1 Oct 2022 16:32:04 +0200 Subject: [PATCH 245/628] Check what will happen if we build ClickHouse with Musl --- PreLoad.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 4879e721ae3..223e7aedd42 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -86,7 +86,7 @@ if (OS MATCHES "Linux" # - compile musl with debug and -fasynchronous-unwind-tables # # But none of this changes anything so far. - set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64.cmake" CACHE INTERNAL "") + set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64-musl.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)") set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-aarch64.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(ppc64le.*|PPC64LE.*)") From 043e9239d69ea54768850d292b4f3fe02efab896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 18:59:54 +0100 Subject: [PATCH 246/628] Update Azure and Sentry --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 096049bf24f..352ff0a61cb 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 096049bf24fffafcaccc132b9367694532716731 +Subproject commit 352ff0a61cb319ac1cc38c4058443ddf70147530 From 6effb1311f31b73d8f3f60560f5efd218b8bc71c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:01:41 +0100 Subject: [PATCH 247/628] Compatibility with Musl in HDFS and RocksDB --- contrib/libhdfs3-cmake/CMakeLists.txt | 5 +++++ contrib/rocksdb-cmake/CMakeLists.txt | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/libhdfs3-cmake/CMakeLists.txt b/contrib/libhdfs3-cmake/CMakeLists.txt index 4278575fd7f..8cd951af746 100644 --- a/contrib/libhdfs3-cmake/CMakeLists.txt +++ b/contrib/libhdfs3-cmake/CMakeLists.txt @@ -26,6 +26,11 @@ ADD_DEFINITIONS(-D__STDC_FORMAT_MACROS) ADD_DEFINITIONS(-D_GNU_SOURCE) ADD_DEFINITIONS(-D_GLIBCXX_USE_NANOSLEEP) ADD_DEFINITIONS(-DHAVE_NANOSLEEP) + +if (USE_MUSL) + ADD_DEFINITIONS(-DSTRERROR_R_RETURN_INT) +endif () + set(HAVE_STEADY_CLOCK 1) set(HAVE_NESTED_EXCEPTION 1) SET(HAVE_BOOST_CHRONO 0) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 7d7666dff87..c4220ba90ac 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -117,7 +117,7 @@ endif() add_definitions(-DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX) -if (OS_LINUX OR OS_FREEBSD) +if ((OS_LINUX OR OS_FREEBSD) AND NOT USE_MUSL) add_definitions(-DROCKSDB_PTHREAD_ADAPTIVE_MUTEX) endif() From 6d4af13f911e2d4e68e8429d68fa84dc5874a85d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:13:12 +0100 Subject: [PATCH 248/628] Update HDFS --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index bdcb91354b1..b9598e60167 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit bdcb91354b1c05b21e73043a112a6f1e3b013497 +Subproject commit b9598e6016720a7c088bfe85ce1fa0410f9d2103 From 9621c519e14745104b04f267ab4b24301f0df11c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:20:47 +0100 Subject: [PATCH 249/628] Clarifications --- contrib/unixodbc-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/unixodbc-cmake/CMakeLists.txt b/contrib/unixodbc-cmake/CMakeLists.txt index 3317654cd67..6fbe8c14ebb 100644 --- a/contrib/unixodbc-cmake/CMakeLists.txt +++ b/contrib/unixodbc-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ option (ENABLE_ODBC "Enable ODBC library" ${ENABLE_LIBRARIES}) -if (NOT OS_LINUX) +if (NOT OS_LINUX OR USE_MUSL) if (ENABLE_ODBC) - message(STATUS "ODBC is only supported on Linux") + message(STATUS "ODBC is only supported on Linux with dynamic linking") endif() set (ENABLE_ODBC OFF CACHE INTERNAL "") endif () From b2d8103d19b5af20a6d1b2f10019d551790ba2cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 23:25:39 +0100 Subject: [PATCH 250/628] Add special build with Musl --- .github/workflows/master.yml | 6 ++++++ .github/workflows/pull_request.yml | 5 +++++ docker/packager/packager | 9 +++++++++ tests/ci/ci_config.py | 8 ++++++++ 4 files changed, 28 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e662a5b6f98..9ffb58a696a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -205,6 +205,12 @@ jobs: with: build_name: binary_amd64_compat checkout_depth: 0 + BuilderBinAmd64Musl: + needs: [ DockerHubPush ] + uses: ./.github/workflows/reusable_build.yml + with: + build_name: binary_amd64_musl + checkout_depth: 0 BuilderBinAarch64V80Compat: needs: [DockerHubPush] uses: ./.github/workflows/reusable_build.yml diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f8f052d9226..1620b00c33f 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -242,6 +242,11 @@ jobs: uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_compat + BuilderBinAmd64Musl: + needs: [ FastTest, StyleCheck ] + uses: ./.github/workflows/reusable_build.yml + with: + build_name: binary_amd64_musl BuilderBinAarch64V80Compat: needs: [FastTest, StyleCheck] uses: ./.github/workflows/reusable_build.yml diff --git a/docker/packager/packager b/docker/packager/packager index b5bcbada1da..ff77892b8b3 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -145,6 +145,7 @@ def parse_env_variables( RISCV_SUFFIX = "-riscv64" S390X_SUFFIX = "-s390x" AMD64_COMPAT_SUFFIX = "-amd64-compat" + AMD64_MUSL_SUFFIX = "-amd64-musl" result = [] result.append("OUTPUT_DIR=/output") @@ -163,6 +164,7 @@ def parse_env_variables( is_cross_s390x = compiler.endswith(S390X_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) is_amd64_compat = compiler.endswith(AMD64_COMPAT_SUFFIX) + is_amd64_musl = compiler.endswith(AMD64_MUSL_SUFFIX) if is_cross_darwin: cc = compiler[: -len(DARWIN_SUFFIX)] @@ -232,6 +234,12 @@ def parse_env_variables( cc = compiler[: -len(AMD64_COMPAT_SUFFIX)] result.append("DEB_ARCH=amd64") cmake_flags.append("-DNO_SSE3_OR_HIGHER=1") + elif is_amd64_musl: + cc = compiler[: -len(AMD64_MUSL_SUFFIX)] + result.append("DEB_ARCH=amd64") + cmake_flags.append( + "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64-musl.cmake" + ) else: cc = compiler result.append("DEB_ARCH=amd64") @@ -396,6 +404,7 @@ def parse_args() -> argparse.Namespace: "clang-17-riscv64", "clang-17-s390x", "clang-17-amd64-compat", + "clang-17-amd64-musl", "clang-17-freebsd", ), default="clang-17", diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d1d2021753e..aa3aa5654aa 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -208,6 +208,13 @@ CI_CONFIG = CiConfig( static_binary_name="amd64compat", comment="SSE2-only build", ), + "binary_amd64_musl": BuildConfig( + name="binary_amd64_musl", + compiler="clang-17-amd64-musl", + package_type="binary", + static_binary_name="amd64musl", + comment="Build with Musl", + ), "binary_riscv64": BuildConfig( name="binary_riscv64", compiler="clang-17-riscv64", @@ -249,6 +256,7 @@ CI_CONFIG = CiConfig( "binary_riscv64", "binary_s390x", "binary_amd64_compat", + "binary_amd64_musl", ], }, test_configs={ From 51fecff769b29413c801b9fbb5dafbedc0bf865c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 02:26:59 +0100 Subject: [PATCH 251/628] Fix style --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 9ffb58a696a..05654926fd7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -206,7 +206,7 @@ jobs: build_name: binary_amd64_compat checkout_depth: 0 BuilderBinAmd64Musl: - needs: [ DockerHubPush ] + needs: [DockerHubPush] uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_musl diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1620b00c33f..c089e9c6d94 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -243,7 +243,7 @@ jobs: with: build_name: binary_amd64_compat BuilderBinAmd64Musl: - needs: [ FastTest, StyleCheck ] + needs: [FastTest, StyleCheck] uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_musl From 8ec4a90cb443d27a333e78938733f9218913890f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 17:55:13 +0100 Subject: [PATCH 252/628] Fix error --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index ff77892b8b3..ade36a55591 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -238,7 +238,7 @@ def parse_env_variables( cc = compiler[: -len(AMD64_MUSL_SUFFIX)] result.append("DEB_ARCH=amd64") cmake_flags.append( - "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64-musl.cmake" + "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake" ) else: cc = compiler From 4a02de4674e289e3a800f991802bcdefb1967fe1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 1 Oct 2023 21:35:42 +0200 Subject: [PATCH 253/628] Add ability to disable checksums for S3 to avoid excessive input file read AWS S3 client can read file multiple times, this is required for: - calculate checksums - calculate signature (done only for HTTP, since ClickHouse uses PayloadSigningPolicy::Never) So this means that for HTTP, to send file to S3 it will be read 3x times, and for HTTPS 2x times. By overriding GetChecksumAlgorithmName() to return empty string, checksums can be disabled, and the input file will be read only once. And even though additional https layer adds extra integrity layer, someone still may find this too risky I guess, even though ClickHouse internal format (for MergeTree) has checksums, and more. Here is an example stacktrace of this excessive read:
stacktrace (lldb) bt * thread 383, name = 'BackupWorker', stop reason = breakpoint 1.1 * frame 0: 0x00000000103c5fc0 clickhouse`DB::StdStreamBufFromReadBuffer::seekpos() + 32 at StdStreamBufFromReadBuffer.cpp:67 frame 1: 0x000000001777f7f8 clickhouse`std::__1::basic_istream>::tellg() [inlined] std::__1::basic_streambuf>::pubseekoff[abi:v15000](this=, __off=0, __way=cur, __which=8) + 120 at streambuf:162 frame 2: 0x000000001777f7e3 clickhouse`std::__1::basic_istream>::tellg() + 99 at istream:1249 frame 3: 0x00000000152e4979 clickhouse`Aws::Utils::Crypto::MD5OpenSSLImpl::Calculate() + 57 at CryptoImpl.cpp:223 frame 4: 0x00000000152dedee clickhouse`Aws::Utils::Crypto::MD5::Calculate() + 14 at MD5.cpp:30 frame 5: 0x00000000152db5ac clickhouse`Aws::Utils::HashingUtils::CalculateMD5() + 44 at HashingUtils.cpp:235 frame 6: 0x000000001528b97b clickhouse`Aws::Client::AWSClient::AddChecksumToRequest() const + 507 at AWSClient.cpp:772 frame 7: 0x000000001528ded2 clickhouse`Aws::Client::AWSClient::BuildHttpRequest() const + 1682 at AWSClient.cpp:930 frame 8: 0x00000000100b864f clickhouse`DB::S3::Client::BuildHttpRequest() const + 15 at Client.cpp:622 frame 9: 0x0000000015286a41 clickhouse`Aws::Client::AWSClient::AttemptOneRequest(this=0x00007ffde2f8f000, httpRequest=, request=, signerName=, signerRegionOverride=, signerServiceNameOverride="s3") const + 65 at AWSClient.cpp:491 frame 10: 0x00000000152845b9 clickhouse`Aws::Client::AWSClient::AttemptExhaustively(this=0x00007ffde2f8f000, uri=0x00007ffdd4d44f38, request=0x00007ffdd4d45d10, method=HTTP_PUT, signerName="SignatureV4", signerRegionOverride="us-east-1", signerServiceNameOverride="s3") const + 1337 at AWSClient.cpp:272 frame 11: 0x0000000015298d0d clickhouse`Aws::Client::AWSXMLClient::MakeRequest() const + 45 at AWSXmlClient.cpp:99 frame 12: 0x0000000015298cb5 clickhouse`Aws::Client::AWSXMLClient::MakeRequest() const + 309 at AWSXmlClient.cpp:66 frame 13: 0x0000000015354b23 clickhouse`Aws::S3::S3Client::PutObject(this=0x00007ffde2f8f000, request=0x00007ffdd4d45d10) const + 2659 at S3Client.cpp:1731 frame 14: 0x00000000100b174f clickhouse`DB::S3::Client::PutObject(DB::S3::ExtendedRequest const&) const [inlined] frame 15: 0x00000000100b173a clickhouse`DB::S3::Client::PutObject(DB::S3::ExtendedRequest const&) const + 41 at Client.cpp:578 frame 16: 0x00000000100b1711 clickhouse`DB::S3::Client::PutObject(DB::S3::ExtendedRequest const&) const + 981 at Client.cpp:508 frame 17: 0x00000000100b133c clickhouse`DB::S3::Client::PutObject(DB::S3::ExtendedRequest const&) const [inlined] frame 18: 0x00000000100b133c clickhouse`DB::S3::Client::PutObject() const + 28 at Client.cpp:418 frame 19: 0x00000000103b96d6 clickhouse`DB::copyDataToS3File()
This new behaviour could be enabled with `s3_disable_checksum=true`. Note, that I've checked this implementation with GCS/R2/S3/MinIO and it works everywhere. --- src/Backups/BackupIO_S3.cpp | 11 ++++--- src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Core/Settings.h | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 10 +++++-- src/IO/S3/Client.cpp | 16 ++++++++--- src/IO/S3/Client.h | 8 ++++-- src/IO/S3/Requests.h | 18 ++++++++++++ src/IO/S3/tests/gtest_aws_s3_client.cpp | 30 ++++++++++++++++++++ src/IO/tests/gtest_writebuffer_s3.cpp | 3 +- src/Storages/StorageS3.cpp | 12 +++++--- 10 files changed, 92 insertions(+), 18 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index fecb97dbbaf..74195a93072 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -48,20 +48,22 @@ namespace } const auto & request_settings = settings.request_settings; + const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); + const Settings & local_settings = context->getSettingsRef(); S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, context->getRemoteHostFilter(), - static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), - static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), - context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + static_cast(global_settings.s3_max_redirects), + static_cast(global_settings.s3_retry_attempts), + global_settings.enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, s3_uri.uri.getScheme()); client_configuration.endpointOverride = s3_uri.endpoint; - client_configuration.maxConnections = static_cast(context->getSettingsRef().s3_max_connections); + client_configuration.maxConnections = static_cast(global_settings.s3_max_connections); /// Increase connect timeout client_configuration.connectTimeoutMs = 10 * 1000; /// Requests in backups can be extremely long, set to one hour @@ -71,6 +73,7 @@ namespace return S3::ClientFactory::instance().create( client_configuration, s3_uri.is_virtual_hosted_style, + local_settings.s3_disable_checksum, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), settings.auth_settings.server_side_encryption_customer_key_base64, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index ee62f6471e7..56f64d58e2f 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -101,6 +101,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auto client = S3::ClientFactory::instance().create( client_configuration, new_uri.is_virtual_hosted_style, + /* disable_checksum= */ false, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), auth_settings.server_side_encryption_customer_key_base64, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b734fa4c390..d48fce8e636 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,6 +104,7 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, s3_disable_checksum, false, "Disable checksums for S3 to avoid excessive reads", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 0232a6eb070..985e4249c32 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -43,6 +43,9 @@ std::unique_ptr getClient( ContextPtr context, const S3ObjectStorageSettings & settings) { + const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); + const Settings & local_settings = context->getSettingsRef(); + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); if (!uri.key.ends_with('/')) @@ -51,9 +54,9 @@ std::unique_ptr getClient( S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), - static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), - static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), - context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + static_cast(global_settings.s3_max_redirects), + static_cast(global_settings.s3_retry_attempts), + global_settings.enable_s3_requests_logging, /* for_disk_s3 = */ true, settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler, @@ -96,6 +99,7 @@ std::unique_ptr getClient( return S3::ClientFactory::instance().create( client_configuration, uri.is_virtual_hosted_style, + local_settings.s3_disable_checksum, config.getString(config_prefix + ".access_key_id", ""), config.getString(config_prefix + ".secret_access_key", ""), config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 28a55580dc5..7658ea5941c 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -125,11 +125,12 @@ std::unique_ptr Client::create( const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing) + bool use_virtual_addressing, + bool disable_checksum) { verifyClientConfiguration(client_configuration); return std::unique_ptr( - new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing)); + new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, use_virtual_addressing, disable_checksum)); } std::unique_ptr Client::clone() const @@ -159,12 +160,14 @@ Client::Client( const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration_, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads_, - bool use_virtual_addressing_) + bool use_virtual_addressing_, + bool disable_checksum_) : Aws::S3::S3Client(credentials_provider_, client_configuration_, sign_payloads_, use_virtual_addressing_) , credentials_provider(credentials_provider_) , client_configuration(client_configuration_) , sign_payloads(sign_payloads_) , use_virtual_addressing(use_virtual_addressing_) + , disable_checksum(disable_checksum_) , max_redirects(max_redirects_) , sse_kms_config(std::move(sse_kms_config_)) , log(&Poco::Logger::get("S3Client")) @@ -210,6 +213,7 @@ Client::Client( , client_configuration(client_configuration_) , sign_payloads(other.sign_payloads) , use_virtual_addressing(other.use_virtual_addressing) + , disable_checksum(other.disable_checksum) , explicit_region(other.explicit_region) , detect_region(other.detect_region) , provider_type(other.provider_type) @@ -511,6 +515,8 @@ Client::doRequest(RequestType & request, RequestFn request_fn) const addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); const auto & bucket = request.GetBucket(); request.setApiMode(api_mode); + if (disable_checksum) + request.disableChecksum(); if (auto region = getRegionForBucket(bucket); !region.empty()) { @@ -844,6 +850,7 @@ ClientFactory & ClientFactory::instance() std::unique_ptr ClientFactory::create( // NOLINT const PocoHTTPClientConfiguration & cfg_, bool is_virtual_hosted_style, + bool disable_checksum, const String & access_key_id, const String & secret_access_key, const String & server_side_encryption_customer_key_base64, @@ -888,7 +895,8 @@ std::unique_ptr ClientFactory::create( // NOLINT credentials_provider, client_configuration, // Client configuration. Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - is_virtual_hosted_style || client_configuration.endpointOverride.empty() /// Use virtual addressing if endpoint is not specified. + is_virtual_hosted_style || client_configuration.endpointOverride.empty(), /// Use virtual addressing if endpoint is not specified. + disable_checksum ); } diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index cf0d53d9c72..d0a21a2dafe 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -116,7 +116,8 @@ public: const std::shared_ptr & credentials_provider, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing); + bool use_virtual_addressing, + bool disable_checksum); std::unique_ptr clone() const; @@ -211,7 +212,8 @@ private: const std::shared_ptr & credentials_provider_, const PocoHTTPClientConfiguration & client_configuration, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads, - bool use_virtual_addressing); + bool use_virtual_addressing, + bool disable_checksum_); Client( const Client & other, const PocoHTTPClientConfiguration & client_configuration); @@ -257,6 +259,7 @@ private: PocoHTTPClientConfiguration client_configuration; Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy sign_payloads; bool use_virtual_addressing; + bool disable_checksum; std::string explicit_region; mutable bool detect_region = true; @@ -287,6 +290,7 @@ public: std::unique_ptr create( const PocoHTTPClientConfiguration & cfg, bool is_virtual_hosted_style, + bool disable_checksum, const String & access_key_id, const String & secret_access_key, const String & server_side_encryption_customer_key_base64, diff --git a/src/IO/S3/Requests.h b/src/IO/S3/Requests.h index 560ba9b2775..eae45491fe6 100644 --- a/src/IO/S3/Requests.h +++ b/src/IO/S3/Requests.h @@ -47,6 +47,17 @@ public: return params; } + Aws::String GetChecksumAlgorithmName() const override + { + /// Return empty string is enough to disable checksums (see + /// AWSClient::AddChecksumToRequest [1] for more details). + /// + /// [1]: https://github.com/aws/aws-sdk-cpp/blob/b0ee1c0d336dbb371c34358b68fba6c56aae2c92/src/aws-cpp-sdk-core/source/client/AWSClient.cpp#L783-L839 + if (!checksum) + return ""; + return BaseRequest::GetChecksumAlgorithmName(); + } + void overrideRegion(std::string region) const { region_override = std::move(region); @@ -67,10 +78,17 @@ public: api_mode = api_mode_; } + /// Disable checksum to avoid extra read of the input stream + void disableChecksum() const + { + checksum = false; + } + protected: mutable std::string region_override; mutable std::optional uri_override; mutable ApiMode api_mode{ApiMode::AWS}; + mutable bool checksum = true; }; class CopyObjectRequest : public ExtendedRequest diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 2afa5d4b8e8..0b44698ac2c 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -107,6 +107,7 @@ using RequestFn = std::function, cons void testServerSideEncryption( RequestFn do_request, + bool disable_checksum, String server_side_encryption_customer_key_base64, DB::S3::ServerSideEncryptionKMSConfig sse_kms_config, String expected_headers) @@ -142,6 +143,7 @@ void testServerSideEncryption( std::shared_ptr client = DB::S3::ClientFactory::instance().create( client_configuration, uri.is_virtual_hosted_style, + disable_checksum, access_key_id, secret_access_key, server_side_encryption_customer_key_base64, @@ -166,6 +168,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersRead) /// See https://github.com/ClickHouse/ClickHouse/pull/19748 testServerSideEncryption( doReadRequest, + /* disable_checksum= */ false, "Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=", {}, "authorization: ... SignedHeaders=" @@ -190,6 +193,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersWrite) /// See https://github.com/ClickHouse/ClickHouse/pull/19748 testServerSideEncryption( doWriteRequest, + /* disable_checksum= */ false, "Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=", {}, "authorization: ... SignedHeaders=" @@ -209,6 +213,30 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeadersWrite) "x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n"); } +TEST(IOTestAwsS3Client, AppendExtraSSECHeadersWriteDisableChecksum) +{ + /// See https://github.com/ClickHouse/ClickHouse/pull/19748 + testServerSideEncryption( + doWriteRequest, + /* disable_checksum= */ true, + "Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=", + {}, + "authorization: ... SignedHeaders=" + "amz-sdk-invocation-id;" + "amz-sdk-request;" + "content-length;" + "content-type;" + "host;" + "x-amz-content-sha256;" + "x-amz-date;" + "x-amz-server-side-encryption-customer-algorithm;" + "x-amz-server-side-encryption-customer-key;" + "x-amz-server-side-encryption-customer-key-md5, ...\n" + "x-amz-server-side-encryption-customer-algorithm: AES256\n" + "x-amz-server-side-encryption-customer-key: Kv/gDqdWVGIT4iDqg+btQvV3lc1idlm4WI+MMOyHOAw=\n" + "x-amz-server-side-encryption-customer-key-md5: fMNuOw6OLU5GG2vc6RTA+g==\n"); +} + TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead) { DB::S3::ServerSideEncryptionKMSConfig sse_kms_config; @@ -218,6 +246,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersRead) // KMS headers shouldn't be set on a read request testServerSideEncryption( doReadRequest, + /* disable_checksum= */ false, "", sse_kms_config, "authorization: ... SignedHeaders=" @@ -239,6 +268,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersWrite) sse_kms_config.bucket_key_enabled = true; testServerSideEncryption( doWriteRequest, + /* disable_checksum= */ false, "", sse_kms_config, "authorization: ... SignedHeaders=" diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 41ba6a2779d..5880b40c408 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -210,7 +210,8 @@ struct Client : DB::S3::Client std::make_shared("", ""), GetClientConfiguration(), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - /* use_virtual_addressing = */ true) + /* use_virtual_addressing = */ true, + /* disable_checksum_= */ false) , store(mock_s3_store) { } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 194f9cb647f..a28143d4f25 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1437,12 +1437,15 @@ bool StorageS3::Configuration::update(ContextPtr context) void StorageS3::Configuration::connect(ContextPtr context) { + const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); + const Settings & local_settings = context->getSettingsRef(); + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( auth_settings.region, context->getRemoteHostFilter(), - static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), - static_cast(context->getGlobalContext()->getSettingsRef().s3_retry_attempts), - context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + static_cast(global_settings.s3_max_redirects), + static_cast(global_settings.s3_retry_attempts), + global_settings.enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, @@ -1450,7 +1453,7 @@ void StorageS3::Configuration::connect(ContextPtr context) client_configuration.endpointOverride = url.endpoint; client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.http_connection_pool_size = context->getGlobalContext()->getSettingsRef().s3_http_connection_pool_size; + client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; auto headers = auth_settings.headers; if (!headers_from_ast.empty()) headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); @@ -1461,6 +1464,7 @@ void StorageS3::Configuration::connect(ContextPtr context) client = S3::ClientFactory::instance().create( client_configuration, url.is_virtual_hosted_style, + local_settings.s3_disable_checksum, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), auth_settings.server_side_encryption_customer_key_base64, From 1168b7910b63bea201fd5acd10af88f2af30cd56 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 12 Oct 2023 16:11:57 +0200 Subject: [PATCH 254/628] Update test_throttling with enabled s3_disable_checksum Signed-off-by: Azat Khuzhin --- .../configs/users_overrides_persistent.xml | 7 +++++++ tests/integration/test_throttling/test.py | 9 ++++++--- 2 files changed, 13 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_throttling/configs/users_overrides_persistent.xml diff --git a/tests/integration/test_throttling/configs/users_overrides_persistent.xml b/tests/integration/test_throttling/configs/users_overrides_persistent.xml new file mode 100644 index 00000000000..b25a656ddb1 --- /dev/null +++ b/tests/integration/test_throttling/configs/users_overrides_persistent.xml @@ -0,0 +1,7 @@ + + + + true + + + diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index ba2e59170ce..04d02cc859d 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -38,7 +38,10 @@ node = cluster.add_instance( "configs/server_overrides.xml", "configs/ssl.xml", ], - user_configs=["configs/users_overrides.xml"], + user_configs=[ + "configs/users_overrides.xml", + "configs/users_overrides_persistent.xml", + ], with_minio=True, minio_certs_dir="minio_certs", ) @@ -256,7 +259,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - 7 * 2, + 7, id="user_local_to_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -266,7 +269,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - 3 * 2, + 3, id="server_local_to_remote_throttling", ), ], From 9d6a9d357356cee118eaa49f0f5408f64ccad2d5 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 26 Nov 2023 19:31:39 +0000 Subject: [PATCH 255/628] do not rethrow load errors during db shutdown --- src/Common/AsyncLoader.cpp | 12 ++----- src/Common/AsyncLoader.h | 36 +++++++++---------- src/Databases/DatabaseAtomic.cpp | 4 +-- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/DatabaseOrdinary.cpp | 4 +-- src/Databases/DatabaseOrdinary.h | 2 +- src/Databases/DatabaseReplicated.cpp | 6 ++-- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/IDatabase.h | 3 +- .../MySQL/DatabaseMaterializedMySQL.cpp | 6 ++-- .../MySQL/DatabaseMaterializedMySQL.h | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 6 ++-- .../DatabaseMaterializedPostgreSQL.h | 2 +- 14 files changed, 41 insertions(+), 48 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 643db8c5cc0..962adb8b052 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -369,20 +369,14 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool) prioritize(job, new_pool, lock); } -void AsyncLoader::wait(const LoadJobPtr & job) +void AsyncLoader::wait(const LoadJobPtr & job, bool no_throw) { std::unique_lock job_lock{job->mutex}; wait(job_lock, job); - if (job->load_exception) + if (!no_throw && job->load_exception) std::rethrow_exception(job->load_exception); } -void AsyncLoader::waitNoThrow(const LoadJobPtr & job) -{ - std::unique_lock job_lock{job->mutex}; - wait(job_lock, job); -} - void AsyncLoader::remove(const LoadJobSet & jobs) { DENY_ALLOCATIONS_IN_SCOPE; @@ -413,7 +407,7 @@ void AsyncLoader::remove(const LoadJobSet & jobs) ALLOW_ALLOCATIONS_IN_SCOPE; chassert(info->second.isExecuting()); lock.unlock(); - waitNoThrow(job); // Wait for job to finish + wait(job, /* no_throw = */ true); // Wait for job to finish lock.lock(); } } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 5a619584785..7b6e3ebfefe 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -79,7 +79,7 @@ public: // Value may change during job execution by `prioritize()`. size_t pool() const; - // Returns number of threads blocked by `wait()` or `waitNoThrow()` calls. + // Returns number of threads blocked by `wait()` calls. size_t waitersCount() const; // Introspection @@ -335,12 +335,9 @@ public: void prioritize(const LoadJobPtr & job, size_t new_pool); // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. - // Throws if job is FAILED or CANCELED. Returns or throws immediately if called on non-pending job. + // Throws if job is FAILED or CANCELED unless `no_throw` is set. Returns or throws immediately if called on non-pending job. // If job was not scheduled, it will be implicitly scheduled before the wait (deadlock auto-resolution). - void wait(const LoadJobPtr & job); - - // Wait for a job to reach any non PENDING status. The same as `wait()` but does not throw. - void waitNoThrow(const LoadJobPtr & job); + void wait(const LoadJobPtr & job, bool no_throw = false); // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them. void remove(const LoadJobSet & jobs); @@ -418,11 +415,12 @@ private: // Note that prioritization may be done // (1) before scheduling (to ensure all jobs are started in the correct pools) // (2) after scheduling (for dynamic prioritization, e.g. when new query arrives) -// waitLoad([loader], pool_id, {jobs|task|tasks}): +// waitLoad([loader], pool_id, {jobs|task|tasks}, [no_throw]): // Prioritize and wait for jobs. // Note that to avoid deadlocks it implicitly schedules all the jobs before waiting for them. // Also to avoid priority inversion you should never wait for a job that has lower priority. // So it prioritizes all jobs, then schedules all jobs and waits every job. +// IMPORTANT: Any load error will be rethrown, unless `no_throw` is set. // Common usage pattern is: // waitLoad(currentPoolOr(foreground_pool_id), tasks); @@ -449,24 +447,24 @@ inline void scheduleLoad(const LoadTaskPtrs & tasks) loader.schedule(tasks); } -inline void waitLoad(AsyncLoader & loader, const LoadJobSet & jobs) +inline void waitLoad(AsyncLoader & loader, const LoadJobSet & jobs, bool no_throw = false) { scheduleLoad(loader, jobs); for (const auto & job : jobs) - loader.wait(job); + loader.wait(job, no_throw); } -inline void waitLoad(const LoadTaskPtr & task) +inline void waitLoad(const LoadTaskPtr & task, bool no_throw = false) { scheduleLoad(task); - waitLoad(task->loader, task->goals()); + waitLoad(task->loader, task->goals(), no_throw); } -inline void waitLoad(const LoadTaskPtrs & tasks) +inline void waitLoad(const LoadTaskPtrs & tasks, bool no_throw = false) { scheduleLoad(tasks); for (const auto & task : tasks) - waitLoad(task->loader, task->goals()); + waitLoad(task->loader, task->goals(), no_throw); } inline void prioritizeLoad(AsyncLoader & loader, size_t pool_id, const LoadJobSet & jobs) @@ -486,22 +484,22 @@ inline void prioritizeLoad(size_t pool_id, const LoadTaskPtrs & tasks) prioritizeLoad(task->loader, pool_id, task->goals()); } -inline void waitLoad(AsyncLoader & loader, size_t pool_id, const LoadJobSet & jobs) +inline void waitLoad(AsyncLoader & loader, size_t pool_id, const LoadJobSet & jobs, bool no_throw = false) { prioritizeLoad(loader, pool_id, jobs); - waitLoad(loader, jobs); + waitLoad(loader, jobs, no_throw); } -inline void waitLoad(size_t pool_id, const LoadTaskPtr & task) +inline void waitLoad(size_t pool_id, const LoadTaskPtr & task, bool no_throw = false) { prioritizeLoad(task->loader, pool_id, task->goals()); - waitLoad(task->loader, task->goals()); + waitLoad(task->loader, task->goals(), no_throw); } -inline void waitLoad(size_t pool_id, const LoadTaskPtrs & tasks) +inline void waitLoad(size_t pool_id, const LoadTaskPtrs & tasks, bool no_throw = false) { prioritizeLoad(pool_id, tasks); - waitLoad(tasks); + waitLoad(tasks, no_throw); } inline LoadJobSet getGoals(const LoadTaskPtrs & tasks, const LoadJobSet & alternative = {}) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 6d10f827916..8f045ee2255 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -470,10 +470,10 @@ LoadTaskPtr DatabaseAtomic::startupDatabaseAsync(AsyncLoader & async_loader, Loa return startup_atomic_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseAtomic::waitDatabaseStarted() const +void DatabaseAtomic::waitDatabaseStarted(bool no_throw) const { if (startup_atomic_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_atomic_database_task); + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_atomic_database_task, no_throw); } void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path, bool if_data_path_exist) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 18274bd50a6..b7bb1b9d7ce 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -51,7 +51,7 @@ public: void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted() const override; + void waitDatabaseStarted(bool no_throw = false) const override; /// Atomic database cannot be detached if there is detached table which still in use void assertCanBeDetached(bool cleanup) override; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index e342da5ef75..a8fb87fd647 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -165,7 +165,7 @@ DatabaseOnDisk::DatabaseOnDisk( void DatabaseOnDisk::shutdown() { - waitDatabaseStarted(); + waitDatabaseStarted(/* no_throw = */ true); DatabaseWithOwnTablesBase::shutdown(); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 02af5d27bc3..e25a04a9e21 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -245,11 +245,11 @@ void DatabaseOrdinary::waitTableStarted(const String & name) const waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), task); } -void DatabaseOrdinary::waitDatabaseStarted() const +void DatabaseOrdinary::waitDatabaseStarted(bool no_throw) const { /// Prioritize load and startup of all tables and database itself and wait for them synchronously if (startup_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_database_task); + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_database_task, no_throw); } DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 071e4279b56..74e027da555 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -51,7 +51,7 @@ public: void waitTableStarted(const String & name) const override; - void waitDatabaseStarted() const override; + void waitDatabaseStarted(bool no_throw = false) const override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c2d7d3dec4e..2509776603c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -578,10 +578,10 @@ LoadTaskPtr DatabaseReplicated::startupDatabaseAsync(AsyncLoader & async_loader, return startup_replicated_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseReplicated::waitDatabaseStarted() const +void DatabaseReplicated::waitDatabaseStarted(bool no_throw) const { if (startup_replicated_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_replicated_database_task); + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_replicated_database_task, no_throw); } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const @@ -1257,7 +1257,7 @@ void DatabaseReplicated::drop(ContextPtr context_) void DatabaseReplicated::stopReplication() { - waitDatabaseStarted(); + waitDatabaseStarted(/* no_throw = */ true); if (ddl_worker) ddl_worker->shutdown(); } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 6e49c19bd2a..65db50e36c5 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -122,7 +122,7 @@ private: UInt64 getMetadataHash(const String & table_name) const; bool checkDigestValid(const ContextPtr & local_context, bool debug_check = true) const TSA_REQUIRES(metadata_mutex); - void waitDatabaseStarted() const override; + void waitDatabaseStarted(bool no_throw = false) const override; String zookeeper_path; String shard_name; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 598e7fd7eeb..057a1c1f02d 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -219,7 +219,8 @@ public: virtual void waitTableStarted(const String & /*name*/) const {} /// Waits for the database to be started up, i.e. task returned by `startupDatabaseAsync()` is done - virtual void waitDatabaseStarted() const {} + /// NOTE: `no_throw` wait should be used during shutdown to (1) prevent race with startup and (2) avoid exceptions if startup failed + virtual void waitDatabaseStarted(bool /*no_throw*/ = false) const {} /// Check the existence of the table in memory (attached). virtual bool isTableExist(const String & name, ContextPtr context) const = 0; diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index b984063eba2..a31e74cc7ae 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -83,10 +83,10 @@ LoadTaskPtr DatabaseMaterializedMySQL::startupDatabaseAsync(AsyncLoader & async_ return startup_mysql_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseMaterializedMySQL::waitDatabaseStarted() const +void DatabaseMaterializedMySQL::waitDatabaseStarted(bool no_throw) const { if (startup_mysql_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_mysql_database_task); + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_mysql_database_task, no_throw); } void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) @@ -174,7 +174,7 @@ void DatabaseMaterializedMySQL::checkIsInternalQuery(ContextPtr context_, const void DatabaseMaterializedMySQL::stopReplication() { - waitDatabaseStarted(); + waitDatabaseStarted(/* no_throw = */ true); materialize_thread.stopSynchronization(); started_up = false; } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 3a33ebc3030..e8a9460fe9c 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -52,7 +52,7 @@ public: String getEngineName() const override { return "MaterializedMySQL"; } LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted() const override; + void waitDatabaseStarted(bool no_throw = false) const override; void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index af4a022ca3f..78be0611631 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -153,10 +153,10 @@ LoadTaskPtr DatabaseMaterializedPostgreSQL::startupDatabaseAsync(AsyncLoader & a return startup_postgresql_database_task = makeLoadTask(async_loader, {job}); } -void DatabaseMaterializedPostgreSQL::waitDatabaseStarted() const +void DatabaseMaterializedPostgreSQL::waitDatabaseStarted(bool no_throw) const { if (startup_postgresql_database_task) - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_postgresql_database_task); + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), startup_postgresql_database_task, no_throw); } void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) @@ -436,7 +436,7 @@ void DatabaseMaterializedPostgreSQL::shutdown() void DatabaseMaterializedPostgreSQL::stopReplication() { - waitDatabaseStarted(); + waitDatabaseStarted(/* no_throw = */ true); std::lock_guard lock(handler_mutex); if (replication_handler) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 1c737093e67..823989a48e7 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -41,7 +41,7 @@ public: String getMetadataPath() const override { return metadata_path; } LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - void waitDatabaseStarted() const override; + void waitDatabaseStarted(bool no_throw = false) const override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; From 62831d235f4f238188ddd2401b675ee6f2e85a62 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 26 Nov 2023 23:28:11 +0000 Subject: [PATCH 256/628] support multiple lines on dashboard charts --- programs/server/dashboard.html | 81 ++++++++++++++++++++-------------- 1 file changed, 48 insertions(+), 33 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 123a15c5706..b43ed357bac 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -478,7 +478,6 @@ * - compress the state for URL's #hash; * - footer with "about" or a link to source code; * - allow to configure a table on a server to save the dashboards; - * - multiple lines on chart; * - if a query returned one value, display this value instead of a diagram; * - if a query returned something unusual, display the table; */ @@ -966,7 +965,7 @@ async function doFetch(query, url_params = '') { user = document.getElementById('user').value; password = document.getElementById('password').value; - let url = `${host}?default_format=JSONCompactColumns&enable_http_compression=1` + let url = `${host}?default_format=JSONColumnsWithMetadata&enable_http_compression=1` if (add_http_cors_header) { // For debug purposes, you may set add_http_cors_header from a browser console @@ -980,14 +979,14 @@ async function doFetch(query, url_params = '') { url += `&password=${encodeURIComponent(password)}`; } - let response, data, error; + let response, reply, error; try { response = await fetch(url + url_params, { method: "POST", body: query }); - data = await response.text(); + reply = await response.text(); if (response.ok) { - data = JSON.parse(data); + reply = JSON.parse(reply); } else { - error = data; + error = reply; } } catch (e) { console.log(e); @@ -1006,7 +1005,7 @@ async function doFetch(query, url_params = '') { } } - return {data, error}; + return {reply, error}; } async function draw(idx, chart, url_params, query) { @@ -1015,17 +1014,21 @@ async function draw(idx, chart, url_params, query) { plots[idx] = null; } - let {data, error} = await doFetch(query, url_params); - + let {reply, error} = await doFetch(query, url_params); if (!error) { - if (!Array.isArray(data)) { - error = "Query should return an array."; - } else if (data.length == 0) { + if (reply.rows.length == 0) { error = "Query returned empty result."; - } else if (data.length != 2) { - error = "Query should return exactly two columns: unix timestamp and value."; - } else if (!Array.isArray(data[0]) || !Array.isArray(data[1]) || data[0].length != data[1].length) { - error = "Wrong data format of the query."; + } else if (reply.meta.length < 2) { + error = "Query should return at least two columns: unix timestamp and value."; + } else { + for (let i = 0; i < reply.meta.length; i++) { + let label = reply.meta[i].name; + let column = reply.data[label]; + if (!Array.isArray(column) || column.length != reply.data[reply.meta[0].name].length) { + error = "Wrong data format of the query."; + break; + } + } } } @@ -1048,19 +1051,32 @@ async function draw(idx, chart, url_params, query) { let sync = uPlot.sync("sync"); - const max_value = Math.max(...data[1]); + let axes = []; + let series = []; + let data = []; + series.push({ label: "x" }); + data.push(reply.data[reply.meta[0].name]); + let max_value = Number.NEGATIVE_INFINITY; + for (let i = 1; i < reply.meta.length; i++) { + let label = reply.meta[i].name; + axes.push({ + stroke: axes_color, + grid: { width: 1 / devicePixelRatio, stroke: grid_color }, + ticks: { width: 1 / devicePixelRatio, stroke: grid_color } + }); + series.push({ label, stroke: line_color, fill: fill_color }); + data.push(reply.data[label]); + max_value = Math.max(max_value, ...reply.data[label]); + } + if (reply.meta.length == 2) { + series[1].label = "y"; + } const opts = { width: chart.clientWidth, height: chart.clientHeight, - axes: [ { stroke: axes_color, - grid: { width: 1 / devicePixelRatio, stroke: grid_color }, - ticks: { width: 1 / devicePixelRatio, stroke: grid_color } }, - { stroke: axes_color, - grid: { width: 1 / devicePixelRatio, stroke: grid_color }, - ticks: { width: 1 / devicePixelRatio, stroke: grid_color } } ], - series: [ { label: "x" }, - { label: "y", stroke: line_color, fill: fill_color } ], + axes, + series, padding: [ null, null, null, (Math.round(max_value * 100) / 100).toString().length * 6 - 10 ], plugins: [ legendAsTooltipPlugin() ], cursor: { @@ -1216,22 +1232,21 @@ function saveState() { } async function searchQueries() { - let {data, error} = await doFetch(search_query); + let {reply, error} = await doFetch(search_query); if (error) { throw new Error(error); } - if (!Array.isArray(data)) { - throw new Error("Search query should return an array."); - } else if (data.length == 0) { + let data = reply.data; + if (reply.rows == 0) { throw new Error("Search query returned empty result."); - } else if (data.length != 2) { + } else if (reply.meta.length != 2 || reply.meta[0].name != "title" || reply.meta[1].name != "query") { throw new Error("Search query should return exactly two columns: title and query."); - } else if (!Array.isArray(data[0]) || !Array.isArray(data[1]) || data[0].length != data[1].length) { + } else if (!Array.isArray(data.title) || !Array.isArray(data.query) || data.title.length != data.query.length) { throw new Error("Wrong data format of the search query."); } - for (let i = 0; i < data[0].length; i++) { - queries.push({title: data[0][i], query: data[1][i]}); + for (let i = 0; i < data.title.length; i++) { + queries.push({title: data.title[i], query: data.query[i]}); } regenerate(); From 346dec71cd291bf3a4aa8e3848721c847ea0a7d2 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sun, 26 Nov 2023 23:03:44 -0800 Subject: [PATCH 257/628] remove hostname from getDefaultOrderBy() --- src/Interpreters/AsynchronousMetricLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 93bbdcdff2c..9a5266cee6e 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -50,7 +50,7 @@ public: void addValues(const AsynchronousMetricValues &); /// This table is usually queried for fixed metric name. - static const char * getDefaultOrderBy() { return "hostname, metric, event_date, event_time"; } + static const char * getDefaultOrderBy() { return "metric, event_date, event_time"; } }; } From cc57e67e83bd0a6135906ec51bbecf76241cfb2e Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Sun, 26 Nov 2023 23:57:33 -0800 Subject: [PATCH 258/628] Fix docs check erorr --- docs/en/operations/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/monitoring.md b/docs/en/operations/monitoring.md index 8cfa665cca0..adc384e21ae 100644 --- a/docs/en/operations/monitoring.md +++ b/docs/en/operations/monitoring.md @@ -17,7 +17,7 @@ You can monitor: ## Built-in observability dashboard -Screenshot 2023-11-12 at 6 08 58 PM +Screenshot 2023-11-12 at 6 08 58 PM ClickHouse comes with a built-in observability dashboard feature which can be accessed by `$HOST:$PORT/dashboard` (requires user and password) that shows the following metrics: - Queries/second From a3e342a477f2ff4268201b9109841ae4a2d695c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 22 Nov 2023 13:08:45 +0100 Subject: [PATCH 259/628] Separate first_value and first_value_respect_nulls states --- .../AggregateFunctionAny.cpp | 25 ++++++++----------- .../AggregateFunctionMinMaxAny.h | 20 ++++++++++++--- src/AggregateFunctions/HelpersMinMaxAny.h | 12 +++++---- src/Parsers/ExpressionListParsers.cpp | 19 ++++++++------ .../02922_respect_nulls_states.reference | 10 ++++++++ .../02922_respect_nulls_states.sql | 15 +++++++++++ 6 files changed, 70 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02922_respect_nulls_states.reference create mode 100644 tests/queries/0_stateless/02922_respect_nulls_states.sql diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index fc8f50efabe..5afdeff1621 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -50,22 +50,19 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory) factory.registerFunction("any", { createAggregateFunctionAny, properties }); factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive); + + factory.registerFunction("any_respect_nulls", {createAggregateFunctionNullableAny, properties}); + factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("anyLast", { createAggregateFunctionAnyLast, properties }); - factory.registerFunction("anyHeavy", { createAggregateFunctionAnyHeavy, properties }); + factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive); - // Synonyms for use as window functions. - factory.registerFunction("first_value", - { createAggregateFunctionAny, properties }, - AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("first_value_respect_nulls", - { createAggregateFunctionNullableAny, properties }, - AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("last_value", - { createAggregateFunctionAnyLast, properties }, - AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("last_value_respect_nulls", - { createAggregateFunctionNullableAnyLast, properties }, - AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionNullableAnyLast, properties}); + factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive); + + factory.registerFunction("anyHeavy", {createAggregateFunctionAnyHeavy, properties}); } } diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 2ca8d530457..21c5730c3d6 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -1084,7 +1084,7 @@ struct AggregateFunctionMaxData : Data #endif }; -template +template struct AggregateFunctionAnyData : Data { using Self = AggregateFunctionAnyData; @@ -1094,7 +1094,13 @@ struct AggregateFunctionAnyData : Data bool changeIfBetter(const Self & to, Arena * arena) { return this->changeFirstTime(to, arena); } void addManyDefaults(const IColumn & column, size_t /*length*/, Arena * arena) { this->changeFirstTime(column, 0, arena); } - static const char * name() { return "any"; } + static const char * name() + { + if constexpr (RespectNulls) + return "any_respect_nulls"; + else + return "any"; + } #if USE_EMBEDDED_COMPILER @@ -1113,7 +1119,7 @@ struct AggregateFunctionAnyData : Data #endif }; -template +template struct AggregateFunctionAnyLastData : Data { using Self = AggregateFunctionAnyLastData; @@ -1122,7 +1128,13 @@ struct AggregateFunctionAnyLastData : Data bool changeIfBetter(const Self & to, Arena * arena) { return this->changeEveryTime(to, arena); } void addManyDefaults(const IColumn & column, size_t /*length*/, Arena * arena) { this->changeEveryTime(column, 0, arena); } - static const char * name() { return "anyLast"; } + static const char * name() + { + if constexpr (RespectNulls) + return "anyLast_respect_nulls"; + else + return "anyLast"; + } #if USE_EMBEDDED_COMPILER diff --git a/src/AggregateFunctions/HelpersMinMaxAny.h b/src/AggregateFunctions/HelpersMinMaxAny.h index 31ae5fdd59a..c20d4ae498b 100644 --- a/src/AggregateFunctions/HelpersMinMaxAny.h +++ b/src/AggregateFunctions/HelpersMinMaxAny.h @@ -14,8 +14,9 @@ namespace DB struct Settings; /// min, max, any, anyLast, anyHeavy, etc... -template