mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #35736 from CurtizJ/quota-written-bytes
Add quota for written bytes
This commit is contained in:
commit
5a47958744
@ -107,6 +107,11 @@ const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type)
|
||||
static const auto info = make_info("EXECUTION_TIME", 1000000000 /* execution_time is stored in nanoseconds */);
|
||||
return info;
|
||||
}
|
||||
case QuotaType::WRITTEN_BYTES:
|
||||
{
|
||||
static const auto info = make_info("WRITTEN_BYTES", 1);
|
||||
return info;
|
||||
}
|
||||
case QuotaType::MAX: break;
|
||||
}
|
||||
throw Exception("Unexpected quota type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -20,6 +20,7 @@ enum class QuotaType
|
||||
READ_ROWS, /// Number of rows read from tables.
|
||||
READ_BYTES, /// Number of bytes read from tables.
|
||||
EXECUTION_TIME, /// Total amount of query execution time in nanoseconds.
|
||||
WRITTEN_BYTES, /// Number of bytes written to tables.
|
||||
|
||||
MAX
|
||||
};
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int QUOTA_EXPIRED;
|
||||
extern const int QUOTA_EXCEEDED;
|
||||
}
|
||||
|
||||
|
||||
@ -33,7 +33,7 @@ struct EnabledQuota::Impl
|
||||
"Quota for user " + backQuote(user_name) + " for " + to_string(duration) + " has been exceeded: "
|
||||
+ type_info.valueToStringWithName(used) + "/" + type_info.valueToString(max) + ". "
|
||||
+ "Interval will end at " + to_string(end_of_interval) + ". " + "Name of quota template: " + backQuote(quota_name),
|
||||
ErrorCodes::QUOTA_EXPIRED);
|
||||
ErrorCodes::QUOTA_EXCEEDED);
|
||||
}
|
||||
|
||||
|
||||
|
@ -208,7 +208,7 @@
|
||||
M(198, DNS_ERROR) \
|
||||
M(199, UNKNOWN_QUOTA) \
|
||||
M(200, QUOTA_DOESNT_ALLOW_KEYS) \
|
||||
M(201, QUOTA_EXPIRED) \
|
||||
M(201, QUOTA_EXCEEDED) \
|
||||
M(202, TOO_MANY_SIMULTANEOUS_QUERIES) \
|
||||
M(203, NO_FREE_CONNECTION) \
|
||||
M(204, CANNOT_FSYNC) \
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorHash.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Access/EnabledQuota.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
@ -197,6 +198,9 @@ void AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
|
||||
copyData(*read_buf, write_buf);
|
||||
}
|
||||
|
||||
if (auto quota = query_context->getQuota())
|
||||
quota->used(QuotaType::WRITTEN_BYTES, bytes.size());
|
||||
|
||||
auto entry = std::make_shared<InsertData::Entry>(std::move(bytes), query_context->getCurrentQueryId());
|
||||
InsertQuery key{query, settings};
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Access/EnabledQuota.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Processors/Transforms/buildPushingToViewsChain.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -51,6 +52,8 @@ InterpreterInsertQuery::InterpreterInsertQuery(
|
||||
, async_insert(async_insert_)
|
||||
{
|
||||
checkStackSize();
|
||||
if (auto quota = getContext()->getQuota())
|
||||
quota->checkExceeded(QuotaType::WRITTEN_BYTES);
|
||||
}
|
||||
|
||||
|
||||
@ -269,7 +272,7 @@ Chain InterpreterInsertQuery::buildChainImpl(
|
||||
table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0));
|
||||
}
|
||||
|
||||
auto counting = std::make_shared<CountingTransform>(out.getInputHeader(), thread_status);
|
||||
auto counting = std::make_shared<CountingTransform>(out.getInputHeader(), thread_status, getContext()->getQuota());
|
||||
counting->setProcessListElement(context_ptr->getProcessListElement());
|
||||
out.addSource(std::move(counting));
|
||||
|
||||
|
@ -7,7 +7,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TOO_MANY_ROWS_OR_BYTES;
|
||||
extern const int QUOTA_EXPIRED;
|
||||
extern const int QUOTA_EXCEEDED;
|
||||
extern const int QUERY_WAS_CANCELLED;
|
||||
}
|
||||
|
||||
@ -34,7 +34,7 @@ static bool checkCanAddAdditionalInfoToException(const DB::Exception & exception
|
||||
{
|
||||
/// Don't add additional info to limits and quota exceptions, and in case of kill query (to pass tests).
|
||||
return exception.code() != ErrorCodes::TOO_MANY_ROWS_OR_BYTES
|
||||
&& exception.code() != ErrorCodes::QUOTA_EXPIRED
|
||||
&& exception.code() != ErrorCodes::QUOTA_EXCEEDED
|
||||
&& exception.code() != ErrorCodes::QUERY_WAS_CANCELLED;
|
||||
}
|
||||
|
||||
|
@ -18,11 +18,12 @@ namespace DB
|
||||
|
||||
void CountingTransform::onConsume(Chunk chunk)
|
||||
{
|
||||
if (quota)
|
||||
quota->used(QuotaType::WRITTEN_BYTES, chunk.bytes());
|
||||
|
||||
Progress local_progress{WriteProgress(chunk.getNumRows(), chunk.bytes())};
|
||||
progress.incrementPiecewiseAtomically(local_progress);
|
||||
|
||||
//std::cerr << "============ counting adding progress for " << static_cast<const void *>(thread_status) << ' ' << chunk.getNumRows() << " rows\n";
|
||||
|
||||
if (thread_status)
|
||||
{
|
||||
thread_status->performance_counters.increment(ProfileEvents::InsertedRows, local_progress.written_rows);
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <IO/Progress.h>
|
||||
#include <Processors/Transforms/ExceptionKeepingTransform.h>
|
||||
#include <Access/EnabledQuota.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,8 +15,12 @@ class ThreadStatus;
|
||||
class CountingTransform final : public ExceptionKeepingTransform
|
||||
{
|
||||
public:
|
||||
explicit CountingTransform(const Block & header, ThreadStatus * thread_status_ = nullptr)
|
||||
: ExceptionKeepingTransform(header, header), thread_status(thread_status_) {}
|
||||
explicit CountingTransform(
|
||||
const Block & header,
|
||||
ThreadStatus * thread_status_ = nullptr,
|
||||
std::shared_ptr<const EnabledQuota> quota_ = nullptr)
|
||||
: ExceptionKeepingTransform(header, header)
|
||||
, thread_status(thread_status_), quota(std::move(quota_)) {}
|
||||
|
||||
String getName() const override { return "CountingTransform"; }
|
||||
|
||||
@ -47,6 +52,9 @@ protected:
|
||||
ProgressCallback progress_callback;
|
||||
QueryStatus * process_elem = nullptr;
|
||||
ThreadStatus * thread_status = nullptr;
|
||||
|
||||
/// Quota is used to limit amount of written bytes.
|
||||
std::shared_ptr<const EnabledQuota> quota;
|
||||
Chunk cur_chunk;
|
||||
};
|
||||
|
||||
|
@ -129,6 +129,7 @@ def test_quota_from_users_xml():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -349,6 +350,7 @@ def test_tracking_quota():
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -454,7 +456,7 @@ def test_exceed_quota():
|
||||
]
|
||||
)
|
||||
system_quota_limits(
|
||||
[["myQuota", 31556952, 0, 1, 1, 1, 1, 1, "\\N", 1, "\\N", "\\N"]]
|
||||
[["myQuota", 31556952, 0, 1, 1, 1, 1, 1, "\\N", 1, "\\N", "\\N", "\\N"]]
|
||||
)
|
||||
system_quota_usage(
|
||||
[
|
||||
@ -545,6 +547,7 @@ def test_exceed_quota():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -634,6 +637,7 @@ def test_add_remove_interval():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -695,6 +699,7 @@ def test_add_remove_interval():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
],
|
||||
[
|
||||
"myQuota",
|
||||
@ -709,6 +714,7 @@ def test_add_remove_interval():
|
||||
"\\N",
|
||||
20000,
|
||||
120,
|
||||
"\\N",
|
||||
],
|
||||
]
|
||||
)
|
||||
@ -842,6 +848,7 @@ def test_add_remove_interval():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1003,6 +1010,7 @@ def test_add_remove_interval():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1064,6 +1072,7 @@ def test_add_remove_quota():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1136,6 +1145,7 @@ def test_add_remove_quota():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
],
|
||||
[
|
||||
"myQuota2",
|
||||
@ -1150,6 +1160,7 @@ def test_add_remove_quota():
|
||||
4000,
|
||||
400000,
|
||||
60,
|
||||
"\\N",
|
||||
],
|
||||
[
|
||||
"myQuota2",
|
||||
@ -1164,6 +1175,7 @@ def test_add_remove_quota():
|
||||
"\\N",
|
||||
"\\N",
|
||||
1800,
|
||||
"\\N",
|
||||
],
|
||||
]
|
||||
)
|
||||
@ -1226,6 +1238,7 @@ def test_add_remove_quota():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1294,6 +1307,7 @@ def test_add_remove_quota():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1356,6 +1370,7 @@ def test_reload_users_xml_by_timer():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
@ -1382,7 +1397,7 @@ def test_reload_users_xml_by_timer():
|
||||
assert_eq_with_retry(
|
||||
instance,
|
||||
"SELECT * FROM system.quota_limits",
|
||||
[["myQuota", 31556952, 0, 1, 1, 1, 1, 1, "\\N", 1, "\\N", "\\N"]],
|
||||
[["myQuota", 31556952, 0, 1, 1, 1, 1, 1, "\\N", 1, "\\N", "\\N", "\\N"]],
|
||||
)
|
||||
|
||||
|
||||
@ -1481,15 +1496,15 @@ def test_dcl_management():
|
||||
== "CREATE QUOTA qA FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default\n"
|
||||
)
|
||||
assert re.match(
|
||||
"qA\\t\\t.*\\t1800\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t0.5\n"
|
||||
"qA\\t\\t.*\\t39446190\\t1\\t321\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t10\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\n",
|
||||
"qA\\t\\t.*\\t1800\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t0.5\\t0\\t\\\\N\n"
|
||||
"qA\\t\\t.*\\t39446190\\t1\\t321\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t10\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\\t0\\t\\\\N\n",
|
||||
instance.query("SHOW QUOTA"),
|
||||
)
|
||||
|
||||
instance.query("SELECT * from test_table")
|
||||
assert re.match(
|
||||
"qA\\t\\t.*\\t1800\\t1\\t\\\\N\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t0.5\n"
|
||||
"qA\\t\\t.*\\t39446190\\t2\\t321\\t2\\t\\\\N\\t0\\t\\\\N\\t0\\t10\\t100\\t\\\\N\\t400\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t.*\\t\\\\N\n",
|
||||
"qA\\t\\t.*\\t1800\\t1\\t\\\\N\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t0.5\\t0\\t\\\\N\n"
|
||||
"qA\\t\\t.*\\t39446190\\t2\\t321\\t2\\t\\\\N\\t0\\t\\\\N\\t0\\t10\\t100\\t\\\\N\\t400\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t.*\\t\\\\N\\t0\\t\\\\N\n",
|
||||
instance.query("SHOW QUOTA"),
|
||||
)
|
||||
|
||||
@ -1503,7 +1518,7 @@ def test_dcl_management():
|
||||
|
||||
instance.query("SELECT * from test_table")
|
||||
assert re.match(
|
||||
"qA\\t\\t.*\\t42075936\\t1\\t\\\\N\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\n",
|
||||
"qA\\t\\t.*\\t42075936\\t1\\t\\\\N\\t1\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\\t0\\t\\\\N\n",
|
||||
instance.query("SHOW QUOTA"),
|
||||
)
|
||||
|
||||
@ -1519,7 +1534,7 @@ def test_dcl_management():
|
||||
|
||||
instance.query("SELECT * from test_table")
|
||||
assert re.match(
|
||||
"qB\\t\\t.*\\t42075936\\t2\\t\\\\N\\t2\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t.*\\t\\\\N\n",
|
||||
"qB\\t\\t.*\\t42075936\\t2\\t\\\\N\\t2\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t100\\t\\\\N\\t400\\t\\\\N\\t.*\\t\\\\N\\t0\\t\\\\N\n",
|
||||
instance.query("SHOW QUOTA"),
|
||||
)
|
||||
|
||||
@ -1563,6 +1578,7 @@ def test_query_inserts():
|
||||
1000,
|
||||
"\\N",
|
||||
"\\N",
|
||||
"\\N",
|
||||
]
|
||||
]
|
||||
)
|
||||
|
@ -57,10 +57,10 @@ q2_01297 local directory [] [5259492] 0 ['r1_01297','u1_01297'] []
|
||||
q3_01297 local directory ['client_key','user_name'] [5259492,15778476] 0 [] []
|
||||
q4_01297 local directory [] [604800] 1 [] ['u1_01297']
|
||||
-- system.quota_limits
|
||||
q2_01297 5259492 0 100 \N \N 11 1000 10000 1001 10001 2.5
|
||||
q3_01297 5259492 0 \N \N \N \N 1002 \N \N \N \N
|
||||
q3_01297 15778476 0 100 \N \N 11 \N \N \N \N \N
|
||||
q4_01297 604800 0 \N \N \N \N \N \N \N \N \N
|
||||
q2_01297 5259492 0 100 \N \N 11 1000 10000 1001 10001 2.5 \N
|
||||
q3_01297 5259492 0 \N \N \N \N 1002 \N \N \N \N \N
|
||||
q3_01297 15778476 0 100 \N \N 11 \N \N \N \N \N \N
|
||||
q4_01297 604800 0 \N \N \N \N \N \N \N \N \N \N
|
||||
-- query_selects query_inserts
|
||||
CREATE QUOTA q1_01297 KEYED BY user_name FOR INTERVAL 1 minute MAX query_selects = 1 TO r1_01297
|
||||
CREATE QUOTA q2_01297 KEYED BY user_name FOR INTERVAL 1 minute MAX query_inserts = 1 TO r1_01297
|
||||
|
@ -39,10 +39,10 @@ CREATE TABLE system.privileges\n(\n `privilege` Enum16(\'SHOW DATABASES\' = 0
|
||||
CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `distributed_depth` UInt64,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.projection_parts_columns\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quota_limits\n(\n `quota_name` String,\n `duration` UInt32,\n `is_randomized_interval` UInt8,\n `max_queries` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotaLimits()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quota_usage\n(\n `quota_name` String,\n `quota_key` String,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotaUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quota_limits\n(\n `quota_name` String,\n `duration` UInt32,\n `is_randomized_interval` UInt8,\n `max_queries` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `max_execution_time` Nullable(Float64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotaLimits()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quota_usage\n(\n `quota_name` String,\n `quota_key` String,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64),\n `written_bytes` Nullable(UInt64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotaUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quotas\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `keys` Array(Enum8(\'user_name\' = 1, \'ip_address\' = 2, \'forwarded_ip_address\' = 3, \'client_key\' = 4)),\n `durations` Array(UInt32),\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemQuotas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quotas_usage\n(\n `quota_name` String,\n `quota_key` String,\n `is_current` UInt8,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotasUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.quotas_usage\n(\n `quota_name` String,\n `quota_key` String,\n `is_current` UInt8,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64),\n `written_bytes` Nullable(UInt64),\n `max_written_bytes` Nullable(UInt64)\n)\nENGINE = SystemQuotasUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.replicas\n(\n `database` String,\n `table` String,\n `engine` String,\n `is_leader` UInt8,\n `can_become_leader` UInt8,\n `is_readonly` UInt8,\n `is_session_expired` UInt8,\n `future_parts` UInt32,\n `parts_to_check` UInt32,\n `zookeeper_path` String,\n `replica_name` String,\n `replica_path` String,\n `columns_version` Int32,\n `queue_size` UInt32,\n `inserts_in_queue` UInt32,\n `merges_in_queue` UInt32,\n `part_mutations_in_queue` UInt32,\n `queue_oldest_time` DateTime,\n `inserts_oldest_time` DateTime,\n `merges_oldest_time` DateTime,\n `part_mutations_oldest_time` DateTime,\n `oldest_part_to_get` String,\n `oldest_part_to_merge_to` String,\n `oldest_part_to_mutate_to` String,\n `log_max_index` UInt64,\n `log_pointer` UInt64,\n `last_queue_update` DateTime,\n `absolute_delay` UInt64,\n `total_replicas` UInt8,\n `active_replicas` UInt8,\n `last_queue_update_exception` String,\n `zookeeper_exception` String,\n `replica_is_active` Map(String, UInt8)\n)\nENGINE = SystemReplicas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.replicated_fetches\n(\n `database` String,\n `table` String,\n `elapsed` Float64,\n `progress` Float64,\n `result_part_name` String,\n `result_part_path` String,\n `partition_id` String,\n `total_size_bytes_compressed` UInt64,\n `bytes_read_compressed` UInt64,\n `source_replica_path` String,\n `source_replica_hostname` String,\n `source_replica_port` UInt16,\n `interserver_scheme` String,\n `URI` String,\n `to_detached` UInt8,\n `thread_id` UInt64\n)\nENGINE = SystemReplicatedFetches()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
CREATE TABLE system.replicated_merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemReplicatedMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
|
||||
|
@ -1,2 +1,2 @@
|
||||
QUOTA_EXPIRED
|
||||
QUOTA_EXCEEDED
|
||||
2
|
||||
|
@ -20,7 +20,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02246 FOR INTERVAL 100 YEAR MAX QUERY INS
|
||||
|
||||
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (1, 'a')"
|
||||
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (2, 'b')"
|
||||
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (3, 'c')" 2>&1 | grep -m1 -o QUOTA_EXPIRED
|
||||
${CLICKHOUSE_CLIENT} --user u02246 --async_insert 1 -q "INSERT INTO async_inserts_02246 VALUES (3, 'c')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
|
||||
|
||||
sleep 1.0
|
||||
|
||||
|
@ -0,0 +1,7 @@
|
||||
QUOTA_EXCEEDED
|
||||
QUOTA_EXCEEDED
|
||||
1
|
||||
2
|
||||
QUOTA_EXCEEDED
|
||||
1
|
||||
50
|
42
tests/queries/0_stateless/02247_written_bytes_quota.sh
Executable file
42
tests/queries/0_stateless/02247_written_bytes_quota.sh
Executable file
@ -0,0 +1,42 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE written_bytes_02247(s String) ENGINE = Memory"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02247"
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE USER u02247"
|
||||
${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247"
|
||||
${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247"
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 25 TO r02247"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
|
||||
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
|
||||
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
|
||||
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 10 FROM system.quotas_usage WHERE quota_name = 'q02247'"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP QUOTA q02247"
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247"
|
||||
${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)"
|
||||
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT count() FROM written_bytes_02247"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS written_bytes_02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP ROLE IF EXISTS r02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP USER IF EXISTS u02247"
|
||||
${CLICKHOUSE_CLIENT} -q "DROP QUOTA IF EXISTS q02247"
|
Loading…
Reference in New Issue
Block a user