Improve syntax of CREATE QUOTA. Now resource types and key types could be written with underscores.

Also rename columns key_type=>keys and source=>storage in table system.quotas.
This commit is contained in:
Vitaly Baranov 2020-06-05 20:57:33 +03:00
parent 7d1951a79b
commit 4bd00b02e2
12 changed files with 244 additions and 116 deletions

View File

@ -2,6 +2,8 @@
#include <Access/IAccessEntity.h>
#include <Access/RolesOrUsersSet.h>
#include <ext/range.h>
#include <boost/algorithm/string/split.hpp>
#include <boost/lexical_cast.hpp>
#include <chrono>
@ -84,7 +86,8 @@ struct Quota : public IAccessEntity
struct KeyTypeInfo
{
const char * const raw_name;
const String name; /// Lowercased with spaces, e.g. "client key".
const String name; /// Lowercased with underscores, e.g. "client_key".
const std::vector<KeyType> base_types; /// For combined types keeps base types, e.g. for CLIENT_KEY_OR_USER_NAME it keeps [KeyType::CLIENT_KEY, KeyType::USER_NAME].
static const KeyTypeInfo & get(KeyType type);
};
@ -195,8 +198,21 @@ inline const Quota::KeyTypeInfo & Quota::KeyTypeInfo::get(KeyType type)
{
String init_name = raw_name_;
boost::to_lower(init_name);
boost::replace_all(init_name, "_", " ");
return KeyTypeInfo{raw_name_, std::move(init_name)};
std::vector<KeyType> init_base_types;
String replaced = boost::algorithm::replace_all_copy(init_name, "_or_", "|");
Strings tokens;
boost::algorithm::split(tokens, replaced, boost::is_any_of("|"));
if (tokens.size() > 1)
{
for (const auto & token : tokens)
for (auto kt : ext::range(KeyType::MAX))
if (KeyTypeInfo::get(kt).name == token)
{
init_base_types.push_back(kt);
break;
}
}
return KeyTypeInfo{raw_name_, std::move(init_name), std::move(init_base_types)};
};
switch (type)

View File

@ -83,6 +83,23 @@ const char * IntervalKind::toKeyword() const
}
const char * IntervalKind::toLowercasedKeyword() const
{
switch (kind)
{
case IntervalKind::Second: return "second";
case IntervalKind::Minute: return "minute";
case IntervalKind::Hour: return "hour";
case IntervalKind::Day: return "day";
case IntervalKind::Week: return "week";
case IntervalKind::Month: return "month";
case IntervalKind::Quarter: return "quarter";
case IntervalKind::Year: return "year";
}
__builtin_unreachable();
}
const char * IntervalKind::toDateDiffUnit() const
{
switch (kind)

View File

@ -37,6 +37,8 @@ struct IntervalKind
/// Returns an uppercased version of what `toString()` returns.
const char * toKeyword() const;
const char * toLowercasedKeyword() const;
/// Returns the string which can be passed to the `unit` parameter of the dateDiff() function.
/// For example, `IntervalKind{IntervalKind::Day}.getDateDiffParameter()` returns "day".
const char * toDateDiffUnit() const;

View File

@ -132,7 +132,9 @@ namespace
query->names.emplace_back(quota.getName());
query->attach = attach_mode;
if (quota.key_type != Quota::KeyType::NONE)
query->key_type = quota.key_type;
query->all_limits.reserve(quota.all_limits.size());
for (const auto & limits : quota.all_limits)

View File

@ -18,8 +18,28 @@ namespace
void formatKeyType(const KeyType & key_type, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " KEYED BY " << (settings.hilite ? IAST::hilite_none : "") << "'"
<< KeyTypeInfo::get(key_type).name << "'";
const auto & type_info = KeyTypeInfo::get(key_type);
if (key_type == KeyType::NONE)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NOT KEYED" << (settings.hilite ? IAST::hilite_none : "");
return;
}
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " KEYED BY " << (settings.hilite ? IAST::hilite_none : "");
if (!type_info.base_types.empty())
{
bool need_comma = false;
for (const auto & base_type : type_info.base_types)
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << KeyTypeInfo::get(base_type).name;
}
return;
}
settings.ostr << type_info.name;
}
@ -43,20 +63,14 @@ namespace
}
void formatLimit(ResourceType resource_type, ResourceAmount max, bool first, const IAST::FormatSettings & settings)
void formatLimit(ResourceType resource_type, ResourceAmount max, const IAST::FormatSettings & settings)
{
if (first)
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MAX" << (settings.hilite ? IAST::hilite_none : "");
else
settings.ostr << ",";
const auto & type_info = ResourceTypeInfo::get(resource_type);
settings.ostr << " " << (settings.hilite ? IAST::hilite_keyword : "") << type_info.keyword
<< (settings.hilite ? IAST::hilite_none : "") << " " << type_info.amountToString(max);
settings.ostr << " " << type_info.name << " = " << type_info.amountToString(max);
}
void formatLimits(const ASTCreateQuotaQuery::Limits & limits, const IAST::FormatSettings & settings)
void formatIntervalWithLimits(const ASTCreateQuotaQuery::Limits & limits, const IAST::FormatSettings & settings)
{
auto interval_kind = IntervalKind::fromAvgSeconds(limits.duration.count());
Int64 num_intervals = limits.duration.count() / interval_kind.toAvgSeconds();
@ -64,11 +78,11 @@ namespace
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "")
<< " FOR"
<< (limits.randomize_interval ? " RANDOMIZED" : "")
<< " INTERVAL "
<< " INTERVAL"
<< (settings.hilite ? IAST::hilite_none : "")
<< num_intervals << " "
<< " " << num_intervals << " "
<< (settings.hilite ? IAST::hilite_keyword : "")
<< interval_kind.toKeyword()
<< interval_kind.toLowercasedKeyword()
<< (settings.hilite ? IAST::hilite_none : "");
if (limits.drop)
@ -81,17 +95,28 @@ namespace
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (limits.max[resource_type])
{
formatLimit(resource_type, *limits.max[resource_type], !limit_found, settings);
limit_found = true;
}
if (limit_found)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MAX" << (settings.hilite ? IAST::hilite_none : "");
bool need_comma = false;
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (limits.max[resource_type])
{
if (std::exchange(need_comma, true))
settings.ostr << ",";
formatLimit(resource_type, *limits.max[resource_type], settings);
}
if (!limit_found)
}
}
else
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TRACKING ONLY" << (settings.hilite ? IAST::hilite_none : "");
}
}
void formatAllLimits(const std::vector<ASTCreateQuotaQuery::Limits> & all_limits, const IAST::FormatSettings & settings)
void formatIntervalsWithLimits(const std::vector<ASTCreateQuotaQuery::Limits> & all_limits, const IAST::FormatSettings & settings)
{
bool need_comma = false;
for (const auto & limits : all_limits)
@ -100,7 +125,7 @@ namespace
settings.ostr << ",";
need_comma = true;
formatLimits(limits, settings);
formatIntervalWithLimits(limits, settings);
}
}
@ -152,7 +177,7 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat
if (key_type)
formatKeyType(*key_type, settings);
formatAllLimits(all_limits, settings);
formatIntervalsWithLimits(all_limits, settings);
if (roles && (!roles->empty() || alter))
formatToRoles(*roles, settings);

View File

@ -11,17 +11,17 @@ class ASTRolesOrUsersSet;
/** CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name
* [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
* [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
* {MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number} [,...] |
* [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED]
* [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day}
* {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] |
* NO LIMITS | TRACKING ONLY} [,...]]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*
* ALTER QUOTA [IF EXISTS] name
* [RENAME TO new_name]
* [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
* [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
* {MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number} [,...] |
* [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED]
* [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day}
* {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] |
* NO LIMITS | TRACKING ONLY} [,...]]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/

View File

@ -6,6 +6,7 @@
#include <Parsers/ParserRolesOrUsersSet.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTRolesOrUsersSet.h>
#include <ext/range.h>
#include <boost/algorithm/string/predicate.hpp>
@ -39,84 +40,126 @@ namespace
});
}
bool parseKeyType(IParserBase::Pos & pos, Expected & expected, std::optional<Quota::KeyType> & key_type)
bool parseKeyType(IParserBase::Pos & pos, Expected & expected, KeyType & key_type)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"KEYED BY"}.ignore(pos, expected))
if (ParserKeyword{"NOT KEYED"}.ignore(pos, expected))
{
key_type = KeyType::NONE;
return true;
}
if (!ParserKeyword{"KEY BY"}.ignore(pos, expected) && !ParserKeyword{"KEYED BY"}.ignore(pos, expected))
return false;
ASTPtr key_type_ast;
if (!ParserStringLiteral().parse(pos, key_type_ast, expected))
Strings names;
if (!parseIdentifiersOrStringLiterals(pos, expected, names))
return false;
const String & key_type_str = key_type_ast->as<ASTLiteral &>().value.safeGet<const String &>();
String name = boost::algorithm::join(names, "_or_");
boost::to_lower(name);
boost::replace_all(name, " ", "_");
for (auto kt : ext::range(Quota::KeyType::MAX))
if (boost::iequals(KeyTypeInfo::get(kt).name, key_type_str))
if (KeyTypeInfo::get(kt).name == name)
{
key_type = kt;
return true;
}
String all_key_types_str;
String all_types_str;
for (auto kt : ext::range(Quota::KeyType::MAX))
all_key_types_str += String(all_key_types_str.empty() ? "" : ", ") + "'" + KeyTypeInfo::get(kt).name + "'";
String msg = "Quota cannot be keyed by '" + key_type_str + "'. Expected one of these literals: " + all_key_types_str;
all_types_str += String(all_types_str.empty() ? "" : ", ") + "'" + KeyTypeInfo::get(kt).name + "'";
String msg = "Quota cannot be keyed by '" + name + "'. Expected one of the following identifiers: " + all_types_str;
throw Exception(msg, ErrorCodes::SYNTAX_ERROR);
});
}
bool parseLimit(IParserBase::Pos & pos, Expected & expected, bool first, ResourceType & resource_type, ResourceAmount & max)
bool parseResourceType(IParserBase::Pos & pos, Expected & expected, ResourceType & resource_type)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (first)
{
if (!ParserKeyword{"MAX"}.ignore(pos, expected))
return false;
}
else
{
if (!ParserToken{TokenType::Comma}.ignore(pos, expected))
return false;
ParserKeyword{"MAX"}.ignore(pos, expected);
}
std::optional<ResourceType> res_resource_type;
for (auto rt : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (ParserKeyword{ResourceTypeInfo::get(rt).keyword.c_str()}.ignore(pos, expected))
{
res_resource_type = rt;
break;
resource_type = rt;
return true;
}
}
if (!res_resource_type)
ASTPtr ast;
if (!ParserIdentifier{}.parse(pos, ast, expected))
return false;
String name = getIdentifierName(ast);
for (auto rt : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (ResourceTypeInfo::get(rt).name == name)
{
resource_type = rt;
return true;
}
}
return false;
});
}
bool parseMaxAmount(IParserBase::Pos & pos, Expected & expected, ResourceType resource_type, ResourceAmount & max)
{
ASTPtr ast;
if (!ParserNumber{}.parse(pos, ast, expected))
return false;
const Field & max_field = ast->as<ASTLiteral &>().value;
const auto & type_info = ResourceTypeInfo::get(resource_type);
if (type_info.output_denominator == 1)
max = applyVisitor(FieldVisitorConvertToNumber<ResourceAmount>(), max_field);
else
max = static_cast<ResourceAmount>(
applyVisitor(FieldVisitorConvertToNumber<double>(), max_field) * type_info.output_denominator);
return true;
}
bool parseLimit(IParserBase::Pos & pos, Expected & expected, bool first, bool & max_prefix_encountered, ResourceType & resource_type, ResourceAmount & max)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!first && !ParserToken{TokenType::Comma}.ignore(pos, expected))
return false;
max_prefix_encountered |= ParserKeyword{"MAX"}.ignore(pos, expected);
ResourceType res_resource_type;
if (!parseResourceType(pos, expected, res_resource_type))
return false;
if (max_prefix_encountered)
{
ParserToken{TokenType::Equals}.ignore(pos, expected);
}
else
{
if (!ParserKeyword{"MAX"}.ignore(pos, expected))
return false;
}
ResourceAmount res_max;
ASTPtr max_ast;
if (ParserNumber{}.parse(pos, max_ast, expected))
{
const Field & max_field = max_ast->as<ASTLiteral &>().value;
const auto & type_info = ResourceTypeInfo::get(*res_resource_type);
if (type_info.output_denominator == 1)
res_max = applyVisitor(FieldVisitorConvertToNumber<ResourceAmount>(), max_field);
else
res_max = static_cast<ResourceAmount>(
applyVisitor(FieldVisitorConvertToNumber<double>(), max_field) * type_info.output_denominator);
}
else
if (!parseMaxAmount(pos, expected, res_resource_type, res_max))
return false;
resource_type = *res_resource_type;
resource_type = res_resource_type;
max = res_max;
return true;
});
}
bool parseLimits(IParserBase::Pos & pos, Expected & expected, ASTCreateQuotaQuery::Limits & limits)
bool parseIntervalWithLimits(IParserBase::Pos & pos, Expected & expected, ASTCreateQuotaQuery::Limits & limits)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -126,8 +169,7 @@ namespace
new_limits.randomize_interval = ParserKeyword{"RANDOMIZED"}.ignore(pos, expected);
if (!ParserKeyword{"INTERVAL"}.ignore(pos, expected))
return false;
ParserKeyword{"INTERVAL"}.ignore(pos, expected);
ASTPtr num_intervals_ast;
if (!ParserNumber{}.parse(pos, num_intervals_ast, expected))
@ -152,11 +194,12 @@ namespace
{
ResourceType resource_type;
ResourceAmount max;
if (!parseLimit(pos, expected, true, resource_type, max))
bool max_prefix_encountered = false;
if (!parseLimit(pos, expected, true, max_prefix_encountered, resource_type, max))
return false;
new_limits.max[resource_type] = max;
while (parseLimit(pos, expected, false, resource_type, max))
while (parseLimit(pos, expected, false, max_prefix_encountered, resource_type, max))
new_limits.max[resource_type] = max;
}
@ -165,7 +208,7 @@ namespace
});
}
bool parseAllLimits(IParserBase::Pos & pos, Expected & expected, std::vector<ASTCreateQuotaQuery::Limits> & all_limits)
bool parseIntervalsWithLimits(IParserBase::Pos & pos, Expected & expected, std::vector<ASTCreateQuotaQuery::Limits> & all_limits)
{
return IParserBase::wrapParseImpl(pos, [&]
{
@ -173,7 +216,7 @@ namespace
do
{
ASTCreateQuotaQuery::Limits limits;
if (!parseLimits(pos, expected, limits))
if (!parseIntervalWithLimits(pos, expected, limits))
{
all_limits.resize(old_size);
return false;
@ -185,6 +228,7 @@ namespace
});
}
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTRolesOrUsersSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
@ -192,7 +236,7 @@ namespace
ASTPtr node;
ParserRolesOrUsersSet roles_p;
roles_p.allowAll().allowRoleNames().allowUserNames().allowCurrentUser().useIDMode(id_mode);
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !roles_p.parse(pos, node, expected))
if (!ParserKeyword{"TO"}.ignore(pos, expected) || !roles_p.parse(pos, node, expected))
return false;
roles = std::static_pointer_cast<ASTRolesOrUsersSet>(node);
@ -256,10 +300,17 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (alter && new_name.empty() && (names.size() == 1) && parseRenameTo(pos, expected, new_name))
continue;
if (!key_type && parseKeyType(pos, expected, key_type))
if (!key_type)
{
KeyType new_key_type;
if (parseKeyType(pos, expected, new_key_type))
{
key_type = new_key_type;
continue;
}
}
if (parseAllLimits(pos, expected, all_limits))
if (parseIntervalsWithLimits(pos, expected, all_limits))
continue;
if (cluster.empty() && parseOnCluster(pos, expected, cluster))

View File

@ -7,17 +7,17 @@ namespace DB
{
/** Parses queries like
* CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name
* [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
* [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
* {MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number} [,...] |
* [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED]
* [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day}
* {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] |
* NO LIMITS | TRACKING ONLY} [,...]]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*
* ALTER QUOTA [IF EXISTS] name
* [RENAME TO new_name]
* [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
* [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
* {MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number} } [,...] |
* [KEYED BY {none | user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED]
* [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day}
* {MAX {{queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number} [,...] |
* NO LIMITS | TRACKING ONLY} [,...]]
* [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
*/

View File

@ -26,7 +26,11 @@ namespace
{
DataTypeEnum8::Values enum_values;
for (auto key_type : ext::range(KeyType::MAX))
enum_values.push_back({KeyTypeInfo::get(key_type).name, static_cast<UInt8>(key_type)});
{
const auto & type_info = KeyTypeInfo::get(key_type);
if ((key_type != KeyType::NONE) && type_info.base_types.empty())
enum_values.push_back({type_info.name, static_cast<Int8>(key_type)});
}
return enum_values;
}
}
@ -37,8 +41,8 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes()
NamesAndTypesList names_and_types{
{"name", std::make_shared<DataTypeString>()},
{"id", std::make_shared<DataTypeUUID>()},
{"source", std::make_shared<DataTypeString>()},
{"key_type", std::make_shared<DataTypeEnum8>(getKeyTypeEnumValues())},
{"storage", std::make_shared<DataTypeString>()},
{"keys", std::make_shared<DataTypeArray>(std::make_shared<DataTypeEnum8>(getKeyTypeEnumValues()))},
{"durations", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>())},
{"apply_to_all", std::make_shared<DataTypeUInt8>()},
{"apply_to_list", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
@ -58,7 +62,8 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context &
auto & column_name = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_id = assert_cast<ColumnUInt128 &>(*res_columns[column_index++]).getData();
auto & column_storage = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_key_type = assert_cast<ColumnInt8 &>(*res_columns[column_index++]).getData();
auto & column_key_types = assert_cast<ColumnInt8 &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData()).getData();
auto & column_key_types_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
auto & column_durations = assert_cast<ColumnUInt32 &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData()).getData();
auto & column_durations_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
auto & column_apply_to_all = assert_cast<ColumnUInt8 &>(*res_columns[column_index++]).getData();
@ -77,7 +82,16 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context &
column_name.insertData(name.data(), name.length());
column_id.push_back(id);
column_storage.insertData(storage_name.data(), storage_name.length());
column_key_type.push_back(static_cast<Int8>(key_type));
if (key_type != KeyType::NONE)
{
const auto & type_info = KeyTypeInfo::get(key_type);
for (auto base_type : type_info.base_types)
column_key_types.push_back(static_cast<Int8>(base_type));
if (type_info.base_types.empty())
column_key_types.push_back(static_cast<Int8>(key_type));
}
column_key_types_offsets.push_back(column_key_types.size());
for (const auto & limits : all_limits)
column_durations.push_back(std::chrono::duration_cast<std::chrono::seconds>(limits.duration).count());

View File

@ -41,7 +41,7 @@ def test_create():
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS PROFILE s1\n"
assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 IDENTIFIED WITH sha256_password HOST LOCAL DEFAULT ROLE rx\n"
assert instance.query("SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a < 1000 TO u1, u2\n"
assert instance.query("SHOW CREATE QUOTA q") == "CREATE QUOTA q KEYED BY \\'none\\' FOR INTERVAL 1 HOUR MAX QUERIES 100 TO ALL EXCEPT rx\n"
assert instance.query("SHOW CREATE QUOTA q") == "CREATE QUOTA q FOR INTERVAL 1 hour MAX queries = 100 TO ALL EXCEPT rx\n"
assert instance.query("SHOW GRANTS FOR u1") == ""
assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx TO u2\n"
assert instance.query("SHOW CREATE ROLE rx") == "CREATE ROLE rx SETTINGS PROFILE s1\n"

View File

@ -61,7 +61,7 @@ def reset_quotas_and_usage_info():
def test_quota_from_users_xml():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", [31556952], 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
@ -76,7 +76,7 @@ def test_quota_from_users_xml():
def test_simpliest_quota():
# Simpliest quota doesn't even track usage.
copy_quota_xml('simpliest.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]
assert system_quota_limits() == ""
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]
@ -87,7 +87,7 @@ def test_simpliest_quota():
def test_tracking_quota():
# Now we're tracking usage.
copy_quota_xml('tracking.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]]
@ -101,7 +101,7 @@ def test_tracking_quota():
def test_exceed_quota():
# Change quota, now the limits are tiny so we will exceed the quota.
copy_quota_xml('tiny_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]]
@ -110,7 +110,7 @@ def test_exceed_quota():
# Change quota, now the limits are enough to execute queries.
copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]]
@ -119,13 +119,13 @@ def test_exceed_quota():
def test_add_remove_interval():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", [31556952], 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
# Add interval.
copy_quota_xml('two_intervals.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952,63113904]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"],
@ -137,7 +137,7 @@ def test_add_remove_interval():
# Remove interval.
copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", [31556952], 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]
@ -146,7 +146,7 @@ def test_add_remove_interval():
# Remove all intervals.
copy_quota_xml('simpliest.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]
assert system_quota_limits() == ""
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]
@ -155,20 +155,21 @@ def test_add_remove_interval():
# Add one interval back.
copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", [31556952], 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
def test_add_remove_quota():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", [31556952], 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
# Add quota.
copy_quota_xml('two_quotas.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"],
["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "client key or user name", "[3600,2629746]", 0, "[]", "[]"]]
print system_quotas()
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"],
["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60],
["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]]
@ -176,7 +177,7 @@ def test_add_remove_quota():
# Drop quota.
copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
@ -188,25 +189,25 @@ def test_add_remove_quota():
# Add one quota back.
copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
def test_reload_users_xml_by_timer():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]]
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]
time.sleep(1) # The modification time of the 'quota.xml' file should be different,
# because config files are reload by timer only when the modification time is changed.
copy_quota_xml('tiny_limits.xml', reload_immediately=False)
assert_eq_with_retry(instance, "SELECT * FROM system.quotas", [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "user name", "[31556952]", 0, "['default']", "[]"]])
assert_eq_with_retry(instance, "SELECT * FROM system.quotas", [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", ['user_name'], "[31556952]", 0, "['default']", "[]"]])
assert_eq_with_retry(instance, "SELECT * FROM system.quota_limits", [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]])
def test_dcl_introspection():
assert instance.query("SHOW QUOTAS") == "myQuota\n"
assert instance.query("SHOW CREATE QUOTA") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES 1000, READ ROWS 1000 TO default\n"
assert instance.query("SHOW CREATE QUOTA") == "CREATE QUOTA myQuota KEYED BY user_name FOR INTERVAL 1 year MAX queries = 1000, read_rows = 1000 TO default\n"
assert re.match("myQuota\\tdefault\\t.*\\t31556952\\t0\\t1000\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t1000\\t0\\t\\\\N\\t.*\\t\\\\N\n",
instance.query("SHOW QUOTA"))
@ -217,7 +218,7 @@ def test_dcl_introspection():
# Add interval.
copy_quota_xml('two_intervals.xml')
assert instance.query("SHOW QUOTAS") == "myQuota\n"
assert instance.query("SHOW CREATE QUOTA") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES 1000, READ ROWS 1000, FOR RANDOMIZED INTERVAL 2 YEAR MAX RESULT BYTES 30000, READ BYTES 20000, EXECUTION TIME 120 TO default\n"
assert instance.query("SHOW CREATE QUOTA") == "CREATE QUOTA myQuota KEYED BY user_name FOR INTERVAL 1 year MAX queries = 1000, read_rows = 1000, FOR RANDOMIZED INTERVAL 2 year MAX result_bytes = 30000, read_bytes = 20000, execution_time = 120 TO default\n"
assert re.match("myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t1000\\t200\\t\\\\N\\t.*\\t\\\\N\n"
"myQuota\\tdefault\\t.*\\t63113904\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t30000\\t0\\t\\\\N\\t0\\t20000\\t.*\\t120",
instance.query("SHOW QUOTA"))
@ -225,8 +226,8 @@ def test_dcl_introspection():
# Drop interval, add quota.
copy_quota_xml('two_quotas.xml')
assert instance.query("SHOW QUOTAS") == "myQuota\nmyQuota2\n"
assert instance.query("SHOW CREATE QUOTA myQuota") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES 1000, READ ROWS 1000 TO default\n"
assert instance.query("SHOW CREATE QUOTA myQuota2") == "CREATE QUOTA myQuota2 KEYED BY \\'client key or user name\\' FOR RANDOMIZED INTERVAL 1 HOUR MAX RESULT ROWS 4000, RESULT BYTES 400000, READ ROWS 4000, READ BYTES 400000, EXECUTION TIME 60, FOR INTERVAL 1 MONTH MAX EXECUTION TIME 1800\n"
assert instance.query("SHOW CREATE QUOTA myQuota") == "CREATE QUOTA myQuota KEYED BY user_name FOR INTERVAL 1 year MAX queries = 1000, read_rows = 1000 TO default\n"
assert instance.query("SHOW CREATE QUOTA myQuota2") == "CREATE QUOTA myQuota2 KEYED BY client_key, user_name FOR RANDOMIZED INTERVAL 1 hour MAX result_rows = 4000, result_bytes = 400000, read_rows = 4000, read_bytes = 400000, execution_time = 60, FOR INTERVAL 1 month MAX execution_time = 1800\n"
assert re.match("myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t1000\\t200\\t\\\\N\\t.*\\t\\\\N\n",
instance.query("SHOW QUOTA"))
@ -242,7 +243,7 @@ def test_dcl_management():
assert instance.query("SHOW QUOTA") == ""
instance.query("CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER")
assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA KEYED BY \\'none\\' FOR INTERVAL 5 QUARTER MAX QUERIES 123 TO default\n"
assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA FOR INTERVAL 5 quarter MAX queries = 123 TO default\n"
assert re.match("qA\\t\\t.*\\t39446190\\t0\\t123\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t\\\\N\n",
instance.query("SHOW QUOTA"))
@ -251,7 +252,7 @@ def test_dcl_management():
instance.query("SHOW QUOTA"))
instance.query("ALTER QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 321, MAX ERRORS 10, FOR INTERVAL 0.5 HOUR MAX EXECUTION TIME 0.5")
assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA KEYED BY \\'none\\' FOR INTERVAL 30 MINUTE MAX EXECUTION TIME 0.5, FOR INTERVAL 5 QUARTER MAX QUERIES 321, ERRORS 10 TO default\n"
assert instance.query("SHOW CREATE QUOTA qA") == "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\\t.*\\t0.5\n"
"qA\\t\\t.*\\t39446190\\t1\\t321\\t0\\t10\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\n",
instance.query("SHOW QUOTA"))
@ -270,7 +271,7 @@ def test_dcl_management():
instance.query("SHOW QUOTA"))
instance.query("ALTER QUOTA qA RENAME TO qB")
assert instance.query("SHOW CREATE QUOTA qB") == "CREATE QUOTA qB KEYED BY \\'none\\' FOR RANDOMIZED INTERVAL 16 MONTH TRACKING ONLY TO default\n"
assert instance.query("SHOW CREATE QUOTA qB") == "CREATE QUOTA qB FOR RANDOMIZED INTERVAL 16 month TRACKING ONLY TO default\n"
assert re.match("qB\\t\\t.*\\t42075936\\t1\\t\\\\N\\t0\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t50\\t\\\\N\\t200\\t\\\\N\\t.*\\t\\\\N\n",
instance.query("SHOW QUOTA"))

View File

@ -1,2 +1,2 @@
default
CREATE QUOTA default KEYED BY \'user name\' FOR INTERVAL 1 HOUR TRACKING ONLY TO default, readonly
CREATE QUOTA default KEYED BY user_name FOR INTERVAL 1 hour TRACKING ONLY TO default, readonly