From bb2716251b46ebacaf5b3767f5d12d1b2870a68c Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 31 Aug 2024 19:44:31 +0000 Subject: [PATCH 001/157] wip: WORKLOADs and RESOURCEs added --- src/CMakeLists.txt | 1 + src/Common/ErrorCodes.cpp | 2 + .../Workload/IWorkloadEntityStorage.h | 77 +++++ .../Workload/WorkloadEntityDiskStorage.cpp | 296 ++++++++++++++++++ .../Workload/WorkloadEntityDiskStorage.h | 48 +++ .../Workload/WorkloadEntityKeeperStorage.cpp | 0 .../Workload/WorkloadEntityKeeperStorage.h | 0 .../Workload/WorkloadEntityStorageBase.cpp | 195 ++++++++++++ .../Workload/WorkloadEntityStorageBase.h | 73 +++++ src/Parsers/ASTCreateResourceQuery.cpp | 47 +++ src/Parsers/ASTCreateResourceQuery.h | 32 ++ src/Parsers/ASTCreateWorkloadQuery.cpp | 67 ++++ src/Parsers/ASTCreateWorkloadQuery.h | 35 +++ src/Parsers/CommonParsers.h | 2 + src/Parsers/ParserCreateResourceQuery.cpp | 62 ++++ src/Parsers/ParserCreateResourceQuery.h | 16 + src/Parsers/ParserCreateWorkloadQuery.cpp | 76 +++++ src/Parsers/ParserCreateWorkloadQuery.h | 16 + 18 files changed, 1045 insertions(+) create mode 100644 src/Common/Scheduler/Workload/IWorkloadEntityStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp create mode 100644 src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h create mode 100644 src/Parsers/ASTCreateResourceQuery.cpp create mode 100644 src/Parsers/ASTCreateResourceQuery.h create mode 100644 src/Parsers/ASTCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ASTCreateWorkloadQuery.h create mode 100644 src/Parsers/ParserCreateResourceQuery.cpp create mode 100644 src/Parsers/ParserCreateResourceQuery.h create mode 100644 src/Parsers/ParserCreateWorkloadQuery.cpp create mode 100644 src/Parsers/ParserCreateWorkloadQuery.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1889bba3b39..5b20bf6c27d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -114,6 +114,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) +add_headers_and_sources(dbms Common/Scheduler/Workload) if (TARGET ch_contrib::amqp_cpp) add_headers_and_sources(dbms Storages/RabbitMQ) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1055b3d34db..492854cba53 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -609,6 +609,8 @@ M(728, UNEXPECTED_DATA_TYPE) \ M(729, ILLEGAL_TIME_SERIES_TAGS) \ M(730, REFRESH_FAILED) \ + M(731, WORKLOAD_ENTITY_ALREADY_EXISTS) \ + M(732, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h new file mode 100644 index 00000000000..65978a71be0 --- /dev/null +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -0,0 +1,77 @@ +#pragma once + +#include + +#include + +#include + + +namespace DB +{ + +class IAST; +struct Settings; + +enum class WorkloadEntityType : uint8_t +{ + Workload, + Resource, +}; + +/// Interface for a storage of workload entities (WORKLOAD and RESOURCE). +class IWorkloadEntityStorage +{ +public: + virtual ~IWorkloadEntityStorage() = default; + + /// Whether this storage can replicate entities to another node. + virtual bool isReplicated() const { return false; } + virtual String getReplicationID() const { return ""; } + + /// Loads all entities. Can be called once - if entities are already loaded the function does nothing. + virtual void loadEntities() = 0; + + /// Get entity by name. If no entity stored with entity_name throws exception. + virtual ASTPtr get(const String & entity_name) const = 0; + + /// Get entity by name. If no entity stored with entity_name return nullptr. + virtual ASTPtr tryGet(const String & entity_name) const = 0; + + /// Check if entity with entity_name is stored. + virtual bool has(const String & entity_name) const = 0; + + /// Get all entity names. + virtual std::vector getAllEntityNames() const = 0; + + /// Get all entities. + virtual std::vector> getAllEntities() const = 0; + + /// Check whether any entity have been stored. + virtual bool empty() const = 0; + + /// Stops watching. + virtual void stopWatching() {} + + /// Immediately reloads all entities, throws an exception if failed. + virtual void reloadEntities() = 0; + + /// Stores an entity. + virtual bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + /// Removes an entity. + virtual bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp new file mode 100644 index 00000000000..d9ca8bca0a0 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -0,0 +1,296 @@ +#include + +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include +#include + +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DIRECTORY_DOESNT_EXIST; + extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; + extern const int UNKNOWN_WORKLOAD_ENTITY; +} + + +namespace +{ + /// Converts a path to an absolute path and append it with a separator. + String makeDirectoryPathCanonical(const String & directory_path) + { + auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path); + if (canonical_directory_path.has_filename()) + canonical_directory_path += std::filesystem::path::preferred_separator; + return canonical_directory_path; + } +} + +WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) + : WorkloadEntityStorageBase(global_context_) + , dir_path{makeDirectoryPathCanonical(dir_path_)} + , log{getLogger("WorkloadEntityDiskStorage")} +{ +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name) +{ + return tryLoadEntity(entity_type, entity_name, getFilePath(entity_type, entity_name), /* check_file_exists= */ true); +} + + +ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & path, bool check_file_exists) +{ + LOG_DEBUG(log, "Loading workload entity {} from file {}", backQuote(entity_name), path); + + try + { + if (check_file_exists && !fs::exists(path)) + return nullptr; + + /// There is .sql file with workload entity creation statement. + ReadBufferFromFile in(path); + + String entity_create_query; + readStringUntilEOF(entity_create_query, in); + + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + ParserCreateWorkloadQuery parser; + ASTPtr ast = parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef().max_parser_depth, + global_context->getSettingsRef().max_parser_backtracks); + return ast; + } + case WorkloadEntityType::Resource: + { + ParserCreateResourceQuery parser; + ASTPtr ast = parseQuery( + parser, + entity_create_query.data(), + entity_create_query.data() + entity_create_query.size(), + "", + 0, + global_context->getSettingsRef().max_parser_depth, + global_context->getSettingsRef().max_parser_backtracks); + return ast; + } + } + } + catch (...) + { + tryLogCurrentException(log, fmt::format("while loading workload entity {} from path {}", backQuote(entity_name), path)); + return nullptr; /// Failed to load this entity, will ignore it + } +} + + +void WorkloadEntityDiskStorage::loadEntities() +{ + if (!entities_loaded) + loadEntitiesImpl(); +} + + +void WorkloadEntityDiskStorage::reloadEntities() +{ + loadEntitiesImpl(); +} + + +void WorkloadEntityDiskStorage::loadEntitiesImpl() +{ + LOG_INFO(log, "Loading workload entities from {}", dir_path); + + if (!std::filesystem::exists(dir_path)) + { + LOG_DEBUG(log, "The directory for workload entities ({}) does not exist: nothing to load", dir_path); + return; + } + + std::vector> entities_name_and_queries; + + Poco::DirectoryIterator dir_end; + for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it) + { + if (it->isDirectory()) + continue; + + const String & file_name = it.name(); + + if (startsWith(file_name, "workload_") && endsWith(file_name, ".sql")) + { + size_t prefix_length = strlen("workload_"); + size_t suffix_length = strlen(".sql"); + String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Workload, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + + if (startsWith(file_name, "resource_") && endsWith(file_name, ".sql")) + { + size_t prefix_length = strlen("resource_"); + size_t suffix_length = strlen(".sql"); + String name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length)); + + if (name.empty()) + continue; + + ASTPtr ast = tryLoadEntity(WorkloadEntityType::Resource, name, dir_path + it.name(), /* check_file_exists= */ false); + if (ast) + entities_name_and_queries.emplace_back(name, ast); + } + } + + setAllEntities(entities_name_and_queries); + entities_loaded = true; + + LOG_DEBUG(log, "Workload entities loaded"); +} + + +void WorkloadEntityDiskStorage::createDirectory() +{ + std::error_code create_dir_error_code; + fs::create_directories(dir_path, create_dir_error_code); + if (!fs::exists(dir_path) || !fs::is_directory(dir_path) || create_dir_error_code) + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "Couldn't create directory {} reason: '{}'", + dir_path, create_dir_error_code.message()); +} + + +bool WorkloadEntityDiskStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + createDirectory(); + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Storing workload entity {} to file {}", backQuote(entity_name), file_path); + + if (fs::exists(file_path)) + { + if (throw_if_exists) + throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + } + + WriteBufferFromOwnString create_statement_buf; + formatAST(*create_entity_query, create_statement_buf, false); + writeChar('\n', create_statement_buf); + String create_statement = create_statement_buf.str(); + + String temp_file_path = file_path + ".tmp"; + + try + { + WriteBufferFromFile out(temp_file_path, create_statement.size()); + writeString(create_statement, out); + out.next(); + if (settings.fsync_metadata) + out.sync(); + out.close(); + + if (replace_if_exists) + fs::rename(temp_file_path, file_path); + else + renameNoReplace(temp_file_path, file_path); + } + catch (...) + { + fs::remove(temp_file_path); + throw; + } + + LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); + return true; +} + + +bool WorkloadEntityDiskStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + String file_path = getFilePath(entity_type, entity_name); + LOG_DEBUG(log, "Removing workload entity {} stored in file {}", backQuote(entity_name), file_path); + + bool existed = fs::remove(file_path); + + if (!existed) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); + return true; +} + + +String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, const String & entity_name) const +{ + String file_path; + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + file_path = dir_path + "workload_" + escapeForFileName(entity_name) + ".sql"; + break; + } + case WorkloadEntityType::Resource: + { + file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; + break; + } + } + return file_path; +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h new file mode 100644 index 00000000000..22c0ea4b83d --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Loads workload entities from a specified folder. +class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); + + void loadEntities() override; + + void reloadEntities() override; + +private: + bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void createDirectory(); + void loadEntitiesImpl(); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name); + ASTPtr tryLoadEntity(WorkloadEntityType entity_type, const String & entity_name, const String & file_path, bool check_file_exists); + String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; + + String dir_path; + LoggerPtr log; + std::atomic entities_loaded = false; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp new file mode 100644 index 00000000000..a0b6ebc9267 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -0,0 +1,195 @@ +#include + +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; + extern const int UNKNOWN_WORKLOAD_ENTITY; +} + +namespace +{ + +ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const ContextPtr & context) +{ + UNUSED(context); + auto ptr = create_query.clone(); + auto & res = typeid_cast(*ptr); // TODO(serxa): we should also check for ASTCreateResourceQuery + res.if_not_exists = false; + res.or_replace = false; + return ptr; +} + +} + +WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) + : global_context(std::move(global_context_)) +{} + +ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const +{ + std::lock_guard lock(mutex); + + auto it = entities.find(entity_name); + if (it == entities.end()) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, + "The workload entity name '{}' is not saved", + entity_name); + + return it->second; +} + +ASTPtr WorkloadEntityStorageBase::tryGet(const std::string & entity_name) const +{ + std::lock_guard lock(mutex); + + auto it = entities.find(entity_name); + if (it == entities.end()) + return nullptr; + + return it->second; +} + +bool WorkloadEntityStorageBase::has(const String & entity_name) const +{ + return tryGet(entity_name) != nullptr; +} + +std::vector WorkloadEntityStorageBase::getAllEntityNames() const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + entity_names.reserve(entities.size()); + + for (const auto & [name, _] : entities) + entity_names.emplace_back(name); + + return entity_names; +} + +bool WorkloadEntityStorageBase::empty() const +{ + std::lock_guard lock(mutex); + return entities.empty(); +} + +bool WorkloadEntityStorageBase::storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) +{ + std::lock_guard lock{mutex}; + auto it = entities.find(entity_name); + if (it != entities.end()) + { + if (throw_if_exists) + throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + } + + bool stored = storeEntityImpl( + current_context, + entity_type, + entity_name, + create_entity_query, + throw_if_exists, + replace_if_exists, + settings); + + if (stored) + entities[entity_name] = create_entity_query; + + return stored; +} + +bool WorkloadEntityStorageBase::removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) +{ + std::lock_guard lock(mutex); + auto it = entities.find(entity_name); + if (it == entities.end()) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + bool removed = removeEntityImpl( + current_context, + entity_type, + entity_name, + throw_if_not_exists); + + if (removed) + entities.erase(entity_name); + + return removed; +} + +std::unique_lock WorkloadEntityStorageBase::getLock() const +{ + return std::unique_lock{mutex}; +} + +void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) +{ + std::unordered_map normalized_entities; + for (const auto & [entity_name, create_query] : new_entities) + normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); + + std::lock_guard lock(mutex); + entities = std::move(normalized_entities); +} + +std::vector> WorkloadEntityStorageBase::getAllEntities() const +{ + std::lock_guard lock{mutex}; + std::vector> all_entities; + all_entities.reserve(entities.size()); + std::copy(entities.begin(), entities.end(), std::back_inserter(all_entities)); + return all_entities; +} + +void WorkloadEntityStorageBase::setEntity(const String & entity_name, const IAST & create_entity_query) +{ + std::lock_guard lock(mutex); + entities[entity_name] = normalizeCreateWorkloadEntityQuery(create_entity_query, global_context); +} + +void WorkloadEntityStorageBase::removeEntity(const String & entity_name) +{ + std::lock_guard lock(mutex); + entities.erase(entity_name); +} + +void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) +{ + boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; + std::lock_guard lock(mutex); + for (auto it = entities.begin(); it != entities.end();) + { + auto current = it++; + if (!names_set_to_keep.contains(current->first)) + entities.erase(current); + } +} + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h new file mode 100644 index 00000000000..f6dafc033c2 --- /dev/null +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -0,0 +1,73 @@ +#pragma once + +#include +#include + +#include +#include + +#include + +namespace DB +{ + +class WorkloadEntityStorageBase : public IWorkloadEntityStorage +{ +public: + explicit WorkloadEntityStorageBase(ContextPtr global_context_); + ASTPtr get(const String & entity_name) const override; + + ASTPtr tryGet(const String & entity_name) const override; + + bool has(const String & entity_name) const override; + + std::vector getAllEntityNames() const override; + + std::vector> getAllEntities() const override; + + bool empty() const override; + + bool storeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + bool removeEntity( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + +protected: + virtual bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) = 0; + + virtual bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) = 0; + + std::unique_lock getLock() const; + void setAllEntities(const std::vector> & new_entities); + void setEntity(const String & entity_name, const IAST & create_entity_query); + void removeEntity(const String & entity_name); + void removeAllEntitiesExcept(const Strings & entity_names_to_keep); + + std::unordered_map entities; // Maps entity name into CREATE entity query + mutable std::recursive_mutex mutex; + + ContextPtr global_context; +}; + +} diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp new file mode 100644 index 00000000000..adb3e0b6e45 --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateResourceQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->resource_name = resource_name->clone(); + res->children.push_back(res->resource_name); + + return res; +} + +void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + settings.ostr << "OR REPLACE "; + + settings.ostr << "RESOURCE "; + + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (settings.hilite ? hilite_none : ""); + + formatOnCluster(settings); +} + +String ASTCreateResourceQuery::getResourceName() const +{ + String name; + tryGetIdentifierNameInto(resource_name, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h new file mode 100644 index 00000000000..3d571807ec4 --- /dev/null +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + ASTPtr resource_name; + // TODO(serxa): add resource definition + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateResourceQuery" + (delim + getResourceName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getResourceName() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..a6906dbcf65 --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -0,0 +1,67 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTCreateWorkloadQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + res->workload_name = workload_name->clone(); + res->children.push_back(res->workload_name); + + // TODO(serxa): clone settings + + return res; +} + +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + settings.ostr << "OR REPLACE "; + + settings.ostr << "WORKLOAD "; + + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (settings.hilite ? hilite_none : ""); + + formatOnCluster(settings); + + if (hasParent()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (settings.hilite ? hilite_none : ""); + } +} + +String ASTCreateWorkloadQuery::getWorkloadName() const +{ + String name; + tryGetIdentifierNameInto(workload_name, name); + return name; +} + +bool ASTCreateWorkloadQuery::hasParent() const +{ + return workload_parent != nullptr; +} + +String ASTCreateWorkloadQuery::getWorkloadParent() const +{ + String name; + tryGetIdentifierNameInto(workload_parent, name); + return name; +} + +} diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h new file mode 100644 index 00000000000..bdd3a831aeb --- /dev/null +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + ASTPtr workload_name; + ASTPtr workload_parent; + // TODO(serxa): add workload settings (weight and priority should also go inside settings, because they can differ for different resources) + + bool or_replace = false; + bool if_not_exists = false; + + String getID(char delim) const override { return "CreateWorkloadQuery" + (delim + getWorkloadName()); } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + String getWorkloadName() const; + bool hasParent() const; + String getWorkloadParent() const; + + QueryKind getQueryKind() const override { return QueryKind::Create; } +}; + +} diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index ab0e70eb0e5..aef505668d6 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -407,6 +407,7 @@ namespace DB MR_MACROS(REPLACE_PARTITION, "REPLACE PARTITION") \ MR_MACROS(REPLACE, "REPLACE") \ MR_MACROS(RESET_SETTING, "RESET SETTING") \ + MR_MACROS(RESOURCE, "RESOURCE") \ MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \ MR_MACROS(RESTORE, "RESTORE") \ MR_MACROS(RESTRICT, "RESTRICT") \ @@ -519,6 +520,7 @@ namespace DB MR_MACROS(WHEN, "WHEN") \ MR_MACROS(WHERE, "WHERE") \ MR_MACROS(WINDOW, "WINDOW") \ + MR_MACROS(WORKLOAD, "WORKLOAD") \ MR_MACROS(QUALIFY, "QUALIFY") \ MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \ MR_MACROS(WITH_CHECK, "WITH CHECK") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp new file mode 100644 index 00000000000..4921debdf52 --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.cpp @@ -0,0 +1,62 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + // TODO(serxa): parse resource definition + + ASTPtr resource_name; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto create_resource_query = std::make_shared(); + node = create_resource_query; + + create_resource_query->resource_name = resource_name; + create_resource_query->children.push_back(resource_name); + + create_resource_query->or_replace = or_replace; + create_resource_query->if_not_exists = if_not_exists; + create_resource_query->cluster = std::move(cluster_str); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateResourceQuery.h b/src/Parsers/ParserCreateResourceQuery.h new file mode 100644 index 00000000000..1b7c9fc4a7f --- /dev/null +++ b/src/Parsers/ParserCreateResourceQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE RESOURCE cache_io (WRITE DISK s3diskWithCache, READ DISK s3diskWithCache) +class ParserCreateResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..ab0b0e3eb36 --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -0,0 +1,76 @@ +#include + +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create(Keyword::CREATE); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_or_replace(Keyword::OR_REPLACE); + ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); + ParserIdentifier workload_name_p; + ParserKeyword s_on(Keyword::ON); + ParserKeyword s_in(Keyword::IN); + // TODO(serxa): parse workload settings + + ASTPtr workload_name; + ASTPtr workload_parent; + + String cluster_str; + bool or_replace = false; + bool if_not_exists = false; + + if (!s_create.ignore(pos, expected)) + return false; + + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (s_in.ignore(pos, expected)) + { + if (!workload_name_p.parse(pos, workload_parent, expected)) + return false; + } + + auto create_workload_query = std::make_shared(); + node = create_workload_query; + + create_workload_query->workload_name = workload_name; + create_workload_query->children.push_back(workload_name); + + if (workload_parent) + { + create_workload_query->workload_parent = workload_parent; + create_workload_query->children.push_back(workload_parent); + } + + create_workload_query->or_replace = or_replace; + create_workload_query->if_not_exists = if_not_exists; + create_workload_query->cluster = std::move(cluster_str); + + return true; +} + +} diff --git a/src/Parsers/ParserCreateWorkloadQuery.h b/src/Parsers/ParserCreateWorkloadQuery.h new file mode 100644 index 00000000000..62c89affeda --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadQuery.h @@ -0,0 +1,16 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ + +/// CREATE WORKLOAD production IN all SETTINGS weight = 3, max_speed = '1G' FOR network_read, max_speed = '2G' FOR network_write +class ParserCreateWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} From 6b6cfd4e1677f23b989449f49a89d15093e543d8 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:08:24 +0000 Subject: [PATCH 002/157] Integrate workload entity storage into server --- programs/server/Server.cpp | 2 + programs/server/config.xml | 4 ++ .../Workload/createWorkloadEntityStorage.cpp | 48 +++++++++++++++++++ .../Workload/createWorkloadEntityStorage.h | 11 +++++ src/Interpreters/Context.cpp | 36 +++++++++++++- src/Interpreters/Context.h | 5 ++ 6 files changed, 105 insertions(+), 1 deletion(-) create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp create mode 100644 src/Common/Scheduler/Workload/createWorkloadEntityStorage.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fb5717ba33f..996542741f9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2088,6 +2088,8 @@ try database_catalog.assertDatabaseExists(default_database); /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); + /// Load WORKLOADs and RESOURCEs. + global_context->getWorkloadEntityStorage().loadObjects(); } catch (...) { diff --git a/programs/server/config.xml b/programs/server/config.xml index 10ad831465a..b41f0344bb2 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1386,6 +1386,10 @@ If not specified they will be stored locally. --> + + + diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp new file mode 100644 index 00000000000..dde995db6e1 --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int INVALID_CONFIG_PARAMETER; +} + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context) +{ + const String zookeeper_path_key = "workload_zookeeper_path"; + const String disk_path_key = "workload_path"; + + const auto & config = global_context->getConfigRef(); + if (config.has(zookeeper_path_key)) + { + if (config.has(disk_path_key)) + { + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "'{}' and '{}' must not be both specified in the config", + zookeeper_path_key, + disk_path_key); + } + abort(); // TODO(serxa): crate WorkloadEntityKeeperStorage object + //return std::make_unique(global_context, config.getString(zookeeper_path_key)); + } + else + { + String default_path = fs::path{global_context->getPath()} / "workload" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); + } +} + +} diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h new file mode 100644 index 00000000000..936e1275010 --- /dev/null +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +std::unique_ptr createWorkloadEntityStorage(const ContextMutablePtr & global_context); + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 373cc91ebcb..9f0ad40f446 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -64,7 +64,6 @@ #include #include #include -#include #include #include #include @@ -89,6 +88,8 @@ #include #include #include +#include +#include #include #include #include @@ -270,6 +271,9 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag user_defined_sql_objects_storage_initialized; mutable std::unique_ptr user_defined_sql_objects_storage; + mutable OnceFlag workload_entity_storage_initialized; + mutable std::unique_ptr workload_entity_storage; + #if USE_NLP mutable OnceFlag synonyms_extensions_initialized; mutable std::optional synonyms_extensions; @@ -609,6 +613,7 @@ struct ContextSharedPart : boost::noncopyable SHUTDOWN(log, "dictionaries loader", external_dictionaries_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "UDFs loader", external_user_defined_executable_functions_loader, enablePeriodicUpdates(false)); SHUTDOWN(log, "another UDFs storage", user_defined_sql_objects_storage, stopWatching()); + SHUTDOWN(log, "workload entity storage", workload_entity_storage, stopWatching()); LOG_TRACE(log, "Shutting down named sessions"); Session::shutdownNamedSessions(); @@ -640,6 +645,7 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr delete_external_dictionaries_loader; std::unique_ptr delete_external_user_defined_executable_functions_loader; std::unique_ptr delete_user_defined_sql_objects_storage; + std::unique_ptr delete_workload_entity_storage; std::unique_ptr delete_buffer_flush_schedule_pool; std::unique_ptr delete_schedule_pool; std::unique_ptr delete_distributed_schedule_pool; @@ -724,6 +730,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader = std::move(external_dictionaries_loader); delete_external_user_defined_executable_functions_loader = std::move(external_user_defined_executable_functions_loader); delete_user_defined_sql_objects_storage = std::move(user_defined_sql_objects_storage); + delete_workload_entity_storage = std::move(workload_entity_storage); delete_buffer_flush_schedule_pool = std::move(buffer_flush_schedule_pool); delete_schedule_pool = std::move(schedule_pool); delete_distributed_schedule_pool = std::move(distributed_schedule_pool); @@ -742,6 +749,7 @@ struct ContextSharedPart : boost::noncopyable delete_external_dictionaries_loader.reset(); delete_external_user_defined_executable_functions_loader.reset(); delete_user_defined_sql_objects_storage.reset(); + delete_workload_entity_storage.reset(); delete_ddl_worker.reset(); delete_buffer_flush_schedule_pool.reset(); delete_schedule_pool.reset(); @@ -2903,6 +2911,32 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } +const IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const +{ + callOnce(shared->workload_entity_storage_initialized, [&] { + shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); + }); + + SharedLockGuard lock(shared->mutex); + return *shared->workload_entity_storage; +} + +IWorkloadEntityStorage & Context::getWorkloadEntityStorage() +{ + callOnce(shared->workload_entity_storage_initialized, [&] { + shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); + }); + + std::lock_guard lock(shared->mutex); + return *shared->workload_entity_storage; +} + +void Context::setWorkloadEntityStorage(std::unique_ptr storage) +{ + std::lock_guard lock(shared->mutex); + shared->workload_entity_storage = std::move(storage); +} + #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fb5337158ba..a6fd119f152 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -70,6 +70,7 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; class IUserDefinedSQLObjectsStorage; +class IWorkloadEntityStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; @@ -879,6 +880,10 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); + const IWorkloadEntityStorage & getWorkloadEntityStorage() const; + IWorkloadEntityStorage & getWorkloadEntityStorage(); + void setWorkloadEntityStorage(std::unique_ptr storage); + #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; Lemmatizers & getLemmatizers() const; From 93bcf2d8bcccba54ec86e187e5f9990a20062f22 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:28:16 +0000 Subject: [PATCH 003/157] add ASTs for DROP WORKLOAD and DROP RESOURCE --- src/Parsers/ASTDropResourceQuery.cpp | 25 +++++++++++++++++++++++++ src/Parsers/ASTDropResourceQuery.h | 28 ++++++++++++++++++++++++++++ src/Parsers/ASTDropWorkloadQuery.cpp | 25 +++++++++++++++++++++++++ src/Parsers/ASTDropWorkloadQuery.h | 28 ++++++++++++++++++++++++++++ 4 files changed, 106 insertions(+) create mode 100644 src/Parsers/ASTDropResourceQuery.cpp create mode 100644 src/Parsers/ASTDropResourceQuery.h create mode 100644 src/Parsers/ASTDropWorkloadQuery.cpp create mode 100644 src/Parsers/ASTDropWorkloadQuery.h diff --git a/src/Parsers/ASTDropResourceQuery.cpp b/src/Parsers/ASTDropResourceQuery.cpp new file mode 100644 index 00000000000..753ac4e30e7 --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropResourceQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP RESOURCE "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(resource_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropResourceQuery.h b/src/Parsers/ASTDropResourceQuery.h new file mode 100644 index 00000000000..e1534ea454a --- /dev/null +++ b/src/Parsers/ASTDropResourceQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropResourceQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String resource_name; + + bool if_exists = false; + + String getID(char) const override { return "DropResourceQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.cpp b/src/Parsers/ASTDropWorkloadQuery.cpp new file mode 100644 index 00000000000..3192223c4b3 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + +namespace DB +{ + +ASTPtr ASTDropWorkloadQuery::clone() const +{ + return std::make_shared(*this); +} + +void ASTDropWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP WORKLOAD "; + + if (if_exists) + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(workload_name) << (settings.hilite ? hilite_none : ""); + formatOnCluster(settings); +} + +} diff --git a/src/Parsers/ASTDropWorkloadQuery.h b/src/Parsers/ASTDropWorkloadQuery.h new file mode 100644 index 00000000000..99c3a011447 --- /dev/null +++ b/src/Parsers/ASTDropWorkloadQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ASTDropWorkloadQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + String workload_name; + + bool if_exists = false; + + String getID(char) const override { return "DropWorkloadQuery"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + QueryKind getQueryKind() const override { return QueryKind::Drop; } +}; + +} From 31e2205c4e5e3b4dc9d8d6ae391d83a8d9f85afe Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:28:56 +0000 Subject: [PATCH 004/157] fix workload entity storage start --- programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 996542741f9..dd6bf291354 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -86,6 +86,7 @@ #include #include #include +#include #include #include #include "MetricsTransmitter.h" @@ -2089,7 +2090,7 @@ try /// Load user-defined SQL functions. global_context->getUserDefinedSQLObjectsStorage().loadObjects(); /// Load WORKLOADs and RESOURCEs. - global_context->getWorkloadEntityStorage().loadObjects(); + global_context->getWorkloadEntityStorage().loadEntities(); } catch (...) { From 3a486d79bfb432f24d83051c8e9c53d8e39d8e8a Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:35:52 +0000 Subject: [PATCH 005/157] add parsers for DROP WORKLOAD and DROP RESOURCE queries --- src/Parsers/ParserDropResourceQuery.cpp | 52 +++++++++++++++++++++++++ src/Parsers/ParserDropResourceQuery.h | 14 +++++++ src/Parsers/ParserDropWorkloadQuery.cpp | 52 +++++++++++++++++++++++++ src/Parsers/ParserDropWorkloadQuery.h | 14 +++++++ 4 files changed, 132 insertions(+) create mode 100644 src/Parsers/ParserDropResourceQuery.cpp create mode 100644 src/Parsers/ParserDropResourceQuery.h create mode 100644 src/Parsers/ParserDropWorkloadQuery.cpp create mode 100644 src/Parsers/ParserDropWorkloadQuery.h diff --git a/src/Parsers/ParserDropResourceQuery.cpp b/src/Parsers/ParserDropResourceQuery.cpp new file mode 100644 index 00000000000..6c078281828 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_resource(Keyword::RESOURCE); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier resource_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr resource_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_resource.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!resource_name_p.parse(pos, resource_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_resource_query = std::make_shared(); + drop_resource_query->if_exists = if_exists; + drop_resource_query->cluster = std::move(cluster_str); + + node = drop_resource_query; + + drop_resource_query->resource_name = resource_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropResourceQuery.h b/src/Parsers/ParserDropResourceQuery.h new file mode 100644 index 00000000000..651603d1e90 --- /dev/null +++ b/src/Parsers/ParserDropResourceQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP RESOURCE resource1 +class ParserDropResourceQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP RESOURCE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/ParserDropWorkloadQuery.cpp b/src/Parsers/ParserDropWorkloadQuery.cpp new file mode 100644 index 00000000000..edc82c8f30a --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserDropWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_drop(Keyword::DROP); + ParserKeyword s_workload(Keyword::WORKLOAD); + ParserKeyword s_if_exists(Keyword::IF_EXISTS); + ParserKeyword s_on(Keyword::ON); + ParserIdentifier workload_name_p; + + String cluster_str; + bool if_exists = false; + + ASTPtr workload_name; + + if (!s_drop.ignore(pos, expected)) + return false; + + if (!s_workload.ignore(pos, expected)) + return false; + + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + + if (!workload_name_p.parse(pos, workload_name, expected)) + return false; + + if (s_on.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + auto drop_workload_query = std::make_shared(); + drop_workload_query->if_exists = if_exists; + drop_workload_query->cluster = std::move(cluster_str); + + node = drop_workload_query; + + drop_workload_query->workload_name = workload_name->as().name(); + + return true; +} + +} diff --git a/src/Parsers/ParserDropWorkloadQuery.h b/src/Parsers/ParserDropWorkloadQuery.h new file mode 100644 index 00000000000..af060caf303 --- /dev/null +++ b/src/Parsers/ParserDropWorkloadQuery.h @@ -0,0 +1,14 @@ +#pragma once + +#include "IParserBase.h" + +namespace DB +{ +/// DROP WORKLOAD workload1 +class ParserDropWorkloadQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP WORKLOAD query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} From a58d27166b22da253ec1e214a48ed3f2177ed85c Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 17:39:42 +0000 Subject: [PATCH 006/157] register workload and resource queries parsers --- src/Parsers/ParserQuery.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 22ddc25019f..b0f4715e2a3 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -1,8 +1,12 @@ #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -48,6 +52,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateSettingsProfileQuery create_settings_profile_p; ParserCreateFunctionQuery create_function_p; ParserDropFunctionQuery drop_function_p; + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; ParserCreateNamedCollectionQuery create_named_collection_p; ParserDropNamedCollectionQuery drop_named_collection_p; ParserAlterNamedCollectionQuery alter_named_collection_p; @@ -74,6 +82,10 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_settings_profile_p.parse(pos, node, expected) || create_function_p.parse(pos, node, expected) || drop_function_p.parse(pos, node, expected) + || create_workload_p.parse(pos, node, expected) + || drop_workload_p.parse(pos, node, expected) + || create_resource_p.parse(pos, node, expected) + || drop_resource_p.parse(pos, node, expected) || create_named_collection_p.parse(pos, node, expected) || drop_named_collection_p.parse(pos, node, expected) || alter_named_collection_p.parse(pos, node, expected) From 90764466172c29867aa148541f27824cbbd966db Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:11:42 +0000 Subject: [PATCH 007/157] add interpreters for CREATE WORKLOAD and RESOURCE queries --- src/Access/Common/AccessType.h | 4 ++ src/Access/ContextAccess.cpp | 6 +- .../InterpreterCreateResourceQuery.cpp | 61 +++++++++++++++++++ .../InterpreterCreateResourceQuery.h | 25 ++++++++ .../InterpreterCreateWorkloadQuery.cpp | 61 +++++++++++++++++++ .../InterpreterCreateWorkloadQuery.h | 25 ++++++++ 6 files changed, 180 insertions(+), 2 deletions(-) create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateResourceQuery.h create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterCreateWorkloadQuery.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..e70229b62e8 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -99,6 +99,8 @@ enum class AccessType : uint8_t M(CREATE_ARBITRARY_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables with arbitrary table engine */\ M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \ + M(CREATE_WORKLOAD, "", GLOBAL, CREATE) /* allows to execute CREATE WORKLOAD */ \ + M(CREATE_RESOURCE, "", GLOBAL, CREATE) /* allows to execute CREATE RESOURCE */ \ M(CREATE_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute CREATE NAMED COLLECTION */ \ M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \ \ @@ -108,6 +110,8 @@ enum class AccessType : uint8_t implicitly enabled by the grant DROP_TABLE */\ M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\ M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\ + M(DROP_WORKLOAD, "", GLOBAL, DROP) /* allows to execute DROP WORKLOAD */\ + M(DROP_RESOURCE, "", GLOBAL, DROP) /* allows to execute DROP RESOURCE */\ M(DROP_NAMED_COLLECTION, "", NAMED_COLLECTION, NAMED_COLLECTION_ADMIN) /* allows to execute DROP NAMED COLLECTION */\ M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\ \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index e50521a0730..d856341fade 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -689,15 +689,17 @@ bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlag const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; + const AccessFlags workload_ddl = AccessType::CREATE_WORKLOAD | AccessType::DROP_WORKLOAD; + const AccessFlags resource_ddl = AccessType::CREATE_RESOURCE | AccessType::DROP_RESOURCE; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; const AccessFlags table_and_dictionary_and_function_ddl = table_ddl | dictionary_ddl | function_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; - const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; + const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | workload_ddl | resource_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; - const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; + const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl | workload_ddl | resource_ddl; const AccessFlags introspection_flags = AccessType::INTROSPECTION; }; static const PrecalculatedFlags precalc; diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp new file mode 100644 index 00000000000..78f5b535cb1 --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.cpp @@ -0,0 +1,61 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateResourceQuery::execute() +{ + ASTCreateResourceQuery & create_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_RESOURCE); + + if (create_resource_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!create_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto resource_name = create_resource_query.getResourceName(); + //bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; + //bool replace_if_exists = create_resource_query.or_replace; + + // TODO(serxa): validate and register entity + + return {}; +} + +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateResourceQuery.h b/src/Interpreters/InterpreterCreateResourceQuery.h new file mode 100644 index 00000000000..4bd427e5e8f --- /dev/null +++ b/src/Interpreters/InterpreterCreateResourceQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp new file mode 100644 index 00000000000..1057fb14604 --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp @@ -0,0 +1,61 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterCreateWorkloadQuery::execute() +{ + ASTCreateWorkloadQuery & create_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::CREATE_WORKLOAD); + + if (create_workload_query.or_replace) + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!create_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + auto workload_name = create_workload_query.getWorkloadName(); + //bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; + //bool replace_if_exists = create_workload_query.or_replace; + + // TODO(serxa): validate and register entity + + return {}; +} + +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterCreateWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.h b/src/Interpreters/InterpreterCreateWorkloadQuery.h new file mode 100644 index 00000000000..319388fb64c --- /dev/null +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +class InterpreterCreateWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterCreateWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) + { + } + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} From 2183c73077a7d1477ca4a5993f9776112be8607c Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:22:18 +0000 Subject: [PATCH 008/157] add interpreters for DROP WORKLOAD and RESOURCE queries --- .../InterpreterDropResourceQuery.cpp | 56 +++++++++++++++++++ .../InterpreterDropResourceQuery.h | 21 +++++++ .../InterpreterDropWorkloadQuery.cpp | 56 +++++++++++++++++++ .../InterpreterDropWorkloadQuery.h | 21 +++++++ 4 files changed, 154 insertions(+) create mode 100644 src/Interpreters/InterpreterDropResourceQuery.cpp create mode 100644 src/Interpreters/InterpreterDropResourceQuery.h create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.cpp create mode 100644 src/Interpreters/InterpreterDropWorkloadQuery.h diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp new file mode 100644 index 00000000000..49071a0a1aa --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.cpp @@ -0,0 +1,56 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropResourceQuery::execute() +{ + ASTDropResourceQuery & drop_resource_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_RESOURCE); + + auto current_context = getContext(); + + if (!drop_resource_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + //bool throw_if_not_exists = !drop_resource_query.if_exists; + + // TODO(serxa): validate and unregister entity + + return {}; +} + +void registerInterpreterDropResourceQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropResourceQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropResourceQuery.h b/src/Interpreters/InterpreterDropResourceQuery.h new file mode 100644 index 00000000000..588f26fb88c --- /dev/null +++ b/src/Interpreters/InterpreterDropResourceQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropResourceQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropResourceQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp new file mode 100644 index 00000000000..da022d4d054 --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.cpp @@ -0,0 +1,56 @@ +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + +BlockIO InterpreterDropWorkloadQuery::execute() +{ + ASTDropWorkloadQuery & drop_workload_query = query_ptr->as(); + + AccessRightsElements access_rights_elements; + access_rights_elements.emplace_back(AccessType::DROP_WORKLOAD); + + auto current_context = getContext(); + + if (!drop_workload_query.cluster.empty()) + { + if (current_context->getWorkloadEntityStorage().isReplicated()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "ON CLUSTER is not allowed because workload entities are replicated automatically"); + + DDLQueryOnClusterParams params; + params.access_to_check = std::move(access_rights_elements); + return executeDDLQueryOnCluster(query_ptr, current_context, params); + } + + current_context->checkAccess(access_rights_elements); + + //bool throw_if_not_exists = !drop_workload_query.if_exists; + + // TODO(serxa): validate and unregister entity + + return {}; +} + +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory) +{ + auto create_fn = [] (const InterpreterFactory::Arguments & args) + { + return std::make_unique(args.query, args.context); + }; + factory.registerInterpreter("InterpreterDropWorkloadQuery", create_fn); +} + +} diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.h b/src/Interpreters/InterpreterDropWorkloadQuery.h new file mode 100644 index 00000000000..1297c95e949 --- /dev/null +++ b/src/Interpreters/InterpreterDropWorkloadQuery.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +class InterpreterDropWorkloadQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterDropWorkloadQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; +}; + +} From 7f6694b3705aa5dc929776bc357863b8769733da Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 1 Sep 2024 18:25:28 +0000 Subject: [PATCH 009/157] register workload entities queries interpreters --- src/Interpreters/InterpreterFactory.cpp | 20 ++++++++++++++++++++ src/Interpreters/registerInterpreters.cpp | 8 ++++++++ 2 files changed, 28 insertions(+) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 12b3b510098..5ae29fbe913 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -3,9 +3,13 @@ #include #include #include +#include +#include #include #include #include +#include +#include #include #include #include @@ -326,6 +330,22 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte { interpreter_name = "InterpreterDropFunctionQuery"; } + else if (query->as()) + { + interpreter_name = "InterpreterCreateWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropWorkloadQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterCreateResourceQuery"; + } + else if (query->as()) + { + interpreter_name = "InterpreterDropResourceQuery"; + } else if (query->as()) { interpreter_name = "InterpreterCreateIndexQuery"; diff --git a/src/Interpreters/registerInterpreters.cpp b/src/Interpreters/registerInterpreters.cpp index 481d0597a85..838b3a669da 100644 --- a/src/Interpreters/registerInterpreters.cpp +++ b/src/Interpreters/registerInterpreters.cpp @@ -52,6 +52,10 @@ void registerInterpreterExternalDDLQuery(InterpreterFactory & factory); void registerInterpreterTransactionControlQuery(InterpreterFactory & factory); void registerInterpreterCreateFunctionQuery(InterpreterFactory & factory); void registerInterpreterDropFunctionQuery(InterpreterFactory & factory); +void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory); +void registerInterpreterCreateResourceQuery(InterpreterFactory & factory); +void registerInterpreterDropResourceQuery(InterpreterFactory & factory); void registerInterpreterCreateIndexQuery(InterpreterFactory & factory); void registerInterpreterCreateNamedCollectionQuery(InterpreterFactory & factory); void registerInterpreterDropIndexQuery(InterpreterFactory & factory); @@ -111,6 +115,10 @@ void registerInterpreters() registerInterpreterTransactionControlQuery(factory); registerInterpreterCreateFunctionQuery(factory); registerInterpreterDropFunctionQuery(factory); + registerInterpreterCreateWorkloadQuery(factory); + registerInterpreterDropWorkloadQuery(factory); + registerInterpreterCreateResourceQuery(factory); + registerInterpreterDropResourceQuery(factory); registerInterpreterCreateIndexQuery(factory); registerInterpreterCreateNamedCollectionQuery(factory); registerInterpreterDropIndexQuery(factory); From 2ef36b36acb1926b70b1d4b64c7d3d83783e483c Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:06:44 +0000 Subject: [PATCH 010/157] add notification system for workload entity changes --- .../Workload/IWorkloadEntityStorage.h | 13 ++ .../Workload/WorkloadEntityDiskStorage.cpp | 2 + .../Workload/WorkloadEntityStorageBase.cpp | 113 +++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 41 ++++++- 4 files changed, 150 insertions(+), 19 deletions(-) diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index 65978a71be0..113cefe3f46 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -17,6 +18,8 @@ enum class WorkloadEntityType : uint8_t { Workload, Resource, + + MAX }; /// Interface for a storage of workload entities (WORKLOAD and RESOURCE). @@ -72,6 +75,16 @@ public: WorkloadEntityType entity_type, const String & entity_name, bool throw_if_not_exists) = 0; + + using OnChangedHandler = std::function; + + /// Subscribes for all changes. + virtual scope_guard subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) = 0; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index d9ca8bca0a0..b14a96c771a 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -108,6 +108,7 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, global_context->getSettingsRef().max_parser_backtracks); return ast; } + case WorkloadEntityType::MAX: return nullptr; } } catch (...) @@ -289,6 +290,7 @@ String WorkloadEntityDiskStorage::getFilePath(WorkloadEntityType entity_type, co file_path = dir_path + "resource_" + escapeForFileName(entity_name) + ".sql"; break; } + case WorkloadEntityType::MAX: break; } return file_path; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index a0b6ebc9267..dfcd5f9b7da 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -111,16 +112,19 @@ bool WorkloadEntityStorageBase::storeEntity( settings); if (stored) + { entities[entity_name] = create_entity_query; + onEntityAdded(entity_type, entity_name, create_entity_query); + } return stored; } bool WorkloadEntityStorageBase::removeEntity( - const ContextPtr & current_context, - WorkloadEntityType entity_type, - const String & entity_name, - bool throw_if_not_exists) + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) { std::lock_guard lock(mutex); auto it = entities.find(entity_name); @@ -139,11 +143,94 @@ bool WorkloadEntityStorageBase::removeEntity( throw_if_not_exists); if (removed) + { entities.erase(entity_name); + onEntityRemoved(entity_type, entity_name); + } return removed; } +scope_guard WorkloadEntityStorageBase::subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) +{ + std::lock_guard lock{handlers->mutex}; + auto & list = handlers->by_type[static_cast(entity_type)]; + list.push_back(handler); + auto handler_it = std::prev(list.end()); + + return [my_handlers = handlers, entity_type, handler_it] + { + std::lock_guard lock2{my_handlers->mutex}; + auto & list2 = my_handlers->by_type[static_cast(entity_type)]; + list2.erase(handler_it); + }; +} + +void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + event.entity = new_entity; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + event.entity = changed_entity; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) +{ + std::lock_guard lock{queue_mutex}; + Event event; + event.name = entity_name; + event.type = entity_type; + queue.push(std::move(event)); +} + +void WorkloadEntityStorageBase::sendNotifications() +{ + /// Only one thread can send notification at any time. + std::lock_guard sending_notifications_lock{sending_notifications}; + + std::unique_lock queue_lock{queue_mutex}; + while (!queue.empty()) + { + auto event = std::move(queue.front()); + queue.pop(); + queue_lock.unlock(); + + std::vector current_handlers; + { + std::lock_guard handlers_lock{handlers->mutex}; + boost::range::copy(handlers->by_type[static_cast(event.type)], std::back_inserter(current_handlers)); + } + + for (const auto & handler : current_handlers) + { + try + { + handler(event.type, event.name, event.entity); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + queue_lock.lock(); + } +} + std::unique_lock WorkloadEntityStorageBase::getLock() const { return std::unique_lock{mutex}; @@ -155,6 +242,11 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector> WorkloadEntityStorageBase::getAllEntities return all_entities; } -void WorkloadEntityStorageBase::setEntity(const String & entity_name, const IAST & create_entity_query) -{ - std::lock_guard lock(mutex); - entities[entity_name] = normalizeCreateWorkloadEntityQuery(create_entity_query, global_context); -} - -void WorkloadEntityStorageBase::removeEntity(const String & entity_name) -{ - std::lock_guard lock(mutex); - entities.erase(entity_name); -} - +// TODO(serxa): add notifications or remove this function void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) { boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index f6dafc033c2..9e9e8170a8e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include +#include #include #include @@ -42,6 +44,10 @@ public: const String & entity_name, bool throw_if_not_exists) override; + virtual scope_guard subscribeForChanges( + WorkloadEntityType entity_type, + const OnChangedHandler & handler) override; + protected: virtual bool storeEntityImpl( const ContextPtr & current_context, @@ -60,12 +66,41 @@ protected: std::unique_lock getLock() const; void setAllEntities(const std::vector> & new_entities); - void setEntity(const String & entity_name, const IAST & create_entity_query); - void removeEntity(const String & entity_name); void removeAllEntitiesExcept(const Strings & entity_names_to_keep); - std::unordered_map entities; // Maps entity name into CREATE entity query + /// Called by derived class after a new workload entity has been added. + void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); + + /// Called by derived class after an workload entity has been changed. + void onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity); + + /// Called by derived class after an workload entity has been removed. + void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); + + /// Sends notifications to subscribers about changes in workload entities + /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). + void sendNotifications(); + + struct Handlers + { + std::mutex mutex; + std::list by_type[static_cast(WorkloadEntityType::MAX)]; + }; + /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. + std::shared_ptr handlers; + + struct Event + { + WorkloadEntityType type; + String name; + ASTPtr entity; + }; + std::queue queue; + std::mutex queue_mutex; + std::mutex sending_notifications; + mutable std::recursive_mutex mutex; + std::unordered_map entities; // Maps entity name into CREATE entity query ContextPtr global_context; }; From 8e61a5c0b6b198dd97e0f21feacb06ce64196b86 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:13:22 +0000 Subject: [PATCH 011/157] fix normalizeCreateWorkloadEntityQuery() --- .../Workload/WorkloadEntityStorageBase.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index dfcd5f9b7da..7e7a4e526f1 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -23,9 +24,16 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const Conte { UNUSED(context); auto ptr = create_query.clone(); - auto & res = typeid_cast(*ptr); // TODO(serxa): we should also check for ASTCreateResourceQuery - res.if_not_exists = false; - res.or_replace = false; + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } + if (auto * res = typeid_cast(ptr.get())) + { + res->if_not_exists = false; + res->or_replace = false; + } return ptr; } From 840d284e36a4717fef6a14ed9d4ee35972374f51 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 09:59:30 +0000 Subject: [PATCH 012/157] attach interpreters to storage --- .../Workload/WorkloadEntityStorageBase.cpp | 3 +++ src/Interpreters/InterpreterCreateResourceQuery.cpp | 13 ++++++++++--- src/Interpreters/InterpreterCreateWorkloadQuery.cpp | 13 ++++++++++--- src/Interpreters/InterpreterDropResourceQuery.cpp | 8 ++++++-- src/Interpreters/InterpreterDropWorkloadQuery.cpp | 8 ++++++-- 5 files changed, 35 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 7e7a4e526f1..33e6227b998 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -101,6 +101,9 @@ bool WorkloadEntityStorageBase::storeEntity( const Settings & settings) { std::lock_guard lock{mutex}; + + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); + auto it = entities.find(entity_name); if (it != entities.end()) { diff --git a/src/Interpreters/InterpreterCreateResourceQuery.cpp b/src/Interpreters/InterpreterCreateResourceQuery.cpp index 78f5b535cb1..c6eca7a90d8 100644 --- a/src/Interpreters/InterpreterCreateResourceQuery.cpp +++ b/src/Interpreters/InterpreterCreateResourceQuery.cpp @@ -41,10 +41,17 @@ BlockIO InterpreterCreateResourceQuery::execute() current_context->checkAccess(access_rights_elements); auto resource_name = create_resource_query.getResourceName(); - //bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; - //bool replace_if_exists = create_resource_query.or_replace; + bool throw_if_exists = !create_resource_query.if_not_exists && !create_resource_query.or_replace; + bool replace_if_exists = create_resource_query.or_replace; - // TODO(serxa): validate and register entity + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Resource, + resource_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); return {}; } diff --git a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp index 1057fb14604..41d0f52c685 100644 --- a/src/Interpreters/InterpreterCreateWorkloadQuery.cpp +++ b/src/Interpreters/InterpreterCreateWorkloadQuery.cpp @@ -41,10 +41,17 @@ BlockIO InterpreterCreateWorkloadQuery::execute() current_context->checkAccess(access_rights_elements); auto workload_name = create_workload_query.getWorkloadName(); - //bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; - //bool replace_if_exists = create_workload_query.or_replace; + bool throw_if_exists = !create_workload_query.if_not_exists && !create_workload_query.or_replace; + bool replace_if_exists = create_workload_query.or_replace; - // TODO(serxa): validate and register entity + current_context->getWorkloadEntityStorage().storeEntity( + current_context, + WorkloadEntityType::Workload, + workload_name, + query_ptr, + throw_if_exists, + replace_if_exists, + current_context->getSettingsRef()); return {}; } diff --git a/src/Interpreters/InterpreterDropResourceQuery.cpp b/src/Interpreters/InterpreterDropResourceQuery.cpp index 49071a0a1aa..848a74fda23 100644 --- a/src/Interpreters/InterpreterDropResourceQuery.cpp +++ b/src/Interpreters/InterpreterDropResourceQuery.cpp @@ -37,9 +37,13 @@ BlockIO InterpreterDropResourceQuery::execute() current_context->checkAccess(access_rights_elements); - //bool throw_if_not_exists = !drop_resource_query.if_exists; + bool throw_if_not_exists = !drop_resource_query.if_exists; - // TODO(serxa): validate and unregister entity + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Resource, + drop_resource_query.resource_name, + throw_if_not_exists); return {}; } diff --git a/src/Interpreters/InterpreterDropWorkloadQuery.cpp b/src/Interpreters/InterpreterDropWorkloadQuery.cpp index da022d4d054..bbaa2beb4cd 100644 --- a/src/Interpreters/InterpreterDropWorkloadQuery.cpp +++ b/src/Interpreters/InterpreterDropWorkloadQuery.cpp @@ -37,9 +37,13 @@ BlockIO InterpreterDropWorkloadQuery::execute() current_context->checkAccess(access_rights_elements); - //bool throw_if_not_exists = !drop_workload_query.if_exists; + bool throw_if_not_exists = !drop_workload_query.if_exists; - // TODO(serxa): validate and unregister entity + current_context->getWorkloadEntityStorage().removeEntity( + current_context, + WorkloadEntityType::Workload, + drop_workload_query.workload_name, + throw_if_not_exists); return {}; } From 85e7641299a6de2614dfc24f2cf932252b6e59c2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 3 Sep 2024 11:03:15 +0000 Subject: [PATCH 013/157] add system.workloads table with test --- .../Workload/IWorkloadEntityStorage.h | 3 ++ .../Workload/WorkloadEntityStorageBase.cpp | 24 ++++++++++ .../Workload/WorkloadEntityStorageBase.h | 1 + .../System/StorageSystemWorkloads.cpp | 48 +++++++++++++++++++ src/Storages/System/StorageSystemWorkloads.h | 29 +++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + .../03232_workload_create_and_drop.reference | 5 ++ .../03232_workload_create_and_drop.sql | 11 +++++ 8 files changed, 123 insertions(+) create mode 100644 src/Storages/System/StorageSystemWorkloads.cpp create mode 100644 src/Storages/System/StorageSystemWorkloads.h create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_workload_create_and_drop.sql diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index 113cefe3f46..cff09a2259d 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -47,6 +47,9 @@ public: /// Get all entity names. virtual std::vector getAllEntityNames() const = 0; + /// Get all entity names of specified type. + virtual std::vector getAllEntityNames(WorkloadEntityType entity_type) const = 0; + /// Get all entities. virtual std::vector> getAllEntities() const = 0; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 33e6227b998..e3bf6d4af7f 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -37,6 +37,16 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const Conte return ptr; } +WorkloadEntityType getEntityType(const ASTPtr & ptr) +{ + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Workload; + if (auto * res = typeid_cast(ptr.get())) + return WorkloadEntityType::Resource; + chassert(false); + return WorkloadEntityType::MAX; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -85,6 +95,20 @@ std::vector WorkloadEntityStorageBase::getAllEntityNames() const return entity_names; } +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +{ + std::vector entity_names; + + std::lock_guard lock(mutex); + for (const auto & [name, entity] : entities) + { + if (getEntityType(entity) == entity_type) + entity_names.emplace_back(name); + } + + return entity_names; +} + bool WorkloadEntityStorageBase::empty() const { std::lock_guard lock(mutex); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 9e9e8170a8e..8ec92675ddb 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -24,6 +24,7 @@ public: bool has(const String & entity_name) const override; std::vector getAllEntityNames() const override; + std::vector getAllEntityNames(WorkloadEntityType entity_type) const override; std::vector> getAllEntities() const override; diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp new file mode 100644 index 00000000000..dad2750d8c0 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -0,0 +1,48 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemWorkloads::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the workload."}, + {"parent", std::make_shared(), "The name of the parent workload."}, + {"create_query", std::make_shared(), "CREATE query of the workload."}, + }; +} + +void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & workload_names = storage.getAllEntityNames(WorkloadEntityType::Workload); + for (const auto & workload_name : workload_names) + { + auto ast = storage.get(workload_name); + auto & workload = typeid_cast(*ast); + res_columns[0]->insert(workload_name); + res_columns[1]->insert(workload.getWorkloadParent()); + res_columns[2]->insert(queryToString(ast)); + } +} + +void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for workloads and resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for workloads and resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemWorkloads.h b/src/Storages/System/StorageSystemWorkloads.h new file mode 100644 index 00000000000..9d4770a02b8 --- /dev/null +++ b/src/Storages/System/StorageSystemWorkloads.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `workloads` system table, which allows you to get a list of all workloads +class StorageSystemWorkloads final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemWorkloads"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 816ba5095b1..728e83135a3 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -229,6 +230,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "s3queue", "Contains in-memory state of S3Queue metadata and currently processed rows per file."); attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); + attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); if (has_zookeeper) { diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference new file mode 100644 index 00000000000..4bac2ef71f2 --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -0,0 +1,5 @@ +all CREATE WORKLOAD `all` +development all CREATE WORKLOAD development IN `all` +production all CREATE WORKLOAD production IN `all` +all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql new file mode 100644 index 00000000000..38a7dad7cbc --- /dev/null +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -0,0 +1,11 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +CREATE OR REPLACE WORKLOAD all; +SELECT name, parent, create_query FROM system.workloads; +CREATE WORKLOAD IF NOT EXISTS production IN all; +CREATE WORKLOAD development IN all; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD IF EXISTS production; +DROP WORKLOAD development; +SELECT name, parent, create_query FROM system.workloads; +DROP WORKLOAD all; From 9edc66d458d2e9376ed52582dc25ab7934ea9085 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 6 Sep 2024 19:22:59 +0000 Subject: [PATCH 014/157] simplify scheduler constraints --- src/Common/Scheduler/ISchedulerConstraint.h | 25 +------------- .../Scheduler/Nodes/SemaphoreConstraint.h | 9 +---- .../Scheduler/Nodes/ThrottlerConstraint.h | 11 ++----- src/Common/Scheduler/ResourceRequest.cpp | 33 +++++++++++++++++-- src/Common/Scheduler/ResourceRequest.h | 18 +++++++--- 5 files changed, 49 insertions(+), 47 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index a976206de74..754f6dd404f 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -15,8 +15,7 @@ namespace DB * When constraint is again satisfied, scheduleActivation() is called from finishRequest(). * * Derived class behaviour requirements: - * - dequeueRequest() must fill `request->constraint` iff it is nullptr; - * - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`. + * - dequeueRequest() must call `request->addConstraint()`. */ class ISchedulerConstraint : public ISchedulerNode { @@ -29,30 +28,8 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; - void setParent(ISchedulerNode * parent_) override - { - ISchedulerNode::setParent(parent_); - - // Assign `parent_constraint` to the nearest parent derived from ISchedulerConstraint - for (ISchedulerNode * node = parent_; node != nullptr; node = node->parent) - { - if (auto * constraint = dynamic_cast(node)) - { - parent_constraint = constraint; - break; - } - } - } - /// For introspection of current state (true = satisfied, false = violated) virtual bool isSatisfied() = 0; - -protected: - // Reference to nearest parent that is also derived from ISchedulerConstraint. - // Request can traverse through multiple constraints while being dequeue from hierarchy, - // while finishing request should traverse the same chain in reverse order. - // NOTE: it must be immutable after initialization, because it is accessed in not thread-safe way from finishRequest() - ISchedulerConstraint * parent_constraint = nullptr; }; } diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 92c6af9db18..46b048ce34c 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -69,10 +69,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + request->addConstraint(this); // Update state on request arrival std::unique_lock lock(mutex); @@ -87,10 +84,6 @@ public: void finishRequest(ResourceRequest * request) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // Update state on request departure std::unique_lock lock(mutex); bool was_active = active(); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 56866336f50..4e2faa6b233 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -79,10 +79,7 @@ public: if (!request) return {nullptr, false}; - // Request has reference to the first (closest to leaf) `constraint`, which can have `parent_constraint`. - // The former is initialized here dynamically and the latter is initialized once during hierarchy construction. - if (!request->constraint) - request->constraint = this; + // We don't do `request->addConstraint(this)` because `finishRequest()` is no-op updateBucket(request->cost); @@ -93,12 +90,8 @@ public: return {request, active()}; } - void finishRequest(ResourceRequest * request) override + void finishRequest(ResourceRequest *) override { - // Recursive traverse of parent flow controls in reverse order - if (parent_constraint) - parent_constraint->finishRequest(request); - // NOTE: Token-bucket constraint does not require any action when consumption ends } diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 26e8084cdfa..91394108f5d 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -1,13 +1,42 @@ #include #include +#include + +#include + namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + void ResourceRequest::finish() { - if (constraint) - constraint->finishRequest(this); + // Iterate over constraints in reverse order + for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints)) + { + if (constraint) + constraint->finishRequest(this); + } +} + +void ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +{ + for (auto & constraint : constraints) + { + if (!constraint) + { + constraint = new_constraint; + return; + } + } + // TODO(serxa): is it possible to validate it during enqueue of resource request to avoid LOGICAL_ERRORs in the scheduler thread? possible but will not cover case of moving queue with requests inside to invalid position + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Max number of simultaneous workload constraints exceeded ({}). Remove extra constraints before using this workload.", + ResourceMaxConstraints); } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d64f624cec5..635353b569b 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -15,6 +16,10 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); +// TODO(serxa): validate hierarchy to avoid too many constrants +/// Max number of constraints for a request to pass though (depth of constaints chain) +constexpr size_t ResourceMaxConstraints = 8; + /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -49,9 +54,10 @@ public: /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; - /// Scheduler node to be notified on consumption finish - /// Auto-filled during request enqueue/dequeue - ISchedulerConstraint * constraint; + /// Scheduler nodes to be notified on consumption finish + /// Auto-filled during request dequeue + /// Vector is not used to avoid allocations in the scheduler thread + std::array constraints; explicit ResourceRequest(ResourceCost cost_ = 1) { @@ -62,7 +68,8 @@ public: void reset(ResourceCost cost_) { cost = cost_; - constraint = nullptr; + for (auto & constraint : constraints) + constraint = nullptr; // Note that list_base_hook should be reset independently (by intrusive list) } @@ -79,6 +86,9 @@ public: /// ResourceRequest should not be destructed or reset before calling to `finish()`. /// WARNING: this function MUST not be called if request was canceled. void finish(); + + /// Is called from the scheduler thread to fill `constraints` chain + void addConstraint(ISchedulerConstraint * new_constraint); }; } From 14542d6779652c7c0b78efca3fa74fb6ae4a66f6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:26:54 +0000 Subject: [PATCH 015/157] added main building block UnifiedSchedulerNode --- src/Common/Priority.h | 5 +- src/Common/Scheduler/ISchedulerConstraint.h | 4 + src/Common/Scheduler/ISchedulerNode.h | 29 +- src/Common/Scheduler/ISchedulerQueue.h | 4 + src/Common/Scheduler/Nodes/FairPolicy.h | 4 + src/Common/Scheduler/Nodes/FifoQueue.h | 4 + src/Common/Scheduler/Nodes/PriorityPolicy.h | 4 + .../Scheduler/Nodes/SemaphoreConstraint.h | 7 + .../Scheduler/Nodes/ThrottlerConstraint.h | 8 + .../Scheduler/Nodes/UnifiedSchedulerNode.h | 346 ++++++++++++++++++ src/Common/Scheduler/SchedulingSettings.h | 38 ++ 11 files changed, 445 insertions(+), 8 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h create mode 100644 src/Common/Scheduler/SchedulingSettings.h diff --git a/src/Common/Priority.h b/src/Common/Priority.h index 8952fe4dd5a..f0e5787ae91 100644 --- a/src/Common/Priority.h +++ b/src/Common/Priority.h @@ -6,6 +6,7 @@ /// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize struct Priority { - Int64 value = 0; /// Note that lower value means higher priority. - constexpr operator Int64() const { return value; } /// NOLINT + using Value = Int64; + Value value = 0; /// Note that lower value means higher priority. + constexpr operator Value() const { return value; } /// NOLINT }; diff --git a/src/Common/Scheduler/ISchedulerConstraint.h b/src/Common/Scheduler/ISchedulerConstraint.h index 754f6dd404f..3bee9c1b424 100644 --- a/src/Common/Scheduler/ISchedulerConstraint.h +++ b/src/Common/Scheduler/ISchedulerConstraint.h @@ -24,6 +24,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + /// Resource consumption by `request` is finished. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void finishRequest(ResourceRequest * request) = 0; diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index c051829e336..6d3132f79c1 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -57,7 +57,13 @@ struct SchedulerNodeInfo SchedulerNodeInfo() = default; - explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + explicit SchedulerNodeInfo(double weight_, Priority priority_ = {}) + { + setWeight(weight_); + setPriority(priority_); + } + + explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config, const String & config_prefix = {}) { setWeight(config.getDouble(config_prefix + ".weight", weight)); setPriority(config.getInt64(config_prefix + ".priority", priority)); @@ -78,6 +84,11 @@ struct SchedulerNodeInfo priority.value = value; } + void setPriority(Priority value) + { + priority = value; + } + // To check if configuration update required bool equals(const SchedulerNodeInfo & o) const { @@ -123,6 +134,11 @@ public: , info(config, config_prefix) {} + ISchedulerNode(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : event_queue(event_queue_) + , info(info_) + {} + virtual ~ISchedulerNode() = default; /// Checks if two nodes configuration is equal @@ -134,10 +150,11 @@ public: /// Attach new child virtual void attachChild(const std::shared_ptr & child) = 0; - /// Detach and destroy child + /// Detach child + /// NOTE: child might be destroyed if the only reference was stored in parent virtual void removeChild(ISchedulerNode * child) = 0; - /// Get attached child by name + /// Get attached child by name (for tests only) virtual ISchedulerNode * getChild(const String & child_name) = 0; /// Activation of child due to the first pending request @@ -147,7 +164,7 @@ public: /// Returns true iff node is active virtual bool isActive() = 0; - /// Returns number of active children + /// Returns number of active children (for introspection only). virtual size_t activeChildren() = 0; /// Returns the first request to be executed as the first component of resulting pair. @@ -155,10 +172,10 @@ public: virtual std::pair dequeueRequest() = 0; /// Returns full path string using names of every parent - String getPath() + String getPath() const { String result; - ISchedulerNode * ptr = this; + const ISchedulerNode * ptr = this; while (ptr->parent) { result = "/" + ptr->basename + result; diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index 532f4bf6c63..e816050a50e 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -21,6 +21,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + ISchedulerQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + // Wrapper for `enqueueRequest()` that should be used to account for available resource budget void enqueueRequestUsingBudget(ResourceRequest * request) { diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index fba637e979e..b6be26bea98 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -48,6 +48,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + FairPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerNode(event_queue_, info_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 9fbc6d1ae65..49f3e268bc8 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -30,6 +30,10 @@ public: : ISchedulerQueue(event_queue_, config, config_prefix) {} + FifoQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_) + : ISchedulerQueue(event_queue_, info_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 91dc95600d5..17fcbfd3139 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -39,6 +39,10 @@ public: : ISchedulerNode(event_queue_, config, config_prefix) {} + explicit PriorityPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & node_info) + : ISchedulerNode(event_queue_, node_info) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 46b048ce34c..a2d8df48065 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/Scheduler/ISchedulerNode.h" #include #include @@ -24,6 +25,12 @@ public: , max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost))) {} + SemaphoreConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, Int64 max_requests_, Int64 max_cost_) + : ISchedulerConstraint(event_queue_, info_) + , max_requests(max_requests_) + , max_cost(max_cost_) + {} + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 4e2faa6b233..7c64dd51ac1 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -28,6 +28,14 @@ public: , tokens(max_burst) {} + ThrottlerConstraint(EventQueue * event_queue_, const SchedulerNodeInfo & info_, double max_speed_, double max_burst_) + : ISchedulerConstraint(event_queue_, info_) + , max_speed(max_speed_) + , max_burst(max_burst_) + , last_update(event_queue_->now()) + , tokens(max_burst) + {} + ~ThrottlerConstraint() override { // We should cancel event on destruction to avoid dangling references from event queue diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h new file mode 100644 index 00000000000..46ea5f0f340 --- /dev/null +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -0,0 +1,346 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +/* + * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. + * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes + * are also present. This approch is easiers for manipulations in runtime than using multiple types of nodes. + * + * Unified node is capable of updating its internal structure based on: + * 1. Number of children (fifo if =0 or fairness/priority if >0). + * 2. Priorities of its children (for subtree structure). + * 3. `SchedulingSettings` associated with unified node (for throttler and semaphore constraints). + * + * In general, unified node has "internal" subtree with the following structure: + * + * THIS <-- UnifiedSchedulerNode object + * | + * THROTTLER <-- [Optional] Throttling scheduling constraint + * | + * [If no children]------ SEMAPHORE <-- [Optional] Semaphore constraint + * | | + * FIFO PRIORITY <-- [Optional] Scheduling policy distinguishing priorities + * .-------' '-------. + * FAIRNESS[p1] ... FAIRNESS[pN] <-- [Optional] Policies for fairness if priorities are equal + * / \ / \ + * CHILD[p1,w1] ... CHILD[p1,wM] CHILD[pN,w1] ... CHILD[pN,wM] <-- Unified children (UnifiedSchedulerNode objects) + * + * NOTE: to distinguish different kinds of children we use the following terms: + * - immediate child: child of unified object (THROTTLER); + * - unified child: leaf of this "internal" subtree (CHILD[p,w]); + * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) + */ +class UnifiedSchedulerNode : public ISchedulerNode +{ +private: + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, const SchedulerNodePtr & new_parent) + { + reparent(node, new_parent.get()); + } + + /// Helper function for managing a parent of a node + static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) + { + if (!new_parent || new_parent == node->parent) + return; + if (node->parent) + node->parent->removeChild(node.get()); + new_parent->attachChild(node); + } + + /// A branch of the tree for a specific priority value + struct FairnessBranch { + SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached + std::unordered_map children; // basename -> child + + SchedulerNodePtr getRoot() + { + chassert(!children.empty()); + if (root) + return root; + return children.begin()->second; // There should be exactly one child + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) + throw Exception( + ErrorCodes::INVALID_SCHEDULER_NODE, + "Can't add another child with the same path: {}", + it->second->getPath()); + + if (children.size() == 2) + { + // Insert fair node if we have just added the second child + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->info.setPriority(child->info.priority); + root->basename = fmt::format("p{}_fair", child->info.priority.value); + for (auto & [_, node] : children) + reparent(node, root); + return root; // New root has been created + } + else if (children.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + }; + + /// Handles all the children nodes with intermediate fair and/or priority nodes + struct ChildrenBranch + { + SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached + std::unordered_map branches; /// Branches for different priority values + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + bool existing_branch = branches.contains(child->info.priority); + auto & child_branch = branches[child->info.priority]; + auto branch_root = child_branch.attachUnifiedChild(event_queue_, child); + + if (existing_branch) + { + if (branch_root) + reparent(branch_root, root); + return {}; + } + else + { + chassert(branch_root); + if (branches.size() == 2) + { + // Insert priority node if we have just added the second branch + chassert(!root); + root = std::make_shared(event_queue_, SchedulerNodeInfo{}); + root->basename = "prio"; + for (auto & [_, branch] : branches) + reparent(branch.getRoot(), root); + return root; // New root has been created + } + else if (branches.size() == 1) + return child; // We have added single child so far and it is the new root + else + reparent(child, root); + return {}; // Root is the same + } + } + }; + + /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. + struct QueueOrChildrenBranch + { + SchedulerNodePtr queue; /// FifoQueue node is used if there are no children + ChildrenBranch branch; /// Used if there is at least one child + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) + { + createQueue(event_queue_); + return queue; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (queue) + removeQueue(event_queue_); + return branch.attachUnifiedChild(event_queue_, child); + } + + private: + void createQueue(EventQueue * event_queue_) + { + queue = std::make_shared(event_queue_, SchedulerNodeInfo{}); + queue->basename = "fifo"; + } + + void removeQueue(EventQueue *) + { + // TODO(serxa): cancel all requests, this unified node is not capable of service resoruce requests now + queue.reset(); + } + }; + + /// Handles all the nodes under this unified node + /// Specifically handles constraints with `QueueOrChildrenBranch` under it + struct ConstraintsBranch + { + SchedulerNodePtr throttler; + SchedulerNodePtr semaphore; + QueueOrChildrenBranch branch; + SchedulingSettings settings; + + // Should be called after constructor, before any other methods + [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_, const SchedulingSettings & settings_) + { + settings = settings_; + SchedulerNodePtr node = branch.initialize(event_queue_); + if (settings.hasSemaphore()) + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_requests, settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + if (settings.hasThrottler()) + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, settings.max_speed, settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + return node; + } + + /// Attaches a new child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + { + if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) + { + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } + }; + +public: + explicit UnifiedSchedulerNode(EventQueue * event_queue_, const SchedulingSettings & settings) + : ISchedulerNode(event_queue_, SchedulerNodeInfo(settings.weight, settings.priority)) + { + immediate_child = impl.initialize(event_queue, settings); + reparent(immediate_child, this); + } + + bool equals(ISchedulerNode *) override + { + assert(false); + return false; + } + + /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate node + /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children + void attachUnifiedChild(const SchedulerNodePtr & child) + { + if (auto new_child = impl.attachUnifiedChild(event_queue, child)) + reparent(new_child, this); + } + + /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) + /// NOTE: Changing a priority of a unified child may lead to change of its parent. + void updateUnifiedChildPriority(const SchedulerNodePtr & child, Priority old_priority, Priority new_priority) + { + UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority + } + + /// Updates scheduling settings. Set of constraints might change. + /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) + void updateSchedulingSettings(const SchedulingSettings & new_settings) + { + UNUSED(new_settings); // TODO: implement updateSchedulingSettings + } + + /// Attaches an immediate child (used through `reparent()`) + void attachChild(const SchedulerNodePtr & child_) override + { + immediate_child = child_; + immediate_child->setParent(this); + + // Activate if required + if (immediate_child->isActive()) + activateChild(immediate_child.get()); + } + + /// Removes an immediate child (used through `reparent()`) + void removeChild(ISchedulerNode * child) override + { + if (immediate_child.get() == child) + { + child_active = false; // deactivate + immediate_child->setParent(nullptr); // detach + immediate_child.reset(); + } + } + + ISchedulerNode * getChild(const String & child_name) override + { + if (immediate_child->basename == child_name) + return immediate_child.get(); + else + return nullptr; + } + + std::pair dequeueRequest() override + { + auto [request, child_now_active] = immediate_child->dequeueRequest(); + if (!request) + return {nullptr, false}; + + child_active = child_now_active; + if (!child_active) + busy_periods++; + incrementDequeued(request->cost); + return {request, child_active}; + } + + bool isActive() override + { + return child_active; + } + + /// Shows number of immediate active children (for introspection) + size_t activeChildren() override + { + return child_active; + } + + /// Activate an immediate child + void activateChild(ISchedulerNode * child) override + { + if (child == immediate_child.get()) + if (!std::exchange(child_active, true) && parent) + parent->activateChild(this); + } + +private: + ConstraintsBranch impl; + SchedulerNodePtr immediate_child; // An immediate child (actually the root of the whole subtree) + bool child_active = false; +}; + +} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h new file mode 100644 index 00000000000..4c6eff2b1e9 --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +#include + +#include + +namespace DB +{ + +struct SchedulingSettings +{ + /// Priority and weight among siblings + double weight = 1.0; + Priority priority; + + /// Throttling constraints. + /// Up to 2 independent throttlers: one for average speed and one for peek speed. + static constexpr double default_burst_seconds = 1.0; + double max_speed = 0; // Zero means unlimited + double max_burst = 0; // default is `default_burst_seconds * max_speed` + + /// Limits total number of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_requests = std::numeric_limits::max(); + Int64 max_requests = default_max_requests; + + /// Limits total cost of concurrent resource requests that are allowed to consume + static constexpr Int64 default_max_cost = std::numeric_limits::max(); + Int64 max_cost = default_max_cost; + + bool hasThrottler() const { return max_speed != 0; } + bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } + + // TODO(serxa): add helper functions for parsing, printing and validating +}; + +} From 7bf7b516a753dd106bfb5d56da71eb814775274e Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 10 Sep 2024 11:27:54 +0000 Subject: [PATCH 016/157] add smoke test for UnifiedSchedulerNode --- .../Scheduler/Nodes/tests/ResourceTest.h | 19 ++++++++++++++-- .../tests/gtest_unified_scheduler_node.cpp | 22 +++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index ea3f9edf765..6583e2beb0f 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -14,10 +14,12 @@ #include #include +#include #include #include #include #include +#include namespace DB { @@ -37,10 +39,17 @@ struct ResourceTestBase Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)}; String config_prefix = "node"; + return add(event_queue, root_node, path, std::ref(*config), config_prefix); + } + + template + static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) + { + if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); - root_node.reset(new TClass(event_queue, *config, config_prefix)); + root_node.reset(new TClass(event_queue, std::forward(args)...)); return static_cast(root_node.get()); } @@ -65,7 +74,7 @@ struct ResourceTestBase } EXPECT_TRUE(!child_name.empty()); // wrong path - SchedulerNodePtr node = std::make_shared(event_queue, *config, config_prefix); + SchedulerNodePtr node = std::make_shared(event_queue, std::forward(args)...); node->basename = child_name; parent->attachChild(node); return static_cast(node.get()); @@ -126,6 +135,12 @@ public: ResourceTestBase::add(&event_queue, root_node, path, xml); } + template + void addCustom(const String & path, Args... args) + { + ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); + } + void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp new file mode 100644 index 00000000000..2acda88ef17 --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -0,0 +1,22 @@ +#include +#include + +#include + +#include +#include + +using namespace DB; + +using ResourceTest = ResourceTestClass; + +TEST(SchedulerUnifiedNode, Smoke) +{ + ResourceTest t; + + t.addCustom("/", SchedulingSettings{}); + + t.enqueue("/fifo", {10, 10}); + t.dequeue(2); + t.consumed("fifo", 20); +} From 4f70f48272444a07514c42268862a952dae29e49 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 11 Sep 2024 19:29:53 +0000 Subject: [PATCH 017/157] add more tests --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 33 ++++++++--- .../Scheduler/Nodes/tests/ResourceTest.h | 39 ++++++++++-- .../tests/gtest_unified_scheduler_node.cpp | 59 +++++++++++++++++++ 3 files changed, 117 insertions(+), 14 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 46ea5f0f340..85b22b02cfa 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -64,7 +64,8 @@ private: /// Helper function for managing a parent of a node static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) { - if (!new_parent || new_parent == node->parent) + chassert(new_parent); + if (new_parent == node->parent) return; if (node->parent) node->parent->removeChild(node.get()); @@ -130,7 +131,12 @@ private: if (existing_branch) { if (branch_root) - reparent(branch_root, root); + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } return {}; } else @@ -247,13 +253,7 @@ public: reparent(immediate_child, this); } - bool equals(ISchedulerNode *) override - { - assert(false); - return false; - } - - /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate node + /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const SchedulerNodePtr & child) { @@ -275,6 +275,19 @@ public: UNUSED(new_settings); // TODO: implement updateSchedulingSettings } + /// Returns the queue to be used for resource requests or `nullptr` if it has unified children + ISchedulerQueue * getQueue() + { + return static_cast(impl.branch.queue.get()); + } + +protected: // Hide all the ISchedulerNode interface methods as an implementation details + bool equals(ISchedulerNode *) override + { + assert(false); + return false; + } + /// Attaches an immediate child (used through `reparent()`) void attachChild(const SchedulerNodePtr & child_) override { @@ -343,4 +356,6 @@ private: bool child_active = false; }; +using UnifiedSchedulerNodePtr = std::shared_ptr; + } diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 6583e2beb0f..4adc0ae7028 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/Scheduler/SchedulingSettings.h" #include #include #include @@ -7,6 +8,7 @@ #include #include #include +#include #include #include @@ -15,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -45,7 +48,6 @@ struct ResourceTestBase template static TClass * add(EventQueue * event_queue, SchedulerNodePtr & root_node, const String & path, Args... args) { - if (path == "/") { EXPECT_TRUE(root_node.get() == nullptr); @@ -141,6 +143,32 @@ public: ResourceTestBase::add(&event_queue, root_node, path, std::forward(args)...); } + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const SchedulingSettings & settings = {}) + { + return createUnifiedNode(basename, {}, settings); + } + + UnifiedSchedulerNodePtr createUnifiedNode(const String & basename, const UnifiedSchedulerNodePtr & parent, const SchedulingSettings & settings = {}) + { + auto node = std::make_shared(&event_queue, settings); + node->basename = basename; + if (parent) + { + parent->attachUnifiedChild(node); + } + else + { + EXPECT_TRUE(root_node.get() == nullptr); + root_node = node; + } + return node; + } + + void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) + { + enqueueImpl(node->getQueue(), costs); + } + void enqueue(const String & path, const std::vector & costs) { ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first @@ -161,13 +189,14 @@ public: pos = String::npos; } } - ISchedulerQueue * queue = dynamic_cast(node); - ASSERT_TRUE(queue != nullptr); // not a queue + enqueueImpl(dynamic_cast(node), costs); + } + void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs) + { + ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - { queue->enqueueRequest(new Request(cost, queue->basename)); - } processEvents(); // to activate queues } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 2acda88ef17..41a5c7f1036 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,6 +1,7 @@ #include #include +#include "Common/Priority.h" #include #include @@ -20,3 +21,61 @@ TEST(SchedulerUnifiedNode, Smoke) t.dequeue(2); t.consumed("fifo", 20); } + +TEST(SchedulerUnifiedNode, Fairness) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.dequeue(); + t.consumed("A", 60); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, Priority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10}); + t.enqueue(b, {10, 10, 10}); + t.enqueue(c, {10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 0); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.dequeue(); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 0); +} From 4401b4dda087ee323871fa2cb5da929c3322e26c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:06:39 +0000 Subject: [PATCH 018/157] fix request naming for tests --- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 4adc0ae7028..762cb64a307 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -166,7 +166,7 @@ public: void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { - enqueueImpl(node->getQueue(), costs); + enqueueImpl(node->getQueue(), costs, node->basename); } void enqueue(const String & path, const std::vector & costs) @@ -192,11 +192,11 @@ public: enqueueImpl(dynamic_cast(node), costs); } - void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs) + void enqueueImpl(ISchedulerQueue * queue, const std::vector & costs, const String & name = {}) { ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - queue->enqueueRequest(new Request(cost, queue->basename)); + queue->enqueueRequest(new Request(cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues } From 86515e1bce1bb1bb7c3829619fb673713d96723c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:27:32 +0000 Subject: [PATCH 019/157] add more tests for fairness and priority --- .../tests/gtest_unified_scheduler_node.cpp | 224 +++++++++++++++++- 1 file changed, 223 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 41a5c7f1036..92c616ff65c 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -22,7 +22,7 @@ TEST(SchedulerUnifiedNode, Smoke) t.consumed("fifo", 20); } -TEST(SchedulerUnifiedNode, Fairness) +TEST(SchedulerUnifiedNode, FairnessWeight) { ResourceTest t; @@ -46,6 +46,152 @@ TEST(SchedulerUnifiedNode, Fairness) t.consumed("B", 20); } +TEST(SchedulerUnifiedNode, FairnessActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + auto c = t.createUnifiedNode("C", all); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 10); + t.consumed("B", 10); + t.consumed("C", 10); + + t.dequeue(4); + t.consumed("A", 30); + t.consumed("B", 0); + t.consumed("C", 10); + + t.enqueue(b, {10, 10}); + t.dequeue(1); + t.consumed("B", 10); + + t.enqueue(c, {10, 10}); + t.dequeue(1); + t.consumed("C", 10); + + t.dequeue(2); // A B or B A + t.consumed("A", 10); + t.consumed("B", 10); +} + +TEST(SchedulerUnifiedNode, FairnessMaxMin) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all); + auto b = t.createUnifiedNode("B", all); + + t.enqueue(a, {10, 10}); // make sure A is never empty + + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + + t.dequeue(2); + t.consumed("A", 20); +} + +TEST(SchedulerUnifiedNode, FairnessHierarchical) +{ + ResourceTest t; + + + auto all = t.createUnifiedNode("all"); + auto x = t.createUnifiedNode("X", all); + auto y = t.createUnifiedNode("Y", all); + auto a = t.createUnifiedNode("A", x); + auto b = t.createUnifiedNode("B", x); + auto c = t.createUnifiedNode("C", y); + auto d = t.createUnifiedNode("D", y); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("B", 40); + t.consumed("C", 20); + t.consumed("D", 20); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(c, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("C", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 20); + t.consumed("B", 20); + t.consumed("D", 40); + } + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(d, {10, 10, 10, 10, 10, 10, 10, 10}); + for (int i = 0; i < 4; i++) + { + t.dequeue(8); + t.consumed("A", 40); + t.consumed("D", 40); + } +} + TEST(SchedulerUnifiedNode, Priority) { ResourceTest t; @@ -79,3 +225,79 @@ TEST(SchedulerUnifiedNode, Priority) t.consumed("B", 0); t.consumed("C", 0); } + +TEST(SchedulerUnifiedNode, PriorityActivation) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.priority = Priority{3}}); + auto b = t.createUnifiedNode("B", all, {.priority = Priority{2}}); + auto c = t.createUnifiedNode("C", all, {.priority = Priority{1}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10}); + t.enqueue(c, {10, 10}); + + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); + + t.enqueue(b, {10, 10, 10}); + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + t.consumed("C", 0); + + t.enqueue(c, {10, 10}); + t.dequeue(3); + t.consumed("A", 0); + t.consumed("B", 10); + t.consumed("C", 20); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + t.consumed("C", 0); +} + +TEST(SchedulerUnifiedNode, List) +{ + ResourceTest t; + + std::list list; + list.push_back(t.createUnifiedNode("all")); + + for (int length = 1; length < 5; length++) + { + String name = fmt::format("L{}", length); + list.push_back(t.createUnifiedNode(name, list.back())); + + for (int i = 0; i < 3; i++) + { + t.enqueue(list.back(), {10, 10}); + t.dequeue(1); + t.consumed(name, 10); + + for (int j = 0; j < 3; j++) + { + t.enqueue(list.back(), {10, 10, 10}); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + t.dequeue(1); + t.consumed(name, 10); + } + + t.dequeue(1); + t.consumed(name, 10); + } + } +} From 3ff86a4347741335e7c7b163e13eadbd7ec24107 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 16:43:48 +0000 Subject: [PATCH 020/157] add tests for max_speed and max_bust --- .../tests/gtest_unified_scheduler_node.cpp | 121 +++++++++++++++++- 1 file changed, 120 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 92c616ff65c..bddfeb19851 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,9 +1,9 @@ #include #include -#include "Common/Priority.h" #include +#include #include #include @@ -301,3 +301,122 @@ TEST(SchedulerUnifiedNode, List) } } } + +TEST(SchedulerUnifiedNode, ThrottlerLeakyBucket) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(3)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(4)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, ThrottlerPacing) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + // Zero burst allows you to send one request of any `size` and than throttle for `size/max_speed` seconds. + // Useful if outgoing traffic should be "paced", i.e. have the least possible burstiness. + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 1.0, .max_burst = 0.0}); + + t.enqueue(all, {1, 2, 3, 1, 2, 1}); + int output[] = {1, 2, 0, 3, 0, 0, 1, 2, 0, 1, 0}; + for (int i = 0; i < std::size(output); i++) + { + t.process(start + std::chrono::seconds(i)); + t.consumed("all", output[i]); + } +} + +TEST(SchedulerUnifiedNode, ThrottlerBucketFilling) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 60); // 5 sec * 10 tokens/sec = 50 tokens + 1 extra request to go below zero + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 40); // Consume rest + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {95, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 101); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} + +TEST(SchedulerUnifiedNode, ThrottlerAndFairness) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + auto a = t.createUnifiedNode("A", all, {.weight = 10.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 90.0, .priority = Priority{}}); + + ResourceCost req_cost = 1; + ResourceCost total_cost = 2000; + for (int i = 0; i < total_cost / req_cost; i++) + { + t.enqueue(a, {req_cost}); + t.enqueue(b, {req_cost}); + } + + double shareA = 0.1; + double shareB = 0.9; + + // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share + auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); + auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + + double consumedA = 0; + double consumedB = 0; + for (int seconds = 0; seconds < 100; seconds++) + { + t.process(start + std::chrono::seconds(seconds)); + double arrival_curve = 100.0 + 10.0 * seconds + req_cost; + t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); + t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); + consumedA = arrival_curve * shareA; + consumedB = arrival_curve * shareB; + } +} From ca1567da0311a58d7ba0217ca3fbb5d3fcf806f5 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 18:01:22 +0000 Subject: [PATCH 021/157] abort resource requests in queue dtor --- src/Common/Scheduler/Nodes/FifoQueue.h | 11 ++++++++++ .../Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- .../Scheduler/Nodes/tests/ResourceTest.h | 7 +++++++ .../Nodes/tests/gtest_resource_scheduler.cpp | 5 +++++ src/Common/Scheduler/ResourceGuard.h | 20 +++++++++++++++++++ src/Common/Scheduler/ResourceRequest.h | 6 +++++- 6 files changed, 50 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 49f3e268bc8..b3c8bbcffbf 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -34,6 +34,17 @@ public: : ISchedulerQueue(event_queue_, info_) {} + ~FifoQueue() override + { + while (!requests.empty()) + { + ResourceRequest * request = &requests.front(); + requests.pop_front(); + request->failed(std::make_exception_ptr( + Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request was destructed"))); + } + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 85b22b02cfa..4bdcaca3cb1 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -12,7 +12,6 @@ #include #include -#include namespace DB { @@ -192,7 +191,8 @@ private: void removeQueue(EventQueue *) { - // TODO(serxa): cancel all requests, this unified node is not capable of service resoruce requests now + // This unified node will not be able to process resource requests any longer + // All remaining resource requests are be aborted on queue destruction queue.reset(); } }; diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 9bf70e42e1a..0e246ed2273 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -119,6 +120,7 @@ class ResourceTestClass : public ResourceTestBase struct Request : public ResourceRequest { String name; + std::exception_ptr exception; Request(ResourceCost cost_, const String & name_) : ResourceRequest(cost_) @@ -128,6 +130,11 @@ class ResourceTestClass : public ResourceTestBase void execute() override { } + + void failed(const std::exception_ptr & ptr) override + { + exception = ptr; + } }; public: diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ddfe0cfbc6f..8eaa4ebb840 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -101,6 +101,11 @@ struct MyRequest : public ResourceRequest if (on_execute) on_execute(); } + + void failed(const std::exception_ptr &) override + { + FAIL(); + } }; TEST(SchedulerRoot, Smoke) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index cf97f7acf93..6ff22edd221 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -12,6 +12,7 @@ #include #include +#include #include @@ -34,6 +35,11 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor @@ -109,12 +115,25 @@ public: dequeued_cv.notify_one(); } + // This function is executed inside scheduler thread and wakes thread issued this `request`. + // That thread will throw an exception. + void failed(const std::exception_ptr & ptr) override + { + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + exception = ptr; + dequeued_cv.notify_one(); + } + void wait() { CurrentMetrics::Increment scheduled(metrics->scheduled_count); auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); + if (exception) + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Resource request failed: {}", getExceptionMessage(exception, /* with_stacktrace = */ false)); } void finish(ResourceCost real_cost_, ResourceLink link_) @@ -151,6 +170,7 @@ public: std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; + std::exception_ptr exception; }; /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d394459819e..24afcc98b57 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -81,10 +82,13 @@ public: /// (e.g. setting an std::promise or creating a job in a thread pool) virtual void execute() = 0; + /// Callback to trigger an error in case if resource is unavailable. + virtual void failed(const std::exception_ptr & ptr) = 0; + /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// WARNING: this function MUST not be called if request was canceled. + /// WARNING: this function MUST not be called if request was canceled or failed. void finish(); /// Is called from the scheduler thread to fill `constraints` chain From 7acc10444cee1a6d4ef5079f5a682e3102fc1535 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 18:20:53 +0000 Subject: [PATCH 022/157] add test for queue destruction --- .../tests/gtest_unified_scheduler_node.cpp | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index bddfeb19851..24a8950d9ae 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -420,3 +420,31 @@ TEST(SchedulerUnifiedNode, ThrottlerAndFairness) consumedB = arrival_curve * shareB; } } + +TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + // This will destory the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} From 6307ada396541f82b143c1a19691fd0589fda32b Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 12 Sep 2024 19:58:31 +0000 Subject: [PATCH 023/157] add purgeQueue() with test --- src/Common/Scheduler/ISchedulerQueue.h | 5 +++ src/Common/Scheduler/Nodes/FifoQueue.h | 26 ++++++++--- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 14 +++++- .../Scheduler/Nodes/tests/ResourceTest.h | 26 ++++++++--- .../tests/gtest_unified_scheduler_node.cpp | 45 +++++++++++++++++++ 5 files changed, 101 insertions(+), 15 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index e07f797cb42..6c77cee6b9d 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -51,6 +51,11 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual bool cancelRequest(ResourceRequest * request) = 0; + /// Fails all the resource requests in queue and marks this queue as not usable. + /// Afterwards any new request will be failed on `enqueueRequest()`. + /// NOTE: This is done for queues that are about to be destructed. + virtual void purgeQueue() = 0; + /// For introspection ResourceCost getBudget() const { diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index b3c8bbcffbf..c95125b21bf 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -36,13 +36,7 @@ public: ~FifoQueue() override { - while (!requests.empty()) - { - ResourceRequest * request = &requests.front(); - requests.pop_front(); - request->failed(std::make_exception_ptr( - Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request was destructed"))); - } + chassert(requests.empty()); } bool equals(ISchedulerNode * other) override @@ -57,6 +51,8 @@ public: void enqueueRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue is about to be destructed"); queue_cost += request->cost; bool was_empty = requests.empty(); requests.push_back(*request); @@ -81,6 +77,8 @@ public: bool cancelRequest(ResourceRequest * request) override { std::lock_guard lock(mutex); + if (is_not_usable) + return false; // Any request should already be failed or executed if (request->is_linked()) { // It's impossible to check that `request` is indeed inserted to this queue and not another queue. @@ -103,6 +101,19 @@ public: return false; } + void purgeQueue() override + { + std::lock_guard lock(mutex); + is_not_usable = true; + while (!requests.empty()) + { + ResourceRequest * request = &requests.front(); + requests.pop_front(); + request->failed(std::make_exception_ptr( + Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Scheduler queue with resource request is about to be destructed"))); + } + } + bool isActive() override { std::lock_guard lock(mutex); @@ -146,6 +157,7 @@ private: std::mutex mutex; Int64 queue_cost = 0; boost::intrusive::list requests; + bool is_not_usable = false; }; } diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 4bdcaca3cb1..fa284ed5254 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -71,6 +72,13 @@ private: new_parent->attachChild(node); } + /// Helper function for managing a parent of a node + static void detach(const SchedulerNodePtr & node) + { + if (node->parent) + node->parent->removeChild(node.get()); + } + /// A branch of the tree for a specific priority value struct FairnessBranch { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached @@ -193,6 +201,8 @@ private: { // This unified node will not be able to process resource requests any longer // All remaining resource requests are be aborted on queue destruction + detach(queue); + std::static_pointer_cast(queue)->purgeQueue(); queue.reset(); } }; @@ -276,9 +286,9 @@ public: } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children - ISchedulerQueue * getQueue() + std::shared_ptr getQueue() { - return static_cast(impl.branch.queue.get()); + return static_pointer_cast(impl.branch.queue); } protected: // Hide all the ISchedulerNode interface methods as an implementation details diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 0e246ed2273..acb8504ce30 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -119,11 +119,12 @@ class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest { + ResourceTestClass * test; String name; - std::exception_ptr exception; - Request(ResourceCost cost_, const String & name_) + Request(ResourceTestClass * test_, ResourceCost cost_, const String & name_) : ResourceRequest(cost_) + , test(test_) , name(name_) {} @@ -131,13 +132,19 @@ class ResourceTestClass : public ResourceTestBase { } - void failed(const std::exception_ptr & ptr) override + void failed(const std::exception_ptr &) override { - exception = ptr; + test->failed_cost += cost; + delete this; } }; public: + ~ResourceTestClass() + { + dequeue(); // Just to avoid any leaks of `Request` object + } + template void add(const String & path, const String & xml = {}) { @@ -173,7 +180,7 @@ public: void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { - enqueueImpl(node->getQueue(), costs, node->basename); + enqueueImpl(node->getQueue().get(), costs, node->basename); } void enqueue(const String & path, const std::vector & costs) @@ -203,7 +210,7 @@ public: { ASSERT_TRUE(queue != nullptr); // not a queue for (ResourceCost cost : costs) - queue->enqueueRequest(new Request(cost, name.empty() ? queue->basename : name)); + queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name)); processEvents(); // to activate queues } @@ -259,6 +266,12 @@ public: consumed_cost[name] -= value; } + void failed(ResourceCost value) + { + EXPECT_EQ(failed_cost, value); + failed_cost -= value; + } + void processEvents() { while (event_queue.tryProcess()) {} @@ -268,6 +281,7 @@ private: EventQueue event_queue; SchedulerNodePtr root_node; std::unordered_map consumed_cost; + ResourceCost failed_cost = 0; }; template diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 24a8950d9ae..faebaa72b71 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include #include #include @@ -448,3 +450,46 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.dequeue(2); t.consumed("A", 20); } + + +TEST(SchedulerUnifiedNode, ResourceGuardException) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + + t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled + + std::thread consumer([queue = all->getQueue()] + { + ResourceLink link{.queue = queue.get()}; + try + { + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link); + FAIL(); + } + catch (...) + { + } + }); + + // This will destory the queue and fail both requests + auto a = t.createUnifiedNode("A", all); + t.failed(20); + consumer.join(); + + // Check that everything works fine after destruction + auto b = t.createUnifiedNode("B", all); + t.enqueue(a, {10, 10}); // make sure A is never empty + for (int i = 0; i < 10; i++) + { + t.enqueue(a, {10, 10, 10, 10}); + t.enqueue(b, {10, 10}); + + t.dequeue(6); + t.consumed("A", 40); + t.consumed("B", 20); + } + t.dequeue(2); + t.consumed("A", 20); +} From f8599391253d2e10679505c6d0879ea4277130eb Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 13 Sep 2024 15:49:17 +0000 Subject: [PATCH 024/157] allow only unified children for unified nodes --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 49 ++++++++++++++----- 1 file changed, 38 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index fa284ed5254..c3c8ca2134a 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -23,6 +23,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class UnifiedSchedulerNode; +using UnifiedSchedulerNodePtr = std::shared_ptr; + /* * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes @@ -82,7 +85,7 @@ private: /// A branch of the tree for a specific priority value struct FairnessBranch { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached - std::unordered_map children; // basename -> child + std::unordered_map children; // basename -> child SchedulerNodePtr getRoot() { @@ -94,7 +97,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (auto [it, inserted] = children.emplace(child->basename, child); !inserted) throw Exception( @@ -129,7 +132,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { bool existing_branch = branches.contains(child->info.priority); auto & child_branch = branches[child->info.priority]; @@ -183,10 +186,10 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (queue) - removeQueue(event_queue_); + removeQueue(); return branch.attachUnifiedChild(event_queue_, child); } @@ -197,7 +200,7 @@ private: queue->basename = "fifo"; } - void removeQueue(EventQueue *) + void removeQueue() { // This unified node will not be able to process resource requests any longer // All remaining resource requests are be aborted on queue destruction @@ -240,7 +243,7 @@ private: /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. - [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const SchedulerNodePtr & child) + [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) { if (auto branch_root = branch.attachUnifiedChild(event_queue_, child)) { @@ -265,7 +268,7 @@ public: /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children - void attachUnifiedChild(const SchedulerNodePtr & child) + void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) { if (auto new_child = impl.attachUnifiedChild(event_queue, child)) reparent(new_child, this); @@ -273,7 +276,7 @@ public: /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) /// NOTE: Changing a priority of a unified child may lead to change of its parent. - void updateUnifiedChildPriority(const SchedulerNodePtr & child, Priority old_priority, Priority new_priority) + void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) { UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority } @@ -291,6 +294,32 @@ public: return static_pointer_cast(impl.branch.queue); } + /// Returns nodes that could be accessed with raw pointers by resource requests (queue and constraints) + /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints + /// for that queue might change in future versions, and `request->constraints` might reference nodes not in + /// the initial set of nodes returned by `getClassifierNodes()`. To avoid destruction of such additinal nodes + /// classifier must (indirectly) hold nodes return by `getClassifierNodes()` for all future versions of all unified nodes. + /// Such a version control is done by `IOResourceManager`. + std::vector getClassifierNodes() + { + std::vector result; + if (impl.branch.queue) + result.push_back(impl.branch.queue); + if (impl.semaphore) + result.push_back(impl.semaphore); + if (impl.throttler) + result.push_back(impl.throttler); + for (auto & [_, branch] : impl.branch.branch.branches) + { + for (auto & [_, child] : branch.children) + { + auto nodes = child->getClassifierNodes(); + result.insert(result.end(), nodes.begin(), nodes.end()); + } + } + return result; + } + protected: // Hide all the ISchedulerNode interface methods as an implementation details bool equals(ISchedulerNode *) override { @@ -366,6 +395,4 @@ private: bool child_active = false; }; -using UnifiedSchedulerNodePtr = std::shared_ptr; - } From 5473b5a051eb90040684eb4f2be4fadb94b26ed9 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 14 Sep 2024 09:45:59 +0000 Subject: [PATCH 025/157] get rid of ResourceManagersFactory --- programs/server/Server.cpp | 2 - .../Nodes/DynamicResourceManager.cpp | 6 -- .../Nodes/registerResourceManagers.cpp | 15 ----- .../Nodes/registerResourceManagers.h | 8 --- .../Scheduler/Nodes/tests/ResourceTest.h | 3 +- src/Common/Scheduler/ResourceManagerFactory.h | 55 ------------------- .../Scheduler/createResourceManager.cpp | 17 ++++++ src/Common/Scheduler/createResourceManager.h | 11 ++++ src/Interpreters/Context.cpp | 4 +- 9 files changed, 31 insertions(+), 90 deletions(-) delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.cpp delete mode 100644 src/Common/Scheduler/Nodes/registerResourceManagers.h delete mode 100644 src/Common/Scheduler/ResourceManagerFactory.h create mode 100644 src/Common/Scheduler/createResourceManager.cpp create mode 100644 src/Common/Scheduler/createResourceManager.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8043b10bead..66651c7ada3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -85,7 +85,6 @@ #include #include #include -#include #include #include #include @@ -781,7 +780,6 @@ try registerFormats(); registerRemoteFileMetadatas(); registerSchedulerNodes(); - registerResourceManagers(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index 6b9f6318903..29b3aefacf1 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include @@ -253,9 +252,4 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) future.get(); } -void registerDynamicResourceManager(ResourceManagerFactory & factory) -{ - factory.registerMethod("dynamic"); -} - } diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp b/src/Common/Scheduler/Nodes/registerResourceManagers.cpp deleted file mode 100644 index c5d5ba5b981..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDynamicResourceManager(ResourceManagerFactory &); - -void registerResourceManagers() -{ - auto & factory = ResourceManagerFactory::instance(); - registerDynamicResourceManager(factory); -} - -} diff --git a/src/Common/Scheduler/Nodes/registerResourceManagers.h b/src/Common/Scheduler/Nodes/registerResourceManagers.h deleted file mode 100644 index 243b25a9587..00000000000 --- a/src/Common/Scheduler/Nodes/registerResourceManagers.h +++ /dev/null @@ -1,8 +0,0 @@ -#pragma once - -namespace DB -{ - -void registerResourceManagers(); - -} diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index acb8504ce30..c8cc0ed0e57 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -10,7 +10,6 @@ #include #include #include -#include #include @@ -32,7 +31,7 @@ struct ResourceTestBase { ResourceTestBase() { - [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }(); + [[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); return true; }(); } template diff --git a/src/Common/Scheduler/ResourceManagerFactory.h b/src/Common/Scheduler/ResourceManagerFactory.h deleted file mode 100644 index 52f271e51b1..00000000000 --- a/src/Common/Scheduler/ResourceManagerFactory.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include -#include - -#include - -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INVALID_SCHEDULER_NODE; -} - -class ResourceManagerFactory : private boost::noncopyable -{ -public: - static ResourceManagerFactory & instance() - { - static ResourceManagerFactory ret; - return ret; - } - - ResourceManagerPtr get(const String & name) - { - std::lock_guard lock{mutex}; - if (auto iter = methods.find(name); iter != methods.end()) - return iter->second(); - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unknown scheduler node type: {}", name); - } - - template - void registerMethod(const String & name) - { - std::lock_guard lock{mutex}; - methods[name] = [] () - { - return std::make_shared(); - }; - } - -private: - std::mutex mutex; - using Method = std::function; - std::unordered_map methods; -}; - -} diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp new file mode 100644 index 00000000000..b0b7f731a89 --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -0,0 +1,17 @@ +#include +#include +#include +#include + +namespace DB +{ + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) +{ + UNUSED(global_context); + // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together + // const auto & config = global_context->getConfigRef(); + return std::make_shared(); +} + +} diff --git a/src/Common/Scheduler/createResourceManager.h b/src/Common/Scheduler/createResourceManager.h new file mode 100644 index 00000000000..d80a17f3bff --- /dev/null +++ b/src/Common/Scheduler/createResourceManager.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context); + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6cacf7bd516..371a8d3900d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -89,7 +89,7 @@ #include #include #include -#include +#include #include #include #include @@ -1677,7 +1677,7 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { callOnce(shared->resource_manager_initialized, [&] { - shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); + shared->resource_manager = createResourceManager(getGlobalContext()); }); return shared->resource_manager; From ec4e0ed1b2c2b355dddd07d18736d8e993a9d620 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 17 Sep 2024 11:18:19 +0000 Subject: [PATCH 026/157] add notification sending --- src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp | 1 + src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index b14a96c771a..c794d2717e4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -128,6 +128,7 @@ void WorkloadEntityDiskStorage::loadEntities() void WorkloadEntityDiskStorage::reloadEntities() { + // TODO(serxa): it does not send notifications, maybe better to remove this method completely loadEntitiesImpl(); } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index e3bf6d4af7f..ad5a3166cf6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -152,6 +152,8 @@ bool WorkloadEntityStorageBase::storeEntity( onEntityAdded(entity_type, entity_name, create_entity_query); } + sendNotifications(); + return stored; } @@ -183,6 +185,8 @@ bool WorkloadEntityStorageBase::removeEntity( onEntityRemoved(entity_type, entity_name); } + sendNotifications(); + return removed; } From ab6bb3b2a60d060ca0ac2a2dfe423721bec765b7 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 17 Sep 2024 11:19:38 +0000 Subject: [PATCH 027/157] initial implementation of IOResourceManager --- src/Common/Scheduler/IResourceManager.h | 2 +- src/Common/Scheduler/ISchedulerNode.h | 2 + .../Nodes/DynamicResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/FairPolicy.h | 6 + src/Common/Scheduler/Nodes/FifoQueue.h | 6 + .../Scheduler/Nodes/IOResourceManager.cpp | 502 ++++++++++++++++++ .../Scheduler/Nodes/IOResourceManager.h | 272 ++++++++++ src/Common/Scheduler/Nodes/PriorityPolicy.h | 6 + .../Scheduler/Nodes/SemaphoreConstraint.h | 6 + .../Scheduler/Nodes/ThrottlerConstraint.h | 8 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 75 ++- .../Nodes/tests/gtest_event_queue.cpp | 6 + src/Common/Scheduler/SchedulerRoot.h | 6 + .../Scheduler/createResourceManager.cpp | 6 +- .../System/StorageSystemScheduler.cpp | 18 +- 15 files changed, 887 insertions(+), 36 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.cpp create mode 100644 src/Common/Scheduler/Nodes/IOResourceManager.h diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index 8a7077ac3d5..c4a5c590ba7 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -51,7 +51,7 @@ public: virtual ClassifierPtr acquire(const String & classifier_name) = 0; /// For introspection, see `system.scheduler` table - using VisitorFunc = std::function; + using VisitorFunc = std::function; virtual void forEachNode(VisitorFunc visitor) = 0; }; diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 6d3132f79c1..d68a32e8290 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -141,6 +141,8 @@ public: virtual ~ISchedulerNode() = default; + virtual const String & getTypeName() const = 0; + /// Checks if two nodes configuration is equal virtual bool equals(ISchedulerNode * other) { diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp index 29b3aefacf1..88b4eec063d 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp @@ -244,7 +244,7 @@ void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { for (auto & [name, resource] : state_ref->resources) for (auto & [path, node] : resource->nodes) - visitor(name, path, node.type, node.ptr); + visitor(name, path, node.ptr.get()); promise.set_value(); }); diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index b6be26bea98..81bfaaadf19 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -52,6 +52,12 @@ public: : ISchedulerNode(event_queue_, info_) {} + const String & getTypeName() const override + { + static String type_name("fair"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index c95125b21bf..79963a45b3b 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -39,6 +39,12 @@ public: chassert(requests.empty()); } + const String & getTypeName() const override + { + static String type_name("fifo"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp new file mode 100644 index 00000000000..9e6b4ebb254 --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -0,0 +1,502 @@ +#include "Common/Scheduler/IResourceManager.h" +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; + extern const int RESOURCE_NOT_FOUND; + extern const int INVALID_SCHEDULER_NODE; + extern const int LOGICAL_ERROR; +} + +namespace +{ + String getEntityName(const ASTPtr & ast) + { + if (auto * create = typeid_cast(ast.get())) + return create->getWorkloadName(); + if (auto * create = typeid_cast(ast.get())) + return create->getResourceName(); + return "unknown-workload-entity"; + } +} + +IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resource_name) +{ + auto * create = typeid_cast(ast.get()); + name = create->getWorkloadName(); + parent = create->getWorkloadParent(); + // TODO(serxa): parse workload settings specifically for `resource_name` + UNUSED(resource_name); +} + +IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) + : resource_entity(resource_entity_) + , resource_name(getEntityName(resource_entity)) +{ + scheduler.start(); +} + +IOResourceManager::Resource::~Resource() +{ + // TODO(serxa): destroy all workloads, purge all queue, abort all resource requests + scheduler.stop(); +} + +void IOResourceManager::Resource::createNode(const NodeInfo & info) +{ + // TODO(serxa): make sure all possible callers validate empty workload name! + if (info.name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", + resource_name); + + // TODO(serxa): make sure all possible callers validate self-reference! + if (info.name == info.parent) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", + info.name, resource_name); + + if (node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", + info.name, resource_name); + + // TODO(serxa): make sure all possible callers validate parent existence, add tests for creating workload with invalid parent + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + // TODO(serxa): make sure all possible callers validate second root, add tests for creating the second root + if (info.parent.empty() && root_node) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", + info.name, root_node->basename, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = std::make_shared(scheduler.event_queue, info.settings); + node->basename = info.name; + if (!info.parent.empty()) + node_for_workload[info.parent]->attachUnifiedChild(node); + else + { + root_node = node; + scheduler.attachChild(root_node); + } + node_for_workload[info.name] = node; + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::deleteNode(const NodeInfo & info) +{ + if (!node_for_workload.contains(info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for removing workload '{}' does not exist in resource '{}'", + info.name, resource_name); + + if (!info.parent.empty() && !node_for_workload.contains(info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for removing workload '{}' does not exist in resource '{}'", + info.parent, info.name, resource_name); + + auto node = node_for_workload[info.name]; + + // TODO(serxa): make sure all possible callers validate that removing workload has no children workloads + if (node->hasUnifiedChildren()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", + info.name, resource_name); + + executeInSchedulerThread([&, this] + { + if (!info.parent.empty()) + node_for_workload[info.parent]->detachUnifiedChild(node); + else + { + chassert(node == root_node); + scheduler.removeChild(root_node.get()); + root_node.reset(); + } + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const NodeInfo & new_info) +{ + if (old_info.name != new_info.name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", + old_info.name, new_info.name, resource_name); + + if (old_info.parent != new_info.parent && (old_info.parent.empty() || old_info.parent.empty())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", + old_info.name, old_info.parent, new_info.parent, resource_name); + + if (!node_for_workload.contains(old_info.name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for updating workload '{}' does not exist in resource '{}'", + old_info.name, resource_name); + + if (!old_info.parent.empty() && !node_for_workload.contains(old_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Old parent node '{}' for updating workload '{}' does not exist in resource '{}'", + old_info.parent, old_info.name, resource_name); + + if (!new_info.parent.empty() && !node_for_workload.contains(new_info.parent)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "New parent node '{}' for updating workload '{}' does not exist in resource '{}'", + new_info.parent, new_info.name, resource_name); + + executeInSchedulerThread([&, this] + { + auto node = node_for_workload[old_info.name]; + bool detached = false; + if (old_info.parent != new_info.parent) + { + node_for_workload[old_info.parent]->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_info.settings); + if (!detached && !old_info.parent.empty() && old_info.settings.priority != new_info.settings.priority) + node_for_workload[old_info.parent]->updateUnifiedChildPriority( + node, + old_info.settings.priority, + new_info.settings.priority); + + if (detached) + node_for_workload[new_info.parent]->attachUnifiedChild(node); + + updateCurrentVersion(); + }); +} + +void IOResourceManager::Resource::updateCurrentVersion() +{ + auto previous_version = current_version; + + // Create a full list of constraints and queues in the current hierarchy + current_version = std::make_shared(); + if (root_node) + root_node->addRawPointerNodes(current_version->nodes); + + // See details in version control section of description in IOResourceManager.h + if (previous_version) + { + previous_version->newer_version = current_version; + // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. + // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it + } +} + +IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_) + : resource_manager(resource_manager_) + , workload_entity(workload_entity_) +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->createNode(NodeInfo(workload_entity, resource_name)); +} + +IOResourceManager::Workload::~Workload() +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->deleteNode(NodeInfo(workload_entity, resource_name)); +} + +void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) +{ + for (auto & [resource_name, resource] : resource_manager->resources) + resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); + workload_entity = new_entity; +} + +String IOResourceManager::Workload::getParent() const +{ + return typeid_cast(workload_entity.get())->getWorkloadParent(); +} + +IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) + : storage(storage_) +{ + workload_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Workload, [this] ( + WorkloadEntityType, + const String & entity_name, + const ASTPtr & entity) + { + try + { + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + } + catch (...) + { + // TODO(serxa): handle CRUD errors + } + }); + resource_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Resource, [this] ( + WorkloadEntityType, + const String & entity_name, + const ASTPtr & entity /* new or changed entity, null if removed */) + { + try + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + } + catch (...) + { + // TODO(serxa): handle CRUD errors + } + }); +} + +void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) +{ + // No-op +} + +void IOResourceManager::createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + workload_iter->second->updateWorkload(ast); + else + workloads.emplace(workload_name, std::make_shared(this, ast)); +} + +void IOResourceManager::deleteWorkload(const String & workload_name) +{ + std::unique_lock lock{mutex}; + if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + workloads.erase(workload_iter); + else + { + // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } +} + +void IOResourceManager::createResource(const String & resource_name, const ASTPtr & ast) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + { + // Resource to be created already exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } + else + { + // Add all workloads into the new resource + auto resource = std::make_shared(ast); + for (Workload * workload : topologicallySortedWorkloads()) + resource->createNode(NodeInfo(workload->workload_entity, resource_name)); + + // Attach the resource + resources.emplace(resource_name, resource); + } +} + +void IOResourceManager::deleteResource(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) + { + resources.erase(resource_iter); + } + else + { + // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + // TODO(serxa): add logging + } +} + +IOResourceManager::Classifier::~Classifier() +{ + // Detach classifier from all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(attachments.size()); + for (auto & [resource_name, attachment] : attachments) + { + futures.emplace_back(attachment.resource->detachClassifier(std::move(attachment.version))); + attachment.link.reset(); // Just in case because it is not valid any longer + } + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // There should not be any exceptions because it just destruct few objects, but let's rethrow just in case + for (auto & future : futures) + future.get(); + + // This unreferences and probably destroys `Resource` objects. + // NOTE: We cannot do it in the scheduler threads (because thread cannot join itself). + attachments.clear(); +} + +std::future IOResourceManager::Resource::detachClassifier(VersionPtr && version) +{ + auto detach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semanticss + auto future = detach_promise->get_future(); + scheduler.event_queue->enqueue([detached_version = std::move(version), promise = std::move(detach_promise)] mutable + { + try + { + // Unreferences and probably destroys the version and scheduler nodes it owns. + // The main reason from moving destruction into the scheduler thread is to + // free memory in the same thread it was allocated to avoid memtrackers drift. + detached_version.reset(); + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +ResourceLink IOResourceManager::Classifier::get(const String & resource_name) +{ + std::unique_lock lock{mutex}; + if (auto iter = attachments.find(resource_name); iter != attachments.end()) + return iter->second.link; + else + throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Access denied to resource '{}'", resource_name); +} + +void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link) +{ + std::unique_lock lock{mutex}; + chassert(!attachments.contains(resource->getName())); + attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; +} + +std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) +{ + auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics + auto future = attach_promise->get_future(); + scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] mutable + { + try + { + if (auto iter = node_for_workload.find(workload_name); iter != node_for_workload.end()) + { + auto queue = iter->second->getQueue(); + if (!queue) + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to use workload '{}' that have children for resource '{}'", + workload_name, resource_name); + classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); + } + else + throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to find workload '{}' for resource '{}'", workload_name, resource_name); + promise->set_value(); + } + catch (...) + { + promise->set_exception(std::current_exception()); + } + }); + return future; +} + +ClassifierPtr IOResourceManager::acquire(const String & workload_name) +{ + auto classifier = std::make_shared(); + + // Attach classifier to all resources in parallel (executed in every scheduler thread) + std::vector> futures; + { + std::unique_lock lock{mutex}; + futures.reserve(resources.size()); + for (auto & [resource_name, resource] : resources) + futures.emplace_back(resource->attachClassifier(*classifier, workload_name)); + } + + // Wait for all tasks to finish (to avoid races in case of exceptions) + for (auto & future : futures) + future.wait(); + + // Rethrow exceptions if any + for (auto & future : futures) + future.get(); + + return classifier; +} + +void IOResourceManager::Resource::forEachResourceNode(IResourceManager::VisitorFunc & visitor) +{ + executeInSchedulerThread([&, this] + { + for (auto & [path, node] : node_for_workload) + { + node->forEachSchedulerNode([&] (ISchedulerNode * scheduler_node) + { + visitor(resource_name, scheduler_node->getPath(), scheduler_node); + }); + } + }); +} + +void IOResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +{ + // Gather resource upfront to avoid holding mutex for a long time + std::map sorted_resources; + { + std::unique_lock lock{mutex}; + for (auto & [resource_name, resource] : resources) + sorted_resources[resource_name] = resource; + } + + /// Run tasks one by one to avoid concurrent calls to visitor + for (auto & [resource_name, resource] : sorted_resources) + resource->forEachResourceNode(visitor); +} + +void IOResourceManager::topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads) +{ + if (visited.contains(workload)) + return; + visited.insert(workload); + + // Recurse into parent (if any) + String parent = workload->getParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + chassert(parent_iter != workloads.end()); // validations check that all parents exist + topologicallySortedWorkloadsImpl(parent_iter->second.get(), visited, sorted_workloads); + } + + sorted_workloads.push_back(workload); +} + +std::vector IOResourceManager::topologicallySortedWorkloads() +{ + std::vector sorted_workloads; + std::unordered_set visited; + for (auto & [workload_name, workload] : workloads) + topologicallySortedWorkloadsImpl(workload.get(), visited, sorted_workloads); + return sorted_workloads; +} + +} diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h new file mode 100644 index 00000000000..157507ed56b --- /dev/null +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -0,0 +1,272 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +/* + * Implementation of `IResourceManager` that creates hierarchy of scheduler nodes according to + * workload entities (WORKLOADs and RESOURCEs). It subscribes for updates in IWorkloadEntityStorage and + * creates hierarchy of UnifiedSchedulerNode identical to the hierarchy of WORKLOADs. + * For every RESOURCE an independent hierarchy of scheduler nodes is created. + * + * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. + * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). + * After DROP RESOURCE parts of hierarchy might be keept alive while at least one query uses it. + * + * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having + * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. + * + * Classifiers are used (1) to access IO resources and (2) to keep shared ownership of scheduling nodes. + * This allows `ResourceRequest` and `ResourceLink` to hold raw pointers as long as + * `ClassifierPtr` is acquired and held. + * + * === RESOURCE ARCHITECTURE === + * Let's consider how a single resource is implemented. Every workload is represented by corresponding UnifiedSchedulerNode. + * Every UnifiedSchedulerNode manages its own subtree of ISchedulerNode objects (see details in UnifiedSchedulerNode.h) + * UnifiedSchedulerNode for workload w/o children has a queue, which provide a ResourceLink for consumption. + * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. + * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). + * + * Here is an example of SQL and corresponding heirarchy of scheduler nodes: + * CREATE RESOURCE my_io_resource (...) + * CREATE WORKLOAD all + * CREATE WORKLOAD production PARENT all + * CREATE WORKLOAD development PARENT all + * + * root - SchedulerRoot (with scheduler thread and EventQueue) + * | + * all - UnifiedSchedulerNode + * | + * p0_fair - FairPolicy (part of parent UnifiedSchedulerNode internal structure) + * / \ + * production development - UnifiedSchedulerNode + * | | + * queue queue - FifoQueue (part of parent UnifiedSchedulerNode internal structure) + * + * === UPDATING WORKLOADS === + * Workload may be created, updated or deleted. + * Updating a child of a workload might lead to updating other workloads: + * 1. Workload itself: it's structure depend on settings of children workloads + * (e.g. fifo node of a leaf workload is remove when the first child is added; + * and a fair node is inserted after the first two children are added). + * 2. Other children: for them path to root might be changed (e.g. intermediate priority node is inserted) + * + * === VERSION CONTROL === + * Versions are created on hierarchy updates and hold ownership of nodes that are used through raw pointers. + * Classifier reference version of every resource it use. Older version reference newer version. + * Here is a diagram explaining version control based on Version objects (for 1 resource): + * + * [nodes] [nodes] [nodes] + * ^ ^ ^ + * | | | + * version1 --> version2 -...-> versionN + * ^ ^ ^ + * | | | + * old_classifier new_classifier current_version + * + * Previous version should hold reference to a newer version. It is required for proper handling of updates. + * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). + * It may move a queue to a new position in the hierarchy or create/destry constraints, thus resource requests + * created by old classifier may reference constraints of newer versions through `request->constraints` which + * is filled during dequeueRequst(). + * + * === THREADS === + * scheduler thread: + * - one thread per resource + * - uses event_queue (per resource) for processing w/o holding mutex for every scheduler node + * - handle resource requests + * - node activations + * - scheduler hierarchy updates + * query thread: + * - multiple independent threads + * - send resource requests + * - acquire and release classifiers (via scheduler event queues) + * control thread: + * - modify workload and resources through subscription + * + * === SYNCHRONIZATION === + * List of related sync primitives and their roles: + * IOResourceManager::mutex + * - protects resource manager data structures - resource and workloads + * - serialize control thread actions + * IOResourceManager::Resource::scheduler->event_queue + * - serializes scheduler hierarchy events + * - events are created in control and query threads + * - all events are processed by specific scheduler thread + * - hierarchy-wide actions: requests dequeueing, activations propagation and nodes updates. + * - resource version control management + * FifoQueue::mutex and SemaphoreContraint::mutex + * - serializes query and scheduler threads on specific node accesses + * - resource request processing: enqueueRequest(), dequeueRequest() and finishRequest() + */ +class IOResourceManager : public IResourceManager +{ +public: + explicit IOResourceManager(IWorkloadEntityStorage & storage_); + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + ClassifierPtr acquire(const String & workload_name) override; + void forEachNode(VisitorFunc visitor) override; + +private: + // Forward declarations + struct NodeInfo; + struct Version; + class Resource; + struct Workload; + class Classifier; + + friend struct Workload; + + using VersionPtr = std::shared_ptr; + using ResourcePtr = std::shared_ptr; + using WorkloadPtr = std::shared_ptr; + + /// Helper for parsing workload AST for a specific resource + struct NodeInfo + { + String name; // Workload name + String parent; // Name of parent workload + SchedulingSettings settings; // Settings specific for a given resource + + NodeInfo(const ASTPtr & ast, const String & resource_name); + }; + + /// Ownership control for scheduler nodes, which could be referenced by raw pointers + struct Version + { + std::vector nodes; + VersionPtr newer_version; + }; + + /// Holds a thread and hierarchy of unified scheduler nodes for specific RESOURCE + class Resource : public std::enable_shared_from_this, boost::noncopyable + { + public: + explicit Resource(const ASTPtr & resource_entity_); + ~Resource(); + + const String & getName() const { return resource_name; } + + /// Hierarchy management + void createNode(const NodeInfo & info); + void deleteNode(const NodeInfo & info); + void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); + + /// Updates a classifier to contain a reference for specified workload + std::future attachClassifier(Classifier & classifier, const String & workload_name); + + /// Remove classifier reference. This destroys scheduler nodes in proper scheduler thread + std::future detachClassifier(VersionPtr && version); + + /// Introspection + void forEachResourceNode(IOResourceManager::VisitorFunc & visitor); + + private: + void updateCurrentVersion(); + + template + void executeInSchedulerThread(Task && task) + { + std::promise promise; + auto future = promise.get_future(); + scheduler.event_queue->enqueue([&] + { + try + { + task(); + promise.set_value(); + } + catch (...) + { + promise.set_exception(std::current_exception()); + } + }); + future.get(); // Blocks until execution is done in the scheduler thread + } + + const ASTPtr resource_entity; + const String resource_name; + SchedulerRoot scheduler; + + // TODO(serxa): consider using resource_manager->mutex + scheduler thread for updates and mutex only for reading to avoid slow acquire/release of classifier + /// These field should be accessed only by the scheduler thread + std::unordered_map node_for_workload; + UnifiedSchedulerNodePtr root_node; + VersionPtr current_version; + }; + + struct Workload : boost::noncopyable + { + IOResourceManager * resource_manager; + ASTPtr workload_entity; + + Workload(IOResourceManager * resource_manager_, const ASTPtr & workload_entity_); + ~Workload(); + + void updateWorkload(const ASTPtr & new_entity); + String getParent() const; + }; + + class Classifier : public IClassifier + { + public: + ~Classifier() override; + + /// Implements IClassifier interface + /// NOTE: It is called from query threads (possibly multiple) + ResourceLink get(const String & resource_name) override; + + /// Attaches/detaches a specific resource + /// NOTE: It is called from scheduler threads (possibly multiple) + void attach(const ResourcePtr & resource, const VersionPtr & version, ResourceLink link); + void detach(const ResourcePtr & resource); + + private: + IOResourceManager * resource_manager; + std::mutex mutex; + struct Attachment { + ResourcePtr resource; + VersionPtr version; + ResourceLink link; + }; + std::unordered_map attachments; // TSA_GUARDED_BY(mutex); + }; + + void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); + void deleteWorkload(const String & workload_name); + void createResource(const String & resource_name, const ASTPtr & ast); + void deleteResource(const String & resource_name); + + // Topological sorting of worklaods + void topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set & visited, std::vector & sorted_workloads); + std::vector topologicallySortedWorkloads(); + + IWorkloadEntityStorage & storage; + scope_guard workload_change_subscription; + scope_guard resource_change_subscription; + + std::mutex mutex; + std::unordered_map workloads; // TSA_GUARDED_BY(mutex); + std::unordered_map resources; // TSA_GUARDED_BY(mutex); +}; + +} diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index 17fcbfd3139..ea8bde718a2 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -43,6 +43,12 @@ public: : ISchedulerNode(event_queue_, node_info) {} + const String & getTypeName() const override + { + static String type_name("priority"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index a2d8df48065..eab093f6b00 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -31,6 +31,12 @@ public: , max_cost(max_cost_) {} + const String & getTypeName() const override + { + static String type_name("inflight_limit"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 7c64dd51ac1..40b51f24b98 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -3,8 +3,6 @@ #include #include -#include -#include #include @@ -42,6 +40,12 @@ public: event_queue->cancelPostponed(postponed); } + const String & getTypeName() const override + { + static String type_name("bandwidth_limit"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index c3c8ca2134a..76685319c34 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -266,7 +266,7 @@ public: reparent(immediate_child, this); } - /// Attaches a child as a leaf of internal subtree and insert or update all the intermediate nodes + /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) { @@ -274,18 +274,28 @@ public: reparent(new_child, this); } + /// Detaches unified child and update all the intermediate nodes. + /// Detached child could be safely attached to another parent. + /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children + void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) + { + UNUSED(child); // TODO(serxa): implement detachUnifiedChild() + } + /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) /// NOTE: Changing a priority of a unified child may lead to change of its parent. void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) { - UNUSED(child, old_priority, new_priority); // TODO: implement updateUnifiedChildPriority + UNUSED(child, old_priority, new_priority); // TODO(serxa): implement updateUnifiedChildPriority() } /// Updates scheduling settings. Set of constraints might change. /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) void updateSchedulingSettings(const SchedulingSettings & new_settings) { - UNUSED(new_settings); // TODO: implement updateSchedulingSettings + UNUSED(new_settings); // TODO(serxa): implement updateSchedulingSettings() + info.setPriority(new_settings.priority); + info.setWeight(new_settings.weight); } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children @@ -294,33 +304,58 @@ public: return static_pointer_cast(impl.branch.queue); } - /// Returns nodes that could be accessed with raw pointers by resource requests (queue and constraints) + /// Collects nodes that could be accessed with raw pointers by resource requests (queue and constraints) /// NOTE: This is a building block for classifier. Note that due to possible movement of a queue, set of constraints - /// for that queue might change in future versions, and `request->constraints` might reference nodes not in - /// the initial set of nodes returned by `getClassifierNodes()`. To avoid destruction of such additinal nodes - /// classifier must (indirectly) hold nodes return by `getClassifierNodes()` for all future versions of all unified nodes. - /// Such a version control is done by `IOResourceManager`. - std::vector getClassifierNodes() + /// for that queue might change in future, and `request->constraints` might reference nodes not in + /// the initial set of nodes returned by `addRawPointerNodes()`. To avoid destruction of such additional nodes + /// classifier must (indirectly) hold nodes return by `addRawPointerNodes()` for all future versions of + /// all unified nodes. Such a version control is done by `IOResourceManager`. + void addRawPointerNodes(std::vector & nodes) { - std::vector result; - if (impl.branch.queue) - result.push_back(impl.branch.queue); - if (impl.semaphore) - result.push_back(impl.semaphore); if (impl.throttler) - result.push_back(impl.throttler); + nodes.push_back(impl.throttler); + if (impl.semaphore) + nodes.push_back(impl.semaphore); + if (impl.branch.queue) + nodes.push_back(impl.branch.queue); for (auto & [_, branch] : impl.branch.branch.branches) { for (auto & [_, child] : branch.children) - { - auto nodes = child->getClassifierNodes(); - result.insert(result.end(), nodes.begin(), nodes.end()); - } + child->addRawPointerNodes(nodes); + } + } + + bool hasUnifiedChildren() const + { + return impl.branch.queue == nullptr; + } + + /// Introspection. Calls a visitor for self and every internal node. Do not recurse into unified children. + void forEachSchedulerNode(std::function visitor) + { + visitor(this); + if (impl.throttler) + visitor(impl.throttler.get()); + if (impl.semaphore) + visitor(impl.semaphore.get()); + if (impl.branch.queue) + visitor(impl.branch.queue.get()); + if (impl.branch.branch.root) // priority + visitor(impl.branch.branch.root.get()); + for (auto & [_, branch] : impl.branch.branch.branches) + { + if (branch.root) // fairness + visitor(branch.root.get()); } - return result; } protected: // Hide all the ISchedulerNode interface methods as an implementation details + const String & getTypeName() const override + { + static String type_name("unified"); + return type_name; + } + bool equals(ISchedulerNode *) override { assert(false); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp index 07798f78080..9989215ba7b 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -13,6 +13,12 @@ public: , log(log_) {} + const String & getTypeName() const override + { + static String type_name("fake"); + return type_name; + } + void attachChild(const SchedulerNodePtr & child) override { log += " +" + child->basename; diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 5307aadc3cc..e2ed133f662 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -95,6 +95,12 @@ public: } } + const String & getTypeName() const override + { + static String type_name("scheduler"); + return type_name; + } + bool equals(ISchedulerNode * other) override { if (!ISchedulerNode::equals(other)) diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b0b7f731a89..b71b450979f 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -8,10 +9,9 @@ namespace DB ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - UNUSED(global_context); - // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together + // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled // const auto & config = global_context->getConfigRef(); - return std::make_shared(); + return std::make_shared(global_context->getWorkloadEntityStorage()); } } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index b42c807d6fc..8784ba084ce 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -84,12 +84,12 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription() void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const { - context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, ISchedulerNode * node) { size_t i = 0; res_columns[i++]->insert(resource); res_columns[i++]->insert(path); - res_columns[i++]->insert(type); + res_columns[i++]->insert(node->getTypeName()); res_columns[i++]->insert(node->info.weight); res_columns[i++]->insert(node->info.priority.value); res_columns[i++]->insert(node->isActive()); @@ -118,23 +118,23 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c if (auto * parent = dynamic_cast(node->parent)) { - if (auto value = parent->getChildVRuntime(node.get())) + if (auto value = parent->getChildVRuntime(node)) vruntime = *value; } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) system_vruntime = ptr->getSystemVRuntime(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) budget = ptr->getBudget(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) is_satisfied = ptr->isSatisfied(); - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); std::tie(max_requests, max_cost) = ptr->getLimits(); } - if (auto * ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node)) { std::tie(max_speed, max_burst) = ptr->getParams(); throttling_us = ptr->getThrottlingDuration().count() / 1000; From b83fd18c9cebd4a7af4287f9c1e11b4f5410f21d Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 18:12:43 +0000 Subject: [PATCH 028/157] resolve conflict --- src/Common/ErrorCodes.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3144fb757f3..4c1593e2f2d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -609,12 +609,9 @@ M(728, UNEXPECTED_DATA_TYPE) \ M(729, ILLEGAL_TIME_SERIES_TAGS) \ M(730, REFRESH_FAILED) \ -<<<<<<< HEAD - M(731, WORKLOAD_ENTITY_ALREADY_EXISTS) \ - M(732, UNKNOWN_WORKLOAD_ENTITY) \ -======= M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \ ->>>>>>> master + M(732, WORKLOAD_ENTITY_ALREADY_EXISTS) \ + M(733, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ From 50168629b0838d560abca5d5f07b0277d9eb0385 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 19:07:44 +0000 Subject: [PATCH 029/157] fix subscription handlers --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index ad5a3166cf6..4ba16ade9d5 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -50,7 +50,8 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) - : global_context(std::move(global_context_)) + : handlers(std::make_shared()) + , global_context(std::move(global_context_)) {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const From 32f6699c8f6f0428f5f2b9aee2f2284a8b979222 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 19:09:41 +0000 Subject: [PATCH 030/157] .gitignore /programs/server/workload --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 4bc162c1b0f..8a745655cbf 100644 --- a/.gitignore +++ b/.gitignore @@ -159,6 +159,7 @@ website/package-lock.json /programs/server/store /programs/server/uuid /programs/server/coordination +/programs/server/workload # temporary test files tests/queries/0_stateless/test_* From b60d1427a92cf4ac920e162ea35feb0f440b8bc4 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 21 Sep 2024 22:17:08 +0000 Subject: [PATCH 031/157] fix destruction order --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 9 +++++++++ src/Common/Scheduler/Nodes/IOResourceManager.h | 1 + 2 files changed, 10 insertions(+) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 9e6b4ebb254..e684cb9a16f 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -197,6 +197,7 @@ void IOResourceManager::Resource::updateCurrentVersion() previous_version->newer_version = current_version; // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } } @@ -266,6 +267,14 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) }); } +IOResourceManager::~IOResourceManager() +{ + resource_change_subscription.reset(); + workload_change_subscription.reset(); + resources.clear(); + workloads.clear(); +} + void IOResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration &) { // No-op diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 157507ed56b..02a5e420be9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -122,6 +122,7 @@ class IOResourceManager : public IResourceManager { public: explicit IOResourceManager(IWorkloadEntityStorage & storage_); + ~IOResourceManager() override; void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & workload_name) override; void forEachNode(VisitorFunc visitor) override; From 36b8481793903aaa03d89c2e5f1bbb1a1a6dfb35 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 00:13:55 +0000 Subject: [PATCH 032/157] improve workload entities subscription model --- .../Scheduler/Nodes/IOResourceManager.cpp | 54 +++--- .../Scheduler/Nodes/IOResourceManager.h | 3 +- .../Workload/IWorkloadEntityStorage.h | 20 +-- .../Workload/WorkloadEntityDiskStorage.cpp | 7 - .../Workload/WorkloadEntityDiskStorage.h | 3 - .../Workload/WorkloadEntityStorageBase.cpp | 157 ++++++++++++------ .../Workload/WorkloadEntityStorageBase.h | 20 +-- 7 files changed, 149 insertions(+), 115 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index e684cb9a16f..0c204afa97f 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -1,4 +1,3 @@ -#include "Common/Scheduler/IResourceManager.h" #include #include @@ -231,34 +230,34 @@ String IOResourceManager::Workload::getParent() const IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) : storage(storage_) { - workload_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Workload, [this] ( - WorkloadEntityType, - const String & entity_name, - const ASTPtr & entity) + subscription = storage.getAllEntitiesAndSubscribe( + [this] (const std::vector & events) { try { - if (entity) - createOrUpdateWorkload(entity_name, entity); - else - deleteWorkload(entity_name); - } - catch (...) - { - // TODO(serxa): handle CRUD errors - } - }); - resource_change_subscription = storage.subscribeForChanges(WorkloadEntityType::Resource, [this] ( - WorkloadEntityType, - const String & entity_name, - const ASTPtr & entity /* new or changed entity, null if removed */) - { - try - { - if (entity) - createResource(entity_name, entity); - else - deleteResource(entity_name); + for (auto [entity_type, entity_name, entity] : events) + { + switch (entity_type) + { + case WorkloadEntityType::Workload: + { + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + break; + } + case WorkloadEntityType::Resource: + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + break; + } + case WorkloadEntityType::MAX: break; + } + } } catch (...) { @@ -269,8 +268,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) IOResourceManager::~IOResourceManager() { - resource_change_subscription.reset(); - workload_change_subscription.reset(); + subscription.reset(); resources.clear(); workloads.clear(); } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 02a5e420be9..0cb1887d1cd 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -262,8 +262,7 @@ private: std::vector topologicallySortedWorkloads(); IWorkloadEntityStorage & storage; - scope_guard workload_change_subscription; - scope_guard resource_change_subscription; + scope_guard subscription; std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); diff --git a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h index cff09a2259d..adb3a808eea 100644 --- a/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h +++ b/src/Common/Scheduler/Workload/IWorkloadEntityStorage.h @@ -59,9 +59,6 @@ public: /// Stops watching. virtual void stopWatching() {} - /// Immediately reloads all entities, throws an exception if failed. - virtual void reloadEntities() = 0; - /// Stores an entity. virtual bool storeEntity( const ContextPtr & current_context, @@ -79,15 +76,16 @@ public: const String & entity_name, bool throw_if_not_exists) = 0; - using OnChangedHandler = std::function; + struct Event + { + WorkloadEntityType type; + String name; + ASTPtr entity; /// new or changed entity, null if removed + }; + using OnChangedHandler = std::function &)>; - /// Subscribes for all changes. - virtual scope_guard subscribeForChanges( - WorkloadEntityType entity_type, - const OnChangedHandler & handler) = 0; + /// Gets all current entries, pass them through `handler` and subscribes for all later changes. + virtual scope_guard getAllEntitiesAndSubscribe(const OnChangedHandler & handler) = 0; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index c794d2717e4..51016fac4fb 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -126,13 +126,6 @@ void WorkloadEntityDiskStorage::loadEntities() } -void WorkloadEntityDiskStorage::reloadEntities() -{ - // TODO(serxa): it does not send notifications, maybe better to remove this method completely - loadEntitiesImpl(); -} - - void WorkloadEntityDiskStorage::loadEntitiesImpl() { LOG_INFO(log, "Loading workload entities from {}", dir_path); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index 22c0ea4b83d..ceb736372ae 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -13,11 +13,8 @@ class WorkloadEntityDiskStorage : public WorkloadEntityStorageBase { public: WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_); - void loadEntities() override; - void reloadEntities() override; - private: bool storeEntityImpl( const ContextPtr & current_context, diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4ba16ade9d5..8e7f630365d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -8,6 +8,10 @@ #include #include +#include +#include + + namespace DB { @@ -15,6 +19,7 @@ namespace ErrorCodes { extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int LOGICAL_ERROR; } namespace @@ -47,6 +52,34 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + // Recurse into parent (if any) + String parent = typeid_cast(ast.get())->getWorkloadParent(); + if (!parent.empty()) + { + auto parent_iter = workloads.find(parent); + if (parent_iter == workloads.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload metadata inconsistency: Workload '{}' parent '{}' does not exist. This must be fixed manually.", name, parent); + topologicallySortedWorkloadsImpl(parent, parent_iter->second, workloads, visited, sorted_workloads); + } + + sorted_workloads.emplace_back(name, ast); +} + +std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) +{ + std::vector> sorted_workloads; + std::unordered_set visited; + for (const auto & [name, ast] : workloads) + topologicallySortedWorkloadsImpl(name, ast, workloads, visited, sorted_workloads); + return sorted_workloads; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -125,7 +158,7 @@ bool WorkloadEntityStorageBase::storeEntity( bool replace_if_exists, const Settings & settings) { - std::lock_guard lock{mutex}; + std::unique_lock lock{mutex}; create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); @@ -153,7 +186,7 @@ bool WorkloadEntityStorageBase::storeEntity( onEntityAdded(entity_type, entity_name, create_entity_query); } - sendNotifications(); + unlockAndNotify(lock); return stored; } @@ -164,7 +197,7 @@ bool WorkloadEntityStorageBase::removeEntity( const String & entity_name, bool throw_if_not_exists) { - std::lock_guard lock(mutex); + std::unique_lock lock(mutex); auto it = entities.find(entity_name); if (it == entities.end()) { @@ -186,88 +219,79 @@ bool WorkloadEntityStorageBase::removeEntity( onEntityRemoved(entity_type, entity_name); } - sendNotifications(); + unlockAndNotify(lock); return removed; } -scope_guard WorkloadEntityStorageBase::subscribeForChanges( - WorkloadEntityType entity_type, - const OnChangedHandler & handler) +scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) { - std::lock_guard lock{handlers->mutex}; - auto & list = handlers->by_type[static_cast(entity_type)]; - list.push_back(handler); - auto handler_it = std::prev(list.end()); + scope_guard result; - return [my_handlers = handlers, entity_type, handler_it] + std::vector current_state; { - std::lock_guard lock2{my_handlers->mutex}; - auto & list2 = my_handlers->by_type[static_cast(entity_type)]; - list2.erase(handler_it); - }; + std::unique_lock lock{mutex}; + chassert(queue.empty()); + makeEventsForAllEntities(lock); + current_state = std::move(queue); + + std::lock_guard lock2{handlers->mutex}; + handlers->list.push_back(handler); + auto handler_it = std::prev(handlers->list.end()); + result = [my_handlers = handlers, handler_it] + { + std::lock_guard lock3{my_handlers->mutex}; + my_handlers->list.erase(handler_it); + }; + } + + // When you subscribe you get all the entities back to your handler immediately if already loaded, or later when loaded + handler(current_state); + + return result; } void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - event.entity = new_entity; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); } void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - event.entity = changed_entity; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = changed_entity}); } void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) { - std::lock_guard lock{queue_mutex}; - Event event; - event.name = entity_name; - event.type = entity_type; - queue.push(std::move(event)); + queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); } -void WorkloadEntityStorageBase::sendNotifications() +void WorkloadEntityStorageBase::unlockAndNotify(std::unique_lock & mutex_lock) { - /// Only one thread can send notification at any time. - std::lock_guard sending_notifications_lock{sending_notifications}; - - std::unique_lock queue_lock{queue_mutex}; - while (!queue.empty()) + /// Only one thread can send notification at any time, that is why we need `mutex_lock` + if (!queue.empty()) { - auto event = std::move(queue.front()); - queue.pop(); - queue_lock.unlock(); + auto events = std::move(queue); std::vector current_handlers; { std::lock_guard handlers_lock{handlers->mutex}; - boost::range::copy(handlers->by_type[static_cast(event.type)], std::back_inserter(current_handlers)); + boost::range::copy(handlers->list, std::back_inserter(current_handlers)); } + mutex_lock.unlock(); + for (const auto & handler : current_handlers) { try { - handler(event.type, event.name, event.entity); + handler(events); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } } - - queue_lock.lock(); } } @@ -276,21 +300,54 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } + void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) { + std::unordered_map normalized_entities; for (const auto & [entity_name, create_query] : new_entities) normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); // TODO(serxa): do validation and throw LOGICAL_ERROR if failed - // Note that notifications are not sent, because it is hard to send notifications in right order to maintain invariants. - // Another code path using getAllEntities() should be used for initialization - - std::lock_guard lock(mutex); + std::unique_lock lock(mutex); + chassert(entities.empty()); entities = std::move(normalized_entities); + + // Quick check to avoid extra work + { + std::lock_guard lock2(handlers->mutex); + if (handlers->list.empty()) + return; + } + + makeEventsForAllEntities(lock); + unlockAndNotify(lock); } + +void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock &) +{ + std::unordered_map workloads; + std::unordered_map resources; + for (auto & [entity_name, ast] : entities) + { + if (typeid_cast(ast.get())) + workloads.emplace(entity_name, ast); + else if (typeid_cast(ast.get())) + resources.emplace(entity_name, ast); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + } + + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); + + for (auto & [entity_name, ast] : resources) + onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); +} + + std::vector> WorkloadEntityStorageBase::getAllEntities() const { std::lock_guard lock{mutex}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 8ec92675ddb..bf8a89a67c4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -45,8 +44,7 @@ public: const String & entity_name, bool throw_if_not_exists) override; - virtual scope_guard subscribeForChanges( - WorkloadEntityType entity_type, + virtual scope_guard getAllEntitiesAndSubscribe( const OnChangedHandler & handler) override; protected: @@ -66,7 +64,9 @@ protected: bool throw_if_not_exists) = 0; std::unique_lock getLock() const; + void setAllEntities(const std::vector> & new_entities); + void makeEventsForAllEntities(std::unique_lock & lock); void removeAllEntitiesExcept(const Strings & entity_names_to_keep); /// Called by derived class after a new workload entity has been added. @@ -80,25 +80,17 @@ protected: /// Sends notifications to subscribers about changes in workload entities /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). - void sendNotifications(); + void unlockAndNotify(std::unique_lock & lock); struct Handlers { std::mutex mutex; - std::list by_type[static_cast(WorkloadEntityType::MAX)]; + std::list list; }; /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. std::shared_ptr handlers; - struct Event - { - WorkloadEntityType type; - String name; - ASTPtr entity; - }; - std::queue queue; - std::mutex queue_mutex; - std::mutex sending_notifications; + std::vector queue; mutable std::recursive_mutex mutex; std::unordered_map entities; // Maps entity name into CREATE entity query From 64359a54fd82491a41dba78cdf6259569c6c5e6b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 14:35:48 +0000 Subject: [PATCH 033/157] rename: DynamicResourceManager -> CustomResourceManager --- ...eManager.cpp => CustomResourceManager.cpp} | 26 +++++++++---------- ...ourceManager.h => CustomResourceManager.h} | 8 +++--- ....cpp => gtest_custom_resource_manager.cpp} | 8 +++--- .../Scheduler/createResourceManager.cpp | 4 +-- 4 files changed, 24 insertions(+), 22 deletions(-) rename src/Common/Scheduler/Nodes/{DynamicResourceManager.cpp => CustomResourceManager.cpp} (87%) rename src/Common/Scheduler/Nodes/{DynamicResourceManager.h => CustomResourceManager.h} (90%) rename src/Common/Scheduler/Nodes/tests/{gtest_dynamic_resource_manager.cpp => gtest_custom_resource_manager.cpp} (94%) diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp similarity index 87% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.cpp rename to src/Common/Scheduler/Nodes/CustomResourceManager.cpp index 88b4eec063d..caaae11cdc7 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -20,7 +20,7 @@ namespace ErrorCodes extern const int INVALID_SCHEDULER_NODE; } -DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) +CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config) : classifiers(config) { Poco::Util::AbstractConfiguration::Keys keys; @@ -34,7 +34,7 @@ DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util: } } -DynamicResourceManager::State::Resource::Resource( +CustomResourceManager::State::Resource::Resource( const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, @@ -91,7 +91,7 @@ DynamicResourceManager::State::Resource::Resource( throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name); } -DynamicResourceManager::State::Resource::~Resource() +CustomResourceManager::State::Resource::~Resource() { // NOTE: we should rely on `attached_to` and cannot use `parent`, // NOTE: because `parent` can be `nullptr` in case attachment is still in event queue @@ -105,14 +105,14 @@ DynamicResourceManager::State::Resource::~Resource() } } -DynamicResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +CustomResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) : type(config.getString(config_prefix + ".type", "fifo")) , ptr(SchedulerNodeFactory::instance().get(type, event_queue, config, config_prefix)) { ptr->basename = name; } -bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManager::State::Resource & o) const +bool CustomResourceManager::State::Resource::equals(const CustomResourceManager::State::Resource & o) const { if (nodes.size() != o.nodes.size()) return false; @@ -129,14 +129,14 @@ bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManage return true; } -bool DynamicResourceManager::State::Node::equals(const DynamicResourceManager::State::Node & o) const +bool CustomResourceManager::State::Node::equals(const CustomResourceManager::State::Node & o) const { if (type != o.type) return false; return ptr->equals(o.ptr.get()); } -DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::StatePtr & state_, const String & classifier_name) +CustomResourceManager::Classifier::Classifier(const CustomResourceManager::StatePtr & state_, const String & classifier_name) : state(state_) { // State is immutable, but nodes are mutable and thread-safe @@ -161,7 +161,7 @@ DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::Sta } } -ResourceLink DynamicResourceManager::Classifier::get(const String & resource_name) +ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) return iter->second; @@ -169,13 +169,13 @@ ResourceLink DynamicResourceManager::Classifier::get(const String & resource_nam throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); } -DynamicResourceManager::DynamicResourceManager() +CustomResourceManager::CustomResourceManager() : state(new State()) { scheduler.start(); } -void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { StatePtr new_state = std::make_shared(scheduler.event_queue, config); @@ -217,7 +217,7 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } -ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) +ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state StatePtr state_ref; @@ -229,7 +229,7 @@ ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) return std::make_shared(state_ref, classifier_name); } -void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) +void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { // Acquire a reference to the current state StatePtr state_ref; diff --git a/src/Common/Scheduler/Nodes/DynamicResourceManager.h b/src/Common/Scheduler/Nodes/CustomResourceManager.h similarity index 90% rename from src/Common/Scheduler/Nodes/DynamicResourceManager.h rename to src/Common/Scheduler/Nodes/CustomResourceManager.h index 4b0a3a48b61..c78fe672b33 100644 --- a/src/Common/Scheduler/Nodes/DynamicResourceManager.h +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.h @@ -10,7 +10,9 @@ namespace DB { /* - * Implementation of `IResourceManager` supporting arbitrary dynamic hierarchy of scheduler nodes. + * Implementation of `IResourceManager` supporting arbitrary hierarchy of scheduler nodes. + * Scheduling hierarchies for every resource is described through server xml or yaml configuration. + * Configuration could be changed dynamically without server restart. * All resources are controlled by single root `SchedulerRoot`. * * State of manager is set of resources attached to the scheduler. States are referenced by classifiers. @@ -24,10 +26,10 @@ namespace DB * violation will apply to fairness. Old version exists as long as there is at least one classifier * instance referencing it. Classifiers are typically attached to queries and will be destructed with them. */ -class DynamicResourceManager : public IResourceManager +class CustomResourceManager : public IResourceManager { public: - DynamicResourceManager(); + CustomResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp similarity index 94% rename from src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp rename to src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp index 3328196cced..495654d45ce 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp @@ -2,15 +2,15 @@ #include -#include +#include #include using namespace DB; -using ResourceTest = ResourceTestManager; +using ResourceTest = ResourceTestManager; using TestGuard = ResourceTest::Guard; -TEST(SchedulerDynamicResourceManager, Smoke) +TEST(SchedulerCustomResourceManager, Smoke) { ResourceTest t; @@ -49,7 +49,7 @@ TEST(SchedulerDynamicResourceManager, Smoke) } } -TEST(SchedulerDynamicResourceManager, Fairness) +TEST(SchedulerCustomResourceManager, Fairness) { // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). // Requests from A use `value = 1` and from B `value = -1` is used. diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b71b450979f..b6fc0b4f01c 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -9,7 +9,7 @@ namespace DB ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - // TODO(serxa): combine DynamicResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled + // TODO(serxa): combine CustomResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled // const auto & config = global_context->getConfigRef(); return std::make_shared(global_context->getWorkloadEntityStorage()); } From 1053530a86336ce62fcef81a20bd4bd2a8c47798 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 22 Sep 2024 15:39:01 +0000 Subject: [PATCH 034/157] add ResourceManagerDispatcher to combine io and custom managers --- src/Common/Scheduler/IResourceManager.h | 6 ++ .../Scheduler/Nodes/ClassifiersConfig.cpp | 3 +- .../Scheduler/Nodes/ClassifiersConfig.h | 1 + .../Scheduler/Nodes/CustomResourceManager.cpp | 11 +++ .../Scheduler/Nodes/CustomResourceManager.h | 4 +- .../Scheduler/Nodes/IOResourceManager.cpp | 19 +++- .../Scheduler/Nodes/IOResourceManager.h | 4 +- .../Scheduler/createResourceManager.cpp | 93 ++++++++++++++++++- 8 files changed, 133 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/IResourceManager.h b/src/Common/Scheduler/IResourceManager.h index c4a5c590ba7..b6199c91db7 100644 --- a/src/Common/Scheduler/IResourceManager.h +++ b/src/Common/Scheduler/IResourceManager.h @@ -26,6 +26,9 @@ class IClassifier : private boost::noncopyable public: virtual ~IClassifier() = default; + /// Returns true iff resource access is allowed by this classifier + virtual bool has(const String & resource_name) = 0; + /// Returns ResourceLink that should be used to access resource. /// Returned link is valid until classifier destruction. virtual ResourceLink get(const String & resource_name) = 0; @@ -46,6 +49,9 @@ public: /// Initialize or reconfigure manager. virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0; + /// Returns true iff given resource is controlled though this manager. + virtual bool hasResource(const String & resource_name) const = 0; + /// Obtain a classifier instance required to get access to resources. /// Note that it holds resource configuration, so should be destructed when query is done. virtual ClassifierPtr acquire(const String & classifier_name) = 0; diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 192f97645a0..4b0b0eaccfa 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -31,10 +31,11 @@ ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & c const ClassifierDescription & ClassifiersConfig::get(const String & classifier_name) { + static ClassifierDescription empty; if (auto it = classifiers.find(classifier_name); it != classifiers.end()) return it->second; else - throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name); + return empty; } } diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.h b/src/Common/Scheduler/Nodes/ClassifiersConfig.h index 186c49943ad..62db719568b 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.h +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.h @@ -10,6 +10,7 @@ namespace DB /// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class") struct ClassifierDescription : std::unordered_map { + ClassifierDescription() = default; ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); }; diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp index caaae11cdc7..0559b3cae0a 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.cpp @@ -161,6 +161,11 @@ CustomResourceManager::Classifier::Classifier(const CustomResourceManager::State } } +bool CustomResourceManager::Classifier::has(const String & resource_name) +{ + return resources.find(resource_name) != resources.end(); +} + ResourceLink CustomResourceManager::Classifier::get(const String & resource_name) { if (auto iter = resources.find(resource_name); iter != resources.end()) @@ -217,6 +222,12 @@ void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfig // NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable } +bool CustomResourceManager::hasResource(const String & resource_name) const +{ + std::lock_guard lock{mutex}; + return state->resources.find(resource_name) != state->resources.end(); +} + ClassifierPtr CustomResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state diff --git a/src/Common/Scheduler/Nodes/CustomResourceManager.h b/src/Common/Scheduler/Nodes/CustomResourceManager.h index c78fe672b33..900a9c4e50b 100644 --- a/src/Common/Scheduler/Nodes/CustomResourceManager.h +++ b/src/Common/Scheduler/Nodes/CustomResourceManager.h @@ -31,6 +31,7 @@ class CustomResourceManager : public IResourceManager public: CustomResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & classifier_name) override; void forEachNode(VisitorFunc visitor) override; @@ -81,6 +82,7 @@ private: { public: Classifier(const StatePtr & state_, const String & classifier_name); + bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; private: std::unordered_map resources; // accessible resources by names @@ -88,7 +90,7 @@ private: }; SchedulerRoot scheduler; - std::mutex mutex; + mutable std::mutex mutex; StatePtr state; }; diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 0c204afa97f..e956cca1862 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -382,6 +382,12 @@ std::future IOResourceManager::Resource::detachClassifier(VersionPtr && ve return future; } +bool IOResourceManager::Classifier::has(const String & resource_name) +{ + std::unique_lock lock{mutex}; + return attachments.find(resource_name) != attachments.end(); +} + ResourceLink IOResourceManager::Classifier::get(const String & resource_name) { std::unique_lock lock{mutex}; @@ -402,7 +408,7 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla { auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics auto future = attach_promise->get_future(); - scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] mutable + scheduler.event_queue->enqueue([&, this, promise = std::move(attach_promise)] { try { @@ -415,7 +421,10 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla classifier.attach(shared_from_this(), current_version, ResourceLink{.queue = queue.get()}); } else - throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "Unable to find workload '{}' for resource '{}'", workload_name, resource_name); + { + // This resource does not have specified workload. It is either unknown or managed by another resource manager. + // We leave this resource not attached to the classifier. Access denied will be thrown later on `classifier->get(resource_name)` + } promise->set_value(); } catch (...) @@ -426,6 +435,12 @@ std::future IOResourceManager::Resource::attachClassifier(Classifier & cla return future; } +bool IOResourceManager::hasResource(const String & resource_name) const +{ + std::unique_lock lock{mutex}; + return resources.find(resource_name) != resources.end(); +} + ClassifierPtr IOResourceManager::acquire(const String & workload_name) { auto classifier = std::make_shared(); diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 0cb1887d1cd..f4871379456 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -124,6 +124,7 @@ public: explicit IOResourceManager(IWorkloadEntityStorage & storage_); ~IOResourceManager() override; void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; + bool hasResource(const String & resource_name) const override; ClassifierPtr acquire(const String & workload_name) override; void forEachNode(VisitorFunc visitor) override; @@ -234,6 +235,7 @@ private: /// Implements IClassifier interface /// NOTE: It is called from query threads (possibly multiple) + bool has(const String & resource_name) override; ResourceLink get(const String & resource_name) override; /// Attaches/detaches a specific resource @@ -264,7 +266,7 @@ private: IWorkloadEntityStorage & storage; scope_guard subscription; - std::mutex mutex; + mutable std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); std::unordered_map resources; // TSA_GUARDED_BY(mutex); }; diff --git a/src/Common/Scheduler/createResourceManager.cpp b/src/Common/Scheduler/createResourceManager.cpp index b6fc0b4f01c..fd9743dbf72 100644 --- a/src/Common/Scheduler/createResourceManager.cpp +++ b/src/Common/Scheduler/createResourceManager.cpp @@ -4,14 +4,101 @@ #include #include +#include +#include + + namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + +class ResourceManagerDispatcher : public IResourceManager +{ +private: + class Classifier : public IClassifier + { + public: + void addClassifier(const ClassifierPtr & classifier) + { + classifiers.push_back(classifier); + } + + bool has(const String & resource_name) override + { + for (const auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return true; + } + return false; + } + + ResourceLink get(const String & resource_name) override + { + for (auto & classifier : classifiers) + { + if (classifier->has(resource_name)) + return classifier->get(resource_name); + } + throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name); + } + private: + std::vector classifiers; // should be constant after initialization to avoid races + }; + +public: + void addManager(const ResourceManagerPtr & manager) + { + managers.push_back(manager); + } + + void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override + { + for (auto & manager : managers) + manager->updateConfiguration(config); + } + + bool hasResource(const String & resource_name) const override + { + for (const auto & manager : managers) + { + if (manager->hasResource(resource_name)) + return true; + } + return false; + } + + ClassifierPtr acquire(const String & workload_name) override + { + auto classifier = std::make_shared(); + for (const auto & manager : managers) + classifier->addClassifier(manager->acquire(workload_name)); + return classifier; + } + + void forEachNode(VisitorFunc visitor) override + { + for (const auto & manager : managers) + manager->forEachNode(visitor); + } + +private: + std::vector managers; // Should be constant after initialization to avoid races +}; + ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context) { - // TODO(serxa): combine CustomResourceManager and IOResourceManaged to work together, because now old ResourceManager is disabled - // const auto & config = global_context->getConfigRef(); - return std::make_shared(global_context->getWorkloadEntityStorage()); + auto dispatcher = std::make_shared(); + + // NOTE: if the same resource is described by both managers, then manager added earlier will be used. + dispatcher->addManager(std::make_shared()); + dispatcher->addManager(std::make_shared(global_context->getWorkloadEntityStorage())); + + return dispatcher; } } From 823271ddd666da379623fea99ad299e2afade42d Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 09:34:07 +0000 Subject: [PATCH 035/157] fix build: use new setting access style --- .../Workload/WorkloadEntityDiskStorage.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 51016fac4fb..5ffec270610 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -31,6 +31,13 @@ namespace fs = std::filesystem; namespace DB { +namespace Setting +{ + extern const SettingsUInt64 max_parser_backtracks; + extern const SettingsUInt64 max_parser_depth; + extern const SettingsBool fsync_metadata; +} + namespace ErrorCodes { extern const int DIRECTORY_DOESNT_EXIST; @@ -91,8 +98,8 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, entity_create_query.data() + entity_create_query.size(), "", 0, - global_context->getSettingsRef().max_parser_depth, - global_context->getSettingsRef().max_parser_backtracks); + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); return ast; } case WorkloadEntityType::Resource: @@ -104,8 +111,8 @@ ASTPtr WorkloadEntityDiskStorage::tryLoadEntity(WorkloadEntityType entity_type, entity_create_query.data() + entity_create_query.size(), "", 0, - global_context->getSettingsRef().max_parser_depth, - global_context->getSettingsRef().max_parser_backtracks); + global_context->getSettingsRef()[Setting::max_parser_depth], + global_context->getSettingsRef()[Setting::max_parser_backtracks]); return ast; } case WorkloadEntityType::MAX: return nullptr; @@ -225,7 +232,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( WriteBufferFromFile out(temp_file_path, create_statement.size()); writeString(create_statement, out); out.next(); - if (settings.fsync_metadata) + if (settings[Setting::fsync_metadata]) out.sync(); out.close(); From 956b40ec24c5b143a8d7b70d0bc1618326d4e328 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 09:51:16 +0000 Subject: [PATCH 036/157] add SETTINGS clause in CREATE WORKLOAD query --- .../Scheduler/Nodes/IOResourceManager.cpp | 82 ++++++++++-------- src/Common/Scheduler/SchedulingSettings.cpp | 83 +++++++++++++++++++ src/Common/Scheduler/SchedulingSettings.h | 11 +-- src/Parsers/ASTCreateWorkloadQuery.cpp | 38 ++++++--- src/Parsers/ASTCreateWorkloadQuery.h | 2 +- src/Parsers/ParserCreateWorkloadQuery.cpp | 54 +++++++++++- 6 files changed, 219 insertions(+), 51 deletions(-) create mode 100644 src/Common/Scheduler/SchedulingSettings.cpp diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index e956cca1862..34cdaf55ee6 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -43,8 +43,7 @@ IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resourc auto * create = typeid_cast(ast.get()); name = create->getWorkloadName(); parent = create->getWorkloadParent(); - // TODO(serxa): parse workload settings specifically for `resource_name` - UNUSED(resource_name); + settings.updateFromAST(create->settings, resource_name); } IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) @@ -205,21 +204,45 @@ IOResourceManager::Workload::Workload(IOResourceManager * resource_manager_, con : resource_manager(resource_manager_) , workload_entity(workload_entity_) { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->createNode(NodeInfo(workload_entity, resource_name)); + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->createNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } IOResourceManager::Workload::~Workload() { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->deleteNode(NodeInfo(workload_entity, resource_name)); + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->deleteNode(NodeInfo(workload_entity, resource_name)); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } void IOResourceManager::Workload::updateWorkload(const ASTPtr & new_entity) { - for (auto & [resource_name, resource] : resource_manager->resources) - resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); - workload_entity = new_entity; + try + { + for (auto & [resource_name, resource] : resource_manager->resources) + resource->updateNode(NodeInfo(workload_entity, resource_name), NodeInfo(new_entity, resource_name)); + workload_entity = new_entity; + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error in IOResourceManager: {}", + getCurrentExceptionMessage(/* with_stacktrace = */ true)); + } } String IOResourceManager::Workload::getParent() const @@ -233,36 +256,29 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) { - try + for (auto [entity_type, entity_name, entity] : events) { - for (auto [entity_type, entity_name, entity] : events) + switch (entity_type) { - switch (entity_type) + case WorkloadEntityType::Workload: { - case WorkloadEntityType::Workload: - { - if (entity) - createOrUpdateWorkload(entity_name, entity); - else - deleteWorkload(entity_name); - break; - } - case WorkloadEntityType::Resource: - { - if (entity) - createResource(entity_name, entity); - else - deleteResource(entity_name); - break; - } - case WorkloadEntityType::MAX: break; + if (entity) + createOrUpdateWorkload(entity_name, entity); + else + deleteWorkload(entity_name); + break; } + case WorkloadEntityType::Resource: + { + if (entity) + createResource(entity_name, entity); + else + deleteResource(entity_name); + break; + } + case WorkloadEntityType::MAX: break; } } - catch (...) - { - // TODO(serxa): handle CRUD errors - } }); } diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp new file mode 100644 index 00000000000..c7d8a19ce41 --- /dev/null +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -0,0 +1,83 @@ +#include +#include + + +namespace DB +{ + +void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) +{ + UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST + if (auto * set = typeid_cast(settings.get())) + { + std::optional new_weight; + std::optional new_priority; + std::optional new_max_speed; + std::optional new_max_burst; + std::optional new_max_requests; + std::optional new_max_cost; + + // Read changed setting values + for (const auto & [name, value] : set->changes) + { + // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage + // TODO(serxa): and probably we should add and persist version in filename for future changes + if (name == "weight") + new_weight = value.safeGet(); + else if (name == "priority") + new_priority = Priority{value.safeGet()}; + else if (name == "max_speed") + new_max_speed = value.safeGet(); + else if (name == "max_burst") + new_max_burst = value.safeGet(); + else if (name == "max_requests") + new_max_requests = value.safeGet(); + else if (name == "max_cost") + new_max_cost = value.safeGet(); + } + + // Read setting to be reset to default values + static SchedulingSettings default_settings; + bool reset_max_burst = false; + for (const String & name : set->default_settings) + { + if (name == "weight") + new_weight = default_settings.weight; + else if (name == "priority") + new_priority = default_settings.priority; + else if (name == "max_speed") + new_max_speed = default_settings.max_speed; + else if (name == "max_burst") + reset_max_burst = true; + else if (name == "max_requests") + new_max_requests = default_settings.max_requests; + else if (name == "max_cost") + new_max_cost = default_settings.max_cost; + } + if (reset_max_burst) + new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); + + // Save new values into the `this` object + // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value + if (new_weight) + weight = *new_weight; + if (new_priority) + priority = *new_priority; + if (new_max_speed) + { + max_speed = *new_max_speed; + // We always set max_burst if max_speed is changed. + // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed + if (!new_max_burst) + max_burst = default_burst_seconds * max_speed; + } + if (new_max_burst) + max_burst = *new_max_burst; + if (new_max_requests) + max_requests = *new_max_requests; + if (new_max_cost) + max_cost = *new_max_cost; + } +} + +} diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index 4c6eff2b1e9..6d790b34164 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -3,6 +3,7 @@ #include #include +#include #include @@ -12,14 +13,14 @@ namespace DB struct SchedulingSettings { /// Priority and weight among siblings - double weight = 1.0; + Float64 weight = 1.0; Priority priority; /// Throttling constraints. /// Up to 2 independent throttlers: one for average speed and one for peek speed. - static constexpr double default_burst_seconds = 1.0; - double max_speed = 0; // Zero means unlimited - double max_burst = 0; // default is `default_burst_seconds * max_speed` + static constexpr Float64 default_burst_seconds = 1.0; + Float64 max_speed = 0; // Zero means unlimited + Float64 max_burst = 0; // default is `default_burst_seconds * max_speed` /// Limits total number of concurrent resource requests that are allowed to consume static constexpr Int64 default_max_requests = std::numeric_limits::max(); @@ -32,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - // TODO(serxa): add helper functions for parsing, printing and validating + void updateFromAST(const ASTPtr & settings, const String & resource_name); }; } diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index a6906dbcf65..2d32f499438 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -15,33 +15,49 @@ ASTPtr ASTCreateWorkloadQuery::clone() const res->workload_name = workload_name->clone(); res->children.push_back(res->workload_name); - // TODO(serxa): clone settings + if (workload_parent) + { + res->workload_parent = workload_parent->clone(); + res->children.push_back(res->workload_parent); + } + + if (settings) + { + res->settings = settings->clone(); + res->children.push_back(res->settings); + } return res; } -void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format_settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - settings.ostr << "OR REPLACE "; + format_settings.ostr << "OR REPLACE "; - settings.ostr << "WORKLOAD "; + format_settings.ostr << "WORKLOAD "; if (if_not_exists) - settings.ostr << "IF NOT EXISTS "; + format_settings.ostr << "IF NOT EXISTS "; - settings.ostr << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format_settings.hilite ? hilite_none : ""); - formatOnCluster(settings); + formatOnCluster(format_settings); if (hasParent()) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN " << (settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << " IN " << (format_settings.hilite ? hilite_none : ""); + format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format_settings.hilite ? hilite_none : ""); + } + + if (settings) + { + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' '; + settings->format(format_settings); } } diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h index bdd3a831aeb..a17bc4a11cd 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -12,7 +12,7 @@ class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster public: ASTPtr workload_name; ASTPtr workload_parent; - // TODO(serxa): add workload settings (weight and priority should also go inside settings, because they can differ for different resources) + ASTPtr settings; bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index ab0b0e3eb36..df7342093fd 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -2,13 +2,61 @@ #include #include +#include #include #include +#include +#include +#include namespace DB { +namespace +{ + +bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) + return false; + + SettingsChanges settings_changes; + + auto parse_setting = [&] + { + SettingChange setting; + if (ParserSetQuery::parseNameValuePair(setting, pos, expected)) + { + settings_changes.push_back(std::move(setting)); + // TODO(serxa): parse optional clause: [FOR resource_name] + return true; + } + + return false; + }; + + if (!ParserList::parseUtil(pos, expected, parse_setting, false)) + return false; + + ASTPtr res_settings; + if (!settings_changes.empty()) + { + auto settings_changes_ast = std::make_shared(); + settings_changes_ast->changes = std::move(settings_changes); + settings_changes_ast->is_standalone = false; + res_settings = settings_changes_ast; + } + + settings = std::move(res_settings); + return true; + }); +} + +} + bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create(Keyword::CREATE); @@ -18,7 +66,6 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp ParserIdentifier workload_name_p; ParserKeyword s_on(Keyword::ON); ParserKeyword s_in(Keyword::IN); - // TODO(serxa): parse workload settings ASTPtr workload_name; ASTPtr workload_parent; @@ -54,6 +101,9 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } + ASTPtr settings; + parseSettings(pos, expected, settings); + auto create_workload_query = std::make_shared(); node = create_workload_query; @@ -70,6 +120,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_workload_query->if_not_exists = if_not_exists; create_workload_query->cluster = std::move(cluster_str); + create_workload_query->settings = std::move(settings); + return true; } From 32a1766d15603b1b2f59b1a25214d8d0d3cefac4 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 11:37:46 +0000 Subject: [PATCH 037/157] add resource operations AST and parser --- src/Parsers/ASTCreateResourceQuery.cpp | 47 +++++++++++--- src/Parsers/ASTCreateResourceQuery.h | 13 +++- src/Parsers/ASTCreateWorkloadQuery.cpp | 24 ++++---- src/Parsers/CommonParsers.h | 2 + src/Parsers/ParserCreateResourceQuery.cpp | 75 ++++++++++++++++++++++- src/Parsers/ParserCreateWorkloadQuery.cpp | 19 +++++- 6 files changed, 155 insertions(+), 25 deletions(-) diff --git a/src/Parsers/ASTCreateResourceQuery.cpp b/src/Parsers/ASTCreateResourceQuery.cpp index adb3e0b6e45..73d9514bdd0 100644 --- a/src/Parsers/ASTCreateResourceQuery.cpp +++ b/src/Parsers/ASTCreateResourceQuery.cpp @@ -15,26 +15,57 @@ ASTPtr ASTCreateResourceQuery::clone() const res->resource_name = resource_name->clone(); res->children.push_back(res->resource_name); + res->operations = operations; + return res; } -void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - settings.ostr << "OR REPLACE "; + format.ostr << "OR REPLACE "; - settings.ostr << "RESOURCE "; + format.ostr << "RESOURCE "; if (if_not_exists) - settings.ostr << "IF NOT EXISTS "; + format.ostr << "IF NOT EXISTS "; - settings.ostr << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : ""); - formatOnCluster(settings); + formatOnCluster(format); + + format.ostr << " ("; + + bool first = true; + for (const auto & operation : operations) + { + if (!first) + format.ostr << ", "; + else + first = false; + + switch (operation.mode) + { + case AccessMode::Read: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "READ DISK "; + break; + } + case AccessMode::Write: + { + format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE DISK "; + break; + } + } + format.ostr << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(operation.disk) << (format.hilite ? hilite_none : ""); + } + + format.ostr << ")"; } String ASTCreateResourceQuery::getResourceName() const diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index 3d571807ec4..e1713e6b063 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -10,8 +10,19 @@ namespace DB class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster { public: + enum class AccessMode { + Read, + Write + }; + struct Operation { + AccessMode mode; + String disk; + }; + + using Operations = std::vector; + ASTPtr resource_name; - // TODO(serxa): add resource definition + Operations operations; /// List of operations that require this resource bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index 2d32f499438..869dc64daf7 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -30,34 +30,34 @@ ASTPtr ASTCreateWorkloadQuery::clone() const return res; } -void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format_settings, IAST::FormatState &, IAST::FormatStateStacked) const +void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const { - format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << "CREATE "; + format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE "; if (or_replace) - format_settings.ostr << "OR REPLACE "; + format.ostr << "OR REPLACE "; - format_settings.ostr << "WORKLOAD "; + format.ostr << "WORKLOAD "; if (if_not_exists) - format_settings.ostr << "IF NOT EXISTS "; + format.ostr << "IF NOT EXISTS "; - format_settings.ostr << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_none : ""); - format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : ""); - formatOnCluster(format_settings); + formatOnCluster(format); if (hasParent()) { - format_settings.ostr << (format_settings.hilite ? hilite_keyword : "") << " IN " << (format_settings.hilite ? hilite_none : ""); - format_settings.ostr << (format_settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format_settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); } if (settings) { - format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' '; - settings->format(format_settings); + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; + settings->format(format); } } diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 823c78c6f19..49708d8a40b 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -389,6 +389,7 @@ namespace DB MR_MACROS(RANDOMIZE_FOR, "RANDOMIZE FOR") \ MR_MACROS(RANDOMIZED, "RANDOMIZED") \ MR_MACROS(RANGE, "RANGE") \ + MR_MACROS(READ, "READ") \ MR_MACROS(READONLY, "READONLY") \ MR_MACROS(REALM, "REALM") \ MR_MACROS(RECOMPRESS, "RECOMPRESS") \ @@ -533,6 +534,7 @@ namespace DB MR_MACROS(WITH, "WITH") \ MR_MACROS(RECURSIVE, "RECURSIVE") \ MR_MACROS(WK, "WK") \ + MR_MACROS(WRITE, "WRITE") \ MR_MACROS(WRITABLE, "WRITABLE") \ MR_MACROS(WW, "WW") \ MR_MACROS(YEAR, "YEAR") \ diff --git a/src/Parsers/ParserCreateResourceQuery.cpp b/src/Parsers/ParserCreateResourceQuery.cpp index 4921debdf52..1abacaee617 100644 --- a/src/Parsers/ParserCreateResourceQuery.cpp +++ b/src/Parsers/ParserCreateResourceQuery.cpp @@ -4,11 +4,79 @@ #include #include #include +#include namespace DB { +namespace +{ + +bool parseOneOperation(ASTCreateResourceQuery::Operation & operation, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier disk_name_p; + + ASTCreateResourceQuery::AccessMode mode; + ASTPtr node; + String disk; + + if (ParserKeyword(Keyword::WRITE).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Write; + else if (ParserKeyword(Keyword::READ).ignore(pos, expected)) + mode = ASTCreateResourceQuery::AccessMode::Read; + else + return false; + + if (!ParserKeyword(Keyword::DISK).ignore(pos, expected)) + return false; + + if (!disk_name_p.parse(pos, node, expected)) + return false; + + if (!tryGetIdentifierNameInto(node, disk)) + return false; + + operation.mode = mode; + operation.disk = std::move(disk); + + return true; +} + +bool parseOperations(IParser::Pos & pos, Expected & expected, ASTCreateResourceQuery::Operations & operations) +{ + return IParserBase::wrapParseImpl(pos, [&] + { + ParserToken s_open(TokenType::OpeningRoundBracket); + ParserToken s_close(TokenType::ClosingRoundBracket); + + if (!s_open.ignore(pos, expected)) + return false; + + ASTCreateResourceQuery::Operations res_operations; + + auto parse_operation = [&] + { + ASTCreateResourceQuery::Operation operation; + if (!parseOneOperation(operation, pos, expected)) + return false; + res_operations.push_back(std::move(operation)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_operation, false)) + return false; + + if (!s_close.ignore(pos, expected)) + return false; + + operations = std::move(res_operations); + return true; + }); +} + +} + bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create(Keyword::CREATE); @@ -17,7 +85,6 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS); ParserKeyword s_on(Keyword::ON); ParserIdentifier resource_name_p; - // TODO(serxa): parse resource definition ASTPtr resource_name; @@ -46,6 +113,10 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } + ASTCreateResourceQuery::Operations operations; + if (!parseOperations(pos, expected, operations)) + return false; + auto create_resource_query = std::make_shared(); node = create_resource_query; @@ -56,6 +127,8 @@ bool ParserCreateResourceQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_resource_query->if_not_exists = if_not_exists; create_resource_query->cluster = std::move(cluster_str); + create_resource_query->operations = std::move(operations); + return true; } diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index df7342093fd..427d9aa40be 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -24,15 +24,28 @@ bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) return false; SettingsChanges settings_changes; + Strings default_settings; auto parse_setting = [&] { SettingChange setting; - if (ParserSetQuery::parseNameValuePair(setting, pos, expected)) + String default_setting; + std::pair parameter; + + if (ParserSetQuery::parseNameValuePairWithParameterOrDefault(setting, default_setting, parameter, pos, expected)) { - settings_changes.push_back(std::move(setting)); + if (!default_setting.empty()) + { + default_settings.push_back(std::move(default_setting)); + return true; + } + if (!setting.name.empty()) + { + settings_changes.push_back(std::move(setting)); + return true; + } // TODO(serxa): parse optional clause: [FOR resource_name] - return true; + return false; // We do not support parameters } return false; From e713cd938159673ed0198f693ab49370e7620b6b Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 12:21:43 +0000 Subject: [PATCH 038/157] better parsing and validation of WORKLOAD settings --- src/Common/Scheduler/SchedulingSettings.cpp | 67 +++++++++++++++++++-- 1 file changed, 62 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index c7d8a19ce41..18a465fe930 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -1,10 +1,17 @@ +#include #include +#include #include namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) { UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST @@ -17,23 +24,68 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r std::optional new_max_requests; std::optional new_max_cost; + auto get_not_negative_float64 = [] (const String & name, const Field & field) { + { + UInt64 val; + if (field.tryGet(val)) + return static_cast(val); // We dont mind slight loss of precision + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return static_cast(val); // We dont mind slight loss of precision + } + } + + return field.safeGet(); + }; + + auto get_not_negative_int64 = [] (const String & name, const Field & field) { + { + UInt64 val; + if (field.tryGet(val)) + { + // Saturate on overflow + if (val > static_cast(std::numeric_limits::max())) + val = std::numeric_limits::max(); + return static_cast(val); + } + } + + { + Int64 val; + if (field.tryGet(val)) + { + if (val < 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected negative Int64 value for workload setting '{}'", name); + return val; + } + } + + return field.safeGet(); + }; + // Read changed setting values for (const auto & [name, value] : set->changes) { // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage // TODO(serxa): and probably we should add and persist version in filename for future changes if (name == "weight") - new_weight = value.safeGet(); + new_weight = get_not_negative_float64(name, value); else if (name == "priority") new_priority = Priority{value.safeGet()}; else if (name == "max_speed") - new_max_speed = value.safeGet(); + new_max_speed = get_not_negative_float64(name, value); else if (name == "max_burst") - new_max_burst = value.safeGet(); + new_max_burst = get_not_negative_float64(name, value); else if (name == "max_requests") - new_max_requests = value.safeGet(); + new_max_requests = get_not_negative_int64(name, value); else if (name == "max_cost") - new_max_cost = value.safeGet(); + new_max_cost = get_not_negative_int64(name, value); } // Read setting to be reset to default values @@ -57,6 +109,11 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r if (reset_max_burst) new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); + // Validate we could use values we read in a scheduler node + { + SchedulerNodeInfo validating_node(new_weight ? *new_weight : weight, new_priority ? *new_priority : priority); + } + // Save new values into the `this` object // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value if (new_weight) From 709c8489f7548aee4730079b0322a19600079e38 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 23 Sep 2024 12:28:07 +0000 Subject: [PATCH 039/157] add simple integration tests --- tests/integration/test_scheduler/test.py | 160 +++++++++++++++++++++++ 1 file changed, 160 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 31cc106a95d..02b1b4c2169 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -54,6 +54,20 @@ def set_default_configs(): yield +@pytest.fixture(scope="function", autouse=True) +def clear_workloads_and_resources(): + node.query( + f""" + -- drop resource if exist io_write; TODO(serxa): uncomment it + -- drop resource if exist io_read; + -- drop workload if exist production; + -- drop workload if exist development; + -- drop workload if exist all; + """ + ) + yield + + def update_workloads_config(**settings): xml = "" for name in settings: @@ -569,3 +583,149 @@ def test_mutation_workload_change(): assert reads_before < reads_after assert writes_before < writes_after + + +def test_create_workload(): + node.query( + f""" + create resource io_write (write disk s3); + create resource io_read (read disk s3); + create workload all settings max_cost = 1000000; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def do_checks(): + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/admin' and type='unified' and priority=0" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production/%' and type='fifo'" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/production' and type='unified' and weight=9" + ) + == "2\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/development/%' and type='fifo'" + ) + == "2\n" + ) + + do_checks() + node.restart_clickhouse() # Check that workloads persist + do_checks() + + + +def test_resource_read_and_write(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + node.query( + f""" + create resource io_write (write disk s3); + create resource io_read (read disk s3); + create workload all settings max_cost = 1000000; + create workload admin in all settings priority = 0; + create workload production in all settings priority = 1, weight = 9; + create workload development in all settings priority = 1, weight = 1; + """ + ) + + def write_query(workload): + try: + node.query( + f"insert into data select * from numbers(1e5) settings workload='{workload}'" + ) + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=write_query, args=["development"]) + thread2 = threading.Thread(target=write_query, args=["production"]) + thread3 = threading.Thread(target=write_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_write' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) + + def read_query(workload): + try: + node.query(f"select sum(key*key) from data settings workload='{workload}'") + except QueryRuntimeException: + pass + + thread1 = threading.Thread(target=read_query, args=["development"]) + thread2 = threading.Thread(target=read_query, args=["production"]) + thread3 = threading.Thread(target=read_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/admin/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/development/%' and type='fifo'" + ) + == "1\n" + ) + assert ( + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='io_read' and path ilike '%/production/%' and type='fifo'" + ) + == "1\n" + ) From 9db958dcdc494a2c08d4cb08f741d9569a4f1dfa Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 15:52:29 +0000 Subject: [PATCH 040/157] integrate DiskObjectStorage and RESOURCEs --- .../ObjectStorages/DiskObjectStorage.cpp | 65 ++++++++++++++++--- src/Disks/ObjectStorages/DiskObjectStorage.h | 9 ++- src/Interpreters/Context.cpp | 18 +---- src/Interpreters/Context.h | 4 +- 4 files changed, 65 insertions(+), 31 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 07e2edac129..a20ee53ff75 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -15,7 +15,8 @@ #include #include #include - +#include +#include namespace DB { @@ -68,8 +69,8 @@ DiskObjectStorage::DiskObjectStorage( , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) - , read_resource_name(config.getString(config_prefix + ".read_resource", "")) - , write_resource_name(config.getString(config_prefix + ".write_resource", "")) + , read_resource_name_from_config(config.getString(config_prefix + ".read_resource", "")) + , write_resource_name_from_config(config.getString(config_prefix + ".write_resource", "")) , metadata_helper(std::make_unique(this, ReadSettings{}, WriteSettings{})) { data_source_description = DataSourceDescription{ @@ -80,6 +81,52 @@ DiskObjectStorage::DiskObjectStorage( .is_encrypted = false, .is_cached = object_storage->supportsCache(), }; + resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe( + [this] (const std::vector & events) + { + std::unique_lock lock{resource_mutex}; + for (auto [entity_type, resource_name, resource] : events) + { + if (entity_type == WorkloadEntityType::Resource) + { + if (resource) // CREATE RESOURCE + { + // We rely on the fact that every disk is allowed to be mentioned at most + // in one RESOURCE for READ and in one RESOURCE for WRITE + // TODO(serxa): add disk operations validation in workload entity storage + auto * create = typeid_cast(resource.get()); + chassert(create); + for (const auto & [mode, disk] : create->operations) + { + if (disk == name) + { + switch (mode) + { + case ASTCreateResourceQuery::AccessMode::Read: + { + read_resource_name_from_sql = resource_name; + break; + } + case ASTCreateResourceQuery::AccessMode::Write: + { + write_resource_name_from_sql = resource_name; + break; + } + } + } + } + } + else // DROP RESOURCE + { + if (read_resource_name_from_sql == resource_name) + read_resource_name_from_sql.clear(); + if (write_resource_name_from_sql == resource_name) + write_resource_name_from_sql.clear(); + } + break; + } + } + }); } StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const @@ -480,13 +527,13 @@ static inline Settings updateIOSchedulingSettings(const Settings & settings, con String DiskObjectStorage::getReadResourceName() const { std::unique_lock lock(resource_mutex); - return read_resource_name; + return read_resource_name_from_config.empty() ? read_resource_name_from_sql : read_resource_name_from_config; } String DiskObjectStorage::getWriteResourceName() const { std::unique_lock lock(resource_mutex); - return write_resource_name; + return write_resource_name_from_config.empty() ? write_resource_name_from_sql : write_resource_name_from_config; } std::unique_ptr DiskObjectStorage::readFile( @@ -551,10 +598,10 @@ void DiskObjectStorage::applyNewSettings( { std::unique_lock lock(resource_mutex); - if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name) - read_resource_name = new_read_resource_name; - if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name) - write_resource_name = new_write_resource_name; + if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name_from_config) + read_resource_name_from_config = new_read_resource_name; + if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name_from_config) + write_resource_name_from_config = new_write_resource_name; } IDisk::applyNewSettings(config, context_, config_prefix, disk_map); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 5c45a258806..d4d4dc11ffa 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -6,6 +6,8 @@ #include #include +#include + #include "config.h" @@ -242,8 +244,11 @@ private: const bool send_metadata; mutable std::mutex resource_mutex; - String read_resource_name; - String write_resource_name; + String read_resource_name_from_config; // specified in disk config.xml + String write_resource_name_from_config; // specified in disk config.xml + String read_resource_name_from_sql; // described by CREATE RESOURCE queries + String write_resource_name_from_sql; // described by CREATE RESOURCE queries + scope_guard resource_changes_subscription; std::unique_ptr metadata_helper; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d3cbbf76156..5de1dece884 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2979,17 +2979,7 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptruser_defined_sql_objects_storage = std::move(storage); } -const IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const -{ - callOnce(shared->workload_entity_storage_initialized, [&] { - shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); - }); - - SharedLockGuard lock(shared->mutex); - return *shared->workload_entity_storage; -} - -IWorkloadEntityStorage & Context::getWorkloadEntityStorage() +IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const { callOnce(shared->workload_entity_storage_initialized, [&] { shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext()); @@ -2999,12 +2989,6 @@ IWorkloadEntityStorage & Context::getWorkloadEntityStorage() return *shared->workload_entity_storage; } -void Context::setWorkloadEntityStorage(std::unique_ptr storage) -{ - std::lock_guard lock(shared->mutex); - shared->workload_entity_storage = std::move(storage); -} - #if USE_NLP SynonymsExtensions & Context::getSynonymsExtensions() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index be963d85757..114e2c96570 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -882,9 +882,7 @@ public: void setUserDefinedSQLObjectsStorage(std::unique_ptr storage); void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config); - const IWorkloadEntityStorage & getWorkloadEntityStorage() const; - IWorkloadEntityStorage & getWorkloadEntityStorage(); - void setWorkloadEntityStorage(std::unique_ptr storage); + IWorkloadEntityStorage & getWorkloadEntityStorage() const; #if USE_NLP SynonymsExtensions & getSynonymsExtensions() const; From d57c28aa52c193a95928aeb6ed6c4c6c635afc6f Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 16:47:54 +0000 Subject: [PATCH 041/157] add logs for resource to disk mapping --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index a20ee53ff75..166435064da 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -104,11 +104,21 @@ DiskObjectStorage::DiskObjectStorage( { case ASTCreateResourceQuery::AccessMode::Read: { + if (read_resource_name_from_config.empty()) + LOG_INFO(log, "Using resource '{}' for READ", resource_name); + else + LOG_INFO(log, "Resource '{}' should be used for READ, but it is overriden by config to resource '{}'", + resource_name, read_resource_name_from_config); read_resource_name_from_sql = resource_name; break; } case ASTCreateResourceQuery::AccessMode::Write: { + if (write_resource_name_from_config.empty()) + LOG_INFO(log, "Using resource '{}' for WRITE", resource_name); + else + LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overriden by config to resource '{}'", + resource_name, write_resource_name_from_config); write_resource_name_from_sql = resource_name; break; } @@ -119,9 +129,15 @@ DiskObjectStorage::DiskObjectStorage( else // DROP RESOURCE { if (read_resource_name_from_sql == resource_name) + { + LOG_INFO(log, "Stop using resource '{}' for READ", resource_name); read_resource_name_from_sql.clear(); + } if (write_resource_name_from_sql == resource_name) + { + LOG_INFO(log, "Stop using resource '{}' for WRITE", resource_name); write_resource_name_from_sql.clear(); + } } break; } From 0a7bd6010bb1cee2fe415901c272b006d401cfd0 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 16:48:29 +0000 Subject: [PATCH 042/157] fix test_resource_read_and_write --- .../configs/storage_configuration.xml | 16 ++++++++++++++++ tests/integration/test_scheduler/test.py | 10 +++++----- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 823a00a05de..16cdf4a5b15 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -12,6 +12,15 @@ network_read network_write + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + @@ -21,6 +30,13 @@ + + +
+ s3_no_resource +
+
+
diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 02b1b4c2169..311bd6d7401 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -588,8 +588,8 @@ def test_mutation_workload_change(): def test_create_workload(): node.query( f""" - create resource io_write (write disk s3); - create resource io_read (read disk s3); + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); create workload all settings max_cost = 1000000; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; @@ -639,14 +639,14 @@ def test_resource_read_and_write(): node.query( f""" drop table if exists data; - create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3_no_resource'; """ ) node.query( f""" - create resource io_write (write disk s3); - create resource io_read (read disk s3); + create resource io_write (write disk s3_no_resource); + create resource io_read (read disk s3_no_resource); create workload all settings max_cost = 1000000; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; From 407e56adb6936b5c8a407edd23caac870b0045c0 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 18:17:27 +0000 Subject: [PATCH 043/157] fix all scheduler integration tests --- tests/integration/test_scheduler/test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 311bd6d7401..401444b2d5a 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -58,11 +58,12 @@ def set_default_configs(): def clear_workloads_and_resources(): node.query( f""" - -- drop resource if exist io_write; TODO(serxa): uncomment it - -- drop resource if exist io_read; - -- drop workload if exist production; - -- drop workload if exist development; - -- drop workload if exist all; + drop resource if exists io_write; + drop resource if exists io_read; + drop workload if exists production; + drop workload if exists development; + drop workload if exists admin; + drop workload if exists all; """ ) yield From 52204768b03c5674601cc8ac7ccccfedf5af9fdf Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 24 Sep 2024 18:19:23 +0000 Subject: [PATCH 044/157] add reference validation in workload entity storage --- .../Scheduler/Nodes/IOResourceManager.cpp | 3 + .../Workload/WorkloadEntityStorageBase.cpp | 67 ++++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 2 + 3 files changed, 61 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 34cdaf55ee6..460693e1935 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -307,7 +307,10 @@ void IOResourceManager::deleteWorkload(const String & workload_name) { std::unique_lock lock{mutex}; if (auto workload_iter = workloads.find(workload_name); workload_iter != workloads.end()) + { + // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent workloads.erase(workload_iter); + } else { // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 8e7f630365d..f0c76b92870 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -52,6 +52,21 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +void forEachReference(const ASTPtr & source_entity, std::function func) +{ + if (auto * res = typeid_cast(source_entity.get())) + { + String parent = res->getWorkloadParent(); + if (!parent.empty()) + func(parent, res->getWorkloadName()); + // TODO(serxa): add references to RESOURCEs mentioned in SETTINGS clause after FOR keyword + } + if (auto * res = typeid_cast(source_entity.get())) + { + // RESOURCE has no references to be validated + } +} + void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) { if (visited.contains(name)) @@ -162,8 +177,7 @@ bool WorkloadEntityStorageBase::storeEntity( create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); - auto it = entities.find(entity_name); - if (it != entities.end()) + if (auto it = entities.find(entity_name); it != entities.end()) { if (throw_if_exists) throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); @@ -171,6 +185,13 @@ bool WorkloadEntityStorageBase::storeEntity( return false; } + forEachReference(create_entity_query, + [this] (const String & target, const String & source) + { + if (auto it = entities.find(target); it == entities.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + }); + bool stored = storeEntityImpl( current_context, entity_type, @@ -182,12 +203,16 @@ bool WorkloadEntityStorageBase::storeEntity( if (stored) { + forEachReference(create_entity_query, + [this] (const String & target, const String & source) + { + references[target].insert(source); + }); entities[entity_name] = create_entity_query; onEntityAdded(entity_type, entity_name, create_entity_query); + unlockAndNotify(lock); } - unlockAndNotify(lock); - return stored; } @@ -207,6 +232,14 @@ bool WorkloadEntityStorageBase::removeEntity( return false; } + if (auto reference_it = references.find(entity_name); reference_it != references.end()) + { + String names; + for (const String & name : reference_it->second) + names += " " + name; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); + } + bool removed = removeEntityImpl( current_context, entity_type, @@ -215,11 +248,18 @@ bool WorkloadEntityStorageBase::removeEntity( if (removed) { - entities.erase(entity_name); + forEachReference(it->second, + [this] (const String & target, const String & source) + { + references[target].erase(source); + if (references[target].empty()) + references.erase(target); + }); + entities.erase(it); onEntityRemoved(entity_type, entity_name); - } - unlockAndNotify(lock); + unlockAndNotify(lock); + } return removed; } @@ -300,10 +340,8 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } - void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) { - std::unordered_map normalized_entities; for (const auto & [entity_name, create_query] : new_entities) normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); @@ -313,6 +351,15 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector &) { std::unordered_map workloads; @@ -347,7 +393,6 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock> WorkloadEntityStorageBase::getAllEntities() const { std::lock_guard lock{mutex}; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index bf8a89a67c4..a51e2392ea4 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -94,6 +95,7 @@ protected: mutable std::recursive_mutex mutex; std::unordered_map entities; // Maps entity name into CREATE entity query + std::unordered_map> references; // Keep track of references between entities for validation ContextPtr global_context; }; From 53896e1a1f9366fd9c2e38b0c3045c8606a4aaaf Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 25 Sep 2024 13:21:10 +0000 Subject: [PATCH 045/157] add `system.resources` table --- .../System/StorageSystemResources.cpp | 72 +++++++++++++++++++ src/Storages/System/StorageSystemResources.h | 29 ++++++++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 103 insertions(+) create mode 100644 src/Storages/System/StorageSystemResources.cpp create mode 100644 src/Storages/System/StorageSystemResources.h diff --git a/src/Storages/System/StorageSystemResources.cpp b/src/Storages/System/StorageSystemResources.cpp new file mode 100644 index 00000000000..692f89358e7 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.cpp @@ -0,0 +1,72 @@ +#include +#include +#include +#include +#include +#include +#include "Parsers/ASTCreateQuery.h" +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemResources::getColumnsDescription() +{ + return ColumnsDescription + { + {"name", std::make_shared(), "The name of the resource."}, + {"read_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for read operations."}, + {"write_disks", std::make_shared(std::make_shared()), "The list of disk names that uses this resource for write operations."}, + {"create_query", std::make_shared(), "CREATE query of the resource."}, + }; +} + +void StorageSystemResources::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto & storage = context->getWorkloadEntityStorage(); + const auto & resource_names = storage.getAllEntityNames(WorkloadEntityType::Resource); + for (const auto & resource_name : resource_names) + { + auto ast = storage.get(resource_name); + auto & resource = typeid_cast(*ast); + res_columns[0]->insert(resource_name); + { + Array read_disks; + Array write_disks; + for (const auto & [mode, disk] : resource.operations) + { + switch (mode) + { + case DB::ASTCreateResourceQuery::AccessMode::Read: + { + read_disks.emplace_back(disk); + break; + } + case DB::ASTCreateResourceQuery::AccessMode::Write: + { + write_disks.emplace_back(disk); + break; + } + } + } + res_columns[1]->insert(read_disks); + res_columns[2]->insert(write_disks); + } + res_columns[3]->insert(queryToString(ast)); + } +} + +void StorageSystemResources::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add backup for resources + // storage.backup(backup_entries_collector, data_path_in_backup); +} + +void StorageSystemResources::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) +{ + // TODO(serxa): add restore for resources + // storage.restore(restorer, data_path_in_backup); +} + +} diff --git a/src/Storages/System/StorageSystemResources.h b/src/Storages/System/StorageSystemResources.h new file mode 100644 index 00000000000..42bbcd09aa4 --- /dev/null +++ b/src/Storages/System/StorageSystemResources.h @@ -0,0 +1,29 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +/// Implements `resources` system table, which allows you to get a list of all RESOURCEs +class StorageSystemResources final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemResources"; } + + static ColumnsDescription getColumnsDescription(); + + void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) override; + void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional & partitions) override; + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 728e83135a3..93d846a1d40 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -231,6 +232,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dashboards", "Contains queries used by /dashboard page accessible though HTTP interface. This table can be useful for monitoring and troubleshooting. The table contains a row for every chart in a dashboard."); attach(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server."); attach(context, system_database, "workloads", "Contains a list of all currently existing workloads."); + attach(context, system_database, "resources", "Contains a list of all currently existing resources."); if (has_zookeeper) { From fc49a4a146dcc7d447be476dc93eb4d8f29567a4 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 25 Sep 2024 13:21:48 +0000 Subject: [PATCH 046/157] add stateless test for CREATE and DROP RESOURCE queries --- .../0_stateless/03232_resource_create_and_drop.reference | 5 +++++ .../0_stateless/03232_resource_create_and_drop.sql | 9 +++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.reference create mode 100644 tests/queries/0_stateless/03232_resource_create_and_drop.sql diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.reference b/tests/queries/0_stateless/03232_resource_create_and_drop.reference new file mode 100644 index 00000000000..2a1045d314c --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.reference @@ -0,0 +1,5 @@ +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) +03232_resource_2 ['03232_disk_2'] [] CREATE RESOURCE `03232_resource_2` (READ DISK `03232_disk_2`) +03232_resource_3 [] ['03232_disk_2'] CREATE RESOURCE `03232_resource_3` (WRITE DISK `03232_disk_2`) +03232_resource_1 ['03232_disk_1'] ['03232_disk_1'] CREATE RESOURCE `03232_resource_1` (WRITE DISK `03232_disk_1`, READ DISK `03232_disk_1`) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql new file mode 100644 index 00000000000..eb356e12448 --- /dev/null +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.sql @@ -0,0 +1,9 @@ +CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); +CREATE RESOURCE 03232_resource_3 (WRITE DISK 03232_disk_2); +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE IF EXISTS 03232_resource_2; +DROP RESOURCE 03232_resource_3; +SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; +DROP RESOURCE 03232_resource_1; From 5e772899f69b9429f05e3257116af0652266fc5d Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 29 Sep 2024 18:38:58 +0000 Subject: [PATCH 047/157] add resource-specific workload settings --- .../Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/SchedulingSettings.cpp | 118 ++++++++---------- src/Common/Scheduler/SchedulingSettings.h | 4 +- src/Parsers/ASTCreateWorkloadQuery.cpp | 26 ++-- src/Parsers/ASTCreateWorkloadQuery.h | 22 +++- src/Parsers/ParserCreateWorkloadQuery.cpp | 88 +++++++------ .../System/StorageSystemWorkloads.cpp | 4 +- 7 files changed, 150 insertions(+), 114 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 460693e1935..cf6b041c9f1 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -43,7 +43,7 @@ IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resourc auto * create = typeid_cast(ast.get()); name = create->getWorkloadName(); parent = create->getWorkloadParent(); - settings.updateFromAST(create->settings, resource_name); + settings.updateFromChanges(create->changes, resource_name); } IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index 18a465fe930..352e61fb560 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -12,11 +12,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & resource_name) +// TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage +// TODO(serxa): and probably we should add and persist version in filename for future changes +void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) { - UNUSED(resource_name); // TODO(serxa): read resource specific settings from AST - if (auto * set = typeid_cast(settings.get())) - { + struct { std::optional new_weight; std::optional new_priority; std::optional new_max_speed; @@ -24,7 +24,8 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r std::optional new_max_requests; std::optional new_max_cost; - auto get_not_negative_float64 = [] (const String & name, const Field & field) { + static Float64 getNotNegativeFloat64(const String & name, const Field & field) + { { UInt64 val; if (field.tryGet(val)) @@ -42,9 +43,10 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r } return field.safeGet(); - }; + } - auto get_not_negative_int64 = [] (const String & name, const Field & field) { + static Int64 getNotNegativeInt64(const String & name, const Field & field) + { { UInt64 val; if (field.tryGet(val)) @@ -67,74 +69,64 @@ void SchedulingSettings::updateFromAST(const ASTPtr & settings, const String & r } return field.safeGet(); - }; + } - // Read changed setting values - for (const auto & [name, value] : set->changes) + void read(const String & name, const Field & value) { - // TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage - // TODO(serxa): and probably we should add and persist version in filename for future changes if (name == "weight") - new_weight = get_not_negative_float64(name, value); + new_weight = getNotNegativeFloat64(name, value); else if (name == "priority") new_priority = Priority{value.safeGet()}; else if (name == "max_speed") - new_max_speed = get_not_negative_float64(name, value); + new_max_speed = getNotNegativeFloat64(name, value); else if (name == "max_burst") - new_max_burst = get_not_negative_float64(name, value); + new_max_burst = getNotNegativeFloat64(name, value); else if (name == "max_requests") - new_max_requests = get_not_negative_int64(name, value); + new_max_requests = getNotNegativeInt64(name, value); else if (name == "max_cost") - new_max_cost = get_not_negative_int64(name, value); + new_max_cost = getNotNegativeInt64(name, value); } + } regular, specific; - // Read setting to be reset to default values - static SchedulingSettings default_settings; - bool reset_max_burst = false; - for (const String & name : set->default_settings) - { - if (name == "weight") - new_weight = default_settings.weight; - else if (name == "priority") - new_priority = default_settings.priority; - else if (name == "max_speed") - new_max_speed = default_settings.max_speed; - else if (name == "max_burst") - reset_max_burst = true; - else if (name == "max_requests") - new_max_requests = default_settings.max_requests; - else if (name == "max_cost") - new_max_cost = default_settings.max_cost; - } - if (reset_max_burst) - new_max_burst = default_burst_seconds * (new_max_speed ? *new_max_speed : max_speed); - - // Validate we could use values we read in a scheduler node - { - SchedulerNodeInfo validating_node(new_weight ? *new_weight : weight, new_priority ? *new_priority : priority); - } - - // Save new values into the `this` object - // Leave previous value intentionally for ALTER query to be able to skip not mentioned setting value - if (new_weight) - weight = *new_weight; - if (new_priority) - priority = *new_priority; - if (new_max_speed) - { - max_speed = *new_max_speed; - // We always set max_burst if max_speed is changed. - // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed - if (!new_max_burst) - max_burst = default_burst_seconds * max_speed; - } - if (new_max_burst) - max_burst = *new_max_burst; - if (new_max_requests) - max_requests = *new_max_requests; - if (new_max_cost) - max_cost = *new_max_cost; + // Read changed setting values + for (const auto & [name, value, resource] : changes) + { + if (resource.empty()) + regular.read(name, value); + else if (resource == resource_name) + specific.read(name, value); } + + auto get_value = [] (const std::optional & specific_new, const std::optional & regular_new, T & old) + { + if (specific_new) + return *specific_new; + if (regular_new) + return *regular_new; + return old; + }; + + // Validate that we could use values read in a scheduler node + { + SchedulerNodeInfo validating_node( + get_value(specific.new_weight, regular.new_weight, weight), + get_value(specific.new_priority, regular.new_priority, priority)); + } + + // Commit new values. + // Previous values are left intentionally for ALTER query to be able to skip not mentioned setting values + weight = get_value(specific.new_weight, regular.new_weight, weight); + priority = get_value(specific.new_priority, regular.new_priority, priority); + if (specific.new_max_speed || regular.new_max_speed) + { + max_speed = get_value(specific.new_max_speed, regular.new_max_speed, max_speed); + // We always set max_burst if max_speed is changed. + // This is done for users to be able to ignore more advanced max_burst setting and rely only on max_speed + max_burst = default_burst_seconds * max_speed; + } + max_burst = get_value(specific.new_max_burst, regular.new_max_burst, max_burst); + max_requests = get_value(specific.new_max_requests, regular.new_max_requests, max_requests); + max_cost = get_value(specific.new_max_cost, regular.new_max_cost, max_cost); } } diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index 6d790b34164..cda6b546dfc 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include @@ -33,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - void updateFromAST(const ASTPtr & settings, const String & resource_name); + void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name); }; } diff --git a/src/Parsers/ASTCreateWorkloadQuery.cpp b/src/Parsers/ASTCreateWorkloadQuery.cpp index 869dc64daf7..972ce733651 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.cpp +++ b/src/Parsers/ASTCreateWorkloadQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -21,11 +22,7 @@ ASTPtr ASTCreateWorkloadQuery::clone() const res->children.push_back(res->workload_parent); } - if (settings) - { - res->settings = settings->clone(); - res->children.push_back(res->settings); - } + res->changes = changes; return res; } @@ -54,10 +51,25 @@ void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAS format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : ""); } - if (settings) + if (!changes.empty()) { format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' '; - settings->format(format); + + bool first = true; + + for (const auto & change : changes) + { + if (!first) + format.ostr << ", "; + else + first = false; + format.ostr << change.name << " = " << applyVisitor(FieldVisitorToString(), change.value); + if (!change.resource.empty()) + { + format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "FOR" << (format.hilite ? hilite_none : "") << ' '; + format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(change.resource) << (format.hilite ? hilite_none : ""); + } + } } } diff --git a/src/Parsers/ASTCreateWorkloadQuery.h b/src/Parsers/ASTCreateWorkloadQuery.h index a17bc4a11cd..71e27295bc1 100644 --- a/src/Parsers/ASTCreateWorkloadQuery.h +++ b/src/Parsers/ASTCreateWorkloadQuery.h @@ -1,8 +1,9 @@ #pragma once +#include #include #include - +#include namespace DB { @@ -12,7 +13,24 @@ class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster public: ASTPtr workload_name; ASTPtr workload_parent; - ASTPtr settings; + + /// Special version of settings that support optional `FOR resource` clause + struct SettingChange + { + String name; + Field value; + String resource; + + SettingChange() = default; + SettingChange(std::string_view name_, const Field & value_, std::string_view resource_) : name(name_), value(value_), resource(resource_) {} + SettingChange(std::string_view name_, Field && value_, std::string_view resource_) : name(name_), value(std::move(value_)), resource(resource_) {} + + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.resource == rhs.resource); } + friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } + }; + + using SettingsChanges = std::vector; + SettingsChanges changes; bool or_replace = false; bool if_not_exists = false; diff --git a/src/Parsers/ParserCreateWorkloadQuery.cpp b/src/Parsers/ParserCreateWorkloadQuery.cpp index 427d9aa40be..9caf474741c 100644 --- a/src/Parsers/ParserCreateWorkloadQuery.cpp +++ b/src/Parsers/ParserCreateWorkloadQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -16,54 +17,67 @@ namespace DB namespace { -bool parseSettings(IParser::Pos & pos, Expected & expected, ASTPtr & settings) +bool parseWorkloadSetting( + ASTCreateWorkloadQuery::SettingChange & change, IParser::Pos & pos, Expected & expected) +{ + ParserIdentifier name_p; + ParserLiteral value_p; + ParserToken s_eq(TokenType::Equals); + ParserIdentifier resource_name_p; + + ASTPtr name_node; + ASTPtr value_node; + ASTPtr resource_name_node; + + String name; + String resource_name; + + if (!name_p.parse(pos, name_node, expected)) + return false; + tryGetIdentifierNameInto(name_node, name); + + if (!s_eq.ignore(pos, expected)) + return false; + + if (!value_p.parse(pos, value_node, expected)) + return false; + + if (ParserKeyword(Keyword::FOR).ignore(pos, expected)) + { + if (!resource_name_p.parse(pos, resource_name_node, expected)) + return false; + tryGetIdentifierNameInto(resource_name_node, resource_name); + } + + change.name = std::move(name); + change.value = value_node->as().value; + change.resource = std::move(resource_name); + + return true; +} + +bool parseSettings(IParser::Pos & pos, Expected & expected, ASTCreateWorkloadQuery::SettingsChanges & changes) { return IParserBase::wrapParseImpl(pos, [&] { if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected)) return false; - SettingsChanges settings_changes; - Strings default_settings; + ASTCreateWorkloadQuery::SettingsChanges res_changes; auto parse_setting = [&] { - SettingChange setting; - String default_setting; - std::pair parameter; - - if (ParserSetQuery::parseNameValuePairWithParameterOrDefault(setting, default_setting, parameter, pos, expected)) - { - if (!default_setting.empty()) - { - default_settings.push_back(std::move(default_setting)); - return true; - } - if (!setting.name.empty()) - { - settings_changes.push_back(std::move(setting)); - return true; - } - // TODO(serxa): parse optional clause: [FOR resource_name] - return false; // We do not support parameters - } - - return false; + ASTCreateWorkloadQuery::SettingChange change; + if (!parseWorkloadSetting(change, pos, expected)) + return false; + res_changes.push_back(std::move(change)); + return true; }; if (!ParserList::parseUtil(pos, expected, parse_setting, false)) return false; - ASTPtr res_settings; - if (!settings_changes.empty()) - { - auto settings_changes_ast = std::make_shared(); - settings_changes_ast->changes = std::move(settings_changes); - settings_changes_ast->is_standalone = false; - res_settings = settings_changes_ast; - } - - settings = std::move(res_settings); + changes = std::move(res_changes); return true; }); } @@ -114,8 +128,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp return false; } - ASTPtr settings; - parseSettings(pos, expected, settings); + ASTCreateWorkloadQuery::SettingsChanges changes; + parseSettings(pos, expected, changes); auto create_workload_query = std::make_shared(); node = create_workload_query; @@ -132,8 +146,8 @@ bool ParserCreateWorkloadQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_workload_query->or_replace = or_replace; create_workload_query->if_not_exists = if_not_exists; create_workload_query->cluster = std::move(cluster_str); + create_workload_query->changes = std::move(changes); - create_workload_query->settings = std::move(settings); return true; } diff --git a/src/Storages/System/StorageSystemWorkloads.cpp b/src/Storages/System/StorageSystemWorkloads.cpp index dad2750d8c0..ebb7e693e26 100644 --- a/src/Storages/System/StorageSystemWorkloads.cpp +++ b/src/Storages/System/StorageSystemWorkloads.cpp @@ -35,13 +35,13 @@ void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr c void StorageSystemWorkloads::backupData(BackupEntriesCollector & /*backup_entries_collector*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) { - // TODO(serxa): add backup for workloads and resources + // TODO(serxa): add backup for workloads // storage.backup(backup_entries_collector, data_path_in_backup); } void StorageSystemWorkloads::restoreDataFromBackup(RestorerFromBackup & /*restorer*/, const String & /*data_path_in_backup*/, const std::optional & /* partitions */) { - // TODO(serxa): add restore for workloads and resources + // TODO(serxa): add restore for workloads // storage.restore(restorer, data_path_in_backup); } From a8c164ece7427dd4382fd543598cc14ff773379e Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 29 Sep 2024 18:39:30 +0000 Subject: [PATCH 048/157] add test for resource-specific settings --- tests/integration/test_scheduler/test.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 401444b2d5a..0eee9d968ba 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -591,7 +591,7 @@ def test_create_workload(): f""" create resource io_write (write disk s3_no_resource); create resource io_read (read disk s3_no_resource); - create workload all settings max_cost = 1000000; + create workload all settings max_cost = 1000000 for io_write, max_cost = 2000000 for io_read; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; @@ -629,6 +629,18 @@ def test_create_workload(): ) == "2\n" ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_write' and max_cost=1000000" + ) + == "1\n" + ) + assert ( + node.query( + f"select count() from system.scheduler where path ilike '%/all/%' and type='inflight_limit' and resource='io_read' and max_cost=2000000" + ) + == "1\n" + ) do_checks() node.restart_clickhouse() # Check that workloads persist From ffe61c3007ec216a9b8010f3238c351834b1e068 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 30 Sep 2024 11:18:28 +0000 Subject: [PATCH 049/157] working on validations and tests --- .../Scheduler/Nodes/IOResourceManager.cpp | 3 - src/Common/Scheduler/SchedulingSettings.h | 2 +- .../Workload/WorkloadEntityStorageBase.cpp | 76 ++++++++++++++++--- .../Workload/WorkloadEntityStorageBase.h | 7 +- .../03232_workload_create_and_drop.reference | 2 +- .../03232_workload_create_and_drop.sql | 6 +- .../03232_workloads_and_resources.reference | 0 .../03232_workloads_and_resources.sql | 17 +++++ 8 files changed, 92 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.reference create mode 100644 tests/queries/0_stateless/03232_workloads_and_resources.sql diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index cf6b041c9f1..07929e855ce 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -61,12 +61,10 @@ IOResourceManager::Resource::~Resource() void IOResourceManager::Resource::createNode(const NodeInfo & info) { - // TODO(serxa): make sure all possible callers validate empty workload name! if (info.name.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'", resource_name); - // TODO(serxa): make sure all possible callers validate self-reference! if (info.name == info.parent) throw Exception(ErrorCodes::LOGICAL_ERROR, "Self-referencing workload '{}' is not allowed in resource '{}'", info.name, resource_name); @@ -80,7 +78,6 @@ void IOResourceManager::Resource::createNode(const NodeInfo & info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", info.parent, info.name, resource_name); - // TODO(serxa): make sure all possible callers validate second root, add tests for creating the second root if (info.parent.empty() && root_node) throw Exception(ErrorCodes::LOGICAL_ERROR, "The second root workload '{}' is not allowed (current root '{}') in resource '{}'", info.name, root_node->basename, resource_name); diff --git a/src/Common/Scheduler/SchedulingSettings.h b/src/Common/Scheduler/SchedulingSettings.h index cda6b546dfc..6db3ef0dce9 100644 --- a/src/Common/Scheduler/SchedulingSettings.h +++ b/src/Common/Scheduler/SchedulingSettings.h @@ -33,7 +33,7 @@ struct SchedulingSettings bool hasThrottler() const { return max_speed != 0; } bool hasSemaphore() const { return max_requests != default_max_requests || max_cost != default_max_cost; } - void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name); + void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name = {}); }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index f0c76b92870..8679c8639f6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,13 +1,14 @@ #include -#include -#include - +#include #include #include #include #include +#include +#include + #include #include @@ -52,18 +53,33 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } -void forEachReference(const ASTPtr & source_entity, std::function func) +enum class ReferenceType +{ + Parent, ForResource +}; + +void forEachReference(const ASTPtr & source_entity, std::function func) { if (auto * res = typeid_cast(source_entity.get())) { + // Parent reference String parent = res->getWorkloadParent(); if (!parent.empty()) - func(parent, res->getWorkloadName()); - // TODO(serxa): add references to RESOURCEs mentioned in SETTINGS clause after FOR keyword + func(parent, res->getWorkloadName(), ReferenceType::Parent); + + // References to RESOURCEs mentioned in SETTINGS clause after FOR keyword + std::unordered_set resources; + for (const auto & [name, value, resource] : res->changes) + { + if (!resource.empty()) + resources.insert(resource); + } + for (const String & resource : resources) + func(resource, res->getWorkloadName(), ReferenceType::ForResource); } if (auto * res = typeid_cast(source_entity.get())) { - // RESOURCE has no references to be validated + // RESOURCE has no references to be validated, we allow mentioned disks to be created later } } @@ -173,6 +189,16 @@ bool WorkloadEntityStorageBase::storeEntity( bool replace_if_exists, const Settings & settings) { + if (entity_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); + + auto * workload = typeid_cast(create_entity_query.get()); + if (workload) + { + if (entity_name == workload->getWorkloadParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Self-referencing workloads are not allowed."); + } + std::unique_lock lock{mutex}; create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); @@ -185,11 +211,35 @@ bool WorkloadEntityStorageBase::storeEntity( return false; } + std::optional new_root_name; + + // Validate workload + if (workload) + { + if (!workload->hasParent()) + { + if (!root_name.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); + new_root_name = workload->getWorkloadName(); + } + + SchedulingSettings validator; + validator.updateFromChanges(workload->changes); + } + forEachReference(create_entity_query, - [this] (const String & target, const String & source) + [this, workload] (const String & target, const String & source, ReferenceType type) { if (auto it = entities.find(target); it == entities.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + + // Validate that we could parse the settings for specific resource + if (type == ReferenceType::ForResource) + { + // TODO(serxa): check this is a target is a resource, not workload + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + } }); bool stored = storeEntityImpl( @@ -203,8 +253,10 @@ bool WorkloadEntityStorageBase::storeEntity( if (stored) { + if (new_root_name) + root_name = *new_root_name; forEachReference(create_entity_query, - [this] (const String & target, const String & source) + [this] (const String & target, const String & source, ReferenceType) { references[target].insert(source); }); @@ -248,8 +300,10 @@ bool WorkloadEntityStorageBase::removeEntity( if (removed) { + if (entity_name == root_name) + root_name.clear(); forEachReference(it->second, - [this] (const String & target, const String & source) + [this] (const String & target, const String & source, ReferenceType) { references[target].erase(source); if (references[target].empty()) @@ -354,7 +408,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector queue; mutable std::recursive_mutex mutex; - std::unordered_map entities; // Maps entity name into CREATE entity query - std::unordered_map> references; // Keep track of references between entities for validation + std::unordered_map entities; /// Maps entity name into CREATE entity query + + // Validation + std::unordered_map> references; /// Keep track of references between entities + String root_name; /// current root workload name ContextPtr global_context; }; diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.reference b/tests/queries/0_stateless/03232_workload_create_and_drop.reference index 4bac2ef71f2..923e8652a35 100644 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.reference +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.reference @@ -1,5 +1,5 @@ all CREATE WORKLOAD `all` +all CREATE WORKLOAD `all` development all CREATE WORKLOAD development IN `all` production all CREATE WORKLOAD production IN `all` all CREATE WORKLOAD `all` -all CREATE WORKLOAD `all` diff --git a/tests/queries/0_stateless/03232_workload_create_and_drop.sql b/tests/queries/0_stateless/03232_workload_create_and_drop.sql index 38a7dad7cbc..1d8f97baf4c 100644 --- a/tests/queries/0_stateless/03232_workload_create_and_drop.sql +++ b/tests/queries/0_stateless/03232_workload_create_and_drop.sql @@ -1,11 +1,11 @@ -- Tags: no-parallel -- Do not run this test in parallel because `all` workload might affect other queries execution process CREATE OR REPLACE WORKLOAD all; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; CREATE WORKLOAD IF NOT EXISTS production IN all; CREATE WORKLOAD development IN all; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; DROP WORKLOAD IF EXISTS production; DROP WORKLOAD development; -SELECT name, parent, create_query FROM system.workloads; +SELECT name, parent, create_query FROM system.workloads ORDER BY name; DROP WORKLOAD all; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.reference b/tests/queries/0_stateless/03232_workloads_and_resources.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql new file mode 100644 index 00000000000..1653659bcc4 --- /dev/null +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -0,0 +1,17 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because `all` workload might affect other queries execution process +create resource 03232_write (write disk 03232_fake_disk); +create resource 03232_read (read disk 03232_fake_disk); +create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} +create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create workload admin in all settings priority = 0; +create workload production in all settings priority = 1, weight = 9; +create workload development in all settings priority = 1, weight = 1; +create workload another_root; -- {serverError BAD_ARGUMENTS} + +drop workload if exists production; +drop workload if exists development; +drop workload if exists admin; +drop workload if exists all; +drop resource if exists 03232_write; +drop resource if exists 03232_read; From 21931b1d99c618227e66a83cd719e1492e0e7dc0 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 30 Sep 2024 21:47:46 +0000 Subject: [PATCH 050/157] add more worklaod validation and testing --- src/Common/Scheduler/ISchedulerNode.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 -- .../Scheduler/Workload/WorkloadEntityStorageBase.cpp | 4 +++- .../0_stateless/03232_workloads_and_resources.sql | 12 +++++++++++- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index d68a32e8290..73fd0759c6a 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -74,7 +74,7 @@ struct SchedulerNodeInfo if (value <= 0 || !isfinite(value)) throw Exception( ErrorCodes::INVALID_SCHEDULER_NODE, - "Negative and non-finite node weights are not allowed: {}", + "Zero, negative and non-finite node weights are not allowed: {}", value); weight = value; } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 07929e855ce..55defbd2432 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -73,7 +73,6 @@ void IOResourceManager::Resource::createNode(const NodeInfo & info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Node for creating workload '{}' already exist in resource '{}'", info.name, resource_name); - // TODO(serxa): make sure all possible callers validate parent existence, add tests for creating workload with invalid parent if (!info.parent.empty() && !node_for_workload.contains(info.parent)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Parent node '{}' for creating workload '{}' does not exist in resource '{}'", info.parent, info.name, resource_name); @@ -111,7 +110,6 @@ void IOResourceManager::Resource::deleteNode(const NodeInfo & info) auto node = node_for_workload[info.name]; - // TODO(serxa): make sure all possible callers validate that removing workload has no children workloads if (node->hasUnifiedChildren()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'", info.name, resource_name); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 8679c8639f6..6f633893d70 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -236,7 +236,9 @@ bool WorkloadEntityStorageBase::storeEntity( // Validate that we could parse the settings for specific resource if (type == ReferenceType::ForResource) { - // TODO(serxa): check this is a target is a resource, not workload + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); + SchedulingSettings validator; validator.updateFromChanges(workload->changes, target); } diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index 1653659bcc4..ae0061b3bd5 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -2,12 +2,22 @@ -- Do not run this test in parallel because `all` workload might affect other queries execution process create resource 03232_write (write disk 03232_fake_disk); create resource 03232_read (read disk 03232_fake_disk); -create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; + create workload another_root; -- {serverError BAD_ARGUMENTS} +create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} +drop workload all; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} +create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} +create workload invalid in all settings weight = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} +create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} drop workload if exists production; drop workload if exists development; From b25b711a7135fa82bd3bd41a2547d316bb5c9ff1 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 10:52:16 +0000 Subject: [PATCH 051/157] implement detach of a unified node --- src/Common/Scheduler/SchedulingSettings.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/Scheduler/SchedulingSettings.cpp b/src/Common/Scheduler/SchedulingSettings.cpp index 352e61fb560..60319cdd54c 100644 --- a/src/Common/Scheduler/SchedulingSettings.cpp +++ b/src/Common/Scheduler/SchedulingSettings.cpp @@ -12,8 +12,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -// TODO(serxa): we should validate workloads with this function before storing in WorkloadEntityStorage -// TODO(serxa): and probably we should add and persist version in filename for future changes void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name) { struct { From 1ccdc196b702a0c02ba91501fbf7716c8905c03a Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 10:53:09 +0000 Subject: [PATCH 052/157] implement detach of a unified node --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 98 ++++++++++++++++++- 1 file changed, 97 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 76685319c34..2b2eb320e0a 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -87,6 +87,8 @@ private: SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached std::unordered_map children; // basename -> child + bool empty() const { return children.empty(); } + SchedulerNodePtr getRoot() { chassert(!children.empty()); @@ -122,6 +124,29 @@ private: reparent(child, root); return {}; // Root is the same } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue *, const UnifiedSchedulerNodePtr & child) + { + auto it = children.find(child->basename); + if (it == children.end()) + return {}; // unknown child + + children.erase(it); + if (children.size() == 1) + { + // Remove fair if the only child has left + chassert(root); + root.reset(); // it will be still alive because it is attached to hierarchy for now + return children.begin()->second; // The last child is a new root now + } + else if (children.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children have left) + } }; /// Handles all the children nodes with intermediate fair and/or priority nodes @@ -130,6 +155,9 @@ private: SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached std::unordered_map branches; /// Branches for different priority values + // Returns true iff there are no unified children attached + bool empty() const { return branches.empty(); } + /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) @@ -169,6 +197,42 @@ private: return {}; // Root is the same } } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + /// NOTE: It could also return null if `empty()` after detaching + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + auto it = branches.find(child->info.priority); + if (it == branches.end()) + return {}; // unknown child + + auto & child_branch = it->second; + auto branch_root = child_branch.detachUnifiedChild(event_queue_, child); + if (child_branch.empty()) + { + branches.erase(it); + if (branches.size() == 1) + { + // Remove priority node if the only child-branch has left + chassert(root); + root.reset(); // it will be still alive because it is attached to hierarchy for now + return branches.begin()->second.getRoot(); // The last child-branch is a new root now + } + else if (branches.empty()) + return {}; // We have detached the last child + else + return {}; // Root is the same (two or more children-branches have left) + } + if (branch_root) + { + if (root) + reparent(branch_root, root); + else + return branch_root; + } + return {}; // Root is the same + } }; /// Handles degenerate case of zero children (a fifo queue) or delegate to `ChildrenBranch`. @@ -193,6 +257,21 @@ private: return branch.attachUnifiedChild(event_queue_, child); } + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (queue) + return {}; // No-op, it already has no children + auto branch_root = branch.detachUnifiedChild(event_queue_, child); + if (branch.empty()) + { + createQueue(event_queue_); + return queue; + } + return branch_root; + } + private: void createQueue(EventQueue * event_queue_) { @@ -256,6 +335,22 @@ private: } return {}; } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr detachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) + { + if (auto branch_root = branch.detachUnifiedChild(event_queue_, child)) + { + if (semaphore) + reparent(branch_root, semaphore); + else if (throttler) + reparent(branch_root, throttler); + else + return branch_root; + } + return {}; + } }; public: @@ -279,7 +374,8 @@ public: /// NOTE: Do not confuse with `removeChild()` which is used only for immediate children void detachUnifiedChild(const UnifiedSchedulerNodePtr & child) { - UNUSED(child); // TODO(serxa): implement detachUnifiedChild() + if (auto new_child = impl.detachUnifiedChild(event_queue, child)) + reparent(new_child, this); } /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) From 809f0ee0a2b8f928cb655b98e654a1304717abb6 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Oct 2024 16:36:32 +0000 Subject: [PATCH 053/157] fix test: correct drop order --- tests/integration/test_scheduler/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 0eee9d968ba..58d8ab44457 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -58,12 +58,12 @@ def set_default_configs(): def clear_workloads_and_resources(): node.query( f""" - drop resource if exists io_write; - drop resource if exists io_read; drop workload if exists production; drop workload if exists development; drop workload if exists admin; drop workload if exists all; + drop resource if exists io_write; + drop resource if exists io_read; """ ) yield From 7722a5e4fa78763a7d88f69b826b7103d0f306a3 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 2 Oct 2024 17:54:43 +0000 Subject: [PATCH 054/157] fixes and tests for DROP WORKLOAD query --- src/Common/Scheduler/Nodes/FairPolicy.h | 7 +++ .../Scheduler/Nodes/IOResourceManager.cpp | 2 + src/Common/Scheduler/Nodes/PriorityPolicy.h | 7 +++ .../Scheduler/Nodes/SemaphoreConstraint.h | 7 +++ .../Scheduler/Nodes/ThrottlerConstraint.h | 4 ++ .../Scheduler/Nodes/UnifiedSchedulerNode.h | 14 ++++- src/Common/Scheduler/SchedulerRoot.h | 24 +++---- tests/integration/test_scheduler/test.py | 63 +++++++++++++++++++ 8 files changed, 115 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index 81bfaaadf19..007a2416fae 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -52,6 +52,13 @@ public: : ISchedulerNode(event_queue_, info_) {} + ~FairPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + const String & getTypeName() const override { static String type_name("fair"); diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 55defbd2432..cf67bf2dfcb 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -125,6 +125,8 @@ void IOResourceManager::Resource::deleteNode(const NodeInfo & info) root_node.reset(); } + node_for_workload.erase(info.name); + updateCurrentVersion(); }); } diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index ea8bde718a2..f7d4ee93a36 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -43,6 +43,13 @@ public: : ISchedulerNode(event_queue_, node_info) {} + ~PriorityPolicy() override + { + // We need to clear `parent` in all children to avoid dangling references + while (!children.empty()) + removeChild(children.begin()->second.get()); + } + const String & getTypeName() const override { static String type_name("priority"); diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index eab093f6b00..20a10f5da7d 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -31,6 +31,13 @@ public: , max_cost(max_cost_) {} + ~SemaphoreConstraint() override + { + // We need to clear `parent` in child to avoid dangling references + if (child) + removeChild(child.get()); + } + const String & getTypeName() const override { static String type_name("inflight_limit"); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index 40b51f24b98..eaa26b2da54 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -38,6 +38,10 @@ public: { // We should cancel event on destruction to avoid dangling references from event queue event_queue->cancelPostponed(postponed); + + // We need to clear `parent` in child to avoid dangling reference + if (child) + removeChild(child.get()); } const String & getTypeName() const override diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 2b2eb320e0a..2de5131efbb 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -67,6 +67,7 @@ private: /// Helper function for managing a parent of a node static void reparent(const SchedulerNodePtr & node, ISchedulerNode * new_parent) { + chassert(node); chassert(new_parent); if (new_parent == node->parent) return; @@ -139,7 +140,8 @@ private: { // Remove fair if the only child has left chassert(root); - root.reset(); // it will be still alive because it is attached to hierarchy for now + detach(root); + root.reset(); return children.begin()->second; // The last child is a new root now } else if (children.empty()) @@ -216,7 +218,8 @@ private: { // Remove priority node if the only child-branch has left chassert(root); - root.reset(); // it will be still alive because it is attached to hierarchy for now + detach(root); + root.reset(); return branches.begin()->second.getRoot(); // The last child-branch is a new root now } else if (branches.empty()) @@ -361,6 +364,13 @@ public: reparent(immediate_child, this); } + ~UnifiedSchedulerNode() override + { + // We need to clear `parent` in child to avoid dangling references + if (immediate_child) + removeChild(immediate_child.get()); + } + /// Attaches a unified child as a leaf of internal subtree and insert or update all the intermediate nodes /// NOTE: Do not confuse with `attachChild()` which is used only for immediate children void attachUnifiedChild(const UnifiedSchedulerNodePtr & child) diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index e2ed133f662..8bb25f80139 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -31,24 +31,24 @@ namespace ErrorCodes class SchedulerRoot : public ISchedulerNode { private: - struct TResource + struct Resource { SchedulerNodePtr root; // Intrusive cyclic list of active resources - TResource * next = nullptr; - TResource * prev = nullptr; + Resource * next = nullptr; + Resource * prev = nullptr; - explicit TResource(const SchedulerNodePtr & root_) + explicit Resource(const SchedulerNodePtr & root_) : root(root_) { root->info.parent.ptr = this; } // Get pointer stored by ctor in info - static TResource * get(SchedulerNodeInfo & info) + static Resource * get(SchedulerNodeInfo & info) { - return reinterpret_cast(info.parent.ptr); + return reinterpret_cast(info.parent.ptr); } }; @@ -60,6 +60,8 @@ public: ~SchedulerRoot() override { stop(); + while (!children.empty()) + removeChild(children.begin()->first); } /// Runs separate scheduler thread @@ -185,7 +187,7 @@ public: void activateChild(ISchedulerNode * child) override { - activate(TResource::get(child->info)); + activate(Resource::get(child->info)); } void setParent(ISchedulerNode *) override @@ -194,7 +196,7 @@ public: } private: - void activate(TResource * value) + void activate(Resource * value) { assert(value->next == nullptr && value->prev == nullptr); if (current == nullptr) // No active children @@ -212,7 +214,7 @@ private: } } - void deactivate(TResource * value) + void deactivate(Resource * value) { if (value->next == nullptr) return; // Already deactivated @@ -257,8 +259,8 @@ private: request->execute(); } - TResource * current = nullptr; // round-robin pointer - std::unordered_map children; // resources by pointer + Resource * current = nullptr; // round-robin pointer + std::unordered_map children; // resources by pointer std::atomic stop_flag = false; EventQueue events; ThreadFromGlobalPool scheduler; diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 58d8ab44457..79d9466eb59 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -5,6 +5,7 @@ import time import threading import pytest +import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -647,6 +648,68 @@ def test_create_workload(): do_checks() +def test_workload_hierarchy_changes(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + for iteration in range(3): + split_idx = random.randint(1, len(queries) - 2) + for query_idx in range(0, split_idx): + node.query(queries[query_idx]) + node.query("create resource io_test (write disk non_existent_disk, read disk non_existent_disk);") + node.query("drop resource io_test;") + for query_idx in range(split_idx, len(queries)): + node.query(queries[query_idx]) + def test_resource_read_and_write(): node.query( From 0d8ff26706c1879debbb4cfa029fbaa9239cd004 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 00:18:42 +0000 Subject: [PATCH 055/157] add unittests for IOResourceManager --- .../Scheduler/Nodes/tests/ResourceTest.h | 6 + .../Nodes/tests/gtest_io_resource_manager.cpp | 237 ++++++++++++++++++ .../Workload/WorkloadEntityStorageBase.cpp | 7 +- 3 files changed, 246 insertions(+), 4 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index c8cc0ed0e57..aa490b38f47 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -328,6 +328,12 @@ struct ResourceTestManager : public ResourceTestBase , busy_period(thread_count) {} + enum DoNotInitManagerEnum { DoNotInitManager }; + + explicit ResourceTestManager(size_t thread_count, DoNotInitManagerEnum) + : busy_period(thread_count) + {} + ~ResourceTestManager() { for (auto & thread : threads) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp new file mode 100644 index 00000000000..f8c973deb3b --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -0,0 +1,237 @@ +#include + +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +class WorkloadEntityTestStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityTestStorage() + : WorkloadEntityStorageBase(Context::getGlobalContextInstance()) + {} + + virtual void loadEntities() override {} + + void executeQuery(const String & query) + { + ParserCreateWorkloadQuery create_workload_p; + ParserDropWorkloadQuery drop_workload_p; + ParserCreateResourceQuery create_resource_p; + ParserDropResourceQuery drop_resource_p; + + auto parse = [&] (IParser & parser) + { + String error; + const char * end = query.data(); + return tryParseQuery( + parser, + end, + query.data() + query.size(), + error, + false, + "", + false, + 0, + DBMS_DEFAULT_MAX_PARSER_DEPTH, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, + true); + }; + + if (ASTPtr create_workload = parse(create_workload_p)) + { + auto & parsed = create_workload->as(); + auto workload_name = parsed.getWorkloadName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Workload, + workload_name, + create_workload, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr create_resource = parse(create_resource_p)) + { + auto & parsed = create_resource->as(); + auto resource_name = parsed.getResourceName(); + bool throw_if_exists = !parsed.if_not_exists && !parsed.or_replace; + bool replace_if_exists = parsed.or_replace; + + storeEntity( + nullptr, + WorkloadEntityType::Resource, + resource_name, + create_resource, + throw_if_exists, + replace_if_exists, + {}); + } + else if (ASTPtr drop_workload = parse(drop_workload_p)) + { + auto & parsed = drop_workload->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Workload, + parsed.workload_name, + throw_if_not_exists); + } + else if (ASTPtr drop_resource = parse(drop_resource_p)) + { + auto & parsed = drop_resource->as(); + bool throw_if_not_exists = !parsed.if_exists; + removeEntity( + nullptr, + WorkloadEntityType::Resource, + parsed.resource_name, + throw_if_not_exists); + } + FAIL(); + } + +private: + bool storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override + { + UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); + return true; + } + + bool removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override + { + UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); + return true; + } +}; + +struct ResourceTest : ResourceTestManager +{ + WorkloadEntityTestStorage storage; + + explicit ResourceTest(size_t thread_count = 1) + : ResourceTestManager(thread_count, DoNotInitManager) + { + manager = std::make_shared(storage); + } + + void query(const String & query_str) + { + storage.executeQuery(query_str); + } +}; + +using TestGuard = ResourceTest::Guard; + +TEST(SchedulerIOResourceManager, Smoke) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); + t.query("CREATE WORKLOAD A in all"); + t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); + + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); + + for (int i = 0; i < 10; i++) + { + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); + + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); + + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); + } +} + +TEST(SchedulerIOResourceManager, Fairness) +{ + // Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1). + // Requests from A use `value = 1` and from B `value = -1` is used. + std::atomic unfairness = 0; + auto fairness_diff = [&] (Int64 value) + { + Int64 cur_unfairness = unfairness.fetch_add(value, std::memory_order_relaxed) + value; + EXPECT_NEAR(cur_unfairness, 0, 1); + }; + + constexpr size_t threads_per_queue = 2; + int requests_per_thread = 100; + ResourceTest t(2 * threads_per_queue + 1); + + t.query("CREATE RESOURCE res1"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD A in all"); + t.query("CREATE WORKLOAD B in all"); + t.query("CREATE WORKLOAD leader in all"); + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("A"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(1); + } + }); + } + + for (int thread = 0; thread < threads_per_queue; thread++) + { + t.threads.emplace_back([&] + { + ClassifierPtr c = t.manager->acquire("B"); + ResourceLink link = c->get("res1"); + t.startBusyPeriod(link, 1, requests_per_thread); + for (int request = 0; request < requests_per_thread; request++) + { + TestGuard g(t, link, 1); + fairness_diff(-1); + } + }); + } + + ClassifierPtr c = t.manager->acquire("leader"); + ResourceLink link = c->get("res1"); + t.blockResource(link); +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 6f633893d70..91f418449ed 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -26,9 +26,8 @@ namespace ErrorCodes namespace { -ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query, const ContextPtr & context) +ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) { - UNUSED(context); auto ptr = create_query.clone(); if (auto * res = typeid_cast(ptr.get())) { @@ -201,7 +200,7 @@ bool WorkloadEntityStorageBase::storeEntity( std::unique_lock lock{mutex}; - create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query, global_context); + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); if (auto it = entities.find(entity_name); it != entities.end()) { @@ -400,7 +399,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector normalized_entities; for (const auto & [entity_name, create_query] : new_entities) - normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query, global_context); + normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); // TODO(serxa): do validation and throw LOGICAL_ERROR if failed From 4c02ddcf3bf68398a210f0492d5aba6d15943571 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 00:18:58 +0000 Subject: [PATCH 056/157] style --- .../tests/gtest_custom_resource_manager.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp index 495654d45ce..37432128606 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_custom_resource_manager.cpp @@ -31,21 +31,21 @@ TEST(SchedulerCustomResourceManager, Smoke) )CONFIG"); - ClassifierPtr cA = t.manager->acquire("A"); - ClassifierPtr cB = t.manager->acquire("B"); + ClassifierPtr c_a = t.manager->acquire("A"); + ClassifierPtr c_b = t.manager->acquire("B"); for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); - gA.lock(); - gA.consume(1); - gA.unlock(); + ResourceGuard g_a(ResourceGuard::Metrics::getIOWrite(), c_a->get("res1"), 1, ResourceGuard::Lock::Defer); + g_a.lock(); + g_a.consume(1); + g_a.unlock(); - ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); - gB.unlock(); + ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1")); + g_b.unlock(); - ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); - gB.consume(2); + ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1")); + g_b.consume(2); } } From d6b3c106db864f9ae1b471c84c67a8032862d52f Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 3 Oct 2024 07:51:47 +0000 Subject: [PATCH 057/157] fix unittests --- .../Scheduler/Nodes/tests/gtest_io_resource_manager.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index f8c973deb3b..31dd98eafc5 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -107,7 +107,8 @@ public: parsed.resource_name, throw_if_not_exists); } - FAIL(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query in WorkloadEntityTestStorage: {}", query); } private: @@ -157,7 +158,7 @@ TEST(SchedulerIOResourceManager, Smoke) { ResourceTest t; - t.query("CREATE RESOURCE res1"); + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 10"); t.query("CREATE WORKLOAD A in all"); t.query("CREATE WORKLOAD B in all SETTINGS weight = 3"); @@ -195,7 +196,7 @@ TEST(SchedulerIOResourceManager, Fairness) int requests_per_thread = 100; ResourceTest t(2 * threads_per_queue + 1); - t.query("CREATE RESOURCE res1"); + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); t.query("CREATE WORKLOAD A in all"); t.query("CREATE WORKLOAD B in all"); From ee200fa3d965d88ae5a23e0186b9cf42c7c23d12 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 5 Oct 2024 13:48:19 +0000 Subject: [PATCH 058/157] test resource request failure --- .../Scheduler/Nodes/tests/ResourceTest.h | 43 +++++++- .../Nodes/tests/gtest_io_resource_manager.cpp | 97 ++++++++++++++++++- 2 files changed, 134 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index aa490b38f47..bbe0df4872e 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -1,6 +1,8 @@ #pragma once -#include "Common/Scheduler/SchedulingSettings.h" +#include + +#include #include #include #include @@ -283,6 +285,8 @@ private: ResourceCost failed_cost = 0; }; +enum EnqueueOnlyEnum { EnqueueOnly }; + template struct ResourceTestManager : public ResourceTestBase { @@ -294,16 +298,49 @@ struct ResourceTestManager : public ResourceTestBase struct Guard : public ResourceGuard { ResourceTestManager & t; + ResourceCost cost; - Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) + /// Works like regular ResourceGuard, ready for consumption after constructor + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) , t(t_) + , cost(cost_) { t.onEnqueue(link); + waitExecute(); + } + + /// Just enqueue resource request, do not block (neede for tests to sync). Call `waitExecuted()` afterwards + Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) + , t(t_) + , cost(cost_) + { + t.onEnqueue(link); + } + + /// Waits for ResourceRequest::execute() to be called for enqueued requet + void waitExecute() + { lock(); t.onExecute(link); consume(cost); } + + /// Waits for ResourceRequest::failure() to be called for enqueued request + void waitFailed(const String & pattern) + { + try + { + lock(); + FAIL(); + } + catch (Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::RESOURCE_ACCESS_DENIED); + ASSERT_TRUE(e.message().contains(pattern)); + } + } }; struct TItem diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 31dd98eafc5..93c8439bdae 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -150,6 +150,27 @@ struct ResourceTest : ResourceTestManager { storage.executeQuery(query_str); } + + template + void async(const String & workload, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + func2(classifier); + }); + } + + template + void async(const String & workload, const String & resource, Func func) + { + threads.emplace_back([=, this, func2 = std::move(func)] + { + ClassifierPtr classifier = manager->acquire(workload); + ResourceLink link = classifier->get(resource); + func2(link); + }); + } }; using TestGuard = ResourceTest::Guard; @@ -198,9 +219,9 @@ TEST(SchedulerIOResourceManager, Fairness) t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); - t.query("CREATE WORKLOAD A in all"); - t.query("CREATE WORKLOAD B in all"); - t.query("CREATE WORKLOAD leader in all"); + t.query("CREATE WORKLOAD A IN all"); + t.query("CREATE WORKLOAD B IN all"); + t.query("CREATE WORKLOAD leader IN all"); for (int thread = 0; thread < threads_per_queue; thread++) { @@ -236,3 +257,73 @@ TEST(SchedulerIOResourceManager, Fairness) ResourceLink link = c->get("res1"); t.blockResource(link); } + +TEST(SchedulerIOResourceManager, DropNotEmptyQueue) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(3); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // 2nd resource request is enqueued + g.waitFailed("is about to be destructed"); + }); + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); +} + +TEST(SchedulerIOResourceManager, DropNotEmptyQueueLong) +{ + ResourceTest t; + + t.query("CREATE RESOURCE res1 (WRITE DISK disk, READ DISK disk)"); + t.query("CREATE WORKLOAD all SETTINGS max_requests = 1"); + t.query("CREATE WORKLOAD intermediate IN all"); + + static constexpr int queue_size = 100; + std::barrier sync_before_enqueue(2); + std::barrier sync_before_drop(2 + queue_size); + std::barrier sync_after_drop(2); + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1); + sync_before_enqueue.arrive_and_wait(); + sync_before_drop.arrive_and_wait(); // 1st resource request is consuming + sync_after_drop.arrive_and_wait(); // 1st resource request is still consuming + }); + + sync_before_enqueue.arrive_and_wait(); // to maintain correct order of resource requests + + for (int i = 0; i < queue_size; i++) + { + t.async("intermediate", "res1", [&] (ResourceLink link) + { + TestGuard g(t, link, 1, EnqueueOnly); + sync_before_drop.arrive_and_wait(); // many resource requests are enqueued + g.waitFailed("is about to be destructed"); + }); + } + + sync_before_drop.arrive_and_wait(); // main thread triggers FifoQueue destruction by adding a unified child + t.query("CREATE WORKLOAD leaf IN intermediate"); + sync_after_drop.arrive_and_wait(); +} From 37bb566be1189f52f5fbf6148b8e70811118a5d5 Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 5 Oct 2024 14:13:54 +0000 Subject: [PATCH 059/157] fix misleading comments --- src/Common/Scheduler/ResourceRequest.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index 24afcc98b57..03bdaec6a2b 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -45,8 +45,7 @@ constexpr size_t ResourceMaxConstraints = 8; * * Request can also be canceled before (3) using ISchedulerQueue::cancelRequest(). * Returning false means it is too late for request to be canceled. It should be processed in a regular way. - * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen - * and step (6) MUST be omitted. + * Returning true means successful cancel and therefore steps (4) and (5) are not going to happen. */ class ResourceRequest : public boost::intrusive::list_base_hook<> { @@ -88,7 +87,7 @@ public: /// Stop resource consumption and notify resource scheduler. /// Should be called when resource consumption is finished by consumer. /// ResourceRequest should not be destructed or reset before calling to `finish()`. - /// WARNING: this function MUST not be called if request was canceled or failed. + /// It is okay to call finish() even for failed and canceled requests (it will be no-op) void finish(); /// Is called from the scheduler thread to fill `constraints` chain From a74185806cd488a61dabbe385e8a0b8d7dee465f Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 6 Oct 2024 12:36:38 +0000 Subject: [PATCH 060/157] CREATE OR REPLACE WORKLOAD support --- src/Common/ErrorCodes.cpp | 2 - .../Workload/WorkloadEntityDiskStorage.cpp | 7 +- .../Workload/WorkloadEntityStorageBase.cpp | 156 +++++++++++++----- .../Workload/WorkloadEntityStorageBase.h | 11 +- .../03232_workloads_and_resources.sql | 1 + 5 files changed, 129 insertions(+), 48 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f441426e2f0..9f07c3ed5d5 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -611,8 +611,6 @@ M(730, REFRESH_FAILED) \ M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \ M(733, TABLE_IS_BEING_RESTARTED) \ - M(734, WORKLOAD_ENTITY_ALREADY_EXISTS) \ - M(735, UNKNOWN_WORKLOAD_ENTITY) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 5ffec270610..190b2928fe0 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -41,8 +41,7 @@ namespace Setting namespace ErrorCodes { extern const int DIRECTORY_DOESNT_EXIST; - extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; - extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int BAD_ARGUMENTS; } @@ -215,7 +214,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( if (fs::exists(file_path)) { if (throw_if_exists) - throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) return false; } @@ -266,7 +265,7 @@ bool WorkloadEntityDiskStorage::removeEntityImpl( if (!existed) { if (throw_if_not_exists) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else return false; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 91f418449ed..4e0c4f8dbbd 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -18,8 +19,7 @@ namespace DB namespace ErrorCodes { - extern const int WORKLOAD_ENTITY_ALREADY_EXISTS; - extern const int UNKNOWN_WORKLOAD_ENTITY; + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } @@ -123,7 +123,7 @@ ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const auto it = entities.find(entity_name); if (it == entities.end()) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The workload entity name '{}' is not saved", entity_name); @@ -191,23 +191,34 @@ bool WorkloadEntityStorageBase::storeEntity( if (entity_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity name should not be empty."); + create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); auto * workload = typeid_cast(create_entity_query.get()); - if (workload) - { - if (entity_name == workload->getWorkloadParent()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Self-referencing workloads are not allowed."); - } + auto * resource = typeid_cast(create_entity_query.get()); std::unique_lock lock{mutex}; - create_entity_query = normalizeCreateWorkloadEntityQuery(*create_entity_query); - + ASTPtr old_entity; // entity to be REPLACED if (auto it = entities.find(entity_name); it != entities.end()) { if (throw_if_exists) - throw Exception(ErrorCodes::WORKLOAD_ENTITY_ALREADY_EXISTS, "Workload entity '{}' already exists", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) return false; + else + old_entity = it->second; + } + + // Validate CREATE OR REPLACE + if (old_entity) + { + auto * old_workload = typeid_cast(old_entity.get()); + auto * old_resource = typeid_cast(old_entity.get()); + if (workload && !old_workload) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); + if (resource && !old_resource) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); + if (workload && !old_workload->hasParent() && workload->hasParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); } std::optional new_root_name; @@ -217,7 +228,7 @@ bool WorkloadEntityStorageBase::storeEntity( { if (!workload->hasParent()) { - if (!root_name.empty()) + if (!root_name.empty() && root_name != workload->getWorkloadName()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); new_root_name = workload->getWorkloadName(); } @@ -232,15 +243,31 @@ bool WorkloadEntityStorageBase::storeEntity( if (auto it = entities.find(target); it == entities.end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); - // Validate that we could parse the settings for specific resource - if (type == ReferenceType::ForResource) + switch (type) { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); + case ReferenceType::Parent: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); + break; + } + case ReferenceType::ForResource: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - SchedulingSettings validator; - validator.updateFromChanges(workload->changes, target); + // Validate that we could parse the settings for specific resource + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + break; + } } + + // Detect reference cycles. + // The only way to create a cycle is to add an edge that will be a part of a new cycle. + // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. + if (isIndirectlyReferenced(source, target)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); }); bool stored = storeEntityImpl( @@ -256,12 +283,17 @@ bool WorkloadEntityStorageBase::storeEntity( { if (new_root_name) root_name = *new_root_name; - forEachReference(create_entity_query, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].insert(source); - }); + + // Remove references of a replaced entity (only for CREATE OR REPLACE) + removeReferences(old_entity); + + // Insert references of created entity + insertReferences(create_entity_query); + + // Store in memory entities[entity_name] = create_entity_query; + + // Process notifications onEntityAdded(entity_type, entity_name, create_entity_query); unlockAndNotify(lock); } @@ -280,7 +312,7 @@ bool WorkloadEntityStorageBase::removeEntity( if (it == entities.end()) { if (throw_if_not_exists) - throw Exception(ErrorCodes::UNKNOWN_WORKLOAD_ENTITY, "Workload entity '{}' doesn't exist", entity_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else return false; } @@ -303,16 +335,15 @@ bool WorkloadEntityStorageBase::removeEntity( { if (entity_name == root_name) root_name.clear(); - forEachReference(it->second, - [this] (const String & target, const String & source, ReferenceType) - { - references[target].erase(source); - if (references[target].empty()) - references.erase(target); - }); - entities.erase(it); - onEntityRemoved(entity_type, entity_name); + // Clean up references + removeReferences(it->second); + + // Remove from memory + entities.erase(it); + + // Process notifications + onEntityRemoved(entity_type, entity_name); unlockAndNotify(lock); } @@ -407,14 +438,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector bfs; + std::unordered_set visited; + visited.insert(target); + bfs.push(target); + while (!bfs.empty()) + { + String current = bfs.front(); + bfs.pop(); + if (current == source) + return true; + if (auto it = references.find(current); it != references.end()) + { + for (const String & node : it->second) + { + if (visited.contains(node)) + continue; + visited.insert(node); + bfs.push(node); + } + } + } + return false; +} + +void WorkloadEntityStorageBase::insertReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].insert(source); + }); +} + +void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) +{ + if (!entity) + return; + forEachReference(entity, + [this] (const String & target, const String & source, ReferenceType) + { + references[target].erase(source); + if (references[target].empty()) + references.erase(target); + }); +} + } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 9b81e5bdff6..7bfc28b3263 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -83,6 +83,15 @@ protected: /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). void unlockAndNotify(std::unique_lock & lock); + /// Return true iff `references` has a path from `source` to `target` + bool isIndirectlyReferenced(const String & target, const String & source); + + /// Adds references that are described by `entity` to `references` + void insertReferences(const ASTPtr & entity); + + /// Removes references that are described by `entity` from `references` + void removeReferences(const ASTPtr & entity); + struct Handlers { std::mutex mutex; @@ -97,7 +106,7 @@ protected: std::unordered_map entities; /// Maps entity name into CREATE entity query // Validation - std::unordered_map> references; /// Keep track of references between entities + std::unordered_map> references; /// Keep track of references between entities. Key is target. Values is set of sources String root_name; /// current root workload name ContextPtr global_context; diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index ae0061b3bd5..3e12d70b7ff 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -10,6 +10,7 @@ create workload development in all settings priority = 1, weight = 1; create workload another_root; -- {serverError BAD_ARGUMENTS} create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} drop workload all; -- {serverError BAD_ARGUMENTS} +create workload invalid in 03232_write; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings priority = 0 for all; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings priority = 'invalid_value'; -- {serverError BAD_GET} create workload invalid in all settings weight = 0; -- {serverError INVALID_SCHEDULER_NODE} From 93d0ed126a4c489ab36bc46ffbf34b8721716094 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:08:29 +0000 Subject: [PATCH 061/157] support update of constraints --- .../Scheduler/Nodes/SemaphoreConstraint.h | 32 +++++++++++++++++-- .../Scheduler/Nodes/ThrottlerConstraint.h | 23 ++++++++++--- 2 files changed, 48 insertions(+), 7 deletions(-) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 20a10f5da7d..feac9654e70 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -123,6 +123,32 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation or deactivation + void updateConstraints(const SchedulerNodePtr & self, Int64 new_max_requests, UInt64 new_max_cost) + { + std::unique_lock lock(mutex); + bool was_active = active(); + max_requests = new_max_requests; + max_cost = new_max_cost; + + if (parent) + { + // Activate on transition from inactive state + if (!was_active && active()) + parent->activateChild(this); + // Deactivate on transition into inactive state + else if (was_active && !active()) + { + // Node deactivation is usually done in dequeueRequest(), but we do not want to + // do extra call to active() on every request just to make sure there was no update(). + // There is no interface method to do deactivation, so we do the following trick. + parent->removeChild(this); + parent->attachChild(self); // This call is the only reason we have `recursive_mutex` + } + } + } + bool isActive() override { std::unique_lock lock(mutex); @@ -164,10 +190,10 @@ private: return satisfied() && child_active; } - const Int64 max_requests = default_max_requests; - const Int64 max_cost = default_max_cost; + Int64 max_requests = default_max_requests; + Int64 max_cost = default_max_cost; - std::mutex mutex; + std::recursive_mutex mutex; Int64 requests = 0; Int64 cost = 0; bool child_active = false; diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index eaa26b2da54..7071b0221ae 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -118,6 +118,21 @@ public: parent->activateChild(this); } + /// Update limits. + /// Should be called from the scheduler thread because it could lead to activation + void updateConstraints(double new_max_speed, double new_max_burst) + { + event_queue->cancelPostponed(postponed); + postponed = EventQueue::not_postponed; + bool was_active = active(); + updateBucket(0, true); // To apply previous params for duration since `last_update` + max_speed = new_max_speed; + max_burst = new_max_burst; + updateBucket(0, false); // To postpone (if needed) using new params + if (!was_active && active() && parent) + parent->activateChild(this); + } + bool isActive() override { return active(); @@ -160,7 +175,7 @@ private: parent->activateChild(this); } - void updateBucket(ResourceCost use = 0) + void updateBucket(ResourceCost use = 0, bool do_not_postpone = false) { auto now = event_queue->now(); if (max_speed > 0.0) @@ -170,7 +185,7 @@ private: tokens -= use; // This is done outside min() to avoid passing large requests w/o token consumption after long idle period // Postpone activation until there is positive amount of tokens - if (tokens < 0.0) + if (!do_not_postpone && tokens < 0.0) { auto delay_ns = std::chrono::nanoseconds(static_cast(-tokens / max_speed * 1e9)); if (postponed == EventQueue::not_postponed) @@ -194,8 +209,8 @@ private: return satisfied() && child_active; } - const double max_speed{0}; /// in tokens per second - const double max_burst{0}; /// in tokens + double max_speed{0}; /// in tokens per second + double max_burst{0}; /// in tokens EventQueue::TimePoint last_update; UInt64 postponed = EventQueue::not_postponed; From b3b0e4fef643fba711f6c11519634ab8354d1869 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:21:14 +0000 Subject: [PATCH 062/157] manager support for CREATE OR REPLACE --- .../Scheduler/Nodes/IOResourceManager.cpp | 33 ++++---- .../Scheduler/Nodes/IOResourceManager.h | 7 +- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 76 +++++++++++++++++-- 3 files changed, 90 insertions(+), 26 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index cf67bf2dfcb..101a0fa4c32 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -137,7 +137,7 @@ void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const No throw Exception(ErrorCodes::LOGICAL_ERROR, "Updating a name of workload '{}' to '{}' is not allowed in resource '{}'", old_info.name, new_info.name, resource_name); - if (old_info.parent != new_info.parent && (old_info.parent.empty() || old_info.parent.empty())) + if (old_info.parent != new_info.parent && (old_info.parent.empty() || new_info.parent.empty())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload '{}' invalid update of parent from '{}' to '{}' in resource '{}'", old_info.name, old_info.parent, new_info.parent, resource_name); @@ -157,22 +157,20 @@ void IOResourceManager::Resource::updateNode(const NodeInfo & old_info, const No { auto node = node_for_workload[old_info.name]; bool detached = false; - if (old_info.parent != new_info.parent) + if (UnifiedSchedulerNode::updateRequiresDetach(old_info.parent, new_info.parent, old_info.settings, new_info.settings)) { - node_for_workload[old_info.parent]->detachUnifiedChild(node); + if (!old_info.parent.empty()) + node_for_workload[old_info.parent]->detachUnifiedChild(node); detached = true; } node->updateSchedulingSettings(new_info.settings); - if (!detached && !old_info.parent.empty() && old_info.settings.priority != new_info.settings.priority) - node_for_workload[old_info.parent]->updateUnifiedChildPriority( - node, - old_info.settings.priority, - new_info.settings.priority); if (detached) - node_for_workload[new_info.parent]->attachUnifiedChild(node); - + { + if (!new_info.parent.empty()) + node_for_workload[new_info.parent]->attachUnifiedChild(node); + } updateCurrentVersion(); }); } @@ -268,7 +266,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) case WorkloadEntityType::Resource: { if (entity) - createResource(entity_name, entity); + createOrUpdateResource(entity_name, entity); else deleteResource(entity_name); break; @@ -315,14 +313,11 @@ void IOResourceManager::deleteWorkload(const String & workload_name) } } -void IOResourceManager::createResource(const String & resource_name, const ASTPtr & ast) +void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) { std::unique_lock lock{mutex}; if (auto resource_iter = resources.find(resource_name); resource_iter != resources.end()) - { - // Resource to be created already exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + resource_iter->second->updateResource(ast); else { // Add all workloads into the new resource @@ -420,6 +415,12 @@ void IOResourceManager::Classifier::attach(const ResourcePtr & resource, const V attachments[resource->getName()] = Attachment{.resource = resource, .version = version, .link = link}; } +void IOResourceManager::Resource::updateResource(const ASTPtr & new_resource_entity) +{ + chassert(getEntityName(new_resource_entity) == resource_name); + resource_entity = new_resource_entity; +} + std::future IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name) { auto attach_promise = std::make_shared>(); // event queue task is std::function, which requires copy semantics diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index f4871379456..dc57b985455 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -173,6 +173,9 @@ private: void deleteNode(const NodeInfo & info); void updateNode(const NodeInfo & old_info, const NodeInfo & new_info); + /// Updates resource entity + void updateResource(const ASTPtr & new_resource_entity); + /// Updates a classifier to contain a reference for specified workload std::future attachClassifier(Classifier & classifier, const String & workload_name); @@ -205,7 +208,7 @@ private: future.get(); // Blocks until execution is done in the scheduler thread } - const ASTPtr resource_entity; + ASTPtr resource_entity; const String resource_name; SchedulerRoot scheduler; @@ -256,7 +259,7 @@ private: void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast); void deleteWorkload(const String & workload_name); - void createResource(const String & resource_name, const ASTPtr & ast); + void createOrUpdateResource(const String & resource_name, const ASTPtr & ast); void deleteResource(const String & resource_name); // Topological sorting of worklaods diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 2de5131efbb..f0ec17a8dca 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -160,6 +160,14 @@ private: // Returns true iff there are no unified children attached bool empty() const { return branches.empty(); } + SchedulerNodePtr getRoot() + { + chassert(!branches.empty()); + if (root) + return root; + return branches.begin()->second.getRoot(); // There should be exactly one child-branch + } + /// Attaches a new child. /// Returns root node if it has been changed to a different node, otherwise returns null. [[nodiscard]] SchedulerNodePtr attachUnifiedChild(EventQueue * event_queue_, const UnifiedSchedulerNodePtr & child) @@ -244,6 +252,14 @@ private: SchedulerNodePtr queue; /// FifoQueue node is used if there are no children ChildrenBranch branch; /// Used if there is at least one child + SchedulerNodePtr getRoot() + { + if (queue) + return queue; + else + return branch.getRoot(); + } + // Should be called after constructor, before any other methods [[nodiscard]] SchedulerNodePtr initialize(EventQueue * event_queue_) { @@ -354,6 +370,52 @@ private: } return {}; } + + /// Detaches a child. + /// Returns root node if it has been changed to a different node, otherwise returns null. + [[nodiscard]] SchedulerNodePtr updateSchedulingSettings(EventQueue * event_queue_, const SchedulingSettings & new_settings) + { + SchedulerNodePtr node = branch.getRoot(); + + if (!settings.hasSemaphore() && new_settings.hasSemaphore()) // Add semaphore + { + semaphore = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_requests, new_settings.max_cost); + semaphore->basename = "semaphore"; + reparent(node, semaphore); + node = semaphore; + } + else if (settings.hasSemaphore() && !new_settings.hasSemaphore()) // Remove semaphore + { + detach(semaphore); + semaphore.reset(); + } + else if (settings.hasSemaphore() && new_settings.hasSemaphore()) // Update semaphore + { + static_cast(*semaphore).updateConstraints(semaphore, new_settings.max_requests, new_settings.max_cost); + node = semaphore; + } + + if (!settings.hasThrottler() && new_settings.hasThrottler()) // Add throttler + { + throttler = std::make_shared(event_queue_, SchedulerNodeInfo{}, new_settings.max_speed, new_settings.max_burst); + throttler->basename = "throttler"; + reparent(node, throttler); + node = throttler; + } + else if (settings.hasThrottler() && !new_settings.hasThrottler()) // Remove throttler + { + detach(throttler); + throttler.reset(); + } + else if (settings.hasThrottler() && new_settings.hasThrottler()) // Update throttler + { + static_cast(*throttler).updateConstraints(new_settings.max_speed, new_settings.max_burst); + node = throttler; + } + + settings = new_settings; + return node; + } }; public: @@ -388,20 +450,19 @@ public: reparent(new_child, this); } - /// Updates intermediate nodes subtree according with new priority (priority is set by the caller beforehand) - /// NOTE: Changing a priority of a unified child may lead to change of its parent. - void updateUnifiedChildPriority(const UnifiedSchedulerNodePtr & child, Priority old_priority, Priority new_priority) + static bool updateRequiresDetach(const String & old_parent, const String & new_parent, const SchedulingSettings & old_settings, const SchedulingSettings & new_settings) { - UNUSED(child, old_priority, new_priority); // TODO(serxa): implement updateUnifiedChildPriority() + return old_parent != new_parent || old_settings.priority != new_settings.priority; } /// Updates scheduling settings. Set of constraints might change. - /// NOTE: Caller is responsible for calling `updateUnifiedChildPriority` in parent unified node (if any) + /// NOTE: Caller is responsible for detaching and attaching if `updateRequiresDetach` returns true void updateSchedulingSettings(const SchedulingSettings & new_settings) { - UNUSED(new_settings); // TODO(serxa): implement updateSchedulingSettings() info.setPriority(new_settings.priority); info.setWeight(new_settings.weight); + if (auto new_child = impl.updateSchedulingSettings(event_queue, new_settings)) + reparent(new_child, this); } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children @@ -418,8 +479,7 @@ public: /// all unified nodes. Such a version control is done by `IOResourceManager`. void addRawPointerNodes(std::vector & nodes) { - if (impl.throttler) - nodes.push_back(impl.throttler); + // NOTE: `impl.throttler` could be skipped, because ThrottlerConstraint does not call `request->addConstraint()` if (impl.semaphore) nodes.push_back(impl.semaphore); if (impl.branch.queue) From 869ac2a20b87489821c3b1426d5ca60561de47e8 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:21:58 +0000 Subject: [PATCH 063/157] clean up --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 7bfc28b3263..51c5d3f0ac6 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -73,14 +73,11 @@ protected: /// Called by derived class after a new workload entity has been added. void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); - /// Called by derived class after an workload entity has been changed. - void onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity); - /// Called by derived class after an workload entity has been removed. void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); /// Sends notifications to subscribers about changes in workload entities - /// (added with previous calls onEntityAdded(), onEntityUpdated(), onEntityRemoved()). + /// (added with previous calls onEntityAdded(), onEntityRemoved()). void unlockAndNotify(std::unique_lock & lock); /// Return true iff `references` has a path from `source` to `target` From cf500575dae2c88563df1211aad4f4a9b03352f9 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 11:26:10 +0000 Subject: [PATCH 064/157] clean up --- .../Workload/WorkloadEntityStorageBase.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4e0c4f8dbbd..b1e426d363e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -382,11 +382,6 @@ void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, co queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); } -void WorkloadEntityStorageBase::onEntityUpdated(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & changed_entity) -{ - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = changed_entity}); -} - void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) { queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); @@ -465,11 +460,13 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lockgetID()); } - for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) - onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); - + // Resources should be created first becase workloads could reference them for (auto & [entity_name, ast] : resources) onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); + + // Workloads should be created in an order such that children are created only after its parent is created + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); } std::vector> WorkloadEntityStorageBase::getAllEntities() const From b20536f7ea3ed6a2f7c51a659ff7a206cfe35ea4 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 12:00:42 +0000 Subject: [PATCH 065/157] add stateless tests for CREATE OR REPLACE --- .../03232_workloads_and_resources.sql | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/tests/queries/0_stateless/03232_workloads_and_resources.sql b/tests/queries/0_stateless/03232_workloads_and_resources.sql index 3e12d70b7ff..a3e46166396 100644 --- a/tests/queries/0_stateless/03232_workloads_and_resources.sql +++ b/tests/queries/0_stateless/03232_workloads_and_resources.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel -- Do not run this test in parallel because `all` workload might affect other queries execution process + +-- Test simple resource and workload hierarchy creation create resource 03232_write (write disk 03232_fake_disk); create resource 03232_read (read disk 03232_fake_disk); create workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; @@ -7,6 +9,7 @@ create workload admin in all settings priority = 0; create workload production in all settings priority = 1, weight = 9; create workload development in all settings priority = 1, weight = 1; +-- Test that illegal actions are not allowed create workload another_root; -- {serverError BAD_ARGUMENTS} create workload self_ref in self_ref; -- {serverError BAD_ARGUMENTS} drop workload all; -- {serverError BAD_ARGUMENTS} @@ -19,7 +22,44 @@ create workload invalid in all settings max_speed = -1; -- {serverError BAD_ARGU create workload invalid in all settings max_cost = -1; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings max_requests = -1; -- {serverError BAD_ARGUMENTS} create workload invalid in all settings max_requests = 1.5; -- {serverError BAD_GET} +create or replace workload all in production; -- {serverError BAD_ARGUMENTS} +-- Test CREATE OR REPLACE WORKLOAD +create or replace workload all settings max_requests = 200 for 03232_write, max_requests = 100 for 03232_read; +create or replace workload admin in all settings priority = 1; +create or replace workload admin in all settings priority = 2; +create or replace workload admin in all settings priority = 0; +create or replace workload production in all settings priority = 1, weight = 90; +create or replace workload production in all settings priority = 0, weight = 9; +create or replace workload production in all settings priority = 2, weight = 9; +create or replace workload development in all settings priority = 1; +create or replace workload development in all settings priority = 0; +create or replace workload development in all settings priority = 2; + +-- Test CREATE OR REPLACE RESOURCE +create or replace resource 03232_write (write disk 03232_fake_disk_2); +create or replace resource 03232_read (read disk 03232_fake_disk_2); + +-- Test update settings with CREATE OR REPLACE WORKLOAD +create or replace workload production in all settings priority = 1, weight = 9, max_requests = 100; +create or replace workload development in all settings priority = 1, weight = 1, max_requests = 10; +create or replace workload production in all settings priority = 1, weight = 9, max_cost = 100000; +create or replace workload development in all settings priority = 1, weight = 1, max_cost = 10000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000; +create or replace workload production in all settings priority = 1, weight = 9, max_speed = 1000000, max_burst = 10000000; +create or replace workload development in all settings priority = 1, weight = 1, max_speed = 100000, max_burst = 1000000; +create or replace workload all settings max_cost = 1000000, max_speed = 100000 for 03232_write, max_speed = 200000 for 03232_read; +create or replace workload all settings max_requests = 100 for 03232_write, max_requests = 200 for 03232_read; +create or replace workload production in all settings priority = 1, weight = 9; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Test change parent with CREATE OR REPLACE WORKLOAD +create or replace workload development in production settings priority = 1, weight = 1; +create or replace workload development in admin settings priority = 1, weight = 1; +create or replace workload development in all settings priority = 1, weight = 1; + +-- Clean up drop workload if exists production; drop workload if exists development; drop workload if exists admin; From d28e41f712dc83d160bcd788eb99115ef5b8e517 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 12:46:54 +0000 Subject: [PATCH 066/157] add unit tests for weight and priority updates --- .../Scheduler/Nodes/UnifiedSchedulerNode.h | 5 ++ .../Scheduler/Nodes/tests/ResourceTest.h | 21 +++++ .../tests/gtest_throttler_constraint.cpp | 20 ++--- .../tests/gtest_unified_scheduler_node.cpp | 80 ++++++++++++++++--- 4 files changed, 106 insertions(+), 20 deletions(-) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index f0ec17a8dca..f7b5d1a2056 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -465,6 +465,11 @@ public: reparent(new_child, this); } + const SchedulingSettings & getSettings() const + { + return impl.settings; + } + /// Returns the queue to be used for resource requests or `nullptr` if it has unified children std::shared_ptr getQueue() { diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index bbe0df4872e..704f7119300 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -179,6 +179,27 @@ public: return node; } + // Updates the parent and/or scheduling settings for a specidfied `node`. + // Unit test implementation must make sure that all needed queues and constraints are not going to be destroyed. + // Normally it is the responsibility of IOResourceManager, but we do not use it here, so manual version control is required. + // (see IOResourceManager::Resource::updateCurrentVersion() fo details) + void updateUnifiedNode(const UnifiedSchedulerNodePtr & node, const UnifiedSchedulerNodePtr & old_parent, const UnifiedSchedulerNodePtr & new_parent, const SchedulingSettings & new_settings) + { + EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported + bool detached = false; + if (UnifiedSchedulerNode::updateRequiresDetach(old_parent->basename, new_parent->basename, node->getSettings(), new_settings)) { + if (old_parent) + old_parent->detachUnifiedChild(node); + detached = true; + } + + node->updateSchedulingSettings(new_settings); + + if (detached && new_parent) + new_parent->attachUnifiedChild(node); + } + + void enqueue(const UnifiedSchedulerNodePtr & node, const std::vector & costs) { enqueueImpl(node->getQueue().get(), costs, node->basename); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 2bc24cdb292..9bb1bc572b8 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -160,22 +160,22 @@ TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness) t.enqueue("/fair/B", {req_cost}); } - double shareA = 0.1; - double shareB = 0.9; + double share_a = 0.1; + double share_b = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); - auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - double consumedA = 0; - double consumedB = 0; + double consumed_a = 0; + double consumed_b = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); - t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); - consumedA = arrival_curve * shareA; - consumedB = arrival_curve * shareB; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index faebaa72b71..cfd837d4f1a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -403,23 +403,23 @@ TEST(SchedulerUnifiedNode, ThrottlerAndFairness) t.enqueue(b, {req_cost}); } - double shareA = 0.1; - double shareB = 0.9; + double share_a = 0.1; + double share_b = 0.9; // Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share - auto max_latencyA = static_cast(req_cost * (1.0 + 1.0 / shareA)); - auto max_latencyB = static_cast(req_cost * (1.0 + 1.0 / shareB)); + auto max_latency_a = static_cast(req_cost * (1.0 + 1.0 / share_a)); + auto max_latency_b = static_cast(req_cost * (1.0 + 1.0 / share_b)); - double consumedA = 0; - double consumedB = 0; + double consumed_a = 0; + double consumed_b = 0; for (int seconds = 0; seconds < 100; seconds++) { t.process(start + std::chrono::seconds(seconds)); double arrival_curve = 100.0 + 10.0 * seconds + req_cost; - t.consumed("A", static_cast(arrival_curve * shareA - consumedA), max_latencyA); - t.consumed("B", static_cast(arrival_curve * shareB - consumedB), max_latencyB); - consumedA = arrival_curve * shareA; - consumedB = arrival_curve * shareB; + t.consumed("A", static_cast(arrival_curve * share_a - consumed_a), max_latency_a); + t.consumed("B", static_cast(arrival_curve * share_b - consumed_b), max_latency_b); + consumed_a = arrival_curve * share_a; + consumed_b = arrival_curve * share_b; } } @@ -493,3 +493,63 @@ TEST(SchedulerUnifiedNode, ResourceGuardException) t.dequeue(2); t.consumed("A", 20); } + +TEST(SchedulerUnifiedNode, UpdateWeight) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 3.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("A", 10); + t.consumed("B", 30); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{}}); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); + + t.dequeue(4); + t.consumed("A", 20); + t.consumed("B", 20); +} + +TEST(SchedulerUnifiedNode, UpdatePriority) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{}}); + + t.enqueue(a, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(b, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-1}}); + + t.dequeue(2); + t.consumed("A", 20); + t.consumed("B", 0); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 0); + t.consumed("B", 20); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{-2}}); + + t.dequeue(2); + t.consumed("A", 10); + t.consumed("B", 10); +} From ee2a5dc4b5794200fb899087b9ff2b75d3b9ee9b Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:11:42 +0000 Subject: [PATCH 067/157] unittest for parent update --- .../tests/gtest_unified_scheduler_node.cpp | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index cfd837d4f1a..5b86b54e9b0 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -553,3 +553,42 @@ TEST(SchedulerUnifiedNode, UpdatePriority) t.consumed("A", 10); t.consumed("B", 10); } + +TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + + auto x = t.createUnifiedNode("X", a, {}); + auto y = t.createUnifiedNode("Y", b, {}); + + t.enqueue(x, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(2); + t.consumed("X", 20); + t.consumed("Y", 0); + + t.updateUnifiedNode(x, a, b, {}); + + t.dequeue(2); + t.consumed("X", 10); + t.consumed("Y", 10); + + t.updateUnifiedNode(y, b, a, {}); + + t.dequeue(2); + t.consumed("X", 0); + t.consumed("Y", 20); + + t.updateUnifiedNode(y, a, all, {}); + t.updateUnifiedNode(x, b, all, {}); + + t.dequeue(4); + t.consumed("X", 20); + t.consumed("Y", 20); +} + From 6f243450cc2fd4ada3a04c50272f7b369b168682 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:12:26 +0000 Subject: [PATCH 068/157] fix parent detaching --- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index f7b5d1a2056..ef10458df0d 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -135,6 +135,7 @@ private: if (it == children.end()) return {}; // unknown child + detach(child); children.erase(it); if (children.size() == 1) { From cd393c7f9d6549a72465d57a7eeee64f4b0b0706 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 13:31:15 +0000 Subject: [PATCH 069/157] add unit tests for intermediate node updates --- .../tests/gtest_unified_scheduler_node.cpp | 83 ++++++++++++++++++- 1 file changed, 81 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 5b86b54e9b0..10f92bc43c3 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -451,7 +451,6 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.consumed("A", 20); } - TEST(SchedulerUnifiedNode, ResourceGuardException) { ResourceTest t; @@ -561,7 +560,6 @@ TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) auto all = t.createUnifiedNode("all"); auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); - auto x = t.createUnifiedNode("X", a, {}); auto y = t.createUnifiedNode("Y", b, {}); @@ -592,3 +590,84 @@ TEST(SchedulerUnifiedNode, UpdateParentOfLeafNode) t.consumed("Y", 20); } +TEST(SchedulerUnifiedNode, UpdatePriorityOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto x1 = t.createUnifiedNode("X1", a, {}); + auto y1 = t.createUnifiedNode("Y1", b, {}); + auto x2 = t.createUnifiedNode("X2", a, {}); + auto y2 = t.createUnifiedNode("Y2", b, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(a, all, all, {.weight = 1.0, .priority = Priority{2}}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(b, all, all, {.weight = 1.0, .priority = Priority{1}}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} + +TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) +{ + ResourceTest t; + + auto all = t.createUnifiedNode("all"); + auto a = t.createUnifiedNode("A", all, {.weight = 1.0, .priority = Priority{1}}); + auto b = t.createUnifiedNode("B", all, {.weight = 1.0, .priority = Priority{2}}); + auto c = t.createUnifiedNode("C", a, {}); + auto d = t.createUnifiedNode("D", b, {}); + auto x1 = t.createUnifiedNode("X1", c, {}); + auto y1 = t.createUnifiedNode("Y1", d, {}); + auto x2 = t.createUnifiedNode("X2", c, {}); + auto y2 = t.createUnifiedNode("Y2", d, {}); + + t.enqueue(x1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y1, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(x2, {10, 10, 10, 10, 10, 10, 10, 10}); + t.enqueue(y2, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.dequeue(4); + t.consumed("X1", 20); + t.consumed("Y1", 0); + t.consumed("X2", 20); + t.consumed("Y2", 0); + + t.updateUnifiedNode(c, a, b, {}); + + t.dequeue(4); + t.consumed("X1", 10); + t.consumed("Y1", 10); + t.consumed("X2", 10); + t.consumed("Y2", 10); + + t.updateUnifiedNode(d, b, a, {}); + + t.dequeue(4); + t.consumed("X1", 0); + t.consumed("Y1", 20); + t.consumed("X2", 0); + t.consumed("Y2", 20); +} From dca5c250fe0a70e8d0bb88714fd42cb7b1e85168 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Oct 2024 18:19:05 +0000 Subject: [PATCH 070/157] add tests for throttler updates --- .../Scheduler/Nodes/tests/ResourceTest.h | 7 +- .../tests/gtest_unified_scheduler_node.cpp | 73 +++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 704f7119300..1bd7824911d 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -187,7 +187,12 @@ public: { EXPECT_TRUE((old_parent && new_parent) || (!old_parent && !new_parent)); // changing root node is not supported bool detached = false; - if (UnifiedSchedulerNode::updateRequiresDetach(old_parent->basename, new_parent->basename, node->getSettings(), new_settings)) { + if (UnifiedSchedulerNode::updateRequiresDetach( + old_parent ? old_parent->basename : "", + new_parent ? new_parent->basename : "", + node->getSettings(), + new_settings)) + { if (old_parent) old_parent->detachUnifiedChild(node); detached = true; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index 10f92bc43c3..dcc29422d7f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -671,3 +671,76 @@ TEST(SchedulerUnifiedNode, UpdateParentOfIntermediateNode) t.consumed("X2", 0); t.consumed("Y2", 20); } + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxSpeed) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 20.0}); + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 30); // It is allowed to go below zero for exactly one resource request + + t.process(start + std::chrono::seconds(1)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 10); + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 1.0, .max_burst = 20.0}); + + t.process(start + std::chrono::seconds(12)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(22)); + t.consumed("all", 10); + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 10); +} + +TEST(SchedulerUnifiedNode, UpdateThrottlerMaxBurst) +{ + ResourceTest t; + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, 0); + + auto all = t.createUnifiedNode("all", {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.enqueue(all, {100}); + + t.process(start + std::chrono::seconds(0)); + t.consumed("all", 100); // consume all tokens, but it is still active (not negative) + + t.process(start + std::chrono::seconds(2)); + t.consumed("all", 0); // There was nothing to consume + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 30.0}); + + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 0); // There was nothing to consume + + t.enqueue(all, {10, 10, 10, 10, 10, 10, 10, 10, 10, 10}); + t.process(start + std::chrono::seconds(5)); + t.consumed("all", 40); // min(30 tokens, 5 sec * 10 tokens/sec) = 30 tokens + 1 extra request to go below zero + + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 10.0, .max_burst = 100.0}); + + t.process(start + std::chrono::seconds(100)); + t.consumed("all", 60); // Consume rest + + t.process(start + std::chrono::seconds(150)); + t.updateUnifiedNode(all, {}, {}, {.priority = Priority{}, .max_speed = 100.0, .max_burst = 200.0}); + + t.process(start + std::chrono::seconds(200)); + + t.enqueue(all, {195, 1, 1, 1, 1, 1, 1, 1, 1, 1}); + t.process(start + std::chrono::seconds(200)); + t.consumed("all", 201); // check we cannot consume more than max_burst + 1 request + + t.process(start + std::chrono::seconds(100500)); + t.consumed("all", 3); +} From 58ec00aea869318a2af81cc5189728376fd3cfb0 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:18:41 +0000 Subject: [PATCH 071/157] fix typos --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.h | 6 +++--- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 2 +- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 4 ++-- .../Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp | 4 ++-- src/Common/Scheduler/ResourceRequest.h | 4 ++-- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 2 +- .../Scheduler/Workload/createWorkloadEntityStorage.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 4 ++-- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 101a0fa4c32..91c54bade3c 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -189,7 +189,7 @@ void IOResourceManager::Resource::updateCurrentVersion() { previous_version->newer_version = current_version; // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. - // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems imposible) + // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems impossible) // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index dc57b985455..f33251ad37c 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -30,7 +30,7 @@ namespace DB * * Manager process updates of WORKLOADs and RESOURCEs: CREATE/DROP/ALTER. * When a RESOURCE is created (dropped) a corresponding scheduler nodes hierarchy is created (destroyed). - * After DROP RESOURCE parts of hierarchy might be keept alive while at least one query uses it. + * After DROP RESOURCE parts of hierarchy might be kept alive while at least one query uses it. * * Manager is specific to IO only because it create scheduler node hierarchies for RESOURCEs having * WRITE DISK and/or READ DISK definitions. CPU and memory resources are managed separately. @@ -46,7 +46,7 @@ namespace DB * Parent of the root workload for a resource is SchedulerRoot with its own scheduler thread. * So every resource has its dedicated thread for processing of resource request and other events (see EventQueue). * - * Here is an example of SQL and corresponding heirarchy of scheduler nodes: + * Here is an example of SQL and corresponding hierarchy of scheduler nodes: * CREATE RESOURCE my_io_resource (...) * CREATE WORKLOAD all * CREATE WORKLOAD production PARENT all @@ -85,7 +85,7 @@ namespace DB * * Previous version should hold reference to a newer version. It is required for proper handling of updates. * Classifiers that were created for any of old versions may use nodes of newer version due to updateNode(). - * It may move a queue to a new position in the hierarchy or create/destry constraints, thus resource requests + * It may move a queue to a new position in the hierarchy or create/destroy constraints, thus resource requests * created by old classifier may reference constraints of newer versions through `request->constraints` which * is filled during dequeueRequst(). * diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index ef10458df0d..1ab187b388e 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -29,7 +29,7 @@ using UnifiedSchedulerNodePtr = std::shared_ptr; /* * Unified scheduler node combines multiple nodes internally to provide all available scheduling policies and constraints. * Whole scheduling hierarchy could "logically" consist of unified nodes only. Physically intermediate "internal" nodes - * are also present. This approch is easiers for manipulations in runtime than using multiple types of nodes. + * are also present. This approach is easiers for manipulations in runtime than using multiple types of nodes. * * Unified node is capable of updating its internal structure based on: * 1. Number of children (fifo if =0 or fairness/priority if >0). diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 1bd7824911d..05797189837 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -336,7 +336,7 @@ struct ResourceTestManager : public ResourceTestBase waitExecute(); } - /// Just enqueue resource request, do not block (neede for tests to sync). Call `waitExecuted()` afterwards + /// Just enqueue resource request, do not block (needed for tests to sync). Call `waitExecuted()` afterwards Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_, EnqueueOnlyEnum) : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer) , t(t_) @@ -345,7 +345,7 @@ struct ResourceTestManager : public ResourceTestBase t.onEnqueue(link); } - /// Waits for ResourceRequest::execute() to be called for enqueued requet + /// Waits for ResourceRequest::execute() to be called for enqueued request void waitExecute() { lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp index dcc29422d7f..159ccc616f4 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_unified_scheduler_node.cpp @@ -431,7 +431,7 @@ TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction) t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled - // This will destory the queue and fail both requests + // This will destroy the queue and fail both requests auto a = t.createUnifiedNode("A", all); t.failed(20); @@ -472,7 +472,7 @@ TEST(SchedulerUnifiedNode, ResourceGuardException) } }); - // This will destory the queue and fail both requests + // This will destroy the queue and fail both requests auto a = t.createUnifiedNode("A", all); t.failed(20); consumer.join(); diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index 03bdaec6a2b..e633af15157 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -17,8 +17,8 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -// TODO(serxa): validate hierarchy to avoid too many constrants -/// Max number of constraints for a request to pass though (depth of constaints chain) +// TODO(serxa): validate hierarchy to avoid too many constraints +/// Max number of constraints for a request to pass though (depth of constraints chain) constexpr size_t ResourceMaxConstraints = 8; /* diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index b1e426d363e..f29d0f45f22 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -460,7 +460,7 @@ void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lockgetID()); } - // Resources should be created first becase workloads could reference them + // Resources should be created first because workloads could reference them for (auto & [entity_name, ast] : resources) onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp index dde995db6e1..8475fe21455 100644 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -34,7 +34,7 @@ std::unique_ptr createWorkloadEntityStorage(const Contex zookeeper_path_key, disk_path_key); } - abort(); // TODO(serxa): crate WorkloadEntityKeeperStorage object + abort(); // TODO(serxa): create WorkloadEntityKeeperStorage object //return std::make_unique(global_context, config.getString(zookeeper_path_key)); } else diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index cd5f1e375d9..03ab0fd8572 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -110,7 +110,7 @@ DiskObjectStorage::DiskObjectStorage( if (read_resource_name_from_config.empty()) LOG_INFO(log, "Using resource '{}' for READ", resource_name); else - LOG_INFO(log, "Resource '{}' should be used for READ, but it is overriden by config to resource '{}'", + LOG_INFO(log, "Resource '{}' should be used for READ, but it is overridden by config to resource '{}'", resource_name, read_resource_name_from_config); read_resource_name_from_sql = resource_name; break; @@ -120,7 +120,7 @@ DiskObjectStorage::DiskObjectStorage( if (write_resource_name_from_config.empty()) LOG_INFO(log, "Using resource '{}' for WRITE", resource_name); else - LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overriden by config to resource '{}'", + LOG_INFO(log, "Resource '{}' should be used for WRITE, but it is overridden by config to resource '{}'", resource_name, write_resource_name_from_config); write_resource_name_from_sql = resource_name; break; From 6170b02c23512ae226c5179f4015b8cec29d0600 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:24:05 +0000 Subject: [PATCH 072/157] fix style --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 1 - src/Common/Scheduler/Nodes/IOResourceManager.h | 3 ++- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 5 +++++ src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h | 1 + src/Parsers/ASTCreateResourceQuery.h | 6 ++++-- tests/integration/test_scheduler/test.py | 2 +- 7 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 91c54bade3c..26f7c65ef55 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { - extern const int RESOURCE_ACCESS_DENIED; extern const int RESOURCE_NOT_FOUND; extern const int INVALID_SCHEDULER_NODE; extern const int LOGICAL_ERROR; diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index f33251ad37c..0bbd14c2ca9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -249,7 +249,8 @@ private: private: IOResourceManager * resource_manager; std::mutex mutex; - struct Attachment { + struct Attachment + { ResourcePtr resource; VersionPtr version; ResourceLink link; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 1ab187b388e..3edca1f70c1 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -20,7 +20,6 @@ namespace DB namespace ErrorCodes { extern const int INVALID_SCHEDULER_NODE; - extern const int LOGICAL_ERROR; } class UnifiedSchedulerNode; @@ -84,7 +83,8 @@ private: } /// A branch of the tree for a specific priority value - struct FairnessBranch { + struct FairnessBranch + { SchedulerNodePtr root; /// FairPolicy node is used if multiple children with the same priority are attached std::unordered_map children; // basename -> child diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 05797189837..b02aa00588a 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -29,6 +29,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int RESOURCE_ACCESS_DENIED; +} + struct ResourceTestBase { ResourceTestBase() diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index e69de29bb2d..6f70f09beec 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -0,0 +1 @@ +#pragma once diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index e1713e6b063..5cf29b31ed0 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -10,11 +10,13 @@ namespace DB class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster { public: - enum class AccessMode { + enum class AccessMode + { Read, Write }; - struct Operation { + struct Operation + { AccessMode mode; String disk; }; diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 2c2fa043f28..24071a29bd6 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -2,11 +2,11 @@ # pylint: disable=redefined-outer-name # pylint: disable=line-too-long +import random import threading import time import pytest -import random from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster From 02867ca2e7630d866be8b9bc8ad90b48af18a3a0 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 00:58:06 +0000 Subject: [PATCH 073/157] style --- src/Common/Scheduler/Nodes/ClassifiersConfig.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp index 4b0b0eaccfa..455d0880aa6 100644 --- a/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp +++ b/src/Common/Scheduler/Nodes/ClassifiersConfig.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int RESOURCE_NOT_FOUND; -} - ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { Poco::Util::AbstractConfiguration::Keys keys; From 511054f92d524b59a2bade3e34965300e69949ee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 9 Oct 2024 01:06:04 +0000 Subject: [PATCH 074/157] Automatic style fix --- tests/integration/test_scheduler/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 24071a29bd6..b78376bffe2 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -645,7 +645,7 @@ def test_create_workload(): ) do_checks() - node.restart_clickhouse() # Check that workloads persist + node.restart_clickhouse() # Check that workloads persist do_checks() @@ -706,7 +706,9 @@ def test_workload_hierarchy_changes(): split_idx = random.randint(1, len(queries) - 2) for query_idx in range(0, split_idx): node.query(queries[query_idx]) - node.query("create resource io_test (write disk non_existent_disk, read disk non_existent_disk);") + node.query( + "create resource io_test (write disk non_existent_disk, read disk non_existent_disk);" + ) node.query("drop resource io_test;") for query_idx in range(split_idx, len(queries)): node.query(queries[query_idx]) From cde94e7afa85d842d9ff6b760acc55ba23a78a03 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 08:20:03 +0000 Subject: [PATCH 075/157] adjust test 01271_show_privileges --- tests/queries/0_stateless/01271_show_privileges.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..feeef5e89fa 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -59,6 +59,8 @@ CREATE DICTIONARY [] DICTIONARY CREATE CREATE TEMPORARY TABLE [] GLOBAL CREATE ARBITRARY TEMPORARY TABLE CREATE ARBITRARY TEMPORARY TABLE [] GLOBAL CREATE CREATE FUNCTION [] GLOBAL CREATE +CREATE WORKLOAD [] GLOBAL CREATE +CREATE RESOURCE [] GLOBAL CREATE CREATE NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN CREATE [] \N ALL DROP DATABASE [] DATABASE DROP @@ -66,6 +68,8 @@ DROP TABLE [] TABLE DROP DROP VIEW [] VIEW DROP DROP DICTIONARY [] DICTIONARY DROP DROP FUNCTION [] GLOBAL DROP +DROP WORKLOAD [] GLOBAL DROP +DROP RESOURCE [] GLOBAL DROP DROP NAMED COLLECTION [] NAMED_COLLECTION NAMED COLLECTION ADMIN DROP [] \N ALL UNDROP TABLE [] TABLE ALL From fd0a9511eb912011d0de530dfe3be94de27d5836 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Oct 2024 16:47:31 +0000 Subject: [PATCH 076/157] fix stateless test --- tests/queries/0_stateless/03232_resource_create_and_drop.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03232_resource_create_and_drop.sql b/tests/queries/0_stateless/03232_resource_create_and_drop.sql index eb356e12448..ceebd557a51 100644 --- a/tests/queries/0_stateless/03232_resource_create_and_drop.sql +++ b/tests/queries/0_stateless/03232_resource_create_and_drop.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel +-- Do not run this test in parallel because creating the same resource twice will fail CREATE OR REPLACE RESOURCE 03232_resource_1 (WRITE DISK 03232_disk_1, READ DISK 03232_disk_1); SELECT name, read_disks, write_disks, create_query FROM system.resources WHERE name ILIKE '03232_%' ORDER BY name; CREATE RESOURCE IF NOT EXISTS 03232_resource_2 (READ DISK 03232_disk_2); From 384ba4217dcb8bc526e50dea7cbf88b9ed7e734d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Oct 2024 05:15:29 +0000 Subject: [PATCH 077/157] Fix logical error in JSONExtract with LowCardinality(Nullable) --- src/Formats/JSONExtractTree.cpp | 2 +- .../0_stateless/03247_json_extract_lc_nullable.reference | 1 + tests/queries/0_stateless/03247_json_extract_lc_nullable.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03247_json_extract_lc_nullable.reference create mode 100644 tests/queries/0_stateless/03247_json_extract_lc_nullable.sql diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 9ea335ee7fe..ae6051823b7 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1085,7 +1085,7 @@ public: } auto & col_lc = assert_cast(column); - auto tmp_nested = col_lc.getDictionary().getNestedColumn()->cloneEmpty(); + auto tmp_nested = removeNullable(col_lc.getDictionary().getNestedColumn()->cloneEmpty())->assumeMutable(); if (!nested->insertResultToColumn(*tmp_nested, element, insert_settings, format_settings, error)) return false; diff --git a/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference b/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference new file mode 100644 index 00000000000..a949a93dfcc --- /dev/null +++ b/tests/queries/0_stateless/03247_json_extract_lc_nullable.reference @@ -0,0 +1 @@ +128 diff --git a/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql b/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql new file mode 100644 index 00000000000..bac1e34c1ab --- /dev/null +++ b/tests/queries/0_stateless/03247_json_extract_lc_nullable.sql @@ -0,0 +1,2 @@ +select JSONExtract('{"a" : 128}', 'a', 'LowCardinality(Nullable(Int128))'); + From c4763389c145416186d128aa5bd03633ab5b383b Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 12 Oct 2024 10:53:16 +0000 Subject: [PATCH 078/157] fix tidy build --- src/Common/Scheduler/Nodes/FairPolicy.h | 2 +- src/Common/Scheduler/Nodes/FifoQueue.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 2 +- src/Common/Scheduler/Nodes/PriorityPolicy.h | 2 +- src/Common/Scheduler/Nodes/SemaphoreConstraint.h | 2 +- src/Common/Scheduler/Nodes/ThrottlerConstraint.h | 2 +- src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h | 4 ++-- src/Common/Scheduler/SchedulerRoot.h | 2 +- .../Workload/WorkloadEntityStorageBase.cpp | 15 +-------------- .../Workload/WorkloadEntityStorageBase.h | 3 +-- src/Parsers/ASTCreateResourceQuery.h | 2 +- 11 files changed, 12 insertions(+), 26 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FairPolicy.h b/src/Common/Scheduler/Nodes/FairPolicy.h index c6151c5727b..a865711c460 100644 --- a/src/Common/Scheduler/Nodes/FairPolicy.h +++ b/src/Common/Scheduler/Nodes/FairPolicy.h @@ -28,7 +28,7 @@ namespace ErrorCodes * of a child is set to vruntime of "start" of the last request. This guarantees immediate processing * of at least single request of newly activated children and thus best isolation and scheduling latency. */ -class FairPolicy : public ISchedulerNode +class FairPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index 3372864402c..ea8985e314f 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -23,7 +23,7 @@ namespace ErrorCodes /* * FIFO queue to hold pending resource requests */ -class FifoQueue : public ISchedulerQueue +class FifoQueue final : public ISchedulerQueue { public: FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 26f7c65ef55..0f015dd22b6 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -250,7 +250,7 @@ IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) { - for (auto [entity_type, entity_name, entity] : events) + for (const auto & [entity_type, entity_name, entity] : events) { switch (entity_type) { diff --git a/src/Common/Scheduler/Nodes/PriorityPolicy.h b/src/Common/Scheduler/Nodes/PriorityPolicy.h index bb2c98d28b1..cfbe242c13e 100644 --- a/src/Common/Scheduler/Nodes/PriorityPolicy.h +++ b/src/Common/Scheduler/Nodes/PriorityPolicy.h @@ -19,7 +19,7 @@ namespace ErrorCodes * Scheduler node that implements priority scheduling policy. * Requests are scheduled in order of priorities. */ -class PriorityPolicy : public ISchedulerNode +class PriorityPolicy final : public ISchedulerNode { /// Scheduling state of a child struct Item diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index c7dccbcdfbb..2454c1ec5bf 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -14,7 +14,7 @@ namespace DB * Limited concurrency constraint. * Blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost` */ -class SemaphoreConstraint : public ISchedulerConstraint +class SemaphoreConstraint final : public ISchedulerConstraint { static constexpr Int64 default_max_requests = std::numeric_limits::max(); static constexpr Int64 default_max_cost = std::numeric_limits::max(); diff --git a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h index e9e4c15b18f..a2594b7ff2e 100644 --- a/src/Common/Scheduler/Nodes/ThrottlerConstraint.h +++ b/src/Common/Scheduler/Nodes/ThrottlerConstraint.h @@ -13,7 +13,7 @@ namespace DB * Limited throughput constraint. Blocks if token-bucket constraint is violated: * i.e. more than `max_burst + duration * max_speed` cost units (aka tokens) dequeued from this node in last `duration` seconds. */ -class ThrottlerConstraint : public ISchedulerConstraint +class ThrottlerConstraint final : public ISchedulerConstraint { public: static constexpr double default_burst_seconds = 1.0; diff --git a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h index 3edca1f70c1..e8e568c9acb 100644 --- a/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h +++ b/src/Common/Scheduler/Nodes/UnifiedSchedulerNode.h @@ -54,7 +54,7 @@ using UnifiedSchedulerNodePtr = std::shared_ptr; * - unified child: leaf of this "internal" subtree (CHILD[p,w]); * - intermediate node: any child that is not UnifiedSchedulerNode (unified child or `this`) */ -class UnifiedSchedulerNode : public ISchedulerNode +class UnifiedSchedulerNode final : public ISchedulerNode { private: /// Helper function for managing a parent of a node @@ -472,7 +472,7 @@ public: } /// Returns the queue to be used for resource requests or `nullptr` if it has unified children - std::shared_ptr getQueue() + std::shared_ptr getQueue() const { return static_pointer_cast(impl.branch.queue); } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 8549a1880fb..45e4309fc81 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -28,7 +28,7 @@ namespace ErrorCodes * Resource scheduler root node with a dedicated thread. * Immediate children correspond to different resources. */ -class SchedulerRoot : public ISchedulerNode +class SchedulerRoot final : public ISchedulerNode { private: struct Resource diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index f29d0f45f22..060bbbd6f87 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -430,7 +430,7 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vector> WorkloadEntityStorageBase::getAllEntities return all_entities; } -// TODO(serxa): add notifications or remove this function -void WorkloadEntityStorageBase::removeAllEntitiesExcept(const Strings & entity_names_to_keep) -{ - boost::container::flat_set names_set_to_keep{entity_names_to_keep.begin(), entity_names_to_keep.end()}; - std::lock_guard lock(mutex); - for (auto it = entities.begin(); it != entities.end();) - { - auto current = it++; - if (!names_set_to_keep.contains(current->first)) - entities.erase(current); - } -} - bool WorkloadEntityStorageBase::isIndirectlyReferenced(const String & target, const String & source) { std::queue bfs; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 51c5d3f0ac6..e1f43181a0c 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -45,7 +45,7 @@ public: const String & entity_name, bool throw_if_not_exists) override; - virtual scope_guard getAllEntitiesAndSubscribe( + scope_guard getAllEntitiesAndSubscribe( const OnChangedHandler & handler) override; protected: @@ -68,7 +68,6 @@ protected: void setAllEntities(const std::vector> & new_entities); void makeEventsForAllEntities(std::unique_lock & lock); - void removeAllEntitiesExcept(const Strings & entity_names_to_keep); /// Called by derived class after a new workload entity has been added. void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index 5cf29b31ed0..b05176837bc 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -33,7 +33,7 @@ public: ASTPtr clone() const override; - void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + void formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked frame) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } From d58395c834c12349e9f7e0774cdede1d593c704e Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 12 Oct 2024 15:53:31 +0000 Subject: [PATCH 079/157] fix tidy build --- .../Scheduler/Nodes/tests/ResourceTest.h | 61 ++++++++++++++++--- 1 file changed, 53 insertions(+), 8 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index b02aa00588a..681453817c6 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -91,15 +91,56 @@ struct ResourceTestBase }; -struct ConstraintTest : public SemaphoreConstraint +struct ConstraintTest final : public ISchedulerConstraint { explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : SemaphoreConstraint(event_queue_, config, config_prefix) + : ISchedulerConstraint(event_queue_, config, config_prefix) + , impl(event_queue_, config, config_prefix) {} + const String & getTypeName() const override + { + return impl.getTypeName(); + } + + bool equals(ISchedulerNode * other) override + { + return impl.equals(other); + } + + void attachChild(const std::shared_ptr & child) override + { + impl.attachChild(child); + } + + void removeChild(ISchedulerNode * child) override + { + impl.removeChild(child); + } + + ISchedulerNode * getChild(const String & child_name) override + { + return impl.getChild(child_name); + } + + void activateChild(ISchedulerNode * child) override + { + impl.activateChild(child); + } + + bool isActive() override + { + return impl.isActive(); + } + + size_t activeChildren() override + { + return impl.activeChildren(); + } + std::pair dequeueRequest() override { - auto [request, active] = SemaphoreConstraint::dequeueRequest(); + auto [request, active] = impl.dequeueRequest(); if (request) { std::unique_lock lock(mutex); @@ -110,13 +151,17 @@ struct ConstraintTest : public SemaphoreConstraint void finishRequest(ResourceRequest * request) override { - { - std::unique_lock lock(mutex); - requests.erase(request); - } - SemaphoreConstraint::finishRequest(request); + impl.finishRequest(request); + std::unique_lock lock(mutex); + requests.erase(request); } + bool isSatisfied() override + { + return impl.isSatisfied(); + } + + SemaphoreConstraint impl; std::mutex mutex; std::set requests; }; From 912d59d2c89e1cd1f0f72280e2fc01c9c9d28255 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 13 Oct 2024 11:14:08 +0000 Subject: [PATCH 080/157] fix unittests --- .../Scheduler/Nodes/tests/ResourceTest.h | 76 ------------------- .../Nodes/tests/gtest_resource_scheduler.cpp | 21 ++--- 2 files changed, 11 insertions(+), 86 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index 681453817c6..3fcbea55ee1 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -90,82 +90,6 @@ struct ResourceTestBase } }; - -struct ConstraintTest final : public ISchedulerConstraint -{ - explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : ISchedulerConstraint(event_queue_, config, config_prefix) - , impl(event_queue_, config, config_prefix) - {} - - const String & getTypeName() const override - { - return impl.getTypeName(); - } - - bool equals(ISchedulerNode * other) override - { - return impl.equals(other); - } - - void attachChild(const std::shared_ptr & child) override - { - impl.attachChild(child); - } - - void removeChild(ISchedulerNode * child) override - { - impl.removeChild(child); - } - - ISchedulerNode * getChild(const String & child_name) override - { - return impl.getChild(child_name); - } - - void activateChild(ISchedulerNode * child) override - { - impl.activateChild(child); - } - - bool isActive() override - { - return impl.isActive(); - } - - size_t activeChildren() override - { - return impl.activeChildren(); - } - - std::pair dequeueRequest() override - { - auto [request, active] = impl.dequeueRequest(); - if (request) - { - std::unique_lock lock(mutex); - requests.insert(request); - } - return {request, active}; - } - - void finishRequest(ResourceRequest * request) override - { - impl.finishRequest(request); - std::unique_lock lock(mutex); - requests.erase(request); - } - - bool isSatisfied() override - { - return impl.isSatisfied(); - } - - SemaphoreConstraint impl; - std::mutex mutex; - std::set requests; -}; - class ResourceTestClass : public ResourceTestBase { struct Request : public ResourceRequest diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index 8eaa4ebb840..85d35fab0a6 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,5 +1,6 @@ #include +#include #include #include @@ -113,14 +114,14 @@ TEST(SchedulerRoot, Smoke) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); r1.registerResource(); ResourceHolder r2(t); - auto * fc2 = r2.add("/", "1"); + auto * fc2 = r2.add("/", "1"); r2.add("/prio"); auto c = r2.addQueue("/prio/C", "-1"); auto d = r2.addQueue("/prio/D", "-2"); @@ -128,25 +129,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); - EXPECT_TRUE(fc1->requests.contains(&rg.request)); + EXPECT_TRUE(fc1->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); - EXPECT_TRUE(fc2->requests.contains(&rg.request)); + EXPECT_TRUE(fc2->getInflights().first == 1); rg.consume(1); } } @@ -156,7 +157,7 @@ TEST(SchedulerRoot, Budget) ResourceTest t; ResourceHolder r1(t); - r1.add("/", "1"); + r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", ""); r1.registerResource(); @@ -181,7 +182,7 @@ TEST(SchedulerRoot, Cancel) ResourceTest t; ResourceHolder r1(t); - auto * fc1 = r1.add("/", "1"); + auto * fc1 = r1.add("/", "1"); r1.add("/prio"); auto a = r1.addQueue("/prio/A", "1"); auto b = r1.addQueue("/prio/B", "2"); @@ -194,7 +195,7 @@ TEST(SchedulerRoot, Cancel) MyRequest request(1,[&] { sync.arrive_and_wait(); // (A) - EXPECT_TRUE(fc1->requests.contains(&request)); + EXPECT_TRUE(fc1->getInflights().first == 1); sync.arrive_and_wait(); // (B) request.finish(); destruct_sync.arrive_and_wait(); // (C) @@ -219,5 +220,5 @@ TEST(SchedulerRoot, Cancel) consumer1.join(); consumer2.join(); - EXPECT_TRUE(fc1->requests.empty()); + EXPECT_TRUE(fc1->getInflights().first == 0); } From bec2db7b79c8b2d7343be7d16d7eac0d732cee20 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 14 Oct 2024 21:17:01 +0000 Subject: [PATCH 081/157] implement workload entity storage based on keeper --- .../Nodes/tests/gtest_io_resource_manager.cpp | 8 +- .../Workload/WorkloadEntityDiskStorage.cpp | 12 +- .../Workload/WorkloadEntityDiskStorage.h | 4 +- .../Workload/WorkloadEntityKeeperStorage.cpp | 274 ++++++++ .../Workload/WorkloadEntityKeeperStorage.h | 69 ++ .../Workload/WorkloadEntityStorageBase.cpp | 624 ++++++++++++------ .../Workload/WorkloadEntityStorageBase.h | 40 +- .../Workload/createWorkloadEntityStorage.cpp | 13 +- src/Parsers/ASTCreateResourceQuery.h | 3 + src/Parsers/ParserCreateWorkloadEntity.cpp | 16 + src/Parsers/ParserCreateWorkloadEntity.h | 17 + .../configs/storage_configuration.xml | 1 + tests/integration/test_scheduler/test.py | 1 + 13 files changed, 852 insertions(+), 230 deletions(-) create mode 100644 src/Parsers/ParserCreateWorkloadEntity.cpp create mode 100644 src/Parsers/ParserCreateWorkloadEntity.h diff --git a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp index 93c8439bdae..15cd6436c47 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_io_resource_manager.cpp @@ -112,7 +112,7 @@ public: } private: - bool storeEntityImpl( + WorkloadEntityStorageBase::OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -122,17 +122,17 @@ private: const Settings & settings) override { UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings); - return true; + return OperationResult::Ok; } - bool removeEntityImpl( + WorkloadEntityStorageBase::OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, bool throw_if_not_exists) override { UNUSED(current_context, entity_type, entity_name, throw_if_not_exists); - return true; + return OperationResult::Ok; } }; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 190b2928fe0..0e67074c84b 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -198,7 +198,7 @@ void WorkloadEntityDiskStorage::createDirectory() } -bool WorkloadEntityDiskStorage::storeEntityImpl( +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::storeEntityImpl( const ContextPtr & /*current_context*/, WorkloadEntityType entity_type, const String & entity_name, @@ -216,7 +216,7 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( if (throw_if_exists) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); else if (!replace_if_exists) - return false; + return OperationResult::Failed; } WriteBufferFromOwnString create_statement_buf; @@ -247,11 +247,11 @@ bool WorkloadEntityDiskStorage::storeEntityImpl( } LOG_TRACE(log, "Entity {} stored", backQuote(entity_name)); - return true; + return OperationResult::Ok; } -bool WorkloadEntityDiskStorage::removeEntityImpl( +WorkloadEntityStorageBase::OperationResult WorkloadEntityDiskStorage::removeEntityImpl( const ContextPtr & /*current_context*/, WorkloadEntityType entity_type, const String & entity_name, @@ -267,11 +267,11 @@ bool WorkloadEntityDiskStorage::removeEntityImpl( if (throw_if_not_exists) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); else - return false; + return OperationResult::Failed; } LOG_TRACE(log, "Entity {} removed", backQuote(entity_name)); - return true; + return OperationResult::Ok; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index ceb736372ae..b60a5075a02 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -16,7 +16,7 @@ public: void loadEntities() override; private: - bool storeEntityImpl( + OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -25,7 +25,7 @@ private: bool replace_if_exists, const Settings & settings) override; - bool removeEntityImpl( + OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index e69de29bb2d..37d1cc568ec 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -0,0 +1,274 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace Setting +{ +extern const SettingsUInt64 max_parser_backtracks; +extern const SettingsUInt64 max_parser_depth; +} + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; +} + +WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( + const ContextPtr & global_context_, const String & zookeeper_path_) + : WorkloadEntityStorageBase(global_context_) + , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} + , zookeeper_path{zookeeper_path_} + , watch_queue{std::make_shared>(std::numeric_limits::max())} + , log{getLogger("WorkloadEntityKeeperStorage")} +{ + if (zookeeper_path.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); + + if (zookeeper_path.back() == '/') + zookeeper_path.resize(zookeeper_path.size() - 1); + + /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. + if (zookeeper_path.front() != '/') + zookeeper_path = "/" + zookeeper_path; +} + +WorkloadEntityKeeperStorage::~WorkloadEntityKeeperStorage() +{ + SCOPE_EXIT_SAFE(stopWatchingThread()); +} + +void WorkloadEntityKeeperStorage::startWatchingThread() +{ + if (!watching_flag.exchange(true)) + watching_thread = ThreadFromGlobalPool(&WorkloadEntityKeeperStorage::processWatchQueue, this); +} + +void WorkloadEntityKeeperStorage::stopWatchingThread() +{ + if (watching_flag.exchange(false)) + { + watch_queue->finish(); + if (watching_thread.joinable()) + watching_thread.join(); + } +} + +zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() +{ + auto [zookeeper, session_status] = zookeeper_getter.getZooKeeper(); + + if (session_status == zkutil::ZooKeeperCachingGetter::SessionStatus::New) + { + /// It's possible that we connected to different [Zoo]Keeper instance + /// so we may read a bit stale state. + zookeeper->sync(zookeeper_path); + + createRootNodes(zookeeper); + refreshAllEntities(zookeeper); + } + + return zookeeper; +} + +void WorkloadEntityKeeperStorage::loadEntities() +{ + /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. + /// However the watching thread must be started anyway in case the connection will be established later. + if (!entities_loaded) + { + try + { + refreshAllEntities(getZooKeeper()); + startWatchingThread(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load workload entities"); + } + } + startWatchingThread(); +} + + +void WorkloadEntityKeeperStorage::processWatchQueue() +{ + LOG_DEBUG(log, "Started watching thread"); + setThreadName("WrkldEntWatch"); + + while (watching_flag) + { + try + { + /// Re-initialize ZooKeeper session if expired + getZooKeeper(); + + bool queued = false; + if (!watch_queue->tryPop(queued, /* timeout_ms: */ 10000)) + continue; + + refreshAllEntities(getZooKeeper()); + } + catch (...) + { + tryLogCurrentException(log, "Will try to restart watching thread after error"); + zookeeper_getter.resetCache(); + sleepForSeconds(5); + } + } + + LOG_DEBUG(log, "Stopped watching thread"); +} + + +void WorkloadEntityKeeperStorage::stopWatching() +{ + stopWatchingThread(); +} + +void WorkloadEntityKeeperStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper) +{ + zookeeper->createAncestors(zookeeper_path); + // If node does not exist we consider it to be equal to empty node: no workload entities + zookeeper->createIfNotExists(zookeeper_path, ""); +} + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool /*throw_if_exists*/, + bool /*replace_if_exists*/, + const Settings &) +{ + LOG_DEBUG(log, "Storing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, create_entity_query}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshAllEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} stored", backQuote(entity_name)); + + return OperationResult::Ok; +} + + +WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEntityImpl( + const ContextPtr & /*current_context*/, + WorkloadEntityType entity_type, + const String & entity_name, + bool /*throw_if_not_exists*/) +{ + LOG_DEBUG(log, "Removing workload entity {}", backQuote(entity_name)); + + String new_data = serializeAllEntities(Event{entity_type, entity_name, {}}); + auto zookeeper = getZooKeeper(); + + Coordination::Stat stat; + auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); + if (code != Coordination::Error::ZOK) + { + refreshAllEntities(zookeeper); + return OperationResult::Retry; + } + + current_version = stat.version; + + LOG_DEBUG(log, "Workload entity {} removed", backQuote(entity_name)); + + return OperationResult::Ok; +} + +std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) +{ + const auto data_watcher = [my_watch_queue = watch_queue](const Coordination::WatchResponse & response) + { + if (response.type == Coordination::Event::CHANGED) + { + [[maybe_unused]] bool inserted = my_watch_queue->emplace(true); + /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). + } + }; + + Coordination::Stat stat; + String data; + bool exists = zookeeper->tryGetWatch(zookeeper_path, data, &stat, data_watcher); + if (!exists) + { + createRootNodes(zookeeper); + data = zookeeper->getWatch(zookeeper_path, &stat, data_watcher); + } + return {data, stat.version}; +} + +void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper) +{ + /// It doesn't make sense to keep the old watch events because we will reread everything in this function. + watch_queue->clear(); + + refreshEntities(zookeeper); + entities_loaded = true; +} + +void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) +{ + LOG_DEBUG(log, "Refreshing workload entities"); + auto [data, version] = getDataAndSetWatch(zookeeper); + + ASTs queries; + ParserCreateWorkloadEntity parser; + const char * begin = data.data(); /// begin of current query + const char * pos = begin; /// parser moves pos from begin to the end of current query + const char * end = begin + data.size(); + while (pos < end) + { + queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS)); + while (isWhitespaceASCII(*pos) || *pos == ';') + ++pos; + } + + /// Read & parse all SQL entities from data we just read from ZooKeeper + std::vector> new_entities; + for (const auto & query : queries) + { + if (auto * create_workload_query = query->as()) + new_entities.emplace_back(create_workload_query->getWorkloadName(), query); + else if (auto * create_resource_query = query->as()) + new_entities.emplace_back(create_resource_query->getResourceName(), query); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity query in keeper storage: {}", query->getID()); + } + + setAllEntities(new_entities); + current_version = version; + + LOG_DEBUG(log, "Workload entities refreshing is done"); +} + +} + diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 6f70f09beec..523be850d8d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -1 +1,70 @@ #pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Loads RESOURCE and WORKLOAD sql objects from Keeper. +class WorkloadEntityKeeperStorage : public WorkloadEntityStorageBase +{ +public: + WorkloadEntityKeeperStorage(const ContextPtr & global_context_, const String & zookeeper_path_); + ~WorkloadEntityKeeperStorage() override; + + bool isReplicated() const override { return true; } + String getReplicationID() const override { return zookeeper_path; } + + void loadEntities() override; + void stopWatching() override; + +private: + OperationResult storeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + ASTPtr create_entity_query, + bool throw_if_exists, + bool replace_if_exists, + const Settings & settings) override; + + OperationResult removeEntityImpl( + const ContextPtr & current_context, + WorkloadEntityType entity_type, + const String & entity_name, + bool throw_if_not_exists) override; + + void processWatchQueue(); + + zkutil::ZooKeeperPtr getZooKeeper(); + + void startWatchingThread(); + void stopWatchingThread(); + + void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); + + std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); + + void refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper); // TODO(serxa): get rid of it + void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); + + zkutil::ZooKeeperCachingGetter zookeeper_getter; + String zookeeper_path; + Int32 current_version = 0; + + ThreadFromGlobalPool watching_thread; + std::atomic entities_loaded = false; + std::atomic watching_flag = false; + + std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue + + LoggerPtr log; +}; + +} diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 060bbbd6f87..0cd872f4890 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include @@ -13,7 +15,6 @@ #include #include - namespace DB { @@ -26,6 +27,7 @@ namespace ErrorCodes namespace { +/// Removes details from a CREATE query to be used as workload entity definition ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) { auto ptr = create_query.clone(); @@ -42,6 +44,7 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query) return ptr; } +/// Returns a type of a workload entity `ptr` WorkloadEntityType getEntityType(const ASTPtr & ptr) { if (auto * res = typeid_cast(ptr.get())) @@ -52,12 +55,38 @@ WorkloadEntityType getEntityType(const ASTPtr & ptr) return WorkloadEntityType::MAX; } +bool entityEquals(const ASTPtr & lhs, const ASTPtr & rhs) +{ + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + { + return std::forward_as_tuple(a->getWorkloadName(), a->getWorkloadParent(), a->changes) + == std::forward_as_tuple(b->getWorkloadName(), b->getWorkloadParent(), b->changes); + } + } + if (auto * a = typeid_cast(lhs.get())) + { + if (auto * b = typeid_cast(rhs.get())) + return std::forward_as_tuple(a->getResourceName(), a->operations) + == std::forward_as_tuple(b->getResourceName(), b->operations); + } + return false; +} + +/// Workload entities could reference each other. +/// This enum defines all possible reference types enum class ReferenceType { - Parent, ForResource + Parent, // Source workload references target workload as a parent + ForResource // Source workload references target resource in its `SETTINGS x = y FOR resource` clause }; -void forEachReference(const ASTPtr & source_entity, std::function func) +/// Runs a `func` callback for every reference from `source` to `target`. +/// This function is the source of truth defining what `target` references are stored in a workload `source_entity` +void forEachReference( + const ASTPtr & source_entity, + std::function func) { if (auto * res = typeid_cast(source_entity.get())) { @@ -82,6 +111,7 @@ void forEachReference(const ASTPtr & source_entity, std::function & workloads, std::unordered_set & visited, std::vector> & sorted_workloads) { if (visited.contains(name)) @@ -101,6 +131,7 @@ void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, c sorted_workloads.emplace_back(name, ast); } +/// Returns pairs {worload_name, create_workload_ast} in order that respect child-parent relation (parent first, then children) std::vector> topologicallySortedWorkloads(const std::unordered_map & workloads) { std::vector> sorted_workloads; @@ -110,6 +141,143 @@ std::vector> topologicallySortedWorkloads(const std::u return sorted_workloads; } +/// Helper for recursive DFS +void topologicallySortedDependenciesImpl( + const String & name, + const std::unordered_map> & dependencies, + std::unordered_set & visited, + std::vector & result) +{ + if (visited.contains(name)) + return; + visited.insert(name); + + if (auto it = dependencies.find(name); it != dependencies.end()) + { + for (const String & dep : it->second) + topologicallySortedDependenciesImpl(dep, dependencies, visited, result); + } + + result.emplace_back(name); +} + +/// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) +std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) { + std::unordered_set visited; // Set to track visited nodes + std::vector result; // Result to store nodes in topologically sorted order + + // Perform DFS for each node in the graph + for (const auto & [name, _] : dependencies) + topologicallySortedDependenciesImpl(name, dependencies, visited, result); + + // Reverse the result to get the correct topological order + std::reverse(result.begin(), result.end()); + + return result; +} + +/// Represents a change of a workload entity (WORKLOAD or RESOURCE) +struct EntityChange +{ + String name; /// Name of entity + ASTPtr before; /// Entity before change (CREATE if not set) + ASTPtr after; /// Entity after change (DROP if not set) + + std::vector toEvents() const + { + if (!after) + return {{getEntityType(before), name, {}}}; + else if (!before) + return {{getEntityType(after), name, after}}; + else + { + auto type_before = getEntityType(before); + auto type_after = getEntityType(after); + // If type changed, we have to remove an old entity and add a new one + if (type_before != type_after) + return {{type_before, name, {}}, {type_after, name, after}}; + else + return {{type_after, name, after}}; + } + } +}; + +/// Returns `changes` ordered for execution. +/// Every intemediate state during execution will be consistent (i.e. all references will be valid) +/// NOTE: It does not validate changes, any problem will be detected during execution. +/// NOTE: There will be no error if valid order does not exist. +std::vector topologicallySortedChanges(const std::vector & changes) +{ + // Construct map from entity name into entity change + std::unordered_map change_by_name; + for (const auto & change : changes) + change_by_name[change.name] = &change; + + // Construct references maps (before changes and after changes) + std::unordered_map> old_sources; // Key is target. Value is set of names of source entities. + std::unordered_map> new_targets; // Key is source. Value is set of names of target entities. + for (const auto & change : changes) + { + if (change.before) + { + forEachReference(change.before, + [&] (const String & target, const String & source, ReferenceType) + { + old_sources[target].insert(source); + }); + } + if (change.after) + { + forEachReference(change.after, + [&] (const String & target, const String & source, ReferenceType) + { + new_targets[source].insert(target); + }); + } + } + + // There are consistency rules that regulate order in which changes must be applied (see below). + // Construct DAG of dependencies between changes. + std::unordered_map> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. + for (const auto & change : changes) + { + for (const auto & event : change.toEvents()) + { + if (!event.entity) // DROP + { + // Rule 1: Entity can only be removed after all existing references to it are removed as well. + for (const String & source : old_sources[event.name]) + { + if (change_by_name.contains(source)) + dependencies[event.name].insert(source); + } + } + else // CREATE || CREATE OR REPLACE + { + // Rule 2: Entity can only be created after all entities it references are created as well. + for (const String & target : new_targets[event.name]) + { + if (auto it = change_by_name.find(target); it != change_by_name.end()) + { + const EntityChange & target_change = *it->second; + // If target is creating, it should be created first. + // (But if target is updating, there is no dependency). + if (!target_change.before) + dependencies[event.name].insert(target); + } + } + } + } + } + + // Topological sort of changes to respect consistency rules + std::vector result; + for (const String & name : topologicallySortedDependencies(dependencies)) + result.push_back(*change_by_name[name]); + + return result; +} + } WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_) @@ -130,7 +298,7 @@ ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const return it->second; } -ASTPtr WorkloadEntityStorageBase::tryGet(const std::string & entity_name) const +ASTPtr WorkloadEntityStorageBase::tryGet(const String & entity_name) const { std::lock_guard lock(mutex); @@ -146,9 +314,9 @@ bool WorkloadEntityStorageBase::has(const String & entity_name) const return tryGet(entity_name) != nullptr; } -std::vector WorkloadEntityStorageBase::getAllEntityNames() const +std::vector WorkloadEntityStorageBase::getAllEntityNames() const { - std::vector entity_names; + std::vector entity_names; std::lock_guard lock(mutex); entity_names.reserve(entities.size()); @@ -159,9 +327,9 @@ std::vector WorkloadEntityStorageBase::getAllEntityNames() const return entity_names; } -std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const +std::vector WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const { - std::vector entity_names; + std::vector entity_names; std::lock_guard lock(mutex); for (const auto & [name, entity] : entities) @@ -195,110 +363,101 @@ bool WorkloadEntityStorageBase::storeEntity( auto * workload = typeid_cast(create_entity_query.get()); auto * resource = typeid_cast(create_entity_query.get()); - std::unique_lock lock{mutex}; - - ASTPtr old_entity; // entity to be REPLACED - if (auto it = entities.find(entity_name); it != entities.end()) + while (true) { - if (throw_if_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); - else if (!replace_if_exists) - return false; - else - old_entity = it->second; - } + std::unique_lock lock{mutex}; - // Validate CREATE OR REPLACE - if (old_entity) - { - auto * old_workload = typeid_cast(old_entity.get()); - auto * old_resource = typeid_cast(old_entity.get()); - if (workload && !old_workload) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); - if (resource && !old_resource) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); - if (workload && !old_workload->hasParent() && workload->hasParent()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); - } - - std::optional new_root_name; - - // Validate workload - if (workload) - { - if (!workload->hasParent()) + ASTPtr old_entity; // entity to be REPLACED + if (auto it = entities.find(entity_name); it != entities.end()) { - if (!root_name.empty() && root_name != workload->getWorkloadName()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); - new_root_name = workload->getWorkloadName(); + if (throw_if_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name); + else if (!replace_if_exists) + return false; + else + old_entity = it->second; } - SchedulingSettings validator; - validator.updateFromChanges(workload->changes); - } - - forEachReference(create_entity_query, - [this, workload] (const String & target, const String & source, ReferenceType type) + // Validate CREATE OR REPLACE + if (old_entity) { - if (auto it = entities.find(target); it == entities.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); + auto * old_workload = typeid_cast(old_entity.get()); + auto * old_resource = typeid_cast(old_entity.get()); + if (workload && !old_workload) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a workload", entity_name); + if (resource && !old_resource) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' already exists, but it is not a resource", entity_name); + if (workload && !old_workload->hasParent() && workload->hasParent()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "It is not allowed to remove root workload"); + } - switch (type) + // Validate workload + if (workload) + { + if (!workload->hasParent()) { - case ReferenceType::Parent: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); - break; - } - case ReferenceType::ForResource: - { - if (typeid_cast(entities[target].get()) == nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - - // Validate that we could parse the settings for specific resource - SchedulingSettings validator; - validator.updateFromChanges(workload->changes, target); - break; - } + if (!root_name.empty() && root_name != workload->getWorkloadName()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second root is not allowed. You should probably add 'PARENT {}' clause.", root_name); } - // Detect reference cycles. - // The only way to create a cycle is to add an edge that will be a part of a new cycle. - // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. - if (isIndirectlyReferenced(source, target)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); - }); + SchedulingSettings validator; + validator.updateFromChanges(workload->changes); + } - bool stored = storeEntityImpl( - current_context, - entity_type, - entity_name, - create_entity_query, - throw_if_exists, - replace_if_exists, - settings); + forEachReference(create_entity_query, + [this, workload] (const String & target, const String & source, ReferenceType type) + { + if (auto it = entities.find(target); it == entities.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' references another workload entity '{}' that doesn't exist", source, target); - if (stored) - { - if (new_root_name) - root_name = *new_root_name; + switch (type) + { + case ReferenceType::Parent: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target); + break; + } + case ReferenceType::ForResource: + { + if (typeid_cast(entities[target].get()) == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload settings should reference resource in FOR clause, not '{}'.", target); - // Remove references of a replaced entity (only for CREATE OR REPLACE) - removeReferences(old_entity); + // Validate that we could parse the settings for specific resource + SchedulingSettings validator; + validator.updateFromChanges(workload->changes, target); + break; + } + } - // Insert references of created entity - insertReferences(create_entity_query); + // Detect reference cycles. + // The only way to create a cycle is to add an edge that will be a part of a new cycle. + // We are going to add an edge: `source` -> `target`, so we ensure there is no path back `target` -> `source`. + if (isIndirectlyReferenced(source, target)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity cycles are not allowed"); + }); - // Store in memory - entities[entity_name] = create_entity_query; + auto result = storeEntityImpl( + current_context, + entity_type, + entity_name, + create_entity_query, + throw_if_exists, + replace_if_exists, + settings); - // Process notifications - onEntityAdded(entity_type, entity_name, create_entity_query); - unlockAndNotify(lock); + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, create_entity_query}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; } - - return stored; } bool WorkloadEntityStorageBase::removeEntity( @@ -307,47 +466,44 @@ bool WorkloadEntityStorageBase::removeEntity( const String & entity_name, bool throw_if_not_exists) { - std::unique_lock lock(mutex); - auto it = entities.find(entity_name); - if (it == entities.end()) + while (true) { - if (throw_if_not_exists) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); - else - return false; + std::unique_lock lock(mutex); + auto it = entities.find(entity_name); + if (it == entities.end()) + { + if (throw_if_not_exists) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name); + else + return false; + } + + if (auto reference_it = references.find(entity_name); reference_it != references.end()) + { + String names; + for (const String & name : reference_it->second) + names += " " + name; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); + } + + auto result = removeEntityImpl( + current_context, + entity_type, + entity_name, + throw_if_not_exists); + + if (result == OperationResult::Retry) + continue; // Entities were updated, we need to rerun all the validations + + if (result == OperationResult::Ok) + { + Event event{entity_type, entity_name, {}}; + applyEvent(lock, event); + unlockAndNotify(lock, {std::move(event)}); + } + + return result == OperationResult::Ok; } - - if (auto reference_it = references.find(entity_name); reference_it != references.end()) - { - String names; - for (const String & name : reference_it->second) - names += " " + name; - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload entity '{}' cannot be dropped. It is referenced by:{}", entity_name, names); - } - - bool removed = removeEntityImpl( - current_context, - entity_type, - entity_name, - throw_if_not_exists); - - if (removed) - { - if (entity_name == root_name) - root_name.clear(); - - // Clean up references - removeReferences(it->second); - - // Remove from memory - entities.erase(it); - - // Process notifications - onEntityRemoved(entity_type, entity_name); - unlockAndNotify(lock); - } - - return removed; } scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler) @@ -357,9 +513,7 @@ scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChange std::vector current_state; { std::unique_lock lock{mutex}; - chassert(queue.empty()); - makeEventsForAllEntities(lock); - current_state = std::move(queue); + current_state = orderEntities(entities); std::lock_guard lock2{handlers->mutex}; handlers->list.push_back(handler); @@ -377,41 +531,30 @@ scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChange return result; } -void WorkloadEntityStorageBase::onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity) +void WorkloadEntityStorageBase::unlockAndNotify( + std::unique_lock & lock, + std::vector tx) { - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = new_entity}); -} + if (tx.empty()) + return; -void WorkloadEntityStorageBase::onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name) -{ - queue.push_back(Event{.type = entity_type, .name = entity_name, .entity = {}}); -} - -void WorkloadEntityStorageBase::unlockAndNotify(std::unique_lock & mutex_lock) -{ - /// Only one thread can send notification at any time, that is why we need `mutex_lock` - if (!queue.empty()) + std::vector current_handlers; { - auto events = std::move(queue); + std::lock_guard handlers_lock{handlers->mutex}; + boost::range::copy(handlers->list, std::back_inserter(current_handlers)); + } - std::vector current_handlers; + lock.unlock(); + + for (const auto & handler : current_handlers) + { + try { - std::lock_guard handlers_lock{handlers->mutex}; - boost::range::copy(handlers->list, std::back_inserter(current_handlers)); + handler(tx); } - - mutex_lock.unlock(); - - for (const auto & handler : current_handlers) + catch (...) { - try - { - handler(events); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + tryLogCurrentException(__PRETTY_FUNCTION__); } } } @@ -421,52 +564,84 @@ std::unique_lock WorkloadEntityStorageBase::getLock() cons return std::unique_lock{mutex}; } -void WorkloadEntityStorageBase::setAllEntities(const std::vector> & new_entities) +void WorkloadEntityStorageBase::setAllEntities(const std::vector> & raw_new_entities) { - std::unordered_map normalized_entities; - for (const auto & [entity_name, create_query] : new_entities) - normalized_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); - - // TODO(serxa): do validation and throw LOGICAL_ERROR if failed + std::unordered_map new_entities; + for (const auto & [entity_name, create_query] : raw_new_entities) + new_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query); std::unique_lock lock(mutex); - chassert(entities.empty()); // TODO(serxa): keeper storage could do full refresh, so we should support it here - entities = std::move(normalized_entities); - for (const auto & [entity_name, entity] : entities) - insertReferences(entity); - // Quick check to avoid extra work + // Fill vector of `changes` based on difference between current `entities` and `new_entities` + std::vector changes; + for (const auto & [entity_name, entity] : entities) { - std::lock_guard lock2(handlers->mutex); - if (handlers->list.empty()) - return; + if (auto it = new_entities.find(entity_name); it != new_entities.end()) + { + if (!entityEquals(entity, it->second)) + changes.emplace_back(entity_name, entity, it->second); // Remove entities that are not present in `new_entities` + } + else + changes.emplace_back(entity_name, entity, ASTPtr{}); // Update entities that are present in both `new_entities` and `entities` + } + for (const auto & [entity_name, entity] : new_entities) + { + if (!entities.contains(entity_name)) + changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` } - makeEventsForAllEntities(lock); - unlockAndNotify(lock); + // Sort `changes` to respect consistency of references and apply them one by one. + std::vector tx; + for (const auto & change : topologicallySortedChanges(changes)) + { + for (const auto & event : change.toEvents()) + { + // TODO(serxa): do validation and throw LOGICAL_ERROR if failed + applyEvent(lock, event); + tx.push_back(event); + } + } + + // Notify subscribers + unlockAndNotify(lock, tx); } -void WorkloadEntityStorageBase::makeEventsForAllEntities(std::unique_lock &) +void WorkloadEntityStorageBase::applyEvent( + std::unique_lock &, + const Event & event) { - std::unordered_map workloads; - std::unordered_map resources; - for (auto & [entity_name, ast] : entities) + if (event.entity) // CREATE || CREATE OR REPLACE { - if (typeid_cast(ast.get())) - workloads.emplace(entity_name, ast); - else if (typeid_cast(ast.get())) - resources.emplace(entity_name, ast); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + auto * workload = typeid_cast(event.entity.get()); + + // Validate workload + if (workload && !workload->hasParent()) + root_name = workload->getWorkloadName(); + + // Remove references of a replaced entity (only for CREATE OR REPLACE) + if (auto it = entities.find(event.name); it != entities.end()) + removeReferences(it->second); + + // Insert references of created entity + insertReferences(event.entity); + + // Store in memory + entities[event.name] = event.entity; } + else // DROP + { + auto it = entities.find(event.name); + chassert(it != entities.end()); - // Resources should be created first because workloads could reference them - for (auto & [entity_name, ast] : resources) - onEntityAdded(WorkloadEntityType::Resource, entity_name, ast); + if (event.name == root_name) + root_name.clear(); - // Workloads should be created in an order such that children are created only after its parent is created - for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) - onEntityAdded(WorkloadEntityType::Workload, entity_name, ast); + // Clean up references + removeReferences(it->second); + + // Remove from memory + entities.erase(it); + } } std::vector> WorkloadEntityStorageBase::getAllEntities() const @@ -528,4 +703,59 @@ void WorkloadEntityStorageBase::removeReferences(const ASTPtr & entity) }); } +std::vector WorkloadEntityStorageBase::orderEntities( + const std::unordered_map & all_entities, + std::optional change) +{ + std::vector result; + + std::unordered_map workloads; + for (auto & [entity_name, ast] : all_entities) + { + if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this workload if it is removed or updated + workloads.emplace(entity_name, ast); + } + else if (typeid_cast(ast.get())) + { + if (change && change->name == entity_name) + continue; // Skip this resource if it is removed or updated + // Resources should go first because workloads could reference them + result.emplace_back(WorkloadEntityType::Resource, entity_name, ast); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid workload entity type '{}'", ast->getID()); + } + + // Introduce new entity described by `change` + if (change && change->entity) + { + if (change->type == WorkloadEntityType::Workload) + workloads.emplace(change->name, change->entity); + else if (change->type == WorkloadEntityType::Resource) + result.emplace_back(WorkloadEntityType::Resource, change->name, change->entity); + } + + // Workloads should go in an order such that children are enlisted only after its parent + for (auto & [entity_name, ast] : topologicallySortedWorkloads(workloads)) + result.emplace_back(WorkloadEntityType::Workload, entity_name, ast); + + return result; +} + +String WorkloadEntityStorageBase::serializeAllEntities(std::optional change) +{ + std::unique_lock lock; + auto ordered_entities = orderEntities(entities, change); + WriteBufferFromOwnString buf; + for (const auto & event : ordered_entities) + { + formatAST(*event.entity, buf, false, true); + buf.write(";\n", 2); + } + return buf.str(); +} + } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index e1f43181a0c..905c80610c2 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -49,7 +49,14 @@ public: const OnChangedHandler & handler) override; protected: - virtual bool storeEntityImpl( + enum class OperationResult + { + Ok, + Failed, + Retry + }; + + virtual OperationResult storeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -58,7 +65,7 @@ protected: bool replace_if_exists, const Settings & settings) = 0; - virtual bool removeEntityImpl( + virtual OperationResult removeEntityImpl( const ContextPtr & current_context, WorkloadEntityType entity_type, const String & entity_name, @@ -66,18 +73,21 @@ protected: std::unique_lock getLock() const; + /// Replace current `entities` with `new_entities` and notifies subscribers. + /// Note that subscribers will be notified with a sequence of events. + /// It is guaranteed that all itermediate states (between every pair of consecutive events) + /// will be consistent (all references between entities will be valid) void setAllEntities(const std::vector> & new_entities); - void makeEventsForAllEntities(std::unique_lock & lock); - /// Called by derived class after a new workload entity has been added. - void onEntityAdded(WorkloadEntityType entity_type, const String & entity_name, const ASTPtr & new_entity); + /// Serialize `entities` stored in memory plus one optional `change` into multiline string + String serializeAllEntities(std::optional change = {}); - /// Called by derived class after an workload entity has been removed. - void onEntityRemoved(WorkloadEntityType entity_type, const String & entity_name); +private: + /// Change state in memory + void applyEvent(std::unique_lock & lock, const Event & event); - /// Sends notifications to subscribers about changes in workload entities - /// (added with previous calls onEntityAdded(), onEntityRemoved()). - void unlockAndNotify(std::unique_lock & lock); + /// Notify subscribers about changes describe by vector of events `tx` + void unlockAndNotify(std::unique_lock & lock, std::vector tx); /// Return true iff `references` has a path from `source` to `target` bool isIndirectlyReferenced(const String & target, const String & source); @@ -88,6 +98,11 @@ protected: /// Removes references that are described by `entity` from `references` void removeReferences(const ASTPtr & entity); + /// Returns an ordered vector of `entities` + std::vector orderEntities( + const std::unordered_map & all_entitites, + std::optional change = {}); + struct Handlers { std::mutex mutex; @@ -96,15 +111,14 @@ protected: /// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed. std::shared_ptr handlers; - std::vector queue; - mutable std::recursive_mutex mutex; std::unordered_map entities; /// Maps entity name into CREATE entity query // Validation - std::unordered_map> references; /// Keep track of references between entities. Key is target. Values is set of sources + std::unordered_map> references; /// Keep track of references between entities. Key is target. Value is set of sources String root_name; /// current root workload name +protected: ContextPtr global_context; }; diff --git a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp index 8475fe21455..5dc1265e31d 100644 --- a/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp +++ b/src/Common/Scheduler/Workload/createWorkloadEntityStorage.cpp @@ -34,15 +34,12 @@ std::unique_ptr createWorkloadEntityStorage(const Contex zookeeper_path_key, disk_path_key); } - abort(); // TODO(serxa): create WorkloadEntityKeeperStorage object - //return std::make_unique(global_context, config.getString(zookeeper_path_key)); - } - else - { - String default_path = fs::path{global_context->getPath()} / "workload" / ""; - String path = config.getString(disk_path_key, default_path); - return std::make_unique(global_context, path); + return std::make_unique(global_context, config.getString(zookeeper_path_key)); } + + String default_path = fs::path{global_context->getPath()} / "workload" / ""; + String path = config.getString(disk_path_key, default_path); + return std::make_unique(global_context, path); } } diff --git a/src/Parsers/ASTCreateResourceQuery.h b/src/Parsers/ASTCreateResourceQuery.h index b05176837bc..f1c762e5bcd 100644 --- a/src/Parsers/ASTCreateResourceQuery.h +++ b/src/Parsers/ASTCreateResourceQuery.h @@ -19,6 +19,9 @@ public: { AccessMode mode; String disk; + + friend bool operator ==(const Operation & lhs, const Operation & rhs) { return lhs.mode == rhs.mode && lhs.disk == rhs.disk; } + friend bool operator !=(const Operation & lhs, const Operation & rhs) { return !(lhs == rhs); } }; using Operations = std::vector; diff --git a/src/Parsers/ParserCreateWorkloadEntity.cpp b/src/Parsers/ParserCreateWorkloadEntity.cpp new file mode 100644 index 00000000000..013210a6d87 --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +bool ParserCreateWorkloadEntity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserCreateWorkloadQuery create_workload_p; + ParserCreateResourceQuery create_resource_p; + + return create_workload_p.parse(pos, node, expected) || create_resource_p.parse(pos, node, expected); +} + +} diff --git a/src/Parsers/ParserCreateWorkloadEntity.h b/src/Parsers/ParserCreateWorkloadEntity.h new file mode 100644 index 00000000000..1e7b78b3ccc --- /dev/null +++ b/src/Parsers/ParserCreateWorkloadEntity.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +/// Special parser for the CREATE WORKLOAD and CREATE RESOURCE queries. +class ParserCreateWorkloadEntity : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE workload entity query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml index 16cdf4a5b15..9498044c836 100644 --- a/tests/integration/test_scheduler/configs/storage_configuration.xml +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -1,4 +1,5 @@ + /clickhouse/workload/definitions.sql diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index b78376bffe2..40c5f7e11ed 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -24,6 +24,7 @@ node = cluster.add_instance( "configs/workloads.xml.default", ], with_minio=True, + with_zookeeper=True, ) From 05eb1ef42b92f4a6150d6bc2f4bb5764d39f9fe8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 15 Oct 2024 01:01:01 +0000 Subject: [PATCH 082/157] style --- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 0cd872f4890..4d09d49c927 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -162,7 +162,8 @@ void topologicallySortedDependenciesImpl( } /// Returns nodes in topological order that respect `dependencies` (key is node name, value is set of dependencies) -std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) { +std::vector topologicallySortedDependencies(const std::unordered_map> & dependencies) +{ std::unordered_set visited; // Set to track visited nodes std::vector result; // Result to store nodes in topologically sorted order From 8a854bd5ac225f2cbb1bef441b1d8c930fd1356f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:10:05 +0000 Subject: [PATCH 083/157] add test for keeper entity storage with partitioning --- tests/integration/test_scheduler/test.py | 121 +++++++++++++++++++++++ 1 file changed, 121 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 40c5f7e11ed..05f38d09245 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -10,6 +10,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) @@ -27,6 +28,20 @@ node = cluster.add_instance( with_zookeeper=True, ) +node2 = cluster.add_instance( + "node2", + stay_alive=True, + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], + with_minio=True, + with_zookeeper=True, +) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): @@ -809,3 +824,109 @@ def test_resource_read_and_write(): ) == "1\n" ) + + +def test_workload_entity_keeper_storage(): + node.query("create resource io_write (write disk s3_no_resource);") + node.query("create resource io_read (read disk s3_no_resource);") + queries = [ + "create workload all;", + "create workload X in all settings priority = 0;", + "create workload Y in all settings priority = 1;", + "create workload A1 in X settings priority = -1;", + "create workload B1 in X settings priority = 1;", + "create workload C1 in Y settings priority = -1;", + "create workload D1 in Y settings priority = 1;", + "create workload A2 in X settings priority = -1;", + "create workload B2 in X settings priority = 1;", + "create workload C2 in Y settings priority = -1;", + "create workload D2 in Y settings priority = 1;", + "drop workload A1;", + "drop workload A2;", + "drop workload B1;", + "drop workload B2;", + "drop workload C1;", + "drop workload C2;", + "drop workload D1;", + "drop workload D2;", + "create workload Z in all;", + "create workload A1 in Z settings priority = -1;", + "create workload A2 in Z settings priority = -1;", + "create workload A3 in Z settings priority = -1;", + "create workload B1 in Z settings priority = 1;", + "create workload B2 in Z settings priority = 1;", + "create workload B3 in Z settings priority = 1;", + "create workload C1 in X settings priority = -1;", + "create workload C2 in X settings priority = -1;", + "create workload C3 in X settings priority = -1;", + "create workload D1 in X settings priority = 1;", + "create workload D2 in X settings priority = 1;", + "create workload D3 in X settings priority = 1;", + "drop workload A1;", + "drop workload B1;", + "drop workload C1;", + "drop workload D1;", + "drop workload A2;", + "drop workload B2;", + "drop workload C2;", + "drop workload D2;", + "drop workload A3;", + "drop workload B3;", + "drop workload C3;", + "drop workload D3;", + "drop workload X;", + "drop workload Y;", + "drop workload Z;", + "drop workload all;", + ] + + def check_consistency(): + checks = [ + "select name, create_query from system.workloads order by all", + "select name, create_query from system.resources order by all", + "select resource, path, type, weight, priority, max_requests, max_cost, max_speed, max_burst from system.scheduler where resource not in ['network_read', 'network_write'] order by all", + ] + attempts = 10 + value1 = "" + value2 = "" + error_query = "" + for attempt in range(attempts): + for query in checks: + value1 = node.query(query) + value2 = node2.query(query) + if value1 != value2: + error_query = query + break # error + else: + break # success + time.sleep(0.5) + else: + raise Exception( + f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" + ) + + + for iteration in range(3): + split_idx_1 = random.randint(1, len(queries) - 3) + split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(0, split_idx_1): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_1, split_idx_2): + node.query(queries[query_idx]) + + check_consistency() + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node2) + for query_idx in range(split_idx_2, len(queries)): + node.query(queries[query_idx]) + + check_consistency() From 23f90fe778ee30b5928d3076becbdc64957b6424 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:12:17 +0000 Subject: [PATCH 084/157] fix bug in topological sorting of entity changes, more logs --- .../Workload/WorkloadEntityDiskStorage.cpp | 2 +- .../Workload/WorkloadEntityDiskStorage.h | 1 - .../Workload/WorkloadEntityKeeperStorage.cpp | 7 ++++-- .../Workload/WorkloadEntityKeeperStorage.h | 2 -- .../Workload/WorkloadEntityStorageBase.cpp | 25 +++++++++++++++---- .../Workload/WorkloadEntityStorageBase.h | 1 + 6 files changed, 27 insertions(+), 11 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp index 0e67074c84b..209d6f06100 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.cpp @@ -60,8 +60,8 @@ namespace WorkloadEntityDiskStorage::WorkloadEntityDiskStorage(const ContextPtr & global_context_, const String & dir_path_) : WorkloadEntityStorageBase(global_context_) , dir_path{makeDirectoryPathCanonical(dir_path_)} - , log{getLogger("WorkloadEntityDiskStorage")} { + log = getLogger("WorkloadEntityDiskStorage"); } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h index b60a5075a02..cb3fb600182 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityDiskStorage.h @@ -38,7 +38,6 @@ private: String getFilePath(WorkloadEntityType entity_type, const String & entity_name) const; String dir_path; - LoggerPtr log; std::atomic entities_loaded = false; }; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index 37d1cc568ec..4aa087e029d 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -35,8 +35,8 @@ WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} , watch_queue{std::make_shared>(std::numeric_limits::max())} - , log{getLogger("WorkloadEntityKeeperStorage")} { + log = getLogger("WorkloadEntityKeeperStorage"); if (zookeeper_path.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty"); @@ -237,9 +237,11 @@ void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) { - LOG_DEBUG(log, "Refreshing workload entities"); auto [data, version] = getDataAndSetWatch(zookeeper); + if (version == current_version) + return; + LOG_DEBUG(log, "Refreshing workload entities from keeper"); ASTs queries; ParserCreateWorkloadEntity parser; const char * begin = data.data(); /// begin of current query @@ -256,6 +258,7 @@ void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & z std::vector> new_entities; for (const auto & query : queries) { + LOG_TRACE(log, "Read keeper entity definition: {}", serializeAST(*query)); if (auto * create_workload_query = query->as()) new_entities.emplace_back(create_workload_query->getWorkloadName(), query); else if (auto * create_resource_query = query->as()) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 523be850d8d..deda5ba909b 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -63,8 +63,6 @@ private: std::atomic watching_flag = false; std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue - - LoggerPtr log; }; } diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index 4d09d49c927..dd4f5365191 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -171,9 +172,6 @@ std::vector topologicallySortedDependencies(const std::unordered_map topologicallySortedChanges(const std::vector> dependencies; // Key is entity name. Value is set of names of entity that should be changed first. for (const auto & change : changes) { + dependencies.emplace(change.name, std::unordered_set{}); // Make sure we create nodes that have no dependencies for (const auto & event : change.toEvents()) { if (!event.entity) // DROP @@ -284,6 +283,7 @@ std::vector topologicallySortedChanges(const std::vector()) , global_context(std::move(global_context_)) + , log{getLogger("WorkloadEntityStorage")} // could be overriden in derived class {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const @@ -580,15 +580,26 @@ void WorkloadEntityStorageBase::setAllEntities(const std::vectorsecond)) - changes.emplace_back(entity_name, entity, it->second); // Remove entities that are not present in `new_entities` + { + changes.emplace_back(entity_name, entity, it->second); // Update entities that are present in both `new_entities` and `entities` + LOG_TRACE(log, "Entity {} was updated", entity_name); + } + else + LOG_TRACE(log, "Entity {} is the same", entity_name); } else - changes.emplace_back(entity_name, entity, ASTPtr{}); // Update entities that are present in both `new_entities` and `entities` + { + changes.emplace_back(entity_name, entity, ASTPtr{}); // Remove entities that are not present in `new_entities` + LOG_TRACE(log, "Entity {} was dropped", entity_name); + } } for (const auto & [entity_name, entity] : new_entities) { if (!entities.contains(entity_name)) + { changes.emplace_back(entity_name, ASTPtr{}, entity); // Create entities that are only present in `new_entities` + LOG_TRACE(log, "Entity {} was created", entity_name); + } } // Sort `changes` to respect consistency of references and apply them one by one. @@ -613,6 +624,8 @@ void WorkloadEntityStorageBase::applyEvent( { if (event.entity) // CREATE || CREATE OR REPLACE { + LOG_DEBUG(log, "Create or replace entity: {}", serializeAST(*event.entity)); + auto * workload = typeid_cast(event.entity.get()); // Validate workload @@ -634,6 +647,8 @@ void WorkloadEntityStorageBase::applyEvent( auto it = entities.find(event.name); chassert(it != entities.end()); + LOG_DEBUG(log, "Drop entity: {}", event.name); + if (event.name == root_name) root_name.clear(); diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h index 905c80610c2..f1ef4124e98 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.h @@ -120,6 +120,7 @@ private: protected: ContextPtr global_context; + LoggerPtr log; }; } From 79a6225a9c2fa4da62cdf752b8553e240c6f2312 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:20:17 +0000 Subject: [PATCH 085/157] abort all requests in queue before destruction --- src/Common/Scheduler/Nodes/FifoQueue.h | 2 +- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/FifoQueue.h b/src/Common/Scheduler/Nodes/FifoQueue.h index ea8985e314f..9502fae1a45 100644 --- a/src/Common/Scheduler/Nodes/FifoQueue.h +++ b/src/Common/Scheduler/Nodes/FifoQueue.h @@ -36,7 +36,7 @@ public: ~FifoQueue() override { - chassert(requests.empty()); + purgeQueue(); } const String & getTypeName() const override diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 0f015dd22b6..6e681632f68 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -54,7 +54,6 @@ IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_) IOResourceManager::Resource::~Resource() { - // TODO(serxa): destroy all workloads, purge all queue, abort all resource requests scheduler.stop(); } From 318215d766d5c787bae0a8f4d5047a3458fc835c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:29:25 +0000 Subject: [PATCH 086/157] log unexpected errors in IOResourceManager --- src/Common/Scheduler/Nodes/IOResourceManager.cpp | 16 ++++++---------- src/Common/Scheduler/Nodes/IOResourceManager.h | 3 +++ 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 6e681632f68..812a49ace60 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -245,6 +246,7 @@ String IOResourceManager::Workload::getParent() const IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_) : storage(storage_) + , log{getLogger("IOResourceManager")} { subscription = storage.getAllEntitiesAndSubscribe( [this] (const std::vector & events) @@ -304,11 +306,8 @@ void IOResourceManager::deleteWorkload(const String & workload_name) // Note that we rely of the fact that workload entity storage will not drop workload that is used as a parent workloads.erase(workload_iter); } - else - { - // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + else // Workload to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete workload that doesn't exist: {}", workload_name); } void IOResourceManager::createOrUpdateResource(const String & resource_name, const ASTPtr & ast) @@ -335,11 +334,8 @@ void IOResourceManager::deleteResource(const String & resource_name) { resources.erase(resource_iter); } - else - { - // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless - // TODO(serxa): add logging - } + else // Resource to be deleted does not exist -- do nothing, throwing exceptions from a subscription is pointless + LOG_ERROR(log, "Delete resource that doesn't exist: {}", resource_name); } IOResourceManager::Classifier::~Classifier() diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.h b/src/Common/Scheduler/Nodes/IOResourceManager.h index 0bbd14c2ca9..d336f012cd1 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.h +++ b/src/Common/Scheduler/Nodes/IOResourceManager.h @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -273,6 +274,8 @@ private: mutable std::mutex mutex; std::unordered_map workloads; // TSA_GUARDED_BY(mutex); std::unordered_map resources; // TSA_GUARDED_BY(mutex); + + LoggerPtr log; }; } From 1c5a8e0008c444a3f70a84e796e6bc99bfd30882 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 15:41:44 +0000 Subject: [PATCH 087/157] ignore constraints beyond limit supported by resource request --- src/Common/Scheduler/Nodes/SemaphoreConstraint.h | 12 +++++++----- src/Common/Scheduler/ResourceRequest.cpp | 9 +++------ src/Common/Scheduler/ResourceRequest.h | 4 ++-- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h index 2454c1ec5bf..e223100a646 100644 --- a/src/Common/Scheduler/Nodes/SemaphoreConstraint.h +++ b/src/Common/Scheduler/Nodes/SemaphoreConstraint.h @@ -88,12 +88,14 @@ public: if (!request) return {nullptr, false}; - request->addConstraint(this); - - // Update state on request arrival std::unique_lock lock(mutex); - requests++; - cost += request->cost; + if (request->addConstraint(this)) + { + // Update state on request arrival + requests++; + cost += request->cost; + } + child_active = child_now_active; if (!active()) busy_periods++; diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 91394108f5d..71731fde9ce 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -23,20 +23,17 @@ void ResourceRequest::finish() } } -void ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) +bool ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint) { for (auto & constraint : constraints) { if (!constraint) { constraint = new_constraint; - return; + return true; } } - // TODO(serxa): is it possible to validate it during enqueue of resource request to avoid LOGICAL_ERRORs in the scheduler thread? possible but will not cover case of moving queue with requests inside to invalid position - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Max number of simultaneous workload constraints exceeded ({}). Remove extra constraints before using this workload.", - ResourceMaxConstraints); + return false; } } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index e633af15157..bb9bfbfc8fd 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -17,7 +17,6 @@ class ISchedulerConstraint; using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -// TODO(serxa): validate hierarchy to avoid too many constraints /// Max number of constraints for a request to pass though (depth of constraints chain) constexpr size_t ResourceMaxConstraints = 8; @@ -91,7 +90,8 @@ public: void finish(); /// Is called from the scheduler thread to fill `constraints` chain - void addConstraint(ISchedulerConstraint * new_constraint); + /// Returns `true` iff constraint was added successfully + bool addConstraint(ISchedulerConstraint * new_constraint); }; } From 882ddb132ec3c360c6f3249ccbf657e0134d5b84 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 16:22:14 +0000 Subject: [PATCH 088/157] simplify keeper entity storage --- .../Workload/WorkloadEntityKeeperStorage.cpp | 57 ++++++++----------- .../Workload/WorkloadEntityKeeperStorage.h | 14 +++-- 2 files changed, 33 insertions(+), 38 deletions(-) diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp index 4aa087e029d..5b1c5d78f86 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.cpp @@ -34,7 +34,7 @@ WorkloadEntityKeeperStorage::WorkloadEntityKeeperStorage( : WorkloadEntityStorageBase(global_context_) , zookeeper_getter{[global_context_]() { return global_context_->getZooKeeper(); }} , zookeeper_path{zookeeper_path_} - , watch_queue{std::make_shared>(std::numeric_limits::max())} + , watch{std::make_shared()} { log = getLogger("WorkloadEntityKeeperStorage"); if (zookeeper_path.empty()) @@ -63,7 +63,7 @@ void WorkloadEntityKeeperStorage::stopWatchingThread() { if (watching_flag.exchange(false)) { - watch_queue->finish(); + watch->cv.notify_one(); if (watching_thread.joinable()) watching_thread.join(); } @@ -80,7 +80,7 @@ zkutil::ZooKeeperPtr WorkloadEntityKeeperStorage::getZooKeeper() zookeeper->sync(zookeeper_path); createRootNodes(zookeeper); - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); } return zookeeper; @@ -90,17 +90,14 @@ void WorkloadEntityKeeperStorage::loadEntities() { /// loadEntities() is called at start from Server::main(), so it's better not to stop here on no connection to ZooKeeper or any other error. /// However the watching thread must be started anyway in case the connection will be established later. - if (!entities_loaded) + try { - try - { - refreshAllEntities(getZooKeeper()); - startWatchingThread(); - } - catch (...) - { - tryLogCurrentException(log, "Failed to load workload entities"); - } + refreshEntities(getZooKeeper()); + startWatchingThread(); + } + catch (...) + { + tryLogCurrentException(log, "Failed to load workload entities"); } startWatchingThread(); } @@ -111,6 +108,7 @@ void WorkloadEntityKeeperStorage::processWatchQueue() LOG_DEBUG(log, "Started watching thread"); setThreadName("WrkldEntWatch"); + UInt64 handled = 0; while (watching_flag) { try @@ -118,11 +116,14 @@ void WorkloadEntityKeeperStorage::processWatchQueue() /// Re-initialize ZooKeeper session if expired getZooKeeper(); - bool queued = false; - if (!watch_queue->tryPop(queued, /* timeout_ms: */ 10000)) - continue; + { + std::unique_lock lock{watch->mutex}; + if (!watch->cv.wait_for(lock, std::chrono::seconds(10), [&] { return !watching_flag || handled != watch->triggered; })) + continue; + handled = watch->triggered; + } - refreshAllEntities(getZooKeeper()); + refreshEntities(getZooKeeper()); } catch (...) { @@ -166,7 +167,7 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::storeEnt auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); if (code != Coordination::Error::ZOK) { - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); return OperationResult::Retry; } @@ -193,7 +194,7 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEn auto code = zookeeper->trySet(zookeeper_path, new_data, current_version, &stat); if (code != Coordination::Error::ZOK) { - refreshAllEntities(zookeeper); + refreshEntities(zookeeper); return OperationResult::Retry; } @@ -206,12 +207,13 @@ WorkloadEntityStorageBase::OperationResult WorkloadEntityKeeperStorage::removeEn std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper) { - const auto data_watcher = [my_watch_queue = watch_queue](const Coordination::WatchResponse & response) + const auto data_watcher = [my_watch = watch](const Coordination::WatchResponse & response) { if (response.type == Coordination::Event::CHANGED) { - [[maybe_unused]] bool inserted = my_watch_queue->emplace(true); - /// `inserted` can be false if `watch_queue` was already finalized (which happens when stopWatching() is called). + std::unique_lock lock{my_watch->mutex}; + my_watch->triggered++; + my_watch->cv.notify_one(); } }; @@ -226,15 +228,6 @@ std::pair WorkloadEntityKeeperStorage::getDataAndSetWatch(const z return {data, stat.version}; } -void WorkloadEntityKeeperStorage::refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper) -{ - /// It doesn't make sense to keep the old watch events because we will reread everything in this function. - watch_queue->clear(); - - refreshEntities(zookeeper); - entities_loaded = true; -} - void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper) { auto [data, version] = getDataAndSetWatch(zookeeper); @@ -254,7 +247,7 @@ void WorkloadEntityKeeperStorage::refreshEntities(const zkutil::ZooKeeperPtr & z ++pos; } - /// Read & parse all SQL entities from data we just read from ZooKeeper + /// Read and parse all SQL entities from data we just read from ZooKeeper std::vector> new_entities; for (const auto & query : queries) { diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index deda5ba909b..0b55f501423 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -3,10 +3,11 @@ #include #include #include -#include #include #include +#include +#include namespace DB { @@ -48,10 +49,7 @@ private: void stopWatchingThread(); void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper); - std::pair getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper); - - void refreshAllEntities(const zkutil::ZooKeeperPtr & zookeeper); // TODO(serxa): get rid of it void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper); zkutil::ZooKeeperCachingGetter zookeeper_getter; @@ -59,10 +57,14 @@ private: Int32 current_version = 0; ThreadFromGlobalPool watching_thread; - std::atomic entities_loaded = false; std::atomic watching_flag = false; - std::shared_ptr> watch_queue; // TODO(serxa): rework it into something that is not a queue + struct WatchEvent { + std::mutex mutex; + std::condition_variable cv; + UInt64 triggered = 0; + }; + std::shared_ptr watch; }; } From 49eaf646298d6e41245ac4cee6c3e77f51228074 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 17:13:40 +0000 Subject: [PATCH 089/157] fix rare race between queue activation and destruction --- src/Common/Scheduler/ISchedulerNode.h | 30 +++++++++++++++---- .../Scheduler/Nodes/IOResourceManager.cpp | 3 -- src/Common/Scheduler/SchedulerRoot.h | 5 ---- 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index d13e0b02c53..5e1239de274 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -139,7 +139,7 @@ public: , info(info_) {} - virtual ~ISchedulerNode() = default; + virtual ~ISchedulerNode(); virtual const String & getTypeName() const = 0; @@ -187,10 +187,7 @@ public: } /// Attach to a parent (used by attachChild) - virtual void setParent(ISchedulerNode * parent_) - { - parent = parent_; - } + void setParent(ISchedulerNode * parent_); protected: /// Notify parents about the first pending request or constraint becoming satisfied. @@ -326,6 +323,15 @@ public: pending.notify_one(); } + /// Removes an activation from queue + void cancelActivation(ISchedulerNode * node) + { + std::unique_lock lock{mutex}; + if (node->is_linked()) + activations.erase(activations.iterator_to(*node)); + node->activation_event_id = 0; + } + /// Process single event if it exists /// Note that postponing constraint are ignored, use it to empty the queue including postponed events on shutdown /// Returns `true` iff event has been processed @@ -490,6 +496,20 @@ private: std::atomic manual_time{TimePoint()}; // for tests only }; +inline ISchedulerNode::~ISchedulerNode() +{ + // Make sure there is no dangling reference in activations queue + event_queue->cancelActivation(this); +} + +inline void ISchedulerNode::setParent(ISchedulerNode * parent_) +{ + parent = parent_; + // Avoid activation of a detached node + if (parent == nullptr) + event_queue->cancelActivation(this); +} + inline void ISchedulerNode::scheduleActivation() { if (likely(parent)) diff --git a/src/Common/Scheduler/Nodes/IOResourceManager.cpp b/src/Common/Scheduler/Nodes/IOResourceManager.cpp index 812a49ace60..80d3650b1b9 100644 --- a/src/Common/Scheduler/Nodes/IOResourceManager.cpp +++ b/src/Common/Scheduler/Nodes/IOResourceManager.cpp @@ -187,9 +187,6 @@ void IOResourceManager::Resource::updateCurrentVersion() if (previous_version) { previous_version->newer_version = current_version; - // TODO(serxa): Node activations might be in event queue on destruction. How to process them? should we just process all events in queue on important updates? add a separate queue for hierarchy modifications? Or maybe everything works as expected, we need unit tests for this. - // Looks like the problem of activations could be solved just by unliking activation from intrusive list on destruction, but we must make sure all destruction are done under event_queue::mutex (which seems impossible) - // Another possible solution is to remove activations from queue on detachChild. It is good because activations are created on attachChild. previous_version.reset(); // Destroys previous version nodes if there are no classifiers referencing it } } diff --git a/src/Common/Scheduler/SchedulerRoot.h b/src/Common/Scheduler/SchedulerRoot.h index 45e4309fc81..451f29f33f2 100644 --- a/src/Common/Scheduler/SchedulerRoot.h +++ b/src/Common/Scheduler/SchedulerRoot.h @@ -190,11 +190,6 @@ public: activate(Resource::get(child->info)); } - void setParent(ISchedulerNode *) override - { - abort(); // scheduler must be the root and this function should not be called - } - private: void activate(Resource * value) { From d0fe70dc1a2023499e651b6aaac10f441d57b71d Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 18:55:42 +0000 Subject: [PATCH 090/157] style --- src/Common/Scheduler/ResourceRequest.cpp | 5 ----- src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h | 3 ++- src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp | 2 +- tests/integration/test_scheduler/test.py | 5 ++--- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Common/Scheduler/ResourceRequest.cpp b/src/Common/Scheduler/ResourceRequest.cpp index 71731fde9ce..674c7650adf 100644 --- a/src/Common/Scheduler/ResourceRequest.cpp +++ b/src/Common/Scheduler/ResourceRequest.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - void ResourceRequest::finish() { // Iterate over constraints in reverse order diff --git a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h index 0b55f501423..25dcd6d8c9a 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h +++ b/src/Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h @@ -59,7 +59,8 @@ private: ThreadFromGlobalPool watching_thread; std::atomic watching_flag = false; - struct WatchEvent { + struct WatchEvent + { std::mutex mutex; std::condition_variable cv; UInt64 triggered = 0; diff --git a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp index dd4f5365191..a42252b1b8e 100644 --- a/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp +++ b/src/Common/Scheduler/Workload/WorkloadEntityStorageBase.cpp @@ -283,7 +283,7 @@ std::vector topologicallySortedChanges(const std::vector()) , global_context(std::move(global_context_)) - , log{getLogger("WorkloadEntityStorage")} // could be overriden in derived class + , log{getLogger("WorkloadEntityStorage")} // could be overridden in derived class {} ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 05f38d09245..3c755860bdb 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -896,16 +896,15 @@ def test_workload_entity_keeper_storage(): value2 = node2.query(query) if value1 != value2: error_query = query - break # error + break # error else: - break # success + break # success time.sleep(0.5) else: raise Exception( f"query '{error_query}' gives different results after {attempts} attempts:\n=== leader node ===\n{value1}\n=== follower node ===\n{value2}" ) - for iteration in range(3): split_idx_1 = random.randint(1, len(queries) - 3) split_idx_2 = random.randint(split_idx_1 + 1, len(queries) - 2) From 3a8bea0932a633f6daf032267b401c8cef94af45 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 16 Oct 2024 20:29:52 +0000 Subject: [PATCH 091/157] add docs --- .../settings.md | 30 +++++++++++- docs/en/operations/system-tables/resources.md | 37 +++++++++++++++ docs/en/operations/system-tables/workloads.md | 40 ++++++++++++++++ docs/en/operations/workload-scheduling.md | 47 +++++++++++++++++++ programs/server/config.xml | 2 +- 5 files changed, 154 insertions(+), 2 deletions(-) create mode 100644 docs/en/operations/system-tables/resources.md create mode 100644 docs/en/operations/system-tables/workloads.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 79407d46ce0..c032f1bac43 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3085,7 +3085,7 @@ By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests ove ### no_proxy By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set. -It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. +It can be set inside the `` clause for list and remote resolvers and as an environment variable for environment resolver. It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does. Example: @@ -3151,6 +3151,34 @@ Default value: "default" **See Also** - [Workload Scheduling](/docs/en/operations/workload-scheduling.md) +## workload_path {#workload_path} + +The directory used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. By default `/workload/` folder under server working directory is used. + +**Example** + +``` xml +/var/lib/clickhouse/workload/ +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_zookeeper_path](#workload_zookeeper_path) + +## workload_zookeeper_path {#workload_zookeeper_path} + +The path to a ZooKeeper node, which is used as a storage for all `CREATE WORKLOAD` and `CREATE RESOURCE` queries. For consistency all SQL definitions are stored as a value of this single znode. By default ZooKeeper is not used and definitions are stored on [disk](#workload_path). + +**Example** + +``` xml +/clickhouse/workload/definitions.sql +``` + +**See Also** +- [Workload Hierarchy](/docs/en/operations/workload-scheduling.md#workloads) +- [workload_path](#workload_path) + ## max_authentication_methods_per_user {#max_authentication_methods_per_user} The maximum number of authentication methods a user can be created with or altered to. diff --git a/docs/en/operations/system-tables/resources.md b/docs/en/operations/system-tables/resources.md new file mode 100644 index 00000000000..6329f05f610 --- /dev/null +++ b/docs/en/operations/system-tables/resources.md @@ -0,0 +1,37 @@ +--- +slug: /en/operations/system-tables/resources +--- +# resources + +Contains information for [resources](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every resource. + +Example: + +``` sql +SELECT * +FROM system.resources +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: io_read +read_disks: ['s3'] +write_disks: [] +create_query: CREATE RESOURCE io_read (READ DISK s3) + +Row 2: +────── +name: io_write +read_disks: [] +write_disks: ['s3'] +create_query: CREATE RESOURCE io_write (WRITE DISK s3) +``` + +Columns: + +- `name` (`String`) - Resource name. +- `read_disks` (`Array(String)`) - The array of disk names that uses this resource for read operations. +- `write_disks` (`Array(String)`) - The array of disk names that uses this resource for write operations. +- `create_query` (`String`) - The definition of the resource. diff --git a/docs/en/operations/system-tables/workloads.md b/docs/en/operations/system-tables/workloads.md new file mode 100644 index 00000000000..d9c62372044 --- /dev/null +++ b/docs/en/operations/system-tables/workloads.md @@ -0,0 +1,40 @@ +--- +slug: /en/operations/system-tables/workloads +--- +# workloads + +Contains information for [workloads](/docs/en/operations/workload-scheduling.md#workload_entity_storage) residing on the local server. The table contains a row for every workload. + +Example: + +``` sql +SELECT * +FROM system.workloads +FORMAT Vertical +``` + +``` text +Row 1: +────── +name: production +parent: all +create_query: CREATE WORKLOAD production IN `all` SETTINGS weight = 9 + +Row 2: +────── +name: development +parent: all +create_query: CREATE WORKLOAD development IN `all` + +Row 3: +────── +name: all +parent: +create_query: CREATE WORKLOAD `all` +``` + +Columns: + +- `name` (`String`) - Workload name. +- `parent` (`String`) - Parent workload name. +- `create_query` (`String`) - The definition of the workload. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 08629492ec6..7dc726d75f3 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -43,6 +43,14 @@ Example: ``` +Alternative way to express which disks are used by a resource is SQL syntax: + +```sql +CREATE RESOURCE (WRITE DISK disk1, READ DISK disk2) +``` + +Note that server configuration options have priority over SQL way to define resources. + ## Workload markup {#workload_markup} Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. @@ -153,9 +161,48 @@ Example: ``` +## Workload hierarchy (SQL only) {#workloads} + +Defining resources and classifiers in XML could be challenging. ClickHouse provides SQL syntax that is much more convenient. All resources that were created with `CREATE RESOURCE` share the same structure of the hierarchy, but could differ in some aspects. Every workload created with `CREATE WORKLOAD` maintain a few automatically created scheduling nodes for every resource. A child workload can be created inside another parent workload. Here is the example that defines exactly the same hierarchy as XML configuration above: + +```sql +CREATE RESOURCE network_write (WRITE DISK s3) +CREATE RESOURCE network_read (READ DISK s3) +CREATE WORKLOAD all SETTINGS max_requests = 100 +CREATE WORKLOAD development IN all +CREATE WORKLOAD production IN all SETTINGS weight = 3 +``` + +Name of a leaf workload without children could be used in query settings `SETTINGS workload = 'name'`. Note that workload classifiers are also created automatically when using SQL syntax. + +To customize workload the following settings could be used: +* `priority` - sibling workloads are served according to static priority values (lower value means higher priority). +* `weight` - sibling workloads having the same static priority share resources according to weights. +* `max_requests` - the limit on the number of concurrent resource requests in this workload. +* `max_cost` - the limit on the total inflight bytes count of concurrent resource requests in this workload. +* `max_speed` - the limit on byte processing rate of this workload (the limit is independent for every resource). +* `max_burst` - maximum number of bytes that could be processed by the workload without being throttled (for every resource independently). + +Note that workload settings are translated into proper set of scheduling nodes. For more details, see description of scheduling node [types and options](#hierarchy). + +There is no way to specify different hierarchy of workloads for different resources. But there is a way to specify differet workload setting value for a specific resource: + +```sql +CREATE OR REPLACE WORKLOAD all SETTINGS max_requests = 100, max_speed = 1000000 FOR network_read, max_speed = 2000000 FOR network_write +``` + +Also note that workload or resource could not be dropped if it is referenced from another workload. To update a definition of a workload use `CREATE OR REPLACE WORKLOAD` query. + +## Workloads and resources storage {#workload_entity_storage} +Definitions of all workload and resource in form of `CREATE WORKLOAD` and `CREATE RESOURCE` queries are stored persistently either on disk at `workload_path` or in ZooKeeper at `workload_zookeeper_path`. ZooKeeper storage is recommended to achieve consistency between nodes. Alternatively `ON CLUSTER` clause could be used along with a disk storage. + ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) + - [system.workloads](/docs/en/operations/system-tables/worklaods.md) + - [system.resources](/docs/en/operations/system-tables/resources.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting + - [workload_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_path) global server setting + - [workload_zookeeper_path](/docs/en/operations/server-configuration-parameters/settings.md#workload_zookeeper_path) global server setting diff --git a/programs/server/config.xml b/programs/server/config.xml index b41f0344bb2..4e05e053d8a 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1388,7 +1388,7 @@ - +