mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 12:14:18 +00:00
Merge pull request #71251 from ClickHouse/revert-69187-sql-for-workload-management
Revert "SQL syntax for workload and resource management"
This commit is contained in:
commit
12718ff024
1
.gitignore
vendored
1
.gitignore
vendored
@ -159,7 +159,6 @@ website/package-lock.json
|
||||
/programs/server/store
|
||||
/programs/server/uuid
|
||||
/programs/server/coordination
|
||||
/programs/server/workload
|
||||
|
||||
# temporary test files
|
||||
tests/queries/0_stateless/test_*
|
||||
|
@ -3224,34 +3224,6 @@ 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
|
||||
<workload_path>/var/lib/clickhouse/workload/</workload_path>
|
||||
```
|
||||
|
||||
**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
|
||||
<workload_zookeeper_path>/clickhouse/workload/definitions.sql</workload_zookeeper_path>
|
||||
```
|
||||
|
||||
**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.
|
||||
|
@ -1,37 +0,0 @@
|
||||
---
|
||||
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.
|
@ -1,40 +0,0 @@
|
||||
---
|
||||
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.
|
@ -43,20 +43,6 @@ Example:
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
An alternative way to express which disks are used by a resource is SQL syntax:
|
||||
|
||||
```sql
|
||||
CREATE RESOURCE resource_name (WRITE DISK disk1, READ DISK disk2)
|
||||
```
|
||||
|
||||
Resource could be used for any number of disk for READ or WRITE or both for READ and WRITE. There a syntax allowing to use a resource for all the disks:
|
||||
|
||||
```sql
|
||||
CREATE RESOURCE all_io (READ ANY DISK, WRITE ANY DISK);
|
||||
```
|
||||
|
||||
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.
|
||||
@ -167,48 +153,9 @@ Example:
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
## 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` maintains 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
|
||||
```
|
||||
|
||||
The 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 a proper set of scheduling nodes. For more details, see the description of the scheduling node [types and options](#hierarchy).
|
||||
|
||||
There is no way to specify different hierarchies of workloads for different resources. But there is a way to specify different 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 workloads and resources in the 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 disk storage.
|
||||
|
||||
## See also
|
||||
- [system.scheduler](/docs/en/operations/system-tables/scheduler.md)
|
||||
- [system.workloads](/docs/en/operations/system-tables/workloads.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
|
||||
|
@ -86,7 +86,7 @@
|
||||
#include <Dictionaries/registerDictionaries.h>
|
||||
#include <Disks/registerDisks.h>
|
||||
#include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
#include <Common/Config/ConfigReloader.h>
|
||||
#include <Server/HTTPHandlerFactory.h>
|
||||
#include "MetricsTransmitter.h"
|
||||
@ -920,6 +920,7 @@ try
|
||||
registerFormats();
|
||||
registerRemoteFileMetadatas();
|
||||
registerSchedulerNodes();
|
||||
registerResourceManagers();
|
||||
|
||||
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision());
|
||||
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
|
||||
@ -2252,8 +2253,6 @@ try
|
||||
database_catalog.assertDatabaseExists(default_database);
|
||||
/// Load user-defined SQL functions.
|
||||
global_context->getUserDefinedSQLObjectsStorage().loadObjects();
|
||||
/// Load WORKLOADs and RESOURCEs.
|
||||
global_context->getWorkloadEntityStorage().loadEntities();
|
||||
|
||||
global_context->getRefreshSet().setRefreshesStopped(false);
|
||||
}
|
||||
|
@ -1399,10 +1399,6 @@
|
||||
If not specified they will be stored locally. -->
|
||||
<!-- <user_defined_zookeeper_path>/clickhouse/user_defined</user_defined_zookeeper_path> -->
|
||||
|
||||
<!-- Path in ZooKeeper to store workload and resource created by the command CREATE WORKLOAD and CREATE REESOURCE.
|
||||
If not specified they will be stored locally. -->
|
||||
<!-- <workload_zookeeper_path>/clickhouse/workload/definitions.sql</workload_zookeeper_path> -->
|
||||
|
||||
<!-- Uncomment if you want data to be compressed 30-100% better.
|
||||
Don't do that if you just started using ClickHouse.
|
||||
-->
|
||||
|
@ -99,8 +99,6 @@ 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} */ \
|
||||
\
|
||||
@ -110,8 +108,6 @@ 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} */\
|
||||
\
|
||||
|
@ -701,17 +701,15 @@ 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 | workload_ddl | resource_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY;
|
||||
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_1_flags = AccessType::CREATE_TEMPORARY_TABLE;
|
||||
|
||||
const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl | workload_ddl | resource_ddl;
|
||||
const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl;
|
||||
const AccessFlags introspection_flags = AccessType::INTROSPECTION;
|
||||
};
|
||||
static const PrecalculatedFlags precalc;
|
||||
|
@ -136,7 +136,6 @@ 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)
|
||||
|
@ -6,7 +6,6 @@
|
||||
/// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize
|
||||
struct Priority
|
||||
{
|
||||
using Value = Int64;
|
||||
Value value = 0; /// Note that lower value means higher priority.
|
||||
constexpr operator Value() const { return value; } /// NOLINT
|
||||
Int64 value = 0; /// Note that lower value means higher priority.
|
||||
constexpr operator Int64() const { return value; } /// NOLINT
|
||||
};
|
||||
|
@ -26,9 +26,6 @@ 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;
|
||||
@ -49,15 +46,12 @@ public:
|
||||
/// Initialize or reconfigure manager.
|
||||
virtual void updateConfiguration(const Poco::Util::AbstractConfiguration & config) = 0;
|
||||
|
||||
/// Returns true iff given resource is controlled through 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;
|
||||
|
||||
/// For introspection, see `system.scheduler` table
|
||||
using VisitorFunc = std::function<void(const String & resource, const String & path, ISchedulerNode * node)>;
|
||||
using VisitorFunc = std::function<void(const String & resource, const String & path, const String & type, const SchedulerNodePtr & node)>;
|
||||
virtual void forEachNode(VisitorFunc visitor) = 0;
|
||||
};
|
||||
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
* When constraint is again satisfied, scheduleActivation() is called from finishRequest().
|
||||
*
|
||||
* Derived class behaviour requirements:
|
||||
* - dequeueRequest() must call `request->addConstraint()`.
|
||||
* - dequeueRequest() must fill `request->constraint` iff it is nullptr;
|
||||
* - finishRequest() must be recursive: call to `parent_constraint->finishRequest()`.
|
||||
*/
|
||||
class ISchedulerConstraint : public ISchedulerNode
|
||||
{
|
||||
@ -24,16 +25,34 @@ 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;
|
||||
|
||||
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<ISchedulerConstraint *>(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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -57,13 +57,7 @@ struct SchedulerNodeInfo
|
||||
|
||||
SchedulerNodeInfo() = default;
|
||||
|
||||
explicit SchedulerNodeInfo(double weight_, Priority priority_ = {})
|
||||
{
|
||||
setWeight(weight_);
|
||||
setPriority(priority_);
|
||||
}
|
||||
|
||||
explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config, const String & config_prefix = {})
|
||||
explicit SchedulerNodeInfo(const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
{
|
||||
setWeight(config.getDouble(config_prefix + ".weight", weight));
|
||||
setPriority(config.getInt64(config_prefix + ".priority", priority));
|
||||
@ -74,7 +68,7 @@ struct SchedulerNodeInfo
|
||||
if (value <= 0 || !isfinite(value))
|
||||
throw Exception(
|
||||
ErrorCodes::INVALID_SCHEDULER_NODE,
|
||||
"Zero, negative and non-finite node weights are not allowed: {}",
|
||||
"Negative and non-finite node weights are not allowed: {}",
|
||||
value);
|
||||
weight = value;
|
||||
}
|
||||
@ -84,11 +78,6 @@ struct SchedulerNodeInfo
|
||||
priority.value = value;
|
||||
}
|
||||
|
||||
void setPriority(Priority value)
|
||||
{
|
||||
priority = value;
|
||||
}
|
||||
|
||||
// To check if configuration update required
|
||||
bool equals(const SchedulerNodeInfo & o) const
|
||||
{
|
||||
@ -134,14 +123,7 @@ public:
|
||||
, info(config, config_prefix)
|
||||
{}
|
||||
|
||||
ISchedulerNode(EventQueue * event_queue_, const SchedulerNodeInfo & info_)
|
||||
: event_queue(event_queue_)
|
||||
, info(info_)
|
||||
{}
|
||||
|
||||
virtual ~ISchedulerNode();
|
||||
|
||||
virtual const String & getTypeName() const = 0;
|
||||
virtual ~ISchedulerNode() = default;
|
||||
|
||||
/// Checks if two nodes configuration is equal
|
||||
virtual bool equals(ISchedulerNode * other)
|
||||
@ -152,11 +134,10 @@ public:
|
||||
/// Attach new child
|
||||
virtual void attachChild(const std::shared_ptr<ISchedulerNode> & child) = 0;
|
||||
|
||||
/// Detach child
|
||||
/// NOTE: child might be destroyed if the only reference was stored in parent
|
||||
/// Detach and destroy child
|
||||
virtual void removeChild(ISchedulerNode * child) = 0;
|
||||
|
||||
/// Get attached child by name (for tests only)
|
||||
/// Get attached child by name
|
||||
virtual ISchedulerNode * getChild(const String & child_name) = 0;
|
||||
|
||||
/// Activation of child due to the first pending request
|
||||
@ -166,7 +147,7 @@ public:
|
||||
/// Returns true iff node is active
|
||||
virtual bool isActive() = 0;
|
||||
|
||||
/// Returns number of active children (for introspection only).
|
||||
/// Returns number of active children
|
||||
virtual size_t activeChildren() = 0;
|
||||
|
||||
/// Returns the first request to be executed as the first component of resulting pair.
|
||||
@ -174,10 +155,10 @@ public:
|
||||
virtual std::pair<ResourceRequest *, bool> dequeueRequest() = 0;
|
||||
|
||||
/// Returns full path string using names of every parent
|
||||
String getPath() const
|
||||
String getPath()
|
||||
{
|
||||
String result;
|
||||
const ISchedulerNode * ptr = this;
|
||||
ISchedulerNode * ptr = this;
|
||||
while (ptr->parent)
|
||||
{
|
||||
result = "/" + ptr->basename + result;
|
||||
@ -187,7 +168,10 @@ public:
|
||||
}
|
||||
|
||||
/// Attach to a parent (used by attachChild)
|
||||
void setParent(ISchedulerNode * parent_);
|
||||
virtual void setParent(ISchedulerNode * parent_)
|
||||
{
|
||||
parent = parent_;
|
||||
}
|
||||
|
||||
protected:
|
||||
/// Notify parents about the first pending request or constraint becoming satisfied.
|
||||
@ -323,15 +307,6 @@ 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
|
||||
@ -496,20 +471,6 @@ private:
|
||||
std::atomic<TimePoint> 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))
|
||||
|
@ -21,10 +21,6 @@ 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
|
||||
// Returns `estimated_cost` that should be passed later to `adjustBudget()`
|
||||
[[ nodiscard ]] ResourceCost enqueueRequestUsingBudget(ResourceRequest * request)
|
||||
@ -51,11 +47,6 @@ 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
|
||||
{
|
||||
|
@ -5,6 +5,11 @@
|
||||
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;
|
||||
@ -26,11 +31,9 @@ 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
|
||||
return empty;
|
||||
throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,7 +10,6 @@ namespace DB
|
||||
/// Mapping of resource name into path string (e.g. "disk1" -> "/path/to/class")
|
||||
struct ClassifierDescription : std::unordered_map<String, String>
|
||||
{
|
||||
ClassifierDescription() = default;
|
||||
ClassifierDescription(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
};
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Common/Scheduler/Nodes/CustomResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/DynamicResourceManager.h>
|
||||
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
@ -20,7 +21,7 @@ namespace ErrorCodes
|
||||
extern const int INVALID_SCHEDULER_NODE;
|
||||
}
|
||||
|
||||
CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config)
|
||||
DynamicResourceManager::State::State(EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config)
|
||||
: classifiers(config)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
@ -34,7 +35,7 @@ CustomResourceManager::State::State(EventQueue * event_queue, const Poco::Util::
|
||||
}
|
||||
}
|
||||
|
||||
CustomResourceManager::State::Resource::Resource(
|
||||
DynamicResourceManager::State::Resource::Resource(
|
||||
const String & name,
|
||||
EventQueue * event_queue,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
@ -91,7 +92,7 @@ CustomResourceManager::State::Resource::Resource(
|
||||
throw Exception(ErrorCodes::INVALID_SCHEDULER_NODE, "undefined root node path '/' for resource '{}'", name);
|
||||
}
|
||||
|
||||
CustomResourceManager::State::Resource::~Resource()
|
||||
DynamicResourceManager::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 +106,14 @@ CustomResourceManager::State::Resource::~Resource()
|
||||
}
|
||||
}
|
||||
|
||||
CustomResourceManager::State::Node::Node(const String & name, EventQueue * event_queue, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
|
||||
DynamicResourceManager::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 CustomResourceManager::State::Resource::equals(const CustomResourceManager::State::Resource & o) const
|
||||
bool DynamicResourceManager::State::Resource::equals(const DynamicResourceManager::State::Resource & o) const
|
||||
{
|
||||
if (nodes.size() != o.nodes.size())
|
||||
return false;
|
||||
@ -129,14 +130,14 @@ bool CustomResourceManager::State::Resource::equals(const CustomResourceManager:
|
||||
return true;
|
||||
}
|
||||
|
||||
bool CustomResourceManager::State::Node::equals(const CustomResourceManager::State::Node & o) const
|
||||
bool DynamicResourceManager::State::Node::equals(const DynamicResourceManager::State::Node & o) const
|
||||
{
|
||||
if (type != o.type)
|
||||
return false;
|
||||
return ptr->equals(o.ptr.get());
|
||||
}
|
||||
|
||||
CustomResourceManager::Classifier::Classifier(const CustomResourceManager::StatePtr & state_, const String & classifier_name)
|
||||
DynamicResourceManager::Classifier::Classifier(const DynamicResourceManager::StatePtr & state_, const String & classifier_name)
|
||||
: state(state_)
|
||||
{
|
||||
// State is immutable, but nodes are mutable and thread-safe
|
||||
@ -161,25 +162,20 @@ CustomResourceManager::Classifier::Classifier(const CustomResourceManager::State
|
||||
}
|
||||
}
|
||||
|
||||
bool CustomResourceManager::Classifier::has(const String & resource_name)
|
||||
{
|
||||
return resources.contains(resource_name);
|
||||
}
|
||||
|
||||
ResourceLink CustomResourceManager::Classifier::get(const String & resource_name)
|
||||
ResourceLink DynamicResourceManager::Classifier::get(const String & resource_name)
|
||||
{
|
||||
if (auto iter = resources.find(resource_name); iter != resources.end())
|
||||
return iter->second;
|
||||
throw Exception(ErrorCodes::RESOURCE_ACCESS_DENIED, "Access denied to resource '{}'", resource_name);
|
||||
}
|
||||
|
||||
CustomResourceManager::CustomResourceManager()
|
||||
DynamicResourceManager::DynamicResourceManager()
|
||||
: state(new State())
|
||||
{
|
||||
scheduler.start();
|
||||
}
|
||||
|
||||
void CustomResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
StatePtr new_state = std::make_shared<State>(scheduler.event_queue, config);
|
||||
|
||||
@ -221,13 +217,7 @@ 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.contains(resource_name);
|
||||
}
|
||||
|
||||
ClassifierPtr CustomResourceManager::acquire(const String & classifier_name)
|
||||
ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name)
|
||||
{
|
||||
// Acquire a reference to the current state
|
||||
StatePtr state_ref;
|
||||
@ -239,7 +229,7 @@ ClassifierPtr CustomResourceManager::acquire(const String & classifier_name)
|
||||
return std::make_shared<Classifier>(state_ref, classifier_name);
|
||||
}
|
||||
|
||||
void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor)
|
||||
void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor)
|
||||
{
|
||||
// Acquire a reference to the current state
|
||||
StatePtr state_ref;
|
||||
@ -254,7 +244,7 @@ void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor)
|
||||
{
|
||||
for (auto & [name, resource] : state_ref->resources)
|
||||
for (auto & [path, node] : resource->nodes)
|
||||
visitor(name, path, node.ptr.get());
|
||||
visitor(name, path, node.type, node.ptr);
|
||||
promise.set_value();
|
||||
});
|
||||
|
||||
@ -262,4 +252,9 @@ void CustomResourceManager::forEachNode(IResourceManager::VisitorFunc visitor)
|
||||
future.get();
|
||||
}
|
||||
|
||||
void registerDynamicResourceManager(ResourceManagerFactory & factory)
|
||||
{
|
||||
factory.registerMethod<DynamicResourceManager>("dynamic");
|
||||
}
|
||||
|
||||
}
|
@ -10,9 +10,7 @@ namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* 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.
|
||||
* Implementation of `IResourceManager` supporting arbitrary dynamic hierarchy of scheduler nodes.
|
||||
* All resources are controlled by single root `SchedulerRoot`.
|
||||
*
|
||||
* State of manager is set of resources attached to the scheduler. States are referenced by classifiers.
|
||||
@ -26,12 +24,11 @@ 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 CustomResourceManager : public IResourceManager
|
||||
class DynamicResourceManager : public IResourceManager
|
||||
{
|
||||
public:
|
||||
CustomResourceManager();
|
||||
DynamicResourceManager();
|
||||
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;
|
||||
|
||||
@ -82,7 +79,6 @@ 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<String, ResourceLink> resources; // accessible resources by names
|
||||
@ -90,7 +86,7 @@ private:
|
||||
};
|
||||
|
||||
SchedulerRoot scheduler;
|
||||
mutable std::mutex mutex;
|
||||
std::mutex mutex;
|
||||
StatePtr state;
|
||||
};
|
||||
|
@ -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 final : public ISchedulerNode
|
||||
class FairPolicy : public ISchedulerNode
|
||||
{
|
||||
/// Scheduling state of a child
|
||||
struct Item
|
||||
@ -48,23 +48,6 @@ public:
|
||||
: ISchedulerNode(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
FairPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & info_)
|
||||
: 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");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
{
|
||||
if (!ISchedulerNode::equals(other))
|
||||
|
@ -23,28 +23,13 @@ namespace ErrorCodes
|
||||
/*
|
||||
* FIFO queue to hold pending resource requests
|
||||
*/
|
||||
class FifoQueue final : public ISchedulerQueue
|
||||
class FifoQueue : public ISchedulerQueue
|
||||
{
|
||||
public:
|
||||
FifoQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
: ISchedulerQueue(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
FifoQueue(EventQueue * event_queue_, const SchedulerNodeInfo & info_)
|
||||
: ISchedulerQueue(event_queue_, info_)
|
||||
{}
|
||||
|
||||
~FifoQueue() override
|
||||
{
|
||||
purgeQueue();
|
||||
}
|
||||
|
||||
const String & getTypeName() const override
|
||||
{
|
||||
static String type_name("fifo");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
{
|
||||
if (!ISchedulerNode::equals(other))
|
||||
@ -57,8 +42,6 @@ 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);
|
||||
@ -83,8 +66,6 @@ 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.
|
||||
@ -107,19 +88,6 @@ 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);
|
||||
@ -163,7 +131,6 @@ private:
|
||||
std::mutex mutex;
|
||||
Int64 queue_cost = 0;
|
||||
boost::intrusive::list<ResourceRequest> requests;
|
||||
bool is_not_usable = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,532 +0,0 @@
|
||||
#include <Common/Scheduler/Nodes/IOResourceManager.h>
|
||||
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Priority.h>
|
||||
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
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<ASTCreateWorkloadQuery *>(ast.get()))
|
||||
return create->getWorkloadName();
|
||||
if (auto * create = typeid_cast<ASTCreateResourceQuery *>(ast.get()))
|
||||
return create->getResourceName();
|
||||
return "unknown-workload-entity";
|
||||
}
|
||||
}
|
||||
|
||||
IOResourceManager::NodeInfo::NodeInfo(const ASTPtr & ast, const String & resource_name)
|
||||
{
|
||||
auto * create = assert_cast<ASTCreateWorkloadQuery *>(ast.get());
|
||||
name = create->getWorkloadName();
|
||||
parent = create->getWorkloadParent();
|
||||
settings.updateFromChanges(create->changes, resource_name);
|
||||
}
|
||||
|
||||
IOResourceManager::Resource::Resource(const ASTPtr & resource_entity_)
|
||||
: resource_entity(resource_entity_)
|
||||
, resource_name(getEntityName(resource_entity))
|
||||
{
|
||||
scheduler.start();
|
||||
}
|
||||
|
||||
IOResourceManager::Resource::~Resource()
|
||||
{
|
||||
scheduler.stop();
|
||||
}
|
||||
|
||||
void IOResourceManager::Resource::createNode(const NodeInfo & info)
|
||||
{
|
||||
if (info.name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Workload must have a name in resource '{}'",
|
||||
resource_name);
|
||||
|
||||
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);
|
||||
|
||||
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);
|
||||
|
||||
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<UnifiedSchedulerNode>(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];
|
||||
|
||||
if (node->hasUnifiedChildren())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Removing workload '{}' with children in resource '{}'",
|
||||
info.name, resource_name);
|
||||
|
||||
executeInSchedulerThread([&]
|
||||
{
|
||||
if (!info.parent.empty())
|
||||
node_for_workload[info.parent]->detachUnifiedChild(node);
|
||||
else
|
||||
{
|
||||
chassert(node == root_node);
|
||||
scheduler.removeChild(root_node.get());
|
||||
root_node.reset();
|
||||
}
|
||||
|
||||
node_for_workload.erase(info.name);
|
||||
|
||||
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() || 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);
|
||||
|
||||
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 (UnifiedSchedulerNode::updateRequiresDetach(old_info.parent, new_info.parent, old_info.settings, new_info.settings))
|
||||
{
|
||||
if (!old_info.parent.empty())
|
||||
node_for_workload[old_info.parent]->detachUnifiedChild(node);
|
||||
detached = true;
|
||||
}
|
||||
|
||||
node->updateSchedulingSettings(new_info.settings);
|
||||
|
||||
if (detached)
|
||||
{
|
||||
if (!new_info.parent.empty())
|
||||
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<Version>();
|
||||
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;
|
||||
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_)
|
||||
{
|
||||
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()
|
||||
{
|
||||
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)
|
||||
{
|
||||
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
|
||||
{
|
||||
return assert_cast<ASTCreateWorkloadQuery *>(workload_entity.get())->getWorkloadParent();
|
||||
}
|
||||
|
||||
IOResourceManager::IOResourceManager(IWorkloadEntityStorage & storage_)
|
||||
: storage(storage_)
|
||||
, log{getLogger("IOResourceManager")}
|
||||
{
|
||||
subscription = storage.getAllEntitiesAndSubscribe(
|
||||
[this] (const std::vector<IWorkloadEntityStorage::Event> & events)
|
||||
{
|
||||
for (const 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)
|
||||
createOrUpdateResource(entity_name, entity);
|
||||
else
|
||||
deleteResource(entity_name);
|
||||
break;
|
||||
}
|
||||
case WorkloadEntityType::MAX: break;
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
IOResourceManager::~IOResourceManager()
|
||||
{
|
||||
subscription.reset();
|
||||
resources.clear();
|
||||
workloads.clear();
|
||||
}
|
||||
|
||||
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<Workload>(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())
|
||||
{
|
||||
// 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
|
||||
LOG_ERROR(log, "Delete workload that doesn't exist: {}", workload_name);
|
||||
}
|
||||
|
||||
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_iter->second->updateResource(ast);
|
||||
else
|
||||
{
|
||||
// Add all workloads into the new resource
|
||||
auto resource = std::make_shared<Resource>(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
|
||||
LOG_ERROR(log, "Delete resource that doesn't exist: {}", resource_name);
|
||||
}
|
||||
|
||||
IOResourceManager::Classifier::~Classifier()
|
||||
{
|
||||
// Detach classifier from all resources in parallel (executed in every scheduler thread)
|
||||
std::vector<std::future<void>> 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<void> IOResourceManager::Resource::detachClassifier(VersionPtr && version)
|
||||
{
|
||||
auto detach_promise = std::make_shared<std::promise<void>>(); // 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;
|
||||
}
|
||||
|
||||
bool IOResourceManager::Classifier::has(const String & resource_name)
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
return attachments.contains(resource_name);
|
||||
}
|
||||
|
||||
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};
|
||||
}
|
||||
|
||||
void IOResourceManager::Resource::updateResource(const ASTPtr & new_resource_entity)
|
||||
{
|
||||
chassert(getEntityName(new_resource_entity) == resource_name);
|
||||
resource_entity = new_resource_entity;
|
||||
}
|
||||
|
||||
std::future<void> IOResourceManager::Resource::attachClassifier(Classifier & classifier, const String & workload_name)
|
||||
{
|
||||
auto attach_promise = std::make_shared<std::promise<void>>(); // 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)]
|
||||
{
|
||||
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
|
||||
{
|
||||
// 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 (...)
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
bool IOResourceManager::hasResource(const String & resource_name) const
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
return resources.contains(resource_name);
|
||||
}
|
||||
|
||||
ClassifierPtr IOResourceManager::acquire(const String & workload_name)
|
||||
{
|
||||
auto classifier = std::make_shared<Classifier>();
|
||||
|
||||
// Attach classifier to all resources in parallel (executed in every scheduler thread)
|
||||
std::vector<std::future<void>> 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)
|
||||
{
|
||||
// Copy resource to avoid holding mutex for a long time
|
||||
std::unordered_map<String, ResourcePtr> resources_copy;
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
resources_copy = resources;
|
||||
}
|
||||
|
||||
/// Run tasks one by one to avoid concurrent calls to visitor
|
||||
for (auto & [resource_name, resource] : resources_copy)
|
||||
resource->forEachResourceNode(visitor);
|
||||
}
|
||||
|
||||
void IOResourceManager::topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set<Workload *> & visited, std::vector<Workload *> & 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::Workload *> IOResourceManager::topologicallySortedWorkloads()
|
||||
{
|
||||
std::vector<Workload *> sorted_workloads;
|
||||
std::unordered_set<Workload *> visited;
|
||||
for (auto & [workload_name, workload] : workloads)
|
||||
topologicallySortedWorkloadsImpl(workload.get(), visited, sorted_workloads);
|
||||
return sorted_workloads;
|
||||
}
|
||||
|
||||
}
|
@ -1,281 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <Common/Logger.h>
|
||||
#include <Common/Scheduler/SchedulingSettings.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/Nodes/UnifiedSchedulerNode.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include <boost/core/noncopyable.hpp>
|
||||
|
||||
#include <exception>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <future>
|
||||
#include <unordered_set>
|
||||
|
||||
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 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.
|
||||
*
|
||||
* 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 hierarchy 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/destroy constraints, thus resource requests
|
||||
* created by old classifier may reference constraints of newer versions through `request->constraints` which
|
||||
* is filled during dequeueRequest().
|
||||
*
|
||||
* === 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_);
|
||||
~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;
|
||||
|
||||
private:
|
||||
// Forward declarations
|
||||
struct NodeInfo;
|
||||
struct Version;
|
||||
class Resource;
|
||||
struct Workload;
|
||||
class Classifier;
|
||||
|
||||
friend struct Workload;
|
||||
|
||||
using VersionPtr = std::shared_ptr<Version>;
|
||||
using ResourcePtr = std::shared_ptr<Resource>;
|
||||
using WorkloadPtr = std::shared_ptr<Workload>;
|
||||
|
||||
/// 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<SchedulerNodePtr> nodes;
|
||||
VersionPtr newer_version;
|
||||
};
|
||||
|
||||
/// Holds a thread and hierarchy of unified scheduler nodes for specific RESOURCE
|
||||
class Resource : public std::enable_shared_from_this<Resource>, 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 resource entity
|
||||
void updateResource(const ASTPtr & new_resource_entity);
|
||||
|
||||
/// Updates a classifier to contain a reference for specified workload
|
||||
std::future<void> attachClassifier(Classifier & classifier, const String & workload_name);
|
||||
|
||||
/// Remove classifier reference. This destroys scheduler nodes in proper scheduler thread
|
||||
std::future<void> detachClassifier(VersionPtr && version);
|
||||
|
||||
/// Introspection
|
||||
void forEachResourceNode(IOResourceManager::VisitorFunc & visitor);
|
||||
|
||||
private:
|
||||
void updateCurrentVersion();
|
||||
|
||||
template <class Task>
|
||||
void executeInSchedulerThread(Task && task)
|
||||
{
|
||||
std::promise<void> 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
|
||||
}
|
||||
|
||||
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<String, UnifiedSchedulerNodePtr> 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)
|
||||
bool has(const String & resource_name) override;
|
||||
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<String, Attachment> attachments; // TSA_GUARDED_BY(mutex);
|
||||
};
|
||||
|
||||
void createOrUpdateWorkload(const String & workload_name, const ASTPtr & ast);
|
||||
void deleteWorkload(const String & workload_name);
|
||||
void createOrUpdateResource(const String & resource_name, const ASTPtr & ast);
|
||||
void deleteResource(const String & resource_name);
|
||||
|
||||
// Topological sorting of workloads
|
||||
void topologicallySortedWorkloadsImpl(Workload * workload, std::unordered_set<Workload *> & visited, std::vector<Workload *> & sorted_workloads);
|
||||
std::vector<Workload *> topologicallySortedWorkloads();
|
||||
|
||||
IWorkloadEntityStorage & storage;
|
||||
scope_guard subscription;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
std::unordered_map<String, WorkloadPtr> workloads; // TSA_GUARDED_BY(mutex);
|
||||
std::unordered_map<String, ResourcePtr> resources; // TSA_GUARDED_BY(mutex);
|
||||
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
||||
* Scheduler node that implements priority scheduling policy.
|
||||
* Requests are scheduled in order of priorities.
|
||||
*/
|
||||
class PriorityPolicy final : public ISchedulerNode
|
||||
class PriorityPolicy : public ISchedulerNode
|
||||
{
|
||||
/// Scheduling state of a child
|
||||
struct Item
|
||||
@ -39,23 +39,6 @@ public:
|
||||
: ISchedulerNode(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
explicit PriorityPolicy(EventQueue * event_queue_, const SchedulerNodeInfo & node_info)
|
||||
: 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");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
{
|
||||
if (!ISchedulerNode::equals(other))
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include "Common/Scheduler/ISchedulerNode.h"
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <mutex>
|
||||
@ -14,7 +13,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 final : public ISchedulerConstraint
|
||||
class SemaphoreConstraint : public ISchedulerConstraint
|
||||
{
|
||||
static constexpr Int64 default_max_requests = std::numeric_limits<Int64>::max();
|
||||
static constexpr Int64 default_max_cost = std::numeric_limits<Int64>::max();
|
||||
@ -25,25 +24,6 @@ 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_)
|
||||
{}
|
||||
|
||||
~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");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
{
|
||||
if (!ISchedulerNode::equals(other))
|
||||
@ -88,14 +68,15 @@ public:
|
||||
if (!request)
|
||||
return {nullptr, false};
|
||||
|
||||
std::unique_lock lock(mutex);
|
||||
if (request->addConstraint(this))
|
||||
{
|
||||
// Update state on request arrival
|
||||
requests++;
|
||||
cost += request->cost;
|
||||
}
|
||||
// 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;
|
||||
|
||||
// Update state on request arrival
|
||||
std::unique_lock lock(mutex);
|
||||
requests++;
|
||||
cost += request->cost;
|
||||
child_active = child_now_active;
|
||||
if (!active())
|
||||
busy_periods++;
|
||||
@ -105,6 +86,10 @@ 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();
|
||||
@ -124,32 +109,6 @@ 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);
|
||||
@ -191,10 +150,10 @@ private:
|
||||
return satisfied() && child_active;
|
||||
}
|
||||
|
||||
Int64 max_requests = default_max_requests;
|
||||
Int64 max_cost = default_max_cost;
|
||||
const Int64 max_requests = default_max_requests;
|
||||
const Int64 max_cost = default_max_cost;
|
||||
|
||||
std::recursive_mutex mutex;
|
||||
std::mutex mutex;
|
||||
Int64 requests = 0;
|
||||
Int64 cost = 0;
|
||||
bool child_active = false;
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <mutex>
|
||||
#include <limits>
|
||||
#include <utility>
|
||||
|
||||
|
||||
@ -13,7 +15,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 final : public ISchedulerConstraint
|
||||
class ThrottlerConstraint : public ISchedulerConstraint
|
||||
{
|
||||
public:
|
||||
static constexpr double default_burst_seconds = 1.0;
|
||||
@ -26,28 +28,10 @@ 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
|
||||
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
|
||||
{
|
||||
static String type_name("bandwidth_limit");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
@ -94,7 +78,10 @@ public:
|
||||
if (!request)
|
||||
return {nullptr, false};
|
||||
|
||||
// We don't do `request->addConstraint(this)` because `finishRequest()` is no-op
|
||||
// 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;
|
||||
|
||||
updateBucket(request->cost);
|
||||
|
||||
@ -105,8 +92,12 @@ public:
|
||||
return {request, active()};
|
||||
}
|
||||
|
||||
void finishRequest(ResourceRequest *) override
|
||||
void finishRequest(ResourceRequest * request) 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
|
||||
}
|
||||
|
||||
@ -117,21 +108,6 @@ 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();
|
||||
@ -174,7 +150,7 @@ private:
|
||||
parent->activateChild(this);
|
||||
}
|
||||
|
||||
void updateBucket(ResourceCost use = 0, bool do_not_postpone = false)
|
||||
void updateBucket(ResourceCost use = 0)
|
||||
{
|
||||
auto now = event_queue->now();
|
||||
if (max_speed > 0.0)
|
||||
@ -184,7 +160,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 (!do_not_postpone && tokens < 0.0)
|
||||
if (tokens < 0.0)
|
||||
{
|
||||
auto delay_ns = std::chrono::nanoseconds(static_cast<Int64>(-tokens / max_speed * 1e9));
|
||||
if (postponed == EventQueue::not_postponed)
|
||||
@ -208,8 +184,8 @@ private:
|
||||
return satisfied() && child_active;
|
||||
}
|
||||
|
||||
double max_speed{0}; /// in tokens per second
|
||||
double max_burst{0}; /// in tokens
|
||||
const double max_speed{0}; /// in tokens per second
|
||||
const double max_burst{0}; /// in tokens
|
||||
|
||||
EventQueue::TimePoint last_update;
|
||||
UInt64 postponed = EventQueue::not_postponed;
|
||||
|
@ -1,606 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Priority.h>
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/ThrottlerConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/SchedulingSettings.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_SCHEDULER_NODE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class UnifiedSchedulerNode;
|
||||
using UnifiedSchedulerNodePtr = std::shared_ptr<UnifiedSchedulerNode>;
|
||||
|
||||
/*
|
||||
* 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 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).
|
||||
* 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 final : 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)
|
||||
{
|
||||
chassert(node);
|
||||
chassert(new_parent);
|
||||
if (new_parent == node->parent)
|
||||
return;
|
||||
if (node->parent)
|
||||
node->parent->removeChild(node.get());
|
||||
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
|
||||
std::unordered_map<String, UnifiedSchedulerNodePtr> children; // basename -> child
|
||||
|
||||
bool empty() const { return children.empty(); }
|
||||
|
||||
SchedulerNodePtr getRoot()
|
||||
{
|
||||
chassert(!children.empty());
|
||||
if (root)
|
||||
return root;
|
||||
chassert(children.size() == 1);
|
||||
return children.begin()->second;
|
||||
}
|
||||
|
||||
/// 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)
|
||||
{
|
||||
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<FairPolicy>(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
|
||||
}
|
||||
|
||||
/// 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
|
||||
|
||||
detach(child);
|
||||
children.erase(it);
|
||||
if (children.size() == 1)
|
||||
{
|
||||
// Remove fair if the only child has left
|
||||
chassert(root);
|
||||
detach(root);
|
||||
root.reset();
|
||||
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
|
||||
struct ChildrenBranch
|
||||
{
|
||||
SchedulerNodePtr root; /// PriorityPolicy node is used if multiple children with different priority are attached
|
||||
std::unordered_map<Priority::Value, FairnessBranch> branches; /// Branches for different priority values
|
||||
|
||||
// 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)
|
||||
{
|
||||
auto [it, new_branch] = branches.try_emplace(child->info.priority);
|
||||
auto & child_branch = it->second;
|
||||
auto branch_root = child_branch.attachUnifiedChild(event_queue_, child);
|
||||
if (!new_branch)
|
||||
{
|
||||
if (branch_root)
|
||||
{
|
||||
if (root)
|
||||
reparent(branch_root, root);
|
||||
else
|
||||
return branch_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<PriorityPolicy>(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
|
||||
}
|
||||
}
|
||||
|
||||
/// 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);
|
||||
detach(root);
|
||||
root.reset();
|
||||
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`.
|
||||
struct QueueOrChildrenBranch
|
||||
{
|
||||
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_)
|
||||
{
|
||||
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 UnifiedSchedulerNodePtr & child)
|
||||
{
|
||||
if (queue)
|
||||
removeQueue();
|
||||
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_)
|
||||
{
|
||||
queue = std::make_shared<FifoQueue>(event_queue_, SchedulerNodeInfo{});
|
||||
queue->basename = "fifo";
|
||||
}
|
||||
|
||||
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
|
||||
detach(queue);
|
||||
std::static_pointer_cast<ISchedulerQueue>(queue)->purgeQueue();
|
||||
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<SemaphoreConstraint>(event_queue_, SchedulerNodeInfo{}, settings.max_requests, settings.max_cost);
|
||||
semaphore->basename = "semaphore";
|
||||
reparent(node, semaphore);
|
||||
node = semaphore;
|
||||
}
|
||||
if (settings.hasThrottler())
|
||||
{
|
||||
throttler = std::make_shared<ThrottlerConstraint>(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 UnifiedSchedulerNodePtr & child)
|
||||
{
|
||||
if (auto branch_root = branch.attachUnifiedChild(event_queue_, child))
|
||||
{
|
||||
// If both semaphore and throttler exist we should reparent to the farthest from the root
|
||||
if (semaphore)
|
||||
reparent(branch_root, semaphore);
|
||||
else if (throttler)
|
||||
reparent(branch_root, throttler);
|
||||
else
|
||||
return branch_root;
|
||||
}
|
||||
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 {};
|
||||
}
|
||||
|
||||
/// Updates constraint-related nodes.
|
||||
/// 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<SemaphoreConstraint>(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<SemaphoreConstraint&>(*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<ThrottlerConstraint>(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<ThrottlerConstraint&>(*throttler).updateConstraints(new_settings.max_speed, new_settings.max_burst);
|
||||
node = throttler;
|
||||
}
|
||||
|
||||
settings = new_settings;
|
||||
return node;
|
||||
}
|
||||
};
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
~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)
|
||||
{
|
||||
if (auto new_child = impl.attachUnifiedChild(event_queue, child))
|
||||
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)
|
||||
{
|
||||
if (auto new_child = impl.detachUnifiedChild(event_queue, child))
|
||||
reparent(new_child, this);
|
||||
}
|
||||
|
||||
static bool updateRequiresDetach(const String & old_parent, const String & new_parent, const SchedulingSettings & old_settings, const SchedulingSettings & new_settings)
|
||||
{
|
||||
return old_parent != new_parent || old_settings.priority != new_settings.priority;
|
||||
}
|
||||
|
||||
/// Updates scheduling settings. Set of constraints might change.
|
||||
/// NOTE: Caller is responsible for detaching and attaching if `updateRequiresDetach` returns true
|
||||
void updateSchedulingSettings(const SchedulingSettings & new_settings)
|
||||
{
|
||||
info.setPriority(new_settings.priority);
|
||||
info.setWeight(new_settings.weight);
|
||||
if (auto new_child = impl.updateSchedulingSettings(event_queue, new_settings))
|
||||
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<ISchedulerQueue> getQueue() const
|
||||
{
|
||||
return static_pointer_cast<ISchedulerQueue>(impl.branch.queue);
|
||||
}
|
||||
|
||||
/// 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, 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<SchedulerNodePtr> & nodes)
|
||||
{
|
||||
// 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)
|
||||
nodes.push_back(impl.branch.queue);
|
||||
for (auto & [_, branch] : impl.branch.branch.branches)
|
||||
{
|
||||
for (auto & [_, child] : branch.children)
|
||||
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<void(ISchedulerNode *)> 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());
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "UnifiedSchedulerNode should not be used with CustomResourceManager");
|
||||
}
|
||||
|
||||
/// 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<ResourceRequest *, bool> 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;
|
||||
};
|
||||
|
||||
}
|
15
src/Common/Scheduler/Nodes/registerResourceManagers.cpp
Normal file
15
src/Common/Scheduler/Nodes/registerResourceManagers.cpp
Normal file
@ -0,0 +1,15 @@
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDynamicResourceManager(ResourceManagerFactory &);
|
||||
|
||||
void registerResourceManagers()
|
||||
{
|
||||
auto & factory = ResourceManagerFactory::instance();
|
||||
registerDynamicResourceManager(factory);
|
||||
}
|
||||
|
||||
}
|
8
src/Common/Scheduler/Nodes/registerResourceManagers.h
Normal file
8
src/Common/Scheduler/Nodes/registerResourceManagers.h
Normal file
@ -0,0 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerResourceManagers();
|
||||
|
||||
}
|
@ -1,8 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Common/Scheduler/SchedulingSettings.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
@ -10,35 +7,26 @@
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/UnifiedSchedulerNode.h>
|
||||
#include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <barrier>
|
||||
#include <exception>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
#include <set>
|
||||
#include <sstream>
|
||||
#include <utility>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int RESOURCE_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
struct ResourceTestBase
|
||||
{
|
||||
ResourceTestBase()
|
||||
{
|
||||
[[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); return true; }();
|
||||
[[maybe_unused]] static bool typesRegistered = [] { registerSchedulerNodes(); registerResourceManagers(); return true; }();
|
||||
}
|
||||
|
||||
template <class TClass>
|
||||
@ -49,16 +37,10 @@ struct ResourceTestBase
|
||||
Poco::AutoPtr config{new Poco::Util::XMLConfiguration(stream)};
|
||||
String config_prefix = "node";
|
||||
|
||||
return add<TClass>(event_queue, root_node, path, std::ref(*config), config_prefix);
|
||||
}
|
||||
|
||||
template <class TClass, class... Args>
|
||||
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, std::forward<Args>(args)...));
|
||||
root_node.reset(new TClass(event_queue, *config, config_prefix));
|
||||
return static_cast<TClass *>(root_node.get());
|
||||
}
|
||||
|
||||
@ -83,114 +65,73 @@ struct ResourceTestBase
|
||||
}
|
||||
|
||||
EXPECT_TRUE(!child_name.empty()); // wrong path
|
||||
SchedulerNodePtr node = std::make_shared<TClass>(event_queue, std::forward<Args>(args)...);
|
||||
SchedulerNodePtr node = std::make_shared<TClass>(event_queue, *config, config_prefix);
|
||||
node->basename = child_name;
|
||||
parent->attachChild(node);
|
||||
return static_cast<TClass *>(node.get());
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct ConstraintTest : public SemaphoreConstraint
|
||||
{
|
||||
explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
: SemaphoreConstraint(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
std::pair<ResourceRequest *, bool> dequeueRequest() override
|
||||
{
|
||||
auto [request, active] = SemaphoreConstraint::dequeueRequest();
|
||||
if (request)
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
requests.insert(request);
|
||||
}
|
||||
return {request, active};
|
||||
}
|
||||
|
||||
void finishRequest(ResourceRequest * request) override
|
||||
{
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
requests.erase(request);
|
||||
}
|
||||
SemaphoreConstraint::finishRequest(request);
|
||||
}
|
||||
|
||||
std::mutex mutex;
|
||||
std::set<ResourceRequest *> requests;
|
||||
};
|
||||
|
||||
class ResourceTestClass : public ResourceTestBase
|
||||
{
|
||||
struct Request : public ResourceRequest
|
||||
{
|
||||
ResourceTestClass * test;
|
||||
String name;
|
||||
|
||||
Request(ResourceTestClass * test_, ResourceCost cost_, const String & name_)
|
||||
Request(ResourceCost cost_, const String & name_)
|
||||
: ResourceRequest(cost_)
|
||||
, test(test_)
|
||||
, name(name_)
|
||||
{}
|
||||
|
||||
void execute() override
|
||||
{
|
||||
}
|
||||
|
||||
void failed(const std::exception_ptr &) override
|
||||
{
|
||||
test->failed_cost += cost;
|
||||
delete this;
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
~ResourceTestClass()
|
||||
{
|
||||
if (root_node)
|
||||
dequeue(); // Just to avoid any leaks of `Request` object
|
||||
}
|
||||
|
||||
template <class TClass>
|
||||
void add(const String & path, const String & xml = {})
|
||||
{
|
||||
ResourceTestBase::add<TClass>(&event_queue, root_node, path, xml);
|
||||
}
|
||||
|
||||
template <class TClass, class... Args>
|
||||
void addCustom(const String & path, Args... args)
|
||||
{
|
||||
ResourceTestBase::add<TClass>(&event_queue, root_node, path, std::forward<Args>(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<UnifiedSchedulerNode>(&event_queue, settings);
|
||||
node->basename = basename;
|
||||
if (parent)
|
||||
{
|
||||
parent->attachUnifiedChild(node);
|
||||
}
|
||||
else
|
||||
{
|
||||
EXPECT_TRUE(root_node.get() == nullptr);
|
||||
root_node = node;
|
||||
}
|
||||
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 ? old_parent->basename : "",
|
||||
new_parent ? 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<ResourceCost> & costs)
|
||||
{
|
||||
enqueueImpl(node->getQueue().get(), costs, node->basename);
|
||||
}
|
||||
|
||||
void enqueue(const String & path, const std::vector<ResourceCost> & costs)
|
||||
{
|
||||
ASSERT_TRUE(root_node.get() != nullptr); // root should be initialized first
|
||||
ISchedulerNode * node = root_node.get();
|
||||
size_t pos = 1;
|
||||
while (node && pos < path.length())
|
||||
while (pos < path.length())
|
||||
{
|
||||
size_t slash = path.find('/', pos);
|
||||
if (slash != String::npos)
|
||||
@ -205,17 +146,13 @@ public:
|
||||
pos = String::npos;
|
||||
}
|
||||
}
|
||||
if (node)
|
||||
enqueueImpl(dynamic_cast<ISchedulerQueue *>(node), costs);
|
||||
}
|
||||
|
||||
void enqueueImpl(ISchedulerQueue * queue, const std::vector<ResourceCost> & costs, const String & name = {})
|
||||
{
|
||||
ISchedulerQueue * queue = dynamic_cast<ISchedulerQueue *>(node);
|
||||
ASSERT_TRUE(queue != nullptr); // not a queue
|
||||
if (!queue)
|
||||
return; // to make clang-analyzer-core.NonNullParamChecker happy
|
||||
|
||||
for (ResourceCost cost : costs)
|
||||
queue->enqueueRequest(new Request(this, cost, name.empty() ? queue->basename : name));
|
||||
{
|
||||
queue->enqueueRequest(new Request(cost, queue->basename));
|
||||
}
|
||||
processEvents(); // to activate queues
|
||||
}
|
||||
|
||||
@ -271,12 +208,6 @@ public:
|
||||
consumed_cost[name] -= value;
|
||||
}
|
||||
|
||||
void failed(ResourceCost value)
|
||||
{
|
||||
EXPECT_EQ(failed_cost, value);
|
||||
failed_cost -= value;
|
||||
}
|
||||
|
||||
void processEvents()
|
||||
{
|
||||
while (event_queue.tryProcess()) {}
|
||||
@ -286,11 +217,8 @@ private:
|
||||
EventQueue event_queue;
|
||||
SchedulerNodePtr root_node;
|
||||
std::unordered_map<String, ResourceCost> consumed_cost;
|
||||
ResourceCost failed_cost = 0;
|
||||
};
|
||||
|
||||
enum EnqueueOnlyEnum { EnqueueOnly };
|
||||
|
||||
template <class TManager>
|
||||
struct ResourceTestManager : public ResourceTestBase
|
||||
{
|
||||
@ -302,49 +230,16 @@ struct ResourceTestManager : public ResourceTestBase
|
||||
struct Guard : public ResourceGuard
|
||||
{
|
||||
ResourceTestManager & t;
|
||||
ResourceCost cost;
|
||||
|
||||
/// Works like regular ResourceGuard, ready for consumption after constructor
|
||||
Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost_)
|
||||
: ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost_, Lock::Defer)
|
||||
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 (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_)
|
||||
, cost(cost_)
|
||||
{
|
||||
t.onEnqueue(link);
|
||||
}
|
||||
|
||||
/// Waits for ResourceRequest::execute() to be called for enqueued request
|
||||
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
|
||||
@ -369,24 +264,10 @@ struct ResourceTestManager : public ResourceTestBase
|
||||
, busy_period(thread_count)
|
||||
{}
|
||||
|
||||
enum DoNotInitManagerEnum { DoNotInitManager };
|
||||
|
||||
explicit ResourceTestManager(size_t thread_count, DoNotInitManagerEnum)
|
||||
: busy_period(thread_count)
|
||||
{}
|
||||
|
||||
~ResourceTestManager()
|
||||
{
|
||||
wait();
|
||||
}
|
||||
|
||||
void wait()
|
||||
{
|
||||
for (auto & thread : threads)
|
||||
{
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
}
|
||||
thread.join();
|
||||
}
|
||||
|
||||
void update(const String & xml)
|
||||
|
@ -2,15 +2,15 @@
|
||||
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <Common/Scheduler/Nodes/CustomResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/DynamicResourceManager.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestManager<CustomResourceManager>;
|
||||
using ResourceTest = ResourceTestManager<DynamicResourceManager>;
|
||||
using TestGuard = ResourceTest::Guard;
|
||||
|
||||
TEST(SchedulerCustomResourceManager, Smoke)
|
||||
TEST(SchedulerDynamicResourceManager, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -31,25 +31,25 @@ TEST(SchedulerCustomResourceManager, Smoke)
|
||||
</clickhouse>
|
||||
)CONFIG");
|
||||
|
||||
ClassifierPtr c_a = t.manager->acquire("A");
|
||||
ClassifierPtr c_b = t.manager->acquire("B");
|
||||
ClassifierPtr cA = t.manager->acquire("A");
|
||||
ClassifierPtr cB = 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 gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer);
|
||||
gA.lock();
|
||||
gA.consume(1);
|
||||
gA.unlock();
|
||||
|
||||
ResourceGuard g_b(ResourceGuard::Metrics::getIOWrite(), c_b->get("res1"));
|
||||
g_b.unlock();
|
||||
ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1"));
|
||||
gB.unlock();
|
||||
|
||||
ResourceGuard g_c(ResourceGuard::Metrics::getIORead(), c_b->get("res1"));
|
||||
g_b.consume(2);
|
||||
ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1"));
|
||||
gB.consume(2);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(SchedulerCustomResourceManager, Fairness)
|
||||
TEST(SchedulerDynamicResourceManager, 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.
|
@ -13,12 +13,6 @@ 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;
|
||||
|
@ -1,335 +0,0 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityStorageBase.h>
|
||||
#include <Common/Scheduler/Nodes/IOResourceManager.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/ASTDropWorkloadQuery.h>
|
||||
#include <Parsers/ASTDropResourceQuery.h>
|
||||
#include <Parsers/ParserCreateWorkloadQuery.h>
|
||||
#include <Parsers/ParserCreateResourceQuery.h>
|
||||
#include <Parsers/ParserDropWorkloadQuery.h>
|
||||
#include <Parsers/ParserDropResourceQuery.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
class WorkloadEntityTestStorage : public WorkloadEntityStorageBase
|
||||
{
|
||||
public:
|
||||
WorkloadEntityTestStorage()
|
||||
: WorkloadEntityStorageBase(Context::getGlobalContextInstance())
|
||||
{}
|
||||
|
||||
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<ASTCreateWorkloadQuery &>();
|
||||
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<ASTCreateResourceQuery &>();
|
||||
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<ASTDropWorkloadQuery &>();
|
||||
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<ASTDropResourceQuery &>();
|
||||
bool throw_if_not_exists = !parsed.if_exists;
|
||||
removeEntity(
|
||||
nullptr,
|
||||
WorkloadEntityType::Resource,
|
||||
parsed.resource_name,
|
||||
throw_if_not_exists);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query in WorkloadEntityTestStorage: {}", query);
|
||||
}
|
||||
|
||||
private:
|
||||
WorkloadEntityStorageBase::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
|
||||
{
|
||||
UNUSED(current_context, entity_type, entity_name, create_entity_query, throw_if_exists, replace_if_exists, settings);
|
||||
return OperationResult::Ok;
|
||||
}
|
||||
|
||||
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 OperationResult::Ok;
|
||||
}
|
||||
};
|
||||
|
||||
struct ResourceTest : ResourceTestManager<IOResourceManager>
|
||||
{
|
||||
WorkloadEntityTestStorage storage;
|
||||
|
||||
explicit ResourceTest(size_t thread_count = 1)
|
||||
: ResourceTestManager(thread_count, DoNotInitManager)
|
||||
{
|
||||
manager = std::make_shared<IOResourceManager>(storage);
|
||||
}
|
||||
|
||||
void query(const String & query_str)
|
||||
{
|
||||
storage.executeQuery(query_str);
|
||||
}
|
||||
|
||||
template <class Func>
|
||||
void async(const String & workload, Func func)
|
||||
{
|
||||
threads.emplace_back([=, this, func2 = std::move(func)]
|
||||
{
|
||||
ClassifierPtr classifier = manager->acquire(workload);
|
||||
func2(classifier);
|
||||
});
|
||||
}
|
||||
|
||||
template <class Func>
|
||||
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;
|
||||
|
||||
TEST(SchedulerIOResourceManager, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
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");
|
||||
|
||||
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<Int64> 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 (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");
|
||||
|
||||
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);
|
||||
|
||||
t.wait(); // Wait for threads to finish before destructing locals
|
||||
}
|
||||
|
||||
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();
|
||||
|
||||
t.wait(); // Wait for threads to finish before destructing locals
|
||||
}
|
||||
|
||||
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();
|
||||
|
||||
t.wait(); // Wait for threads to finish before destructing locals
|
||||
}
|
@ -8,17 +8,18 @@ using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(SchedulerFairPolicy, Factory)
|
||||
/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678
|
||||
|
||||
TEST(DISABLED_SchedulerFairPolicy, Factory)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration();
|
||||
EventQueue event_queue;
|
||||
SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", &event_queue, *cfg, "");
|
||||
SchedulerNodePtr fair = SchedulerNodeFactory::instance().get("fair", /* event_queue = */ nullptr, *cfg, "");
|
||||
EXPECT_TRUE(dynamic_cast<FairPolicy *>(fair.get()) != nullptr);
|
||||
}
|
||||
|
||||
TEST(SchedulerFairPolicy, FairnessWeights)
|
||||
TEST(DISABLED_SchedulerFairPolicy, FairnessWeights)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -42,7 +43,7 @@ TEST(SchedulerFairPolicy, FairnessWeights)
|
||||
t.consumed("B", 20);
|
||||
}
|
||||
|
||||
TEST(SchedulerFairPolicy, Activation)
|
||||
TEST(DISABLED_SchedulerFairPolicy, Activation)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -78,7 +79,7 @@ TEST(SchedulerFairPolicy, Activation)
|
||||
t.consumed("B", 10);
|
||||
}
|
||||
|
||||
TEST(SchedulerFairPolicy, FairnessMaxMin)
|
||||
TEST(DISABLED_SchedulerFairPolicy, FairnessMaxMin)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -102,7 +103,7 @@ TEST(SchedulerFairPolicy, FairnessMaxMin)
|
||||
t.consumed("A", 20);
|
||||
}
|
||||
|
||||
TEST(SchedulerFairPolicy, HierarchicalFairness)
|
||||
TEST(DISABLED_SchedulerFairPolicy, HierarchicalFairness)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
|
@ -8,17 +8,18 @@ using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(SchedulerPriorityPolicy, Factory)
|
||||
/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678
|
||||
|
||||
TEST(DISABLED_SchedulerPriorityPolicy, Factory)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
Poco::AutoPtr cfg = new Poco::Util::XMLConfiguration();
|
||||
EventQueue event_queue;
|
||||
SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", &event_queue, *cfg, "");
|
||||
SchedulerNodePtr prio = SchedulerNodeFactory::instance().get("priority", /* event_queue = */ nullptr, *cfg, "");
|
||||
EXPECT_TRUE(dynamic_cast<PriorityPolicy *>(prio.get()) != nullptr);
|
||||
}
|
||||
|
||||
TEST(SchedulerPriorityPolicy, Priorities)
|
||||
TEST(DISABLED_SchedulerPriorityPolicy, Priorities)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -52,7 +53,7 @@ TEST(SchedulerPriorityPolicy, Priorities)
|
||||
t.consumed("C", 0);
|
||||
}
|
||||
|
||||
TEST(SchedulerPriorityPolicy, Activation)
|
||||
TEST(DISABLED_SchedulerPriorityPolicy, Activation)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -93,7 +94,7 @@ TEST(SchedulerPriorityPolicy, Activation)
|
||||
t.consumed("C", 0);
|
||||
}
|
||||
|
||||
TEST(SchedulerPriorityPolicy, SinglePriority)
|
||||
TEST(DISABLED_SchedulerPriorityPolicy, SinglePriority)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
@ -102,11 +101,6 @@ struct MyRequest : public ResourceRequest
|
||||
if (on_execute)
|
||||
on_execute();
|
||||
}
|
||||
|
||||
void failed(const std::exception_ptr &) override
|
||||
{
|
||||
FAIL();
|
||||
}
|
||||
};
|
||||
|
||||
TEST(SchedulerRoot, Smoke)
|
||||
@ -114,14 +108,14 @@ TEST(SchedulerRoot, Smoke)
|
||||
ResourceTest t;
|
||||
|
||||
ResourceHolder r1(t);
|
||||
auto * fc1 = r1.add<SemaphoreConstraint>("/", "<max_requests>1</max_requests>");
|
||||
auto * fc1 = r1.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
|
||||
r1.add<PriorityPolicy>("/prio");
|
||||
auto a = r1.addQueue("/prio/A", "<priority>1</priority>");
|
||||
auto b = r1.addQueue("/prio/B", "<priority>2</priority>");
|
||||
r1.registerResource();
|
||||
|
||||
ResourceHolder r2(t);
|
||||
auto * fc2 = r2.add<SemaphoreConstraint>("/", "<max_requests>1</max_requests>");
|
||||
auto * fc2 = r2.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
|
||||
r2.add<PriorityPolicy>("/prio");
|
||||
auto c = r2.addQueue("/prio/C", "<priority>-1</priority>");
|
||||
auto d = r2.addQueue("/prio/D", "<priority>-2</priority>");
|
||||
@ -129,25 +123,25 @@ TEST(SchedulerRoot, Smoke)
|
||||
|
||||
{
|
||||
ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a);
|
||||
EXPECT_TRUE(fc1->getInflights().first == 1);
|
||||
EXPECT_TRUE(fc1->requests.contains(&rg.request));
|
||||
rg.consume(1);
|
||||
}
|
||||
|
||||
{
|
||||
ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b);
|
||||
EXPECT_TRUE(fc1->getInflights().first == 1);
|
||||
EXPECT_TRUE(fc1->requests.contains(&rg.request));
|
||||
rg.consume(1);
|
||||
}
|
||||
|
||||
{
|
||||
ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c);
|
||||
EXPECT_TRUE(fc2->getInflights().first == 1);
|
||||
EXPECT_TRUE(fc2->requests.contains(&rg.request));
|
||||
rg.consume(1);
|
||||
}
|
||||
|
||||
{
|
||||
ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d);
|
||||
EXPECT_TRUE(fc2->getInflights().first == 1);
|
||||
EXPECT_TRUE(fc2->requests.contains(&rg.request));
|
||||
rg.consume(1);
|
||||
}
|
||||
}
|
||||
@ -157,7 +151,7 @@ TEST(SchedulerRoot, Budget)
|
||||
ResourceTest t;
|
||||
|
||||
ResourceHolder r1(t);
|
||||
r1.add<SemaphoreConstraint>("/", "<max_requests>1</max_requests>");
|
||||
r1.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
|
||||
r1.add<PriorityPolicy>("/prio");
|
||||
auto a = r1.addQueue("/prio/A", "");
|
||||
r1.registerResource();
|
||||
@ -182,7 +176,7 @@ TEST(SchedulerRoot, Cancel)
|
||||
ResourceTest t;
|
||||
|
||||
ResourceHolder r1(t);
|
||||
auto * fc1 = r1.add<SemaphoreConstraint>("/", "<max_requests>1</max_requests>");
|
||||
auto * fc1 = r1.add<ConstraintTest>("/", "<max_requests>1</max_requests>");
|
||||
r1.add<PriorityPolicy>("/prio");
|
||||
auto a = r1.addQueue("/prio/A", "<priority>1</priority>");
|
||||
auto b = r1.addQueue("/prio/B", "<priority>2</priority>");
|
||||
@ -195,7 +189,7 @@ TEST(SchedulerRoot, Cancel)
|
||||
MyRequest request(1,[&]
|
||||
{
|
||||
sync.arrive_and_wait(); // (A)
|
||||
EXPECT_TRUE(fc1->getInflights().first == 1);
|
||||
EXPECT_TRUE(fc1->requests.contains(&request));
|
||||
sync.arrive_and_wait(); // (B)
|
||||
request.finish();
|
||||
destruct_sync.arrive_and_wait(); // (C)
|
||||
@ -220,5 +214,5 @@ TEST(SchedulerRoot, Cancel)
|
||||
consumer1.join();
|
||||
consumer2.join();
|
||||
|
||||
EXPECT_TRUE(fc1->getInflights().first == 0);
|
||||
EXPECT_TRUE(fc1->requests.empty());
|
||||
}
|
||||
|
@ -10,7 +10,9 @@ using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint)
|
||||
/// Tests disabled because of leaks in the test themselves: https://github.com/ClickHouse/ClickHouse/issues/67678
|
||||
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, LeakyBucketConstraint)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -40,7 +42,7 @@ TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint)
|
||||
t.consumed("A", 10);
|
||||
}
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, Unlimited)
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, Unlimited)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -57,7 +59,7 @@ TEST(SchedulerThrottlerConstraint, Unlimited)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, Pacing)
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, Pacing)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -77,7 +79,7 @@ TEST(SchedulerThrottlerConstraint, Pacing)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, BucketFilling)
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, BucketFilling)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -111,7 +113,7 @@ TEST(SchedulerThrottlerConstraint, BucketFilling)
|
||||
t.consumed("A", 3);
|
||||
}
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits)
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, PeekAndAvgLimits)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -139,7 +141,7 @@ TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness)
|
||||
TEST(DISABLED_SchedulerThrottlerConstraint, ThrottlerAndFairness)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -158,22 +160,22 @@ TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness)
|
||||
t.enqueue("/fair/B", {req_cost});
|
||||
}
|
||||
|
||||
double share_a = 0.1;
|
||||
double share_b = 0.9;
|
||||
double shareA = 0.1;
|
||||
double shareB = 0.9;
|
||||
|
||||
// Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share
|
||||
auto max_latency_a = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / share_a));
|
||||
auto max_latency_b = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / share_b));
|
||||
auto max_latencyA = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / shareA));
|
||||
auto max_latencyB = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / shareB));
|
||||
|
||||
double consumed_a = 0;
|
||||
double consumed_b = 0;
|
||||
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<ResourceCost>(arrival_curve * share_a - consumed_a), max_latency_a);
|
||||
t.consumed("B", static_cast<ResourceCost>(arrival_curve * share_b - consumed_b), max_latency_b);
|
||||
consumed_a = arrival_curve * share_a;
|
||||
consumed_b = arrival_curve * share_b;
|
||||
t.consumed("A", static_cast<ResourceCost>(arrival_curve * shareA - consumedA), max_latencyA);
|
||||
t.consumed("B", static_cast<ResourceCost>(arrival_curve * shareB - consumedB), max_latencyB);
|
||||
consumedA = arrival_curve * shareA;
|
||||
consumedB = arrival_curve * shareB;
|
||||
}
|
||||
}
|
||||
|
@ -1,748 +0,0 @@
|
||||
#include <chrono>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <Common/Priority.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/UnifiedSchedulerNode.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(SchedulerUnifiedNode, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
t.addCustom<UnifiedSchedulerNode>("/", SchedulingSettings{});
|
||||
|
||||
t.enqueue("/fifo", {10, 10});
|
||||
t.dequeue(2);
|
||||
t.consumed("fifo", 20);
|
||||
}
|
||||
|
||||
TEST(SchedulerUnifiedNode, FairnessWeight)
|
||||
{
|
||||
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, 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;
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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<UnifiedSchedulerNodePtr> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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 share_a = 0.1;
|
||||
double share_b = 0.9;
|
||||
|
||||
// Bandwidth-latency coupling due to fairness: worst latency is inversely proportional to share
|
||||
auto max_latency_a = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / share_a));
|
||||
auto max_latency_b = static_cast<ResourceCost>(req_cost * (1.0 + 1.0 / share_b));
|
||||
|
||||
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<ResourceCost>(arrival_curve * share_a - consumed_a), max_latency_a);
|
||||
t.consumed("B", static_cast<ResourceCost>(arrival_curve * share_b - consumed_b), max_latency_b);
|
||||
consumed_a = arrival_curve * share_a;
|
||||
consumed_b = arrival_curve * share_b;
|
||||
}
|
||||
}
|
||||
|
||||
TEST(SchedulerUnifiedNode, QueueWithRequestsDestruction)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
auto all = t.createUnifiedNode("all");
|
||||
|
||||
t.enqueue(all, {10, 10}); // enqueue reqeuests to be canceled
|
||||
|
||||
// This will destroy 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);
|
||||
}
|
||||
|
||||
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()};
|
||||
bool caught = false;
|
||||
try
|
||||
{
|
||||
ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), link);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
caught = true;
|
||||
}
|
||||
ASSERT_TRUE(caught);
|
||||
});
|
||||
|
||||
// This will destroy 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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
@ -12,7 +12,6 @@
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
#include <condition_variable>
|
||||
#include <exception>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
@ -35,11 +34,6 @@ 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
|
||||
@ -115,25 +109,12 @@ public:
|
||||
dequeued_cv.notify_one();
|
||||
}
|
||||
|
||||
// This function is executed inside scheduler thread and wakes thread that 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_)
|
||||
@ -170,7 +151,6 @@ 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`)
|
||||
|
55
src/Common/Scheduler/ResourceManagerFactory.h
Normal file
55
src/Common/Scheduler/ResourceManagerFactory.h
Normal file
@ -0,0 +1,55 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
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 <class TDerived>
|
||||
void registerMethod(const String & name)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
methods[name] = [] ()
|
||||
{
|
||||
return std::make_shared<TDerived>();
|
||||
};
|
||||
}
|
||||
|
||||
private:
|
||||
std::mutex mutex;
|
||||
using Method = std::function<ResourceManagerPtr()>;
|
||||
std::unordered_map<String, Method> methods;
|
||||
};
|
||||
|
||||
}
|
@ -1,34 +1,13 @@
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <ranges>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void ResourceRequest::finish()
|
||||
{
|
||||
// Iterate over constraints in reverse order
|
||||
for (ISchedulerConstraint * constraint : std::ranges::reverse_view(constraints))
|
||||
{
|
||||
if (constraint)
|
||||
constraint->finishRequest(this);
|
||||
}
|
||||
}
|
||||
|
||||
bool ResourceRequest::addConstraint(ISchedulerConstraint * new_constraint)
|
||||
{
|
||||
for (auto & constraint : constraints)
|
||||
{
|
||||
if (!constraint)
|
||||
{
|
||||
constraint = new_constraint;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
if (constraint)
|
||||
constraint->finishRequest(this);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,9 +2,7 @@
|
||||
|
||||
#include <boost/intrusive/list.hpp>
|
||||
#include <base/types.h>
|
||||
#include <array>
|
||||
#include <limits>
|
||||
#include <exception>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -17,9 +15,6 @@ class ISchedulerConstraint;
|
||||
using ResourceCost = Int64;
|
||||
constexpr ResourceCost ResourceCostMax = std::numeric_limits<int>::max();
|
||||
|
||||
/// Max number of constraints for a request to pass though (depth of constraints chain)
|
||||
constexpr size_t ResourceMaxConstraints = 8;
|
||||
|
||||
/*
|
||||
* Request for a resource consumption. The main moving part of the scheduling subsystem.
|
||||
* Resource requests processing workflow:
|
||||
@ -44,7 +39,8 @@ 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.
|
||||
* Returning true means successful cancel and therefore steps (4) and (5) are not going to happen
|
||||
* and step (6) MUST be omitted.
|
||||
*/
|
||||
class ResourceRequest : public boost::intrusive::list_base_hook<>
|
||||
{
|
||||
@ -53,10 +49,9 @@ public:
|
||||
/// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it)
|
||||
ResourceCost cost;
|
||||
|
||||
/// 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<ISchedulerConstraint *, ResourceMaxConstraints> constraints;
|
||||
/// Scheduler node to be notified on consumption finish
|
||||
/// Auto-filled during request enqueue/dequeue
|
||||
ISchedulerConstraint * constraint;
|
||||
|
||||
explicit ResourceRequest(ResourceCost cost_ = 1)
|
||||
{
|
||||
@ -67,8 +62,7 @@ public:
|
||||
void reset(ResourceCost cost_)
|
||||
{
|
||||
cost = cost_;
|
||||
for (auto & constraint : constraints)
|
||||
constraint = nullptr;
|
||||
constraint = nullptr;
|
||||
// Note that list_base_hook should be reset independently (by intrusive list)
|
||||
}
|
||||
|
||||
@ -80,18 +74,11 @@ 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()`.
|
||||
/// It is okay to call finish() even for failed and canceled requests (it will be no-op)
|
||||
/// WARNING: this function MUST not be called if request was canceled.
|
||||
void finish();
|
||||
|
||||
/// Is called from the scheduler thread to fill `constraints` chain
|
||||
/// Returns `true` iff constraint was added successfully
|
||||
bool addConstraint(ISchedulerConstraint * new_constraint);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,27 +28,27 @@ namespace ErrorCodes
|
||||
* Resource scheduler root node with a dedicated thread.
|
||||
* Immediate children correspond to different resources.
|
||||
*/
|
||||
class SchedulerRoot final : public ISchedulerNode
|
||||
class SchedulerRoot : public ISchedulerNode
|
||||
{
|
||||
private:
|
||||
struct Resource
|
||||
struct TResource
|
||||
{
|
||||
SchedulerNodePtr root;
|
||||
|
||||
// Intrusive cyclic list of active resources
|
||||
Resource * next = nullptr;
|
||||
Resource * prev = nullptr;
|
||||
TResource * next = nullptr;
|
||||
TResource * prev = nullptr;
|
||||
|
||||
explicit Resource(const SchedulerNodePtr & root_)
|
||||
explicit TResource(const SchedulerNodePtr & root_)
|
||||
: root(root_)
|
||||
{
|
||||
root->info.parent.ptr = this;
|
||||
}
|
||||
|
||||
// Get pointer stored by ctor in info
|
||||
static Resource * get(SchedulerNodeInfo & info)
|
||||
static TResource * get(SchedulerNodeInfo & info)
|
||||
{
|
||||
return reinterpret_cast<Resource *>(info.parent.ptr);
|
||||
return reinterpret_cast<TResource *>(info.parent.ptr);
|
||||
}
|
||||
};
|
||||
|
||||
@ -60,8 +60,6 @@ public:
|
||||
~SchedulerRoot() override
|
||||
{
|
||||
stop();
|
||||
while (!children.empty())
|
||||
removeChild(children.begin()->first);
|
||||
}
|
||||
|
||||
/// Runs separate scheduler thread
|
||||
@ -97,12 +95,6 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
const String & getTypeName() const override
|
||||
{
|
||||
static String type_name("scheduler");
|
||||
return type_name;
|
||||
}
|
||||
|
||||
bool equals(ISchedulerNode * other) override
|
||||
{
|
||||
if (!ISchedulerNode::equals(other))
|
||||
@ -187,11 +179,16 @@ public:
|
||||
|
||||
void activateChild(ISchedulerNode * child) override
|
||||
{
|
||||
activate(Resource::get(child->info));
|
||||
activate(TResource::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)
|
||||
void activate(TResource * value)
|
||||
{
|
||||
assert(value->next == nullptr && value->prev == nullptr);
|
||||
if (current == nullptr) // No active children
|
||||
@ -209,7 +206,7 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
void deactivate(Resource * value)
|
||||
void deactivate(TResource * value)
|
||||
{
|
||||
if (value->next == nullptr)
|
||||
return; // Already deactivated
|
||||
@ -254,8 +251,8 @@ private:
|
||||
request->execute();
|
||||
}
|
||||
|
||||
Resource * current = nullptr; // round-robin pointer
|
||||
std::unordered_map<ISchedulerNode *, Resource> children; // resources by pointer
|
||||
TResource * current = nullptr; // round-robin pointer
|
||||
std::unordered_map<ISchedulerNode *, TResource> children; // resources by pointer
|
||||
std::atomic<bool> stop_flag = false;
|
||||
EventQueue events;
|
||||
ThreadFromGlobalPool scheduler;
|
||||
|
@ -1,130 +0,0 @@
|
||||
#include <limits>
|
||||
#include <Common/Scheduler/SchedulingSettings.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void SchedulingSettings::updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name)
|
||||
{
|
||||
struct {
|
||||
std::optional<Float64> new_weight;
|
||||
std::optional<Priority> new_priority;
|
||||
std::optional<Float64> new_max_speed;
|
||||
std::optional<Float64> new_max_burst;
|
||||
std::optional<Int64> new_max_requests;
|
||||
std::optional<Int64> new_max_cost;
|
||||
|
||||
static Float64 getNotNegativeFloat64(const String & name, const Field & field)
|
||||
{
|
||||
{
|
||||
UInt64 val;
|
||||
if (field.tryGet(val))
|
||||
return static_cast<Float64>(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<Float64>(val); // We dont mind slight loss of precision
|
||||
}
|
||||
}
|
||||
|
||||
return field.safeGet<Float64>();
|
||||
}
|
||||
|
||||
static Int64 getNotNegativeInt64(const String & name, const Field & field)
|
||||
{
|
||||
{
|
||||
UInt64 val;
|
||||
if (field.tryGet(val))
|
||||
{
|
||||
// Saturate on overflow
|
||||
if (val > static_cast<UInt64>(std::numeric_limits<Int64>::max()))
|
||||
val = std::numeric_limits<Int64>::max();
|
||||
return static_cast<Int64>(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<Int64>();
|
||||
}
|
||||
|
||||
void read(const String & name, const Field & value)
|
||||
{
|
||||
if (name == "weight")
|
||||
new_weight = getNotNegativeFloat64(name, value);
|
||||
else if (name == "priority")
|
||||
new_priority = Priority{value.safeGet<Priority::Value>()};
|
||||
else if (name == "max_speed")
|
||||
new_max_speed = getNotNegativeFloat64(name, value);
|
||||
else if (name == "max_burst")
|
||||
new_max_burst = getNotNegativeFloat64(name, value);
|
||||
else if (name == "max_requests")
|
||||
new_max_requests = getNotNegativeInt64(name, value);
|
||||
else if (name == "max_cost")
|
||||
new_max_cost = getNotNegativeInt64(name, value);
|
||||
}
|
||||
} regular, specific;
|
||||
|
||||
// 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 = [] <typename T> (const std::optional<T> & specific_new, const std::optional<T> & 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);
|
||||
}
|
||||
|
||||
}
|
@ -1,39 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <Common/Priority.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct SchedulingSettings
|
||||
{
|
||||
/// Priority and weight among siblings
|
||||
Float64 weight = 1.0;
|
||||
Priority priority;
|
||||
|
||||
/// Throttling constraints.
|
||||
/// Up to 2 independent throttlers: one for average speed and one for peek 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<Int64>::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<Int64>::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; }
|
||||
|
||||
void updateFromChanges(const ASTCreateWorkloadQuery::SettingsChanges & changes, const String & resource_name = {});
|
||||
};
|
||||
|
||||
}
|
@ -1,91 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IAST;
|
||||
struct Settings;
|
||||
|
||||
enum class WorkloadEntityType : uint8_t
|
||||
{
|
||||
Workload,
|
||||
Resource,
|
||||
|
||||
MAX
|
||||
};
|
||||
|
||||
/// 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<String> getAllEntityNames() const = 0;
|
||||
|
||||
/// Get all entity names of specified type.
|
||||
virtual std::vector<String> getAllEntityNames(WorkloadEntityType entity_type) const = 0;
|
||||
|
||||
/// Get all entities.
|
||||
virtual std::vector<std::pair<String, ASTPtr>> getAllEntities() const = 0;
|
||||
|
||||
/// Check whether any entity have been stored.
|
||||
virtual bool empty() const = 0;
|
||||
|
||||
/// Stops watching.
|
||||
virtual void stopWatching() {}
|
||||
|
||||
/// 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;
|
||||
|
||||
struct Event
|
||||
{
|
||||
WorkloadEntityType type;
|
||||
String name;
|
||||
ASTPtr entity; /// new or changed entity, null if removed
|
||||
};
|
||||
using OnChangedHandler = std::function<void(const std::vector<Event> &)>;
|
||||
|
||||
/// Gets all current entries, pass them through `handler` and subscribes for all later changes.
|
||||
virtual scope_guard getAllEntitiesAndSubscribe(const OnChangedHandler & handler) = 0;
|
||||
};
|
||||
|
||||
}
|
@ -1,287 +0,0 @@
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityDiskStorage.h>
|
||||
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/ParserCreateWorkloadQuery.h>
|
||||
#include <Parsers/ParserCreateResourceQuery.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
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;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
constexpr std::string_view workload_prefix = "workload_";
|
||||
constexpr std::string_view resource_prefix = "resource_";
|
||||
constexpr std::string_view sql_suffix = ".sql";
|
||||
|
||||
/// 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);
|
||||
|
||||
auto parse = [&] (auto parser)
|
||||
{
|
||||
return parseQuery(
|
||||
parser,
|
||||
entity_create_query.data(),
|
||||
entity_create_query.data() + entity_create_query.size(),
|
||||
"",
|
||||
0,
|
||||
global_context->getSettingsRef()[Setting::max_parser_depth],
|
||||
global_context->getSettingsRef()[Setting::max_parser_backtracks]);
|
||||
};
|
||||
|
||||
switch (entity_type)
|
||||
{
|
||||
case WorkloadEntityType::Workload: return parse(ParserCreateWorkloadQuery());
|
||||
case WorkloadEntityType::Resource: return parse(ParserCreateResourceQuery());
|
||||
case WorkloadEntityType::MAX: return nullptr;
|
||||
}
|
||||
}
|
||||
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::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<std::pair<String, ASTPtr>> 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 (file_name.starts_with(workload_prefix) && file_name.ends_with(sql_suffix))
|
||||
{
|
||||
String name = unescapeForFileName(file_name.substr(
|
||||
workload_prefix.size(),
|
||||
file_name.size() - workload_prefix.size() - sql_suffix.size()));
|
||||
|
||||
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 (file_name.starts_with(resource_prefix) && file_name.ends_with(sql_suffix))
|
||||
{
|
||||
String name = unescapeForFileName(file_name.substr(
|
||||
resource_prefix.size(),
|
||||
file_name.size() - resource_prefix.size() - sql_suffix.size()));
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
|
||||
WorkloadEntityStorageBase::OperationResult 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::BAD_ARGUMENTS, "Workload entity '{}' already exists", entity_name);
|
||||
else if (!replace_if_exists)
|
||||
return OperationResult::Failed;
|
||||
}
|
||||
|
||||
|
||||
String temp_file_path = file_path + ".tmp";
|
||||
|
||||
try
|
||||
{
|
||||
WriteBufferFromFile out(temp_file_path);
|
||||
formatAST(*create_entity_query, out, false);
|
||||
writeChar('\n', out);
|
||||
out.next();
|
||||
if (settings[Setting::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 OperationResult::Ok;
|
||||
}
|
||||
|
||||
|
||||
WorkloadEntityStorageBase::OperationResult 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::BAD_ARGUMENTS, "Workload entity '{}' doesn't exist", entity_name);
|
||||
else
|
||||
return OperationResult::Failed;
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Entity {} removed", backQuote(entity_name));
|
||||
return OperationResult::Ok;
|
||||
}
|
||||
|
||||
|
||||
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;
|
||||
}
|
||||
case WorkloadEntityType::MAX: break;
|
||||
}
|
||||
return file_path;
|
||||
}
|
||||
|
||||
}
|
@ -1,44 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityStorageBase.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
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;
|
||||
|
||||
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 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;
|
||||
std::atomic<bool> entities_loaded = false;
|
||||
};
|
||||
|
||||
}
|
@ -1,273 +0,0 @@
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/ParserCreateWorkloadEntity.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <base/sleep.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
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{std::make_shared<WatchEvent>()}
|
||||
{
|
||||
log = getLogger("WorkloadEntityKeeperStorage");
|
||||
if (zookeeper_path.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must be non-empty");
|
||||
|
||||
if (zookeeper_path.back() == '/')
|
||||
zookeeper_path.pop_back();
|
||||
|
||||
/// 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->cv.notify_one();
|
||||
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);
|
||||
|
||||
auto lock = getLock();
|
||||
refreshEntities(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.
|
||||
try
|
||||
{
|
||||
auto lock = getLock();
|
||||
refreshEntities(getZooKeeper());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Failed to load workload entities");
|
||||
}
|
||||
startWatchingThread();
|
||||
}
|
||||
|
||||
|
||||
void WorkloadEntityKeeperStorage::processWatchQueue()
|
||||
{
|
||||
LOG_DEBUG(log, "Started watching thread");
|
||||
setThreadName("WrkldEntWatch");
|
||||
|
||||
UInt64 handled = 0;
|
||||
while (watching_flag)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Re-initialize ZooKeeper session if expired
|
||||
getZooKeeper();
|
||||
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
auto lock = getLock();
|
||||
refreshEntities(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)
|
||||
{
|
||||
refreshEntities(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)
|
||||
{
|
||||
refreshEntities(zookeeper);
|
||||
return OperationResult::Retry;
|
||||
}
|
||||
|
||||
current_version = stat.version;
|
||||
|
||||
LOG_DEBUG(log, "Workload entity {} removed", backQuote(entity_name));
|
||||
|
||||
return OperationResult::Ok;
|
||||
}
|
||||
|
||||
std::pair<String, Int32> WorkloadEntityKeeperStorage::getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper)
|
||||
{
|
||||
const auto data_watcher = [my_watch = watch](const Coordination::WatchResponse & response)
|
||||
{
|
||||
if (response.type == Coordination::Event::CHANGED)
|
||||
{
|
||||
std::unique_lock lock{my_watch->mutex};
|
||||
my_watch->triggered++;
|
||||
my_watch->cv.notify_one();
|
||||
}
|
||||
};
|
||||
|
||||
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::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper)
|
||||
{
|
||||
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
|
||||
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 and parse all SQL entities from data we just read from ZooKeeper
|
||||
std::vector<std::pair<String, ASTPtr>> new_entities;
|
||||
for (const auto & query : queries)
|
||||
{
|
||||
LOG_TRACE(log, "Read keeper entity definition: {}", serializeAST(*query));
|
||||
if (auto * create_workload_query = query->as<ASTCreateWorkloadQuery>())
|
||||
new_entities.emplace_back(create_workload_query->getWorkloadName(), query);
|
||||
else if (auto * create_resource_query = query->as<ASTCreateResourceQuery>())
|
||||
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");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,71 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityStorageBase.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCachingGetter.h>
|
||||
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
||||
|
||||
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<String, Int32> getDataAndSetWatch(const zkutil::ZooKeeperPtr & zookeeper);
|
||||
void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper);
|
||||
|
||||
zkutil::ZooKeeperCachingGetter zookeeper_getter;
|
||||
String zookeeper_path;
|
||||
Int32 current_version = 0;
|
||||
|
||||
ThreadFromGlobalPool watching_thread;
|
||||
std::atomic<bool> watching_flag = false;
|
||||
|
||||
struct WatchEvent
|
||||
{
|
||||
std::mutex mutex;
|
||||
std::condition_variable cv;
|
||||
UInt64 triggered = 0;
|
||||
};
|
||||
std::shared_ptr<WatchEvent> watch;
|
||||
};
|
||||
|
||||
}
|
@ -1,773 +0,0 @@
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityStorageBase.h>
|
||||
|
||||
#include <Common/Scheduler/SchedulingSettings.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <boost/container/flat_set.hpp>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
|
||||
#include <mutex>
|
||||
#include <queue>
|
||||
#include <unordered_set>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
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();
|
||||
if (auto * res = typeid_cast<ASTCreateWorkloadQuery *>(ptr.get()))
|
||||
{
|
||||
res->if_not_exists = false;
|
||||
res->or_replace = false;
|
||||
}
|
||||
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(ptr.get()))
|
||||
{
|
||||
res->if_not_exists = false;
|
||||
res->or_replace = false;
|
||||
}
|
||||
return ptr;
|
||||
}
|
||||
|
||||
/// Returns a type of a workload entity `ptr`
|
||||
WorkloadEntityType getEntityType(const ASTPtr & ptr)
|
||||
{
|
||||
if (auto * res = typeid_cast<ASTCreateWorkloadQuery *>(ptr.get()))
|
||||
return WorkloadEntityType::Workload;
|
||||
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(ptr.get()))
|
||||
return WorkloadEntityType::Resource;
|
||||
chassert(false);
|
||||
return WorkloadEntityType::MAX;
|
||||
}
|
||||
|
||||
bool entityEquals(const ASTPtr & lhs, const ASTPtr & rhs)
|
||||
{
|
||||
if (auto * a = typeid_cast<ASTCreateWorkloadQuery *>(lhs.get()))
|
||||
{
|
||||
if (auto * b = typeid_cast<ASTCreateWorkloadQuery *>(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<ASTCreateResourceQuery *>(lhs.get()))
|
||||
{
|
||||
if (auto * b = typeid_cast<ASTCreateResourceQuery *>(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, // Source workload references target workload as a parent
|
||||
ForResource // Source workload references target resource in its `SETTINGS x = y FOR resource` clause
|
||||
};
|
||||
|
||||
/// 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<void(const String & target, const String & source, ReferenceType type)> func)
|
||||
{
|
||||
if (auto * res = typeid_cast<ASTCreateWorkloadQuery *>(source_entity.get()))
|
||||
{
|
||||
// Parent reference
|
||||
String parent = res->getWorkloadParent();
|
||||
if (!parent.empty())
|
||||
func(parent, res->getWorkloadName(), ReferenceType::Parent);
|
||||
|
||||
// References to RESOURCEs mentioned in SETTINGS clause after FOR keyword
|
||||
std::unordered_set<String> 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<ASTCreateResourceQuery *>(source_entity.get()))
|
||||
{
|
||||
// RESOURCE has no references to be validated, we allow mentioned disks to be created later
|
||||
}
|
||||
}
|
||||
|
||||
/// Helper for recursive DFS
|
||||
void topologicallySortedWorkloadsImpl(const String & name, const ASTPtr & ast, const std::unordered_map<String, ASTPtr> & workloads, std::unordered_set<String> & visited, std::vector<std::pair<String, ASTPtr>> & sorted_workloads)
|
||||
{
|
||||
if (visited.contains(name))
|
||||
return;
|
||||
visited.insert(name);
|
||||
|
||||
// Recurse into parent (if any)
|
||||
String parent = typeid_cast<ASTCreateWorkloadQuery *>(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);
|
||||
}
|
||||
|
||||
/// Returns pairs {worload_name, create_workload_ast} in order that respect child-parent relation (parent first, then children)
|
||||
std::vector<std::pair<String, ASTPtr>> topologicallySortedWorkloads(const std::unordered_map<String, ASTPtr> & workloads)
|
||||
{
|
||||
std::vector<std::pair<String, ASTPtr>> sorted_workloads;
|
||||
std::unordered_set<String> visited;
|
||||
for (const auto & [name, ast] : workloads)
|
||||
topologicallySortedWorkloadsImpl(name, ast, workloads, visited, sorted_workloads);
|
||||
return sorted_workloads;
|
||||
}
|
||||
|
||||
/// Helper for recursive DFS
|
||||
void topologicallySortedDependenciesImpl(
|
||||
const String & name,
|
||||
const std::unordered_map<String, std::unordered_set<String>> & dependencies,
|
||||
std::unordered_set<String> & visited,
|
||||
std::vector<String> & 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<String> topologicallySortedDependencies(const std::unordered_map<String, std::unordered_set<String>> & dependencies)
|
||||
{
|
||||
std::unordered_set<String> visited; // Set to track visited nodes
|
||||
std::vector<String> 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);
|
||||
|
||||
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<IWorkloadEntityStorage::Event> 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<EntityChange> topologicallySortedChanges(const std::vector<EntityChange> & changes)
|
||||
{
|
||||
// Construct map from entity name into entity change
|
||||
std::unordered_map<String, const EntityChange *> 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<String, std::unordered_set<String>> old_sources; // Key is target. Value is set of names of source entities.
|
||||
std::unordered_map<String, std::unordered_set<String>> 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<String, std::unordered_set<String>> 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<String>{}); // Make sure we create nodes that have no dependencies
|
||||
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<EntityChange> result;
|
||||
for (const String & name : topologicallySortedDependencies(dependencies))
|
||||
result.push_back(*change_by_name[name]);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
WorkloadEntityStorageBase::WorkloadEntityStorageBase(ContextPtr global_context_)
|
||||
: handlers(std::make_shared<Handlers>())
|
||||
, global_context(std::move(global_context_))
|
||||
, log{getLogger("WorkloadEntityStorage")} // could be overridden in derived class
|
||||
{}
|
||||
|
||||
ASTPtr WorkloadEntityStorageBase::get(const String & entity_name) const
|
||||
{
|
||||
if (auto result = tryGet(entity_name))
|
||||
return result;
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"The workload entity name '{}' is not saved",
|
||||
entity_name);
|
||||
}
|
||||
|
||||
ASTPtr WorkloadEntityStorageBase::tryGet(const 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<String> WorkloadEntityStorageBase::getAllEntityNames() const
|
||||
{
|
||||
std::vector<String> 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;
|
||||
}
|
||||
|
||||
std::vector<String> WorkloadEntityStorageBase::getAllEntityNames(WorkloadEntityType entity_type) const
|
||||
{
|
||||
std::vector<String> 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);
|
||||
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)
|
||||
{
|
||||
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<ASTCreateWorkloadQuery *>(create_entity_query.get());
|
||||
auto * resource = typeid_cast<ASTCreateResourceQuery *>(create_entity_query.get());
|
||||
|
||||
while (true)
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
|
||||
ASTPtr old_entity; // entity to be REPLACED
|
||||
if (auto it = entities.find(entity_name); it != entities.end())
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
// Validate CREATE OR REPLACE
|
||||
if (old_entity)
|
||||
{
|
||||
auto * old_workload = typeid_cast<ASTCreateWorkloadQuery *>(old_entity.get());
|
||||
auto * old_resource = typeid_cast<ASTCreateResourceQuery *>(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");
|
||||
}
|
||||
|
||||
// Validate workload
|
||||
if (workload)
|
||||
{
|
||||
if (!workload->hasParent())
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
SchedulingSettings validator;
|
||||
validator.updateFromChanges(workload->changes);
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
switch (type)
|
||||
{
|
||||
case ReferenceType::Parent:
|
||||
{
|
||||
if (typeid_cast<ASTCreateWorkloadQuery *>(entities[target].get()) == nullptr)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Workload parent should reference another workload, not '{}'.", target);
|
||||
break;
|
||||
}
|
||||
case ReferenceType::ForResource:
|
||||
{
|
||||
if (typeid_cast<ASTCreateResourceQuery *>(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;
|
||||
}
|
||||
}
|
||||
|
||||
// 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");
|
||||
});
|
||||
|
||||
auto result = storeEntityImpl(
|
||||
current_context,
|
||||
entity_type,
|
||||
entity_name,
|
||||
create_entity_query,
|
||||
throw_if_exists,
|
||||
replace_if_exists,
|
||||
settings);
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
bool WorkloadEntityStorageBase::removeEntity(
|
||||
const ContextPtr & current_context,
|
||||
WorkloadEntityType entity_type,
|
||||
const String & entity_name,
|
||||
bool throw_if_not_exists)
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
scope_guard WorkloadEntityStorageBase::getAllEntitiesAndSubscribe(const OnChangedHandler & handler)
|
||||
{
|
||||
scope_guard result;
|
||||
|
||||
std::vector<Event> current_state;
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
current_state = orderEntities(entities);
|
||||
|
||||
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::unlockAndNotify(
|
||||
std::unique_lock<std::recursive_mutex> & lock,
|
||||
std::vector<Event> tx)
|
||||
{
|
||||
if (tx.empty())
|
||||
return;
|
||||
|
||||
std::vector<OnChangedHandler> current_handlers;
|
||||
{
|
||||
std::lock_guard handlers_lock{handlers->mutex};
|
||||
boost::range::copy(handlers->list, std::back_inserter(current_handlers));
|
||||
}
|
||||
|
||||
lock.unlock();
|
||||
|
||||
for (const auto & handler : current_handlers)
|
||||
{
|
||||
try
|
||||
{
|
||||
handler(tx);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::unique_lock<std::recursive_mutex> WorkloadEntityStorageBase::getLock() const
|
||||
{
|
||||
return std::unique_lock{mutex};
|
||||
}
|
||||
|
||||
void WorkloadEntityStorageBase::setAllEntities(const std::vector<std::pair<String, ASTPtr>> & raw_new_entities)
|
||||
{
|
||||
std::unordered_map<String, ASTPtr> new_entities;
|
||||
for (const auto & [entity_name, create_query] : raw_new_entities)
|
||||
new_entities[entity_name] = normalizeCreateWorkloadEntityQuery(*create_query);
|
||||
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
// Fill vector of `changes` based on difference between current `entities` and `new_entities`
|
||||
std::vector<EntityChange> changes;
|
||||
for (const auto & [entity_name, entity] : entities)
|
||||
{
|
||||
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); // 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{}); // 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.
|
||||
std::vector<Event> 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::applyEvent(
|
||||
std::unique_lock<std::recursive_mutex> &,
|
||||
const Event & event)
|
||||
{
|
||||
if (event.entity) // CREATE || CREATE OR REPLACE
|
||||
{
|
||||
LOG_DEBUG(log, "Create or replace workload entity: {}", serializeAST(*event.entity));
|
||||
|
||||
auto * workload = typeid_cast<ASTCreateWorkloadQuery *>(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());
|
||||
|
||||
LOG_DEBUG(log, "Drop workload entity: {}", event.name);
|
||||
|
||||
if (event.name == root_name)
|
||||
root_name.clear();
|
||||
|
||||
// Clean up references
|
||||
removeReferences(it->second);
|
||||
|
||||
// Remove from memory
|
||||
entities.erase(it);
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<std::pair<String, ASTPtr>> WorkloadEntityStorageBase::getAllEntities() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
std::vector<std::pair<String, ASTPtr>> all_entities;
|
||||
all_entities.reserve(entities.size());
|
||||
std::copy(entities.begin(), entities.end(), std::back_inserter(all_entities));
|
||||
return all_entities;
|
||||
}
|
||||
|
||||
bool WorkloadEntityStorageBase::isIndirectlyReferenced(const String & target, const String & source)
|
||||
{
|
||||
std::queue<String> bfs;
|
||||
std::unordered_set<String> 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);
|
||||
});
|
||||
}
|
||||
|
||||
std::vector<WorkloadEntityStorageBase::Event> WorkloadEntityStorageBase::orderEntities(
|
||||
const std::unordered_map<String, ASTPtr> & all_entities,
|
||||
std::optional<Event> change)
|
||||
{
|
||||
std::vector<Event> result;
|
||||
|
||||
std::unordered_map<String, ASTPtr> workloads;
|
||||
for (const auto & [entity_name, ast] : all_entities)
|
||||
{
|
||||
if (typeid_cast<ASTCreateWorkloadQuery *>(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<ASTCreateResourceQuery *>(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<Event> change)
|
||||
{
|
||||
std::unique_lock<std::recursive_mutex> 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();
|
||||
}
|
||||
|
||||
}
|
@ -1,126 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <list>
|
||||
#include <mutex>
|
||||
#include <unordered_set>
|
||||
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
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<String> getAllEntityNames() const override;
|
||||
std::vector<String> getAllEntityNames(WorkloadEntityType entity_type) const override;
|
||||
|
||||
std::vector<std::pair<String, ASTPtr>> 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;
|
||||
|
||||
scope_guard getAllEntitiesAndSubscribe(
|
||||
const OnChangedHandler & handler) override;
|
||||
|
||||
protected:
|
||||
enum class OperationResult
|
||||
{
|
||||
Ok,
|
||||
Failed,
|
||||
Retry
|
||||
};
|
||||
|
||||
virtual 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) = 0;
|
||||
|
||||
virtual OperationResult removeEntityImpl(
|
||||
const ContextPtr & current_context,
|
||||
WorkloadEntityType entity_type,
|
||||
const String & entity_name,
|
||||
bool throw_if_not_exists) = 0;
|
||||
|
||||
std::unique_lock<std::recursive_mutex> 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<std::pair<String, ASTPtr>> & new_entities);
|
||||
|
||||
/// Serialize `entities` stored in memory plus one optional `change` into multiline string
|
||||
String serializeAllEntities(std::optional<Event> change = {});
|
||||
|
||||
private:
|
||||
/// Change state in memory
|
||||
void applyEvent(std::unique_lock<std::recursive_mutex> & lock, const Event & event);
|
||||
|
||||
/// Notify subscribers about changes describe by vector of events `tx`
|
||||
void unlockAndNotify(std::unique_lock<std::recursive_mutex> & lock, std::vector<Event> tx);
|
||||
|
||||
/// 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);
|
||||
|
||||
/// Returns an ordered vector of `entities`
|
||||
std::vector<Event> orderEntities(
|
||||
const std::unordered_map<String, ASTPtr> & all_entities,
|
||||
std::optional<Event> change = {});
|
||||
|
||||
struct Handlers
|
||||
{
|
||||
std::mutex mutex;
|
||||
std::list<OnChangedHandler> list;
|
||||
};
|
||||
/// shared_ptr is here for safety because WorkloadEntityStorageBase can be destroyed before all subscriptions are removed.
|
||||
std::shared_ptr<Handlers> handlers;
|
||||
|
||||
mutable std::recursive_mutex mutex;
|
||||
std::unordered_map<String, ASTPtr> entities; /// Maps entity name into CREATE entity query
|
||||
|
||||
// Validation
|
||||
std::unordered_map<String, std::unordered_set<String>> 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;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
@ -1,45 +0,0 @@
|
||||
#include <Common/Scheduler/Workload/createWorkloadEntityStorage.h>
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityDiskStorage.h>
|
||||
#include <Common/Scheduler/Workload/WorkloadEntityKeeperStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <filesystem>
|
||||
#include <memory>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_CONFIG_PARAMETER;
|
||||
}
|
||||
|
||||
std::unique_ptr<IWorkloadEntityStorage> 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);
|
||||
}
|
||||
return std::make_unique<WorkloadEntityKeeperStorage>(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<WorkloadEntityDiskStorage>(global_context, path);
|
||||
}
|
||||
|
||||
}
|
@ -1,11 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::unique_ptr<IWorkloadEntityStorage> createWorkloadEntityStorage(const ContextMutablePtr & global_context);
|
||||
|
||||
}
|
@ -1,104 +0,0 @@
|
||||
#include <Common/Scheduler/createResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/CustomResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/IOResourceManager.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
|
||||
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<ClassifierPtr> 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<Classifier>();
|
||||
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<ResourceManagerPtr> managers; // Should be constant after initialization to avoid races
|
||||
};
|
||||
|
||||
ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context)
|
||||
{
|
||||
auto dispatcher = std::make_shared<ResourceManagerDispatcher>();
|
||||
|
||||
// NOTE: if the same resource is described by both managers, then manager added earlier will be used.
|
||||
dispatcher->addManager(std::make_shared<CustomResourceManager>());
|
||||
dispatcher->addManager(std::make_shared<IOResourceManager>(global_context->getWorkloadEntityStorage()));
|
||||
|
||||
return dispatcher;
|
||||
}
|
||||
|
||||
}
|
@ -1,11 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ResourceManagerPtr createResourceManager(const ContextMutablePtr & global_context);
|
||||
|
||||
}
|
@ -18,8 +18,7 @@
|
||||
#include <Disks/FakeDiskTransaction.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -72,8 +71,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_from_config(config.getString(config_prefix + ".read_resource", ""))
|
||||
, write_resource_name_from_config(config.getString(config_prefix + ".write_resource", ""))
|
||||
, read_resource_name(config.getString(config_prefix + ".read_resource", ""))
|
||||
, write_resource_name(config.getString(config_prefix + ".write_resource", ""))
|
||||
, metadata_helper(std::make_unique<DiskObjectStorageRemoteMetadataRestoreHelper>(this, ReadSettings{}, WriteSettings{}))
|
||||
{
|
||||
data_source_description = DataSourceDescription{
|
||||
@ -84,98 +83,6 @@ DiskObjectStorage::DiskObjectStorage(
|
||||
.is_encrypted = false,
|
||||
.is_cached = object_storage->supportsCache(),
|
||||
};
|
||||
resource_changes_subscription = Context::getGlobalContextInstance()->getWorkloadEntityStorage().getAllEntitiesAndSubscribe(
|
||||
[this] (const std::vector<IWorkloadEntityStorage::Event> & events)
|
||||
{
|
||||
std::unique_lock lock{resource_mutex};
|
||||
|
||||
// Sets of matching resource names. Required to resolve possible conflicts in deterministic way
|
||||
std::set<String> new_read_resource_name_from_sql;
|
||||
std::set<String> new_write_resource_name_from_sql;
|
||||
std::set<String> new_read_resource_name_from_sql_any;
|
||||
std::set<String> new_write_resource_name_from_sql_any;
|
||||
|
||||
// Current state
|
||||
if (!read_resource_name_from_sql.empty())
|
||||
new_read_resource_name_from_sql.insert(read_resource_name_from_sql);
|
||||
if (!write_resource_name_from_sql.empty())
|
||||
new_write_resource_name_from_sql.insert(write_resource_name_from_sql);
|
||||
if (!read_resource_name_from_sql_any.empty())
|
||||
new_read_resource_name_from_sql_any.insert(read_resource_name_from_sql_any);
|
||||
if (!write_resource_name_from_sql_any.empty())
|
||||
new_write_resource_name_from_sql_any.insert(write_resource_name_from_sql_any);
|
||||
|
||||
// Process all updates in specified order
|
||||
for (const auto & [entity_type, resource_name, resource] : events)
|
||||
{
|
||||
if (entity_type == WorkloadEntityType::Resource)
|
||||
{
|
||||
if (resource) // CREATE RESOURCE
|
||||
{
|
||||
auto * create = typeid_cast<ASTCreateResourceQuery *>(resource.get());
|
||||
chassert(create);
|
||||
for (const auto & [mode, disk] : create->operations)
|
||||
{
|
||||
if (!disk)
|
||||
{
|
||||
switch (mode)
|
||||
{
|
||||
case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql_any.insert(resource_name); break;
|
||||
case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql_any.insert(resource_name); break;
|
||||
}
|
||||
}
|
||||
else if (*disk == name)
|
||||
{
|
||||
switch (mode)
|
||||
{
|
||||
case ASTCreateResourceQuery::AccessMode::Read: new_read_resource_name_from_sql.insert(resource_name); break;
|
||||
case ASTCreateResourceQuery::AccessMode::Write: new_write_resource_name_from_sql.insert(resource_name); break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else // DROP RESOURCE
|
||||
{
|
||||
new_read_resource_name_from_sql.erase(resource_name);
|
||||
new_write_resource_name_from_sql.erase(resource_name);
|
||||
new_read_resource_name_from_sql_any.erase(resource_name);
|
||||
new_write_resource_name_from_sql_any.erase(resource_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String old_read_resource = getReadResourceNameNoLock();
|
||||
String old_write_resource = getWriteResourceNameNoLock();
|
||||
|
||||
// Apply changes
|
||||
if (!new_read_resource_name_from_sql_any.empty())
|
||||
read_resource_name_from_sql_any = *new_read_resource_name_from_sql_any.begin();
|
||||
else
|
||||
read_resource_name_from_sql_any.clear();
|
||||
|
||||
if (!new_write_resource_name_from_sql_any.empty())
|
||||
write_resource_name_from_sql_any = *new_write_resource_name_from_sql_any.begin();
|
||||
else
|
||||
write_resource_name_from_sql_any.clear();
|
||||
|
||||
if (!new_read_resource_name_from_sql.empty())
|
||||
read_resource_name_from_sql = *new_read_resource_name_from_sql.begin();
|
||||
else
|
||||
read_resource_name_from_sql.clear();
|
||||
|
||||
if (!new_write_resource_name_from_sql.empty())
|
||||
write_resource_name_from_sql = *new_write_resource_name_from_sql.begin();
|
||||
else
|
||||
write_resource_name_from_sql.clear();
|
||||
|
||||
String new_read_resource = getReadResourceNameNoLock();
|
||||
String new_write_resource = getWriteResourceNameNoLock();
|
||||
|
||||
if (old_read_resource != new_read_resource)
|
||||
LOG_INFO(log, "Using resource '{}' instead of '{}' for READ", new_read_resource, old_read_resource);
|
||||
if (old_write_resource != new_write_resource)
|
||||
LOG_INFO(log, "Using resource '{}' instead of '{}' for WRITE", new_write_resource, old_write_resource);
|
||||
});
|
||||
}
|
||||
|
||||
StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) const
|
||||
@ -573,29 +480,13 @@ static inline Settings updateIOSchedulingSettings(const Settings & settings, con
|
||||
String DiskObjectStorage::getReadResourceName() const
|
||||
{
|
||||
std::unique_lock lock(resource_mutex);
|
||||
return getReadResourceNameNoLock();
|
||||
return read_resource_name;
|
||||
}
|
||||
|
||||
String DiskObjectStorage::getWriteResourceName() const
|
||||
{
|
||||
std::unique_lock lock(resource_mutex);
|
||||
return getWriteResourceNameNoLock();
|
||||
}
|
||||
|
||||
String DiskObjectStorage::getReadResourceNameNoLock() const
|
||||
{
|
||||
if (read_resource_name_from_config.empty())
|
||||
return read_resource_name_from_sql.empty() ? read_resource_name_from_sql_any : read_resource_name_from_sql;
|
||||
else
|
||||
return read_resource_name_from_config;
|
||||
}
|
||||
|
||||
String DiskObjectStorage::getWriteResourceNameNoLock() const
|
||||
{
|
||||
if (write_resource_name_from_config.empty())
|
||||
return write_resource_name_from_sql.empty() ? write_resource_name_from_sql_any : write_resource_name_from_sql;
|
||||
else
|
||||
return write_resource_name_from_config;
|
||||
return write_resource_name;
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
@ -716,10 +607,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_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;
|
||||
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;
|
||||
}
|
||||
|
||||
IDisk::applyNewSettings(config, context_, config_prefix, disk_map);
|
||||
|
@ -6,8 +6,6 @@
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
#include <Common/re2.h>
|
||||
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
|
||||
@ -230,8 +228,6 @@ private:
|
||||
|
||||
String getReadResourceName() const;
|
||||
String getWriteResourceName() const;
|
||||
String getReadResourceNameNoLock() const;
|
||||
String getWriteResourceNameNoLock() const;
|
||||
|
||||
const String object_key_prefix;
|
||||
LoggerPtr log;
|
||||
@ -250,13 +246,8 @@ private:
|
||||
const bool send_metadata;
|
||||
|
||||
mutable std::mutex resource_mutex;
|
||||
String read_resource_name_from_config; // specified in disk config.xml read_resource element
|
||||
String write_resource_name_from_config; // specified in disk config.xml write_resource element
|
||||
String read_resource_name_from_sql; // described by CREATE RESOURCE query with READ DISK clause
|
||||
String write_resource_name_from_sql; // described by CREATE RESOURCE query with WRITE DISK clause
|
||||
String read_resource_name_from_sql_any; // described by CREATE RESOURCE query with READ ANY DISK clause
|
||||
String write_resource_name_from_sql_any; // described by CREATE RESOURCE query with WRITE ANY DISK clause
|
||||
scope_guard resource_changes_subscription;
|
||||
String read_resource_name;
|
||||
String write_resource_name;
|
||||
|
||||
std::unique_ptr<DiskObjectStorageRemoteMetadataRestoreHelper> metadata_helper;
|
||||
};
|
||||
|
@ -67,6 +67,7 @@
|
||||
#include <Access/SettingsConstraintsAndProfileIDs.h>
|
||||
#include <Access/ExternalAuthenticators.h>
|
||||
#include <Access/GSSAcceptor.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
#include <Backups/BackupsWorker.h>
|
||||
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
@ -91,8 +92,6 @@
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/Scheduler/createResourceManager.h>
|
||||
#include <Common/Scheduler/Workload/createWorkloadEntityStorage.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
@ -371,9 +370,6 @@ struct ContextSharedPart : boost::noncopyable
|
||||
mutable OnceFlag user_defined_sql_objects_storage_initialized;
|
||||
mutable std::unique_ptr<IUserDefinedSQLObjectsStorage> user_defined_sql_objects_storage;
|
||||
|
||||
mutable OnceFlag workload_entity_storage_initialized;
|
||||
mutable std::unique_ptr<IWorkloadEntityStorage> workload_entity_storage;
|
||||
|
||||
#if USE_NLP
|
||||
mutable OnceFlag synonyms_extensions_initialized;
|
||||
mutable std::optional<SynonymsExtensions> synonyms_extensions;
|
||||
@ -715,7 +711,6 @@ 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();
|
||||
@ -747,7 +742,6 @@ struct ContextSharedPart : boost::noncopyable
|
||||
std::unique_ptr<ExternalDictionariesLoader> delete_external_dictionaries_loader;
|
||||
std::unique_ptr<ExternalUserDefinedExecutableFunctionsLoader> delete_external_user_defined_executable_functions_loader;
|
||||
std::unique_ptr<IUserDefinedSQLObjectsStorage> delete_user_defined_sql_objects_storage;
|
||||
std::unique_ptr<IWorkloadEntityStorage> delete_workload_entity_storage;
|
||||
std::unique_ptr<BackgroundSchedulePool> delete_buffer_flush_schedule_pool;
|
||||
std::unique_ptr<BackgroundSchedulePool> delete_schedule_pool;
|
||||
std::unique_ptr<BackgroundSchedulePool> delete_distributed_schedule_pool;
|
||||
@ -832,7 +826,6 @@ 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);
|
||||
@ -851,7 +844,6 @@ 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();
|
||||
@ -1776,7 +1768,7 @@ std::vector<UUID> Context::getEnabledProfiles() const
|
||||
ResourceManagerPtr Context::getResourceManager() const
|
||||
{
|
||||
callOnce(shared->resource_manager_initialized, [&] {
|
||||
shared->resource_manager = createResourceManager(getGlobalContext());
|
||||
shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic"));
|
||||
});
|
||||
|
||||
return shared->resource_manager;
|
||||
@ -3023,16 +3015,6 @@ void Context::setUserDefinedSQLObjectsStorage(std::unique_ptr<IUserDefinedSQLObj
|
||||
shared->user_defined_sql_objects_storage = std::move(storage);
|
||||
}
|
||||
|
||||
IWorkloadEntityStorage & Context::getWorkloadEntityStorage() const
|
||||
{
|
||||
callOnce(shared->workload_entity_storage_initialized, [&] {
|
||||
shared->workload_entity_storage = createWorkloadEntityStorage(getGlobalContext());
|
||||
});
|
||||
|
||||
std::lock_guard lock(shared->mutex);
|
||||
return *shared->workload_entity_storage;
|
||||
}
|
||||
|
||||
#if USE_NLP
|
||||
|
||||
SynonymsExtensions & Context::getSynonymsExtensions() const
|
||||
|
@ -76,7 +76,6 @@ class EmbeddedDictionaries;
|
||||
class ExternalDictionariesLoader;
|
||||
class ExternalUserDefinedExecutableFunctionsLoader;
|
||||
class IUserDefinedSQLObjectsStorage;
|
||||
class IWorkloadEntityStorage;
|
||||
class InterserverCredentials;
|
||||
using InterserverCredentialsPtr = std::shared_ptr<const InterserverCredentials>;
|
||||
class InterserverIOHandler;
|
||||
@ -894,8 +893,6 @@ public:
|
||||
void setUserDefinedSQLObjectsStorage(std::unique_ptr<IUserDefinedSQLObjectsStorage> storage);
|
||||
void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
IWorkloadEntityStorage & getWorkloadEntityStorage() const;
|
||||
|
||||
#if USE_NLP
|
||||
SynonymsExtensions & getSynonymsExtensions() const;
|
||||
Lemmatizers & getLemmatizers() const;
|
||||
|
@ -1,68 +0,0 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterCreateResourceQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateResourceQuery::execute()
|
||||
{
|
||||
ASTCreateResourceQuery & create_resource_query = query_ptr->as<ASTCreateResourceQuery &>();
|
||||
|
||||
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;
|
||||
|
||||
current_context->getWorkloadEntityStorage().storeEntity(
|
||||
current_context,
|
||||
WorkloadEntityType::Resource,
|
||||
resource_name,
|
||||
query_ptr,
|
||||
throw_if_exists,
|
||||
replace_if_exists,
|
||||
current_context->getSettingsRef());
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void registerInterpreterCreateResourceQuery(InterpreterFactory & factory)
|
||||
{
|
||||
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
||||
{
|
||||
return std::make_unique<InterpreterCreateResourceQuery>(args.query, args.context);
|
||||
};
|
||||
factory.registerInterpreter("InterpreterCreateResourceQuery", create_fn);
|
||||
}
|
||||
|
||||
}
|
@ -1,25 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -1,68 +0,0 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterCreateWorkloadQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateWorkloadQuery::execute()
|
||||
{
|
||||
ASTCreateWorkloadQuery & create_workload_query = query_ptr->as<ASTCreateWorkloadQuery &>();
|
||||
|
||||
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;
|
||||
|
||||
current_context->getWorkloadEntityStorage().storeEntity(
|
||||
current_context,
|
||||
WorkloadEntityType::Workload,
|
||||
workload_name,
|
||||
query_ptr,
|
||||
throw_if_exists,
|
||||
replace_if_exists,
|
||||
current_context->getSettingsRef());
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void registerInterpreterCreateWorkloadQuery(InterpreterFactory & factory)
|
||||
{
|
||||
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
||||
{
|
||||
return std::make_unique<InterpreterCreateWorkloadQuery>(args.query, args.context);
|
||||
};
|
||||
factory.registerInterpreter("InterpreterCreateWorkloadQuery", create_fn);
|
||||
}
|
||||
|
||||
}
|
@ -1,25 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -1,60 +0,0 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterDropResourceQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTDropResourceQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
BlockIO InterpreterDropResourceQuery::execute()
|
||||
{
|
||||
ASTDropResourceQuery & drop_resource_query = query_ptr->as<ASTDropResourceQuery &>();
|
||||
|
||||
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;
|
||||
|
||||
current_context->getWorkloadEntityStorage().removeEntity(
|
||||
current_context,
|
||||
WorkloadEntityType::Resource,
|
||||
drop_resource_query.resource_name,
|
||||
throw_if_not_exists);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void registerInterpreterDropResourceQuery(InterpreterFactory & factory)
|
||||
{
|
||||
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
||||
{
|
||||
return std::make_unique<InterpreterDropResourceQuery>(args.query, args.context);
|
||||
};
|
||||
factory.registerInterpreter("InterpreterDropResourceQuery", create_fn);
|
||||
}
|
||||
|
||||
}
|
@ -1,21 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -1,60 +0,0 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/InterpreterDropWorkloadQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTDropWorkloadQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
BlockIO InterpreterDropWorkloadQuery::execute()
|
||||
{
|
||||
ASTDropWorkloadQuery & drop_workload_query = query_ptr->as<ASTDropWorkloadQuery &>();
|
||||
|
||||
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;
|
||||
|
||||
current_context->getWorkloadEntityStorage().removeEntity(
|
||||
current_context,
|
||||
WorkloadEntityType::Workload,
|
||||
drop_workload_query.workload_name,
|
||||
throw_if_not_exists);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void registerInterpreterDropWorkloadQuery(InterpreterFactory & factory)
|
||||
{
|
||||
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
||||
{
|
||||
return std::make_unique<InterpreterDropWorkloadQuery>(args.query, args.context);
|
||||
};
|
||||
factory.registerInterpreter("InterpreterDropWorkloadQuery", create_fn);
|
||||
}
|
||||
|
||||
}
|
@ -1,21 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -3,13 +3,9 @@
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTDeleteQuery.h>
|
||||
#include <Parsers/ASTDropFunctionQuery.h>
|
||||
#include <Parsers/ASTDropWorkloadQuery.h>
|
||||
#include <Parsers/ASTDropResourceQuery.h>
|
||||
#include <Parsers/ASTDropIndexQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTUndropQuery.h>
|
||||
@ -336,22 +332,6 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
|
||||
{
|
||||
interpreter_name = "InterpreterDropFunctionQuery";
|
||||
}
|
||||
else if (query->as<ASTCreateWorkloadQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterCreateWorkloadQuery";
|
||||
}
|
||||
else if (query->as<ASTDropWorkloadQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterDropWorkloadQuery";
|
||||
}
|
||||
else if (query->as<ASTCreateResourceQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterCreateResourceQuery";
|
||||
}
|
||||
else if (query->as<ASTDropResourceQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterDropResourceQuery";
|
||||
}
|
||||
else if (query->as<ASTCreateIndexQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterCreateIndexQuery";
|
||||
|
@ -52,10 +52,6 @@ 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);
|
||||
@ -115,10 +111,6 @@ void registerInterpreters()
|
||||
registerInterpreterTransactionControlQuery(factory);
|
||||
registerInterpreterCreateFunctionQuery(factory);
|
||||
registerInterpreterDropFunctionQuery(factory);
|
||||
registerInterpreterCreateWorkloadQuery(factory);
|
||||
registerInterpreterDropWorkloadQuery(factory);
|
||||
registerInterpreterCreateResourceQuery(factory);
|
||||
registerInterpreterDropResourceQuery(factory);
|
||||
registerInterpreterCreateIndexQuery(factory);
|
||||
registerInterpreterCreateNamedCollectionQuery(factory);
|
||||
registerInterpreterDropIndexQuery(factory);
|
||||
|
@ -1,83 +0,0 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTCreateResourceQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTCreateResourceQuery>(*this);
|
||||
res->children.clear();
|
||||
|
||||
res->resource_name = resource_name->clone();
|
||||
res->children.push_back(res->resource_name);
|
||||
|
||||
res->operations = operations;
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTCreateResourceQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
|
||||
{
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
|
||||
|
||||
if (or_replace)
|
||||
format.ostr << "OR REPLACE ";
|
||||
|
||||
format.ostr << "RESOURCE ";
|
||||
|
||||
if (if_not_exists)
|
||||
format.ostr << "IF NOT EXISTS ";
|
||||
|
||||
format.ostr << (format.hilite ? hilite_none : "");
|
||||
|
||||
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getResourceName()) << (format.hilite ? hilite_none : "");
|
||||
|
||||
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 ";
|
||||
break;
|
||||
}
|
||||
case AccessMode::Write:
|
||||
{
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "WRITE ";
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (operation.disk)
|
||||
{
|
||||
format.ostr << "DISK " << (format.hilite ? hilite_none : "");
|
||||
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(*operation.disk) << (format.hilite ? hilite_none : "");
|
||||
}
|
||||
else
|
||||
format.ostr << "ANY DISK" << (format.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
format.ostr << ")";
|
||||
}
|
||||
|
||||
String ASTCreateResourceQuery::getResourceName() const
|
||||
{
|
||||
String name;
|
||||
tryGetIdentifierNameInto(resource_name, name);
|
||||
return name;
|
||||
}
|
||||
|
||||
}
|
@ -1,48 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTCreateResourceQuery : public IAST, public ASTQueryWithOnCluster
|
||||
{
|
||||
public:
|
||||
enum class AccessMode
|
||||
{
|
||||
Read,
|
||||
Write
|
||||
};
|
||||
struct Operation
|
||||
{
|
||||
AccessMode mode;
|
||||
std::optional<String> disk; // Applies to all disks if not set
|
||||
|
||||
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<Operation>;
|
||||
|
||||
ASTPtr resource_name;
|
||||
Operations operations; /// List of operations that require this resource
|
||||
|
||||
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 & format, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateResourceQuery>(clone()); }
|
||||
|
||||
String getResourceName() const;
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Create; }
|
||||
};
|
||||
|
||||
}
|
@ -1,95 +0,0 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTCreateWorkloadQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTCreateWorkloadQuery>(*this);
|
||||
res->children.clear();
|
||||
|
||||
res->workload_name = workload_name->clone();
|
||||
res->children.push_back(res->workload_name);
|
||||
|
||||
if (workload_parent)
|
||||
{
|
||||
res->workload_parent = workload_parent->clone();
|
||||
res->children.push_back(res->workload_parent);
|
||||
}
|
||||
|
||||
res->changes = changes;
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTCreateWorkloadQuery::formatImpl(const IAST::FormatSettings & format, IAST::FormatState &, IAST::FormatStateStacked) const
|
||||
{
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << "CREATE ";
|
||||
|
||||
if (or_replace)
|
||||
format.ostr << "OR REPLACE ";
|
||||
|
||||
format.ostr << "WORKLOAD ";
|
||||
|
||||
if (if_not_exists)
|
||||
format.ostr << "IF NOT EXISTS ";
|
||||
|
||||
format.ostr << (format.hilite ? hilite_none : "");
|
||||
|
||||
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadName()) << (format.hilite ? hilite_none : "");
|
||||
|
||||
formatOnCluster(format);
|
||||
|
||||
if (hasParent())
|
||||
{
|
||||
format.ostr << (format.hilite ? hilite_keyword : "") << " IN " << (format.hilite ? hilite_none : "");
|
||||
format.ostr << (format.hilite ? hilite_identifier : "") << backQuoteIfNeed(getWorkloadParent()) << (format.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (!changes.empty())
|
||||
{
|
||||
format.ostr << ' ' << (format.hilite ? hilite_keyword : "") << "SETTINGS" << (format.hilite ? hilite_none : "") << ' ';
|
||||
|
||||
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 : "");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
@ -1,53 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <string_view>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTCreateWorkloadQuery : public IAST, public ASTQueryWithOnCluster
|
||||
{
|
||||
public:
|
||||
ASTPtr workload_name;
|
||||
ASTPtr workload_parent;
|
||||
|
||||
/// 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<SettingChange>;
|
||||
SettingsChanges changes;
|
||||
|
||||
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 & format, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateWorkloadQuery>(clone()); }
|
||||
|
||||
String getWorkloadName() const;
|
||||
bool hasParent() const;
|
||||
String getWorkloadParent() const;
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Create; }
|
||||
};
|
||||
|
||||
}
|
@ -1,25 +0,0 @@
|
||||
#include <Parsers/ASTDropResourceQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTDropResourceQuery::clone() const
|
||||
{
|
||||
return std::make_shared<ASTDropResourceQuery>(*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);
|
||||
}
|
||||
|
||||
}
|
@ -1,28 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
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<ASTDropResourceQuery>(clone()); }
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Drop; }
|
||||
};
|
||||
|
||||
}
|
@ -1,25 +0,0 @@
|
||||
#include <Parsers/ASTDropWorkloadQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTDropWorkloadQuery::clone() const
|
||||
{
|
||||
return std::make_shared<ASTDropWorkloadQuery>(*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);
|
||||
}
|
||||
|
||||
}
|
@ -1,28 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
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<ASTDropWorkloadQuery>(clone()); }
|
||||
|
||||
QueryKind getQueryKind() const override { return QueryKind::Drop; }
|
||||
};
|
||||
|
||||
}
|
@ -392,7 +392,6 @@ 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") \
|
||||
@ -412,7 +411,6 @@ namespace DB
|
||||
MR_MACROS(REPLACE, "REPLACE") \
|
||||
MR_MACROS(RESET_SETTING, "RESET SETTING") \
|
||||
MR_MACROS(RESET_AUTHENTICATION_METHODS_TO_NEW, "RESET AUTHENTICATION METHODS TO NEW") \
|
||||
MR_MACROS(RESOURCE, "RESOURCE") \
|
||||
MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \
|
||||
MR_MACROS(RESTORE, "RESTORE") \
|
||||
MR_MACROS(RESTRICT, "RESTRICT") \
|
||||
@ -525,7 +523,6 @@ 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") \
|
||||
@ -538,7 +535,6 @@ 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") \
|
||||
|
@ -1,144 +0,0 @@
|
||||
#include <Parsers/ParserCreateResourceQuery.h>
|
||||
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
bool parseOneOperation(ASTCreateResourceQuery::Operation & operation, IParser::Pos & pos, Expected & expected)
|
||||
{
|
||||
ParserIdentifier disk_name_p;
|
||||
|
||||
ASTCreateResourceQuery::AccessMode mode;
|
||||
ASTPtr node;
|
||||
std::optional<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::ANY).ignore(pos, expected))
|
||||
{
|
||||
if (!ParserKeyword(Keyword::DISK).ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!ParserKeyword(Keyword::DISK).ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!disk_name_p.parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
disk.emplace();
|
||||
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);
|
||||
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;
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
ASTCreateResourceQuery::Operations operations;
|
||||
if (!parseOperations(pos, expected, operations))
|
||||
return false;
|
||||
|
||||
auto create_resource_query = std::make_shared<ASTCreateResourceQuery>();
|
||||
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);
|
||||
|
||||
create_resource_query->operations = std::move(operations);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
@ -1,16 +0,0 @@
|
||||
#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;
|
||||
};
|
||||
|
||||
}
|
@ -1,16 +0,0 @@
|
||||
#include <Parsers/ParserCreateWorkloadEntity.h>
|
||||
#include <Parsers/ParserCreateWorkloadQuery.h>
|
||||
#include <Parsers/ParserCreateResourceQuery.h>
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
@ -1,17 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
@ -1,155 +0,0 @@
|
||||
#include <Parsers/ParserCreateWorkloadQuery.h>
|
||||
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
#include <Common/SettingsChanges.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
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<ASTLiteral &>().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;
|
||||
|
||||
ASTCreateWorkloadQuery::SettingsChanges res_changes;
|
||||
|
||||
auto parse_setting = [&]
|
||||
{
|
||||
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;
|
||||
|
||||
changes = std::move(res_changes);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
ASTCreateWorkloadQuery::SettingsChanges changes;
|
||||
parseSettings(pos, expected, changes);
|
||||
|
||||
auto create_workload_query = std::make_shared<ASTCreateWorkloadQuery>();
|
||||
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);
|
||||
create_workload_query->changes = std::move(changes);
|
||||
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
@ -1,16 +0,0 @@
|
||||
#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;
|
||||
};
|
||||
|
||||
}
|
@ -1,52 +0,0 @@
|
||||
#include <Parsers/ASTDropResourceQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ParserDropResourceQuery.h>
|
||||
|
||||
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<ASTDropResourceQuery>();
|
||||
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<ASTIdentifier &>().name();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
@ -1,14 +0,0 @@
|
||||
#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;
|
||||
};
|
||||
}
|
@ -1,52 +0,0 @@
|
||||
#include <Parsers/ASTDropWorkloadQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ParserDropWorkloadQuery.h>
|
||||
|
||||
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<ASTDropWorkloadQuery>();
|
||||
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<ASTIdentifier &>().name();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
@ -1,14 +0,0 @@
|
||||
#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;
|
||||
};
|
||||
}
|
@ -1,12 +1,8 @@
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/ParserCreateFunctionQuery.h>
|
||||
#include <Parsers/ParserCreateWorkloadQuery.h>
|
||||
#include <Parsers/ParserCreateResourceQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserCreateIndexQuery.h>
|
||||
#include <Parsers/ParserDropFunctionQuery.h>
|
||||
#include <Parsers/ParserDropWorkloadQuery.h>
|
||||
#include <Parsers/ParserDropResourceQuery.h>
|
||||
#include <Parsers/ParserDropIndexQuery.h>
|
||||
#include <Parsers/ParserDropNamedCollectionQuery.h>
|
||||
#include <Parsers/ParserAlterNamedCollectionQuery.h>
|
||||
@ -55,10 +51,6 @@ 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;
|
||||
@ -90,10 +82,6 @@ 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)
|
||||
|
@ -1,71 +0,0 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/System/StorageSystemResources.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Parsers/ASTCreateResourceQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ColumnsDescription StorageSystemResources::getColumnsDescription()
|
||||
{
|
||||
return ColumnsDescription
|
||||
{
|
||||
{"name", std::make_shared<DataTypeString>(), "The name of the resource."},
|
||||
{"read_disks", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "The list of disk names that uses this resource for read operations."},
|
||||
{"write_disks", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "The list of disk names that uses this resource for write operations."},
|
||||
{"create_query", std::make_shared<DataTypeString>(), "CREATE query of the resource."},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemResources::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) 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<ASTCreateResourceQuery &>(*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 ? *disk : "ANY");
|
||||
break;
|
||||
}
|
||||
case DB::ASTCreateResourceQuery::AccessMode::Write:
|
||||
{
|
||||
write_disks.emplace_back(disk ? *disk : "ANY");
|
||||
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<ASTs> & /* 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<ASTs> & /* partitions */)
|
||||
{
|
||||
// TODO(serxa): add restore for resources
|
||||
// storage.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
||||
}
|
@ -1,29 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
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<ASTs> & partitions) override;
|
||||
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
||||
|
||||
protected:
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const override;
|
||||
};
|
||||
|
||||
}
|
@ -84,12 +84,12 @@ ColumnsDescription StorageSystemScheduler::getColumnsDescription()
|
||||
|
||||
void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const
|
||||
{
|
||||
context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, ISchedulerNode * node)
|
||||
context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node)
|
||||
{
|
||||
size_t i = 0;
|
||||
res_columns[i++]->insert(resource);
|
||||
res_columns[i++]->insert(path);
|
||||
res_columns[i++]->insert(node->getTypeName());
|
||||
res_columns[i++]->insert(type);
|
||||
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<FairPolicy *>(node->parent))
|
||||
{
|
||||
if (auto value = parent->getChildVRuntime(node))
|
||||
if (auto value = parent->getChildVRuntime(node.get()))
|
||||
vruntime = *value;
|
||||
}
|
||||
if (auto * ptr = dynamic_cast<FairPolicy *>(node))
|
||||
if (auto * ptr = dynamic_cast<FairPolicy *>(node.get()))
|
||||
system_vruntime = ptr->getSystemVRuntime();
|
||||
if (auto * ptr = dynamic_cast<FifoQueue *>(node))
|
||||
if (auto * ptr = dynamic_cast<FifoQueue *>(node.get()))
|
||||
std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost();
|
||||
if (auto * ptr = dynamic_cast<ISchedulerQueue *>(node))
|
||||
if (auto * ptr = dynamic_cast<ISchedulerQueue *>(node.get()))
|
||||
budget = ptr->getBudget();
|
||||
if (auto * ptr = dynamic_cast<ISchedulerConstraint *>(node))
|
||||
if (auto * ptr = dynamic_cast<ISchedulerConstraint *>(node.get()))
|
||||
is_satisfied = ptr->isSatisfied();
|
||||
if (auto * ptr = dynamic_cast<SemaphoreConstraint *>(node))
|
||||
if (auto * ptr = dynamic_cast<SemaphoreConstraint *>(node.get()))
|
||||
{
|
||||
std::tie(inflight_requests, inflight_cost) = ptr->getInflights();
|
||||
std::tie(max_requests, max_cost) = ptr->getLimits();
|
||||
}
|
||||
if (auto * ptr = dynamic_cast<ThrottlerConstraint *>(node))
|
||||
if (auto * ptr = dynamic_cast<ThrottlerConstraint *>(node.get()))
|
||||
{
|
||||
std::tie(max_speed, max_burst) = ptr->getParams();
|
||||
throttling_us = ptr->getThrottlingDuration().count() / 1000;
|
||||
|
@ -1,48 +0,0 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Storages/System/StorageSystemWorkloads.h>
|
||||
#include <Common/Scheduler/Workload/IWorkloadEntityStorage.h>
|
||||
#include <Parsers/ASTCreateWorkloadQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ColumnsDescription StorageSystemWorkloads::getColumnsDescription()
|
||||
{
|
||||
return ColumnsDescription
|
||||
{
|
||||
{"name", std::make_shared<DataTypeString>(), "The name of the workload."},
|
||||
{"parent", std::make_shared<DataTypeString>(), "The name of the parent workload."},
|
||||
{"create_query", std::make_shared<DataTypeString>(), "CREATE query of the workload."},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemWorkloads::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) 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<ASTCreateWorkloadQuery &>(*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<ASTs> & /* partitions */)
|
||||
{
|
||||
// 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<ASTs> & /* partitions */)
|
||||
{
|
||||
// TODO(serxa): add restore for workloads
|
||||
// storage.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
||||
}
|
@ -1,29 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
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<ASTs> & partitions) override;
|
||||
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
||||
|
||||
protected:
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const override;
|
||||
};
|
||||
|
||||
}
|
@ -23,8 +23,6 @@
|
||||
#include <Storages/System/StorageSystemEvents.h>
|
||||
#include <Storages/System/StorageSystemFormats.h>
|
||||
#include <Storages/System/StorageSystemFunctions.h>
|
||||
#include <Storages/System/StorageSystemWorkloads.h>
|
||||
#include <Storages/System/StorageSystemResources.h>
|
||||
#include <Storages/System/StorageSystemGraphite.h>
|
||||
#include <Storages/System/StorageSystemMacros.h>
|
||||
#include <Storages/System/StorageSystemMerges.h>
|
||||
@ -232,8 +230,6 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
|
||||
attach<StorageSystemObjectStorageQueueSettings<ObjectStorageType::Azure>>(context, system_database, "azure_queue_settings", "Contains a list of settings of AzureQueue tables.");
|
||||
attach<StorageSystemDashboards>(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<StorageSystemViewRefreshes>(context, system_database, "view_refreshes", "Lists all Refreshable Materialized Views of current server.");
|
||||
attach<StorageSystemWorkloads>(context, system_database, "workloads", "Contains a list of all currently existing workloads.");
|
||||
attach<StorageSystemResources>(context, system_database, "resources", "Contains a list of all currently existing resources.");
|
||||
|
||||
if (has_zookeeper)
|
||||
{
|
||||
|
@ -1,5 +1,4 @@
|
||||
<clickhouse>
|
||||
<workload_zookeeper_path>/clickhouse/workload/definitions.sql</workload_zookeeper_path>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<s3>
|
||||
@ -13,15 +12,6 @@
|
||||
<read_resource>network_read</read_resource>
|
||||
<write_resource>network_write</write_resource>
|
||||
</s3>
|
||||
<s3_no_resource>
|
||||
<type>s3</type>
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
<s3_max_single_part_upload_size>33554432</s3_max_single_part_upload_size>
|
||||
<s3_max_put_rps>10</s3_max_put_rps>
|
||||
<s3_max_get_rps>10</s3_max_get_rps>
|
||||
</s3_no_resource>
|
||||
</disks>
|
||||
<policies>
|
||||
<s3>
|
||||
@ -31,13 +21,6 @@
|
||||
</main>
|
||||
</volumes>
|
||||
</s3>
|
||||
<s3_no_resource>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>s3_no_resource</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</s3_no_resource>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
||||
|
@ -2,7 +2,6 @@
|
||||
# pylint: disable=redefined-outer-name
|
||||
# pylint: disable=line-too-long
|
||||
|
||||
import random
|
||||
import threading
|
||||
import time
|
||||
|
||||
@ -10,7 +9,6 @@ import pytest
|
||||
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
@ -25,21 +23,6 @@ node = cluster.add_instance(
|
||||
"configs/workloads.xml.default",
|
||||
],
|
||||
with_minio=True,
|
||||
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,
|
||||
)
|
||||
|
||||
|
||||
@ -72,22 +55,6 @@ def set_default_configs():
|
||||
yield
|
||||
|
||||
|
||||
@pytest.fixture(scope="function", autouse=True)
|
||||
def clear_workloads_and_resources():
|
||||
node.query(
|
||||
f"""
|
||||
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;
|
||||
drop resource if exists io;
|
||||
"""
|
||||
)
|
||||
yield
|
||||
|
||||
|
||||
def update_workloads_config(**settings):
|
||||
xml = ""
|
||||
for name in settings:
|
||||
@ -603,364 +570,3 @@ 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_no_resource);
|
||||
create resource io_read (read disk s3_no_resource);
|
||||
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;
|
||||
"""
|
||||
)
|
||||
|
||||
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"
|
||||
)
|
||||
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
|
||||
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(
|
||||
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_no_resource';
|
||||
"""
|
||||
)
|
||||
|
||||
node.query(
|
||||
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 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"
|
||||
)
|
||||
|
||||
|
||||
def test_resource_any_disk():
|
||||
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_no_resource';
|
||||
"""
|
||||
)
|
||||
|
||||
node.query(
|
||||
f"""
|
||||
create resource io (write any disk, read any disk);
|
||||
create workload all settings max_cost = 1000000;
|
||||
"""
|
||||
)
|
||||
|
||||
node.query(f"insert into data select * from numbers(1e5) settings workload='all'")
|
||||
|
||||
assert (
|
||||
node.query(
|
||||
f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'"
|
||||
)
|
||||
== "1\n"
|
||||
)
|
||||
|
||||
node.query(f"select sum(key*key) from data settings workload='all'")
|
||||
|
||||
assert (
|
||||
node.query(
|
||||
f"select dequeued_requests>0 from system.scheduler where resource='io' and path ilike '%/all/%' and type='fifo'"
|
||||
)
|
||||
== "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()
|
||||
|
@ -59,8 +59,6 @@ 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
|
||||
@ -68,8 +66,6 @@ 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
|
||||
|
@ -1,5 +0,0 @@
|
||||
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`)
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user