Rename Quota::KeyType -> QuotaKeyType and move it to Access/Common.

This commit is contained in:
Vitaly Baranov 2021-11-18 21:43:02 +03:00
parent 6634fcbac7
commit 06077a2952
14 changed files with 162 additions and 155 deletions

View File

@ -1,7 +1,12 @@
#include <Access/Common/QuotaDefs.h>
#include <Common/Exception.h>
#include <base/range.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/classification.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/lexical_cast.hpp>
@ -13,6 +18,12 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
String toString(QuotaType type)
{
return QuotaTypeInfo::get(type).raw_name;
}
String QuotaTypeInfo::valueToString(QuotaValue value) const
{
if (!(value % output_denominator))
@ -101,4 +112,78 @@ const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type)
throw Exception("Unexpected quota type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
String toString(QuotaKeyType type)
{
return QuotaKeyTypeInfo::get(type).raw_name;
}
const QuotaKeyTypeInfo & QuotaKeyTypeInfo::get(QuotaKeyType type)
{
static constexpr auto make_info = [](const char * raw_name_)
{
String init_name = raw_name_;
boost::to_lower(init_name);
std::vector<QuotaKeyType> 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 : collections::range(QuotaKeyType::MAX))
{
if (QuotaKeyTypeInfo::get(kt).name == token)
{
init_base_types.push_back(kt);
break;
}
}
}
}
return QuotaKeyTypeInfo{raw_name_, std::move(init_name), std::move(init_base_types)};
};
switch (type)
{
case QuotaKeyType::NONE:
{
static const auto info = make_info("NONE");
return info;
}
case QuotaKeyType::USER_NAME:
{
static const auto info = make_info("USER_NAME");
return info;
}
case QuotaKeyType::IP_ADDRESS:
{
static const auto info = make_info("IP_ADDRESS");
return info;
}
case QuotaKeyType::FORWARDED_IP_ADDRESS:
{
static const auto info = make_info("FORWARDED_IP_ADDRESS");
return info;
}
case QuotaKeyType::CLIENT_KEY:
{
static const auto info = make_info("CLIENT_KEY");
return info;
}
case QuotaKeyType::CLIENT_KEY_OR_USER_NAME:
{
static const auto info = make_info("CLIENT_KEY_OR_USER_NAME");
return info;
}
case QuotaKeyType::CLIENT_KEY_OR_IP_ADDRESS:
{
static const auto info = make_info("CLIENT_KEY_OR_IP_ADDRESS");
return info;
}
case QuotaKeyType::MAX: break;
}
throw Exception("Unexpected quota key type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -5,8 +5,10 @@
namespace DB
{
/// We use UInt64 to count used resources.
using QuotaValue = UInt64;
/// Kinds of resource what we wish to quota.
enum class QuotaType
{
QUERIES, /// Number of queries.
@ -22,6 +24,8 @@ enum class QuotaType
MAX
};
String toString(QuotaType type);
struct QuotaTypeInfo
{
const char * const raw_name = "";
@ -35,6 +39,29 @@ struct QuotaTypeInfo
static const QuotaTypeInfo & get(QuotaType type);
};
String toString(QuotaType type);
/// Key to share quota consumption.
/// Users with the same key share the same amount of resource.
enum class QuotaKeyType
{
NONE, /// All users share the same quota.
USER_NAME, /// Connections with the same user name share the same quota.
IP_ADDRESS, /// Connections from the same IP share the same quota.
FORWARDED_IP_ADDRESS, /// Use X-Forwarded-For HTTP header instead of IP address.
CLIENT_KEY, /// Client should explicitly supply a key to use.
CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key.
CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key.
MAX
};
String toString(QuotaKeyType type);
struct QuotaKeyTypeInfo
{
const char * const raw_name;
const String name; /// Lowercased with underscores, e.g. "client_key".
const std::vector<QuotaKeyType> 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 QuotaKeyTypeInfo & get(QuotaKeyType type);
};
}

View File

@ -1,13 +1,15 @@
#pragma once
#include <Access/Quota.h>
#include <Access/Common/QuotaDefs.h>
#include <Core/UUID.h>
#include <Poco/Net/IPAddress.h>
#include <boost/container/flat_set.hpp>
#include <boost/noncopyable.hpp>
#include <boost/smart_ptr/atomic_shared_ptr.hpp>
#include <atomic>
#include <chrono>
#include <memory>
#include <optional>
namespace DB

View File

@ -3,19 +3,11 @@
#include <Access/IAccessEntity.h>
#include <Access/Common/QuotaDefs.h>
#include <Access/RolesOrUsersSet.h>
#include <base/range.h>
#include <boost/algorithm/string/split.hpp>
#include <boost/lexical_cast.hpp>
#include <chrono>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Quota for resources consumption for specific interval.
* Used to limit resource usage by user.
@ -44,28 +36,7 @@ struct Quota : public IAccessEntity
/// Key to share quota consumption.
/// Users with the same key share the same amount of resource.
enum class KeyType
{
NONE, /// All users share the same quota.
USER_NAME, /// Connections with the same user name share the same quota.
IP_ADDRESS, /// Connections from the same IP share the same quota.
FORWARDED_IP_ADDRESS, /// Use X-Forwarded-For HTTP header instead of IP address.
CLIENT_KEY, /// Client should explicitly supply a key to use.
CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key.
CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key.
MAX
};
struct KeyTypeInfo
{
const char * const raw_name;
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);
};
KeyType key_type = KeyType::NONE;
QuotaKeyType key_type = QuotaKeyType::NONE;
/// Which roles or users should use this quota.
RolesOrUsersSet to_roles;
@ -76,79 +47,5 @@ struct Quota : public IAccessEntity
Type getType() const override { return TYPE; }
};
inline String toString(Quota::KeyType type)
{
return Quota::KeyTypeInfo::get(type).raw_name;
}
inline const Quota::KeyTypeInfo & Quota::KeyTypeInfo::get(KeyType type)
{
static constexpr auto make_info = [](const char * raw_name_)
{
String init_name = raw_name_;
boost::to_lower(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 : collections::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)
{
case KeyType::NONE:
{
static const auto info = make_info("NONE");
return info;
}
case KeyType::USER_NAME:
{
static const auto info = make_info("USER_NAME");
return info;
}
case KeyType::IP_ADDRESS:
{
static const auto info = make_info("IP_ADDRESS");
return info;
}
case KeyType::FORWARDED_IP_ADDRESS:
{
static const auto info = make_info("FORWARDED_IP_ADDRESS");
return info;
}
case KeyType::CLIENT_KEY:
{
static const auto info = make_info("CLIENT_KEY");
return info;
}
case KeyType::CLIENT_KEY_OR_USER_NAME:
{
static const auto info = make_info("CLIENT_KEY_OR_USER_NAME");
return info;
}
case KeyType::CLIENT_KEY_OR_IP_ADDRESS:
{
static const auto info = make_info("CLIENT_KEY_OR_IP_ADDRESS");
return info;
}
case KeyType::MAX: break;
}
throw Exception("Unexpected quota key type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
using QuotaPtr = std::shared_ptr<const Quota>;
}

View File

@ -1,4 +1,5 @@
#include <Access/EnabledQuota.h>
#include <Access/Quota.h>
#include <Access/QuotaCache.h>
#include <Access/QuotaUsage.h>
#include <Access/AccessControl.h>
@ -44,26 +45,25 @@ void QuotaCache::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota
String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
{
const auto & params = enabled.params;
using KeyType = Quota::KeyType;
switch (quota->key_type)
{
case KeyType::NONE:
case QuotaKeyType::NONE:
{
return "";
}
case KeyType::USER_NAME:
case QuotaKeyType::USER_NAME:
{
return params.user_name;
}
case KeyType::IP_ADDRESS:
case QuotaKeyType::IP_ADDRESS:
{
return params.client_address.toString();
}
case KeyType::FORWARDED_IP_ADDRESS:
case QuotaKeyType::FORWARDED_IP_ADDRESS:
{
return params.forwarded_address;
}
case KeyType::CLIENT_KEY:
case QuotaKeyType::CLIENT_KEY:
{
if (!params.client_key.empty())
return params.client_key;
@ -71,19 +71,19 @@ String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const
"Quota " + quota->getName() + " (for user " + params.user_name + ") requires a client supplied key.",
ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY);
}
case KeyType::CLIENT_KEY_OR_USER_NAME:
case QuotaKeyType::CLIENT_KEY_OR_USER_NAME:
{
if (!params.client_key.empty())
return params.client_key;
return params.user_name;
}
case KeyType::CLIENT_KEY_OR_IP_ADDRESS:
case QuotaKeyType::CLIENT_KEY_OR_IP_ADDRESS:
{
if (!params.client_key.empty())
return params.client_key;
return params.client_address.toString();
}
case KeyType::MAX: break;
case QuotaKeyType::MAX: break;
}
throw Exception("Unexpected quota key type: " + std::to_string(static_cast<int>(quota->key_type)), ErrorCodes::LOGICAL_ERROR);
}

View File

@ -11,7 +11,9 @@
namespace DB
{
class AccessControl;
struct Quota;
using QuotaPtr = std::shared_ptr<const Quota>;
struct RolesOrUsersSet;
/// Stores information how much amount of resources have been consumed and how much are left.
class QuotaCache

View File

@ -1,7 +1,8 @@
#pragma once
#include <Access/Quota.h>
#include <Access/Common/QuotaDefs.h>
#include <chrono>
#include <optional>
namespace DB

View File

@ -223,16 +223,15 @@ namespace
auto quota = std::make_shared<Quota>();
quota->setName(quota_name);
using KeyType = Quota::KeyType;
String quota_config = "quotas." + quota_name;
if (config.has(quota_config + ".keyed_by_ip"))
quota->key_type = KeyType::IP_ADDRESS;
quota->key_type = QuotaKeyType::IP_ADDRESS;
else if (config.has(quota_config + ".keyed_by_forwarded_ip"))
quota->key_type = KeyType::FORWARDED_IP_ADDRESS;
quota->key_type = QuotaKeyType::FORWARDED_IP_ADDRESS;
else if (config.has(quota_config + ".keyed"))
quota->key_type = KeyType::CLIENT_KEY_OR_USER_NAME;
quota->key_type = QuotaKeyType::CLIENT_KEY_OR_USER_NAME;
else
quota->key_type = KeyType::USER_NAME;
quota->key_type = QuotaKeyType::USER_NAME;
Poco::Util::AbstractConfiguration::Keys interval_keys;
config.keys(quota_config, interval_keys);

View File

@ -3,6 +3,7 @@
#include <Parsers/Access/ASTRolesOrUsersSet.h>
#include <Access/AccessControl.h>
#include <Access/Common/AccessFlags.h>
#include <Access/Quota.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <base/range.h>

View File

@ -14,11 +14,12 @@
#include <Parsers/parseQuery.h>
#include <Access/AccessControl.h>
#include <Access/EnabledQuota.h>
#include <Access/Quota.h>
#include <Access/QuotaUsage.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/SettingsProfile.h>
#include <Access/User.h>
#include <Columns/ColumnString.h>
#include <Common/StringUtils/StringUtils.h>
#include <Core/Defines.h>
@ -149,7 +150,7 @@ namespace
query->names.emplace_back(quota.getName());
query->attach = attach_mode;
if (quota.key_type != Quota::KeyType::NONE)
if (quota.key_type != QuotaKeyType::NONE)
query->key_type = quota.key_type;
query->all_limits.reserve(quota.all_limits.size());

View File

@ -10,13 +10,10 @@ namespace DB
{
namespace
{
using KeyType = Quota::KeyType;
using KeyTypeInfo = Quota::KeyTypeInfo;
void formatKeyType(const KeyType & key_type, const IAST::FormatSettings & settings)
void formatKeyType(const QuotaKeyType & key_type, const IAST::FormatSettings & settings)
{
const auto & type_info = KeyTypeInfo::get(key_type);
if (key_type == KeyType::NONE)
const auto & type_info = QuotaKeyTypeInfo::get(key_type);
if (key_type == QuotaKeyType::NONE)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NOT KEYED" << (settings.hilite ? IAST::hilite_none : "");
return;
@ -31,7 +28,7 @@ namespace
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << KeyTypeInfo::get(base_type).name;
settings.ostr << QuotaKeyTypeInfo::get(base_type).name;
}
return;
}

View File

@ -2,7 +2,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Access/Quota.h>
#include <Access/Common/QuotaDefs.h>
namespace DB
@ -35,11 +35,9 @@ public:
bool if_not_exists = false;
bool or_replace = false;
using KeyType = Quota::KeyType;
Strings names;
String new_name;
std::optional<KeyType> key_type;
std::optional<QuotaKeyType> key_type;
struct Limits
{

View File

@ -11,7 +11,10 @@
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <base/range.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/predicate.hpp>
#include <boost/algorithm/string/replace.hpp>
namespace DB
@ -24,9 +27,6 @@ namespace ErrorCodes
namespace
{
using KeyType = Quota::KeyType;
using KeyTypeInfo = Quota::KeyTypeInfo;
bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name)
{
return IParserBase::wrapParseImpl(pos, [&]
@ -38,13 +38,13 @@ namespace
});
}
bool parseKeyType(IParserBase::Pos & pos, Expected & expected, KeyType & key_type)
bool parseKeyType(IParserBase::Pos & pos, Expected & expected, QuotaKeyType & key_type)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword{"NOT KEYED"}.ignore(pos, expected))
{
key_type = KeyType::NONE;
key_type = QuotaKeyType::NONE;
return true;
}
@ -59,9 +59,9 @@ namespace
boost::to_lower(name);
boost::replace_all(name, " ", "_");
for (auto kt : collections::range(Quota::KeyType::MAX))
for (auto kt : collections::range(QuotaKeyType::MAX))
{
if (KeyTypeInfo::get(kt).name == name)
if (QuotaKeyTypeInfo::get(kt).name == name)
{
key_type = kt;
return true;
@ -69,8 +69,8 @@ namespace
}
String all_types_str;
for (auto kt : collections::range(Quota::KeyType::MAX))
all_types_str += String(all_types_str.empty() ? "" : ", ") + "'" + KeyTypeInfo::get(kt).name + "'";
for (auto kt : collections::range(QuotaKeyType::MAX))
all_types_str += String(all_types_str.empty() ? "" : ", ") + "'" + QuotaKeyTypeInfo::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);
});
@ -278,7 +278,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
return false;
String new_name;
std::optional<KeyType> key_type;
std::optional<QuotaKeyType> key_type;
std::vector<ASTCreateQuotaQuery::Limits> all_limits;
String cluster;
@ -289,7 +289,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (!key_type)
{
KeyType new_key_type;
QuotaKeyType new_key_type;
if (parseKeyType(pos, expected, new_key_type))
{
key_type = new_key_type;

View File

@ -19,16 +19,13 @@ namespace DB
{
namespace
{
using KeyType = Quota::KeyType;
using KeyTypeInfo = Quota::KeyTypeInfo;
DataTypeEnum8::Values getKeyTypeEnumValues()
{
DataTypeEnum8::Values enum_values;
for (auto key_type : collections::range(KeyType::MAX))
for (auto key_type : collections::range(QuotaKeyType::MAX))
{
const auto & type_info = KeyTypeInfo::get(key_type);
if ((key_type != KeyType::NONE) && type_info.base_types.empty())
const auto & type_info = QuotaKeyTypeInfo::get(key_type);
if ((key_type != QuotaKeyType::NONE) && type_info.base_types.empty())
enum_values.push_back({type_info.name, static_cast<Int8>(key_type)});
}
return enum_values;
@ -76,16 +73,16 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr cont
const UUID & id,
const String & storage_name,
const std::vector<Quota::Limits> & all_limits,
KeyType key_type,
QuotaKeyType key_type,
const RolesOrUsersSet & apply_to)
{
column_name.insertData(name.data(), name.length());
column_id.push_back(id.toUnderType());
column_storage.insertData(storage_name.data(), storage_name.length());
if (key_type != KeyType::NONE)
if (key_type != QuotaKeyType::NONE)
{
const auto & type_info = KeyTypeInfo::get(key_type);
const auto & type_info = QuotaKeyTypeInfo::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())