提交 421120b9 编写于 作者: V Vitaly Baranov

Implement SQL queries to manipulate users and grants.

上级 c604ce1c
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
namespace DB
{
BlockIO InterpreterCreateUserQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQuery(*updated_user, query);
return updated_user;
};
if (query.if_exists)
{
if (auto id = access_control.find<User>(query.name))
access_control.tryUpdate(*id, update_func);
}
else
access_control.update(access_control.getID<User>(query.name), update_func);
}
else
{
auto new_user = std::make_shared<User>();
updateUserFromQuery(*new_user, query);
if (query.if_not_exists)
access_control.tryInsert(new_user);
else if (query.or_replace)
access_control.insertOrReplace(new_user);
else
access_control.insert(new_user);
}
return {};
}
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query)
{
if (query.alter)
{
if (!query.new_name.empty())
user.setName(query.new_name);
}
else
user.setName(query.name);
if (query.authentication)
user.authentication = *query.authentication;
if (query.hosts)
user.allowed_client_hosts = *query.hosts;
if (query.remove_hosts)
user.allowed_client_hosts.remove(*query.remove_hosts);
if (query.add_hosts)
user.allowed_client_hosts.add(*query.add_hosts);
if (query.profile)
user.profile = *query.profile;
}
}
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ASTCreateUserQuery;
struct User;
class InterpreterCreateUserQuery : public IInterpreter
{
public:
InterpreterCreateUserQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
BlockIO execute() override;
private:
void updateUserFromQuery(User & quota, const ASTCreateUserQuery & query);
ASTPtr query_ptr;
Context & context;
};
}
......@@ -5,6 +5,7 @@
#include <Access/AccessFlags.h>
#include <Access/Quota.h>
#include <Access/RowPolicy.h>
#include <Access/User.h>
#include <boost/range/algorithm/transform.hpp>
......@@ -18,6 +19,16 @@ BlockIO InterpreterDropAccessEntityQuery::execute()
switch (query.kind)
{
case Kind::USER:
{
context.checkAccess(AccessType::DROP_USER);
if (query.if_exists)
access_control.tryRemove(access_control.find<User>(query.names));
else
access_control.remove(access_control.getIDs<User>(query.names));
return {};
}
case Kind::QUOTA:
{
context.checkAccess(AccessType::DROP_QUOTA);
......@@ -27,6 +38,7 @@ BlockIO InterpreterDropAccessEntityQuery::execute()
access_control.remove(access_control.getIDs<Quota>(query.names));
return {};
}
case Kind::ROW_POLICY:
{
context.checkAccess(AccessType::DROP_POLICY);
......
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTDropAccessEntityQuery.h>
......@@ -14,6 +15,7 @@
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/ASTShowProcesslistQuery.h>
#include <Parsers/ASTShowGrantsQuery.h>
#include <Parsers/ASTShowQuotasQuery.h>
#include <Parsers/ASTShowRowPoliciesQuery.h>
#include <Parsers/ASTShowTablesQuery.h>
......@@ -21,10 +23,12 @@
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/InterpreterCheckQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
......@@ -43,12 +47,14 @@
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
#include <Interpreters/InterpreterShowProcesslistQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Interpreters/InterpreterShowQuotasQuery.h>
#include <Interpreters/InterpreterShowRowPoliciesQuery.h>
#include <Interpreters/InterpreterShowTablesQuery.h>
#include <Interpreters/InterpreterSystemQuery.h>
#include <Interpreters/InterpreterUseQuery.h>
#include <Interpreters/InterpreterWatchQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Parsers/ASTSystemQuery.h>
......@@ -176,6 +182,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterWatchQuery>(query, context);
}
else if (query->as<ASTCreateUserQuery>())
{
return std::make_unique<InterpreterCreateUserQuery>(query, context);
}
else if (query->as<ASTCreateQuotaQuery>())
{
return std::make_unique<InterpreterCreateQuotaQuery>(query, context);
......@@ -188,10 +198,18 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterDropAccessEntityQuery>(query, context);
}
else if (query->as<ASTGrantQuery>())
{
return std::make_unique<InterpreterGrantQuery>(query, context);
}
else if (query->as<ASTShowCreateAccessEntityQuery>())
{
return std::make_unique<InterpreterShowCreateAccessEntityQuery>(query, context);
}
else if (query->as<ASTShowGrantsQuery>())
{
return std::make_unique<InterpreterShowGrantsQuery>(query, context);
}
else if (query->as<ASTShowQuotasQuery>())
{
return std::make_unique<InterpreterShowQuotasQuery>(query, context);
......
#include <Interpreters/InterpreterGrantQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Interpreters/Context.h>
#include <Access/AccessControlManager.h>
#include <Access/AccessRightsContext.h>
#include <Access/User.h>
namespace DB
{
BlockIO InterpreterGrantQuery::execute()
{
const auto & query = query_ptr->as<const ASTGrantQuery &>();
auto & access_control = context.getAccessControlManager();
context.getAccessRights()->checkGrantOption(query.access_rights_elements);
using Kind = ASTGrantQuery::Kind;
if (query.to_roles->all_roles)
throw Exception(
"Cannot " + String((query.kind == Kind::GRANT) ? "GRANT to" : "REVOKE from") + " ALL", ErrorCodes::NOT_IMPLEMENTED);
String current_database = context.getCurrentDatabase();
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
if (query.kind == Kind::GRANT)
{
updated_user->access.grant(query.access_rights_elements, current_database);
if (query.grant_option)
updated_user->access_with_grant_option.grant(query.access_rights_elements, current_database);
}
else
{
updated_user->access_with_grant_option.revoke(query.access_rights_elements, current_database);
if (!query.grant_option)
updated_user->access.revoke(query.access_rights_elements, current_database);
}
return updated_user;
};
std::vector<UUID> ids = access_control.getIDs<User>(query.to_roles->roles);
if (query.to_roles->current_user)
ids.push_back(context.getUserID());
access_control.update(ids, update_func);
return {};
}
}
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class InterpreterGrantQuery : public IInterpreter
{
public:
InterpreterGrantQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
Context & context;
};
}
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
......@@ -9,6 +10,7 @@
#include <Parsers/parseQuery.h>
#include <Access/AccessControlManager.h>
#include <Access/QuotaContext.h>
#include <Access/User.h>
#include <Columns/ColumnString.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h>
......@@ -58,6 +60,7 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreat
using Kind = ASTShowCreateAccessEntityQuery::Kind;
switch (show_query.kind)
{
case Kind::USER: return getCreateUserQuery(show_query);
case Kind::QUOTA: return getCreateQuotaQuery(show_query);
case Kind::ROW_POLICY: return getCreateRowPolicyQuery(show_query);
}
......@@ -65,6 +68,27 @@ ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreat
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
UserPtr user;
if (show_query.current_user)
user = context.getUser();
else
user = context.getAccessControlManager().getUser(show_query.name);
auto create_query = std::make_shared<ASTCreateUserQuery>();
create_query->name = user->getName();
if (!user->allowed_client_hosts.containsAnyHost())
create_query->hosts = user->allowed_client_hosts;
if (!user->profile.empty())
create_query->profile = user->profile;
return create_query;
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
auto & access_control = context.getAccessControlManager();
......
......@@ -29,6 +29,7 @@ private:
BlockInputStreamPtr executeImpl();
ASTPtr getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
};
......
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Parsers/ASTShowGrantsQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/formatAST.h>
#include <Interpreters/Context.h>
#include <Columns/ColumnString.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h>
#include <Access/AccessControlManager.h>
#include <Access/User.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/copy.hpp>
namespace DB
{
namespace
{
std::vector<AccessRightsElements> groupByTable(AccessRightsElements && elements)
{
using Key = std::tuple<String, bool, String, bool>;
std::map<Key, AccessRightsElements> grouping_map;
for (auto & element : elements)
{
Key key(element.database, element.any_database, element.table, element.any_table);
grouping_map[key].emplace_back(std::move(element));
}
std::vector<AccessRightsElements> res;
res.reserve(grouping_map.size());
boost::range::copy(grouping_map | boost::adaptors::map_values, std::back_inserter(res));
return res;
}
struct GroupedGrantsAndPartialRevokes
{
std::vector<AccessRightsElements> grants;
std::vector<AccessRightsElements> partial_revokes;
};
GroupedGrantsAndPartialRevokes groupByTable(AccessRights::Elements && elements)
{
GroupedGrantsAndPartialRevokes res;
res.grants = groupByTable(std::move(elements.grants));
res.partial_revokes = groupByTable(std::move(elements.partial_revokes));
return res;
}
}
BlockIO InterpreterShowGrantsQuery::execute()
{
BlockIO res;
res.in = executeImpl();
return res;
}
BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl()
{
const auto & show_query = query_ptr->as<ASTShowGrantsQuery &>();
/// Build a create query.
ASTs grant_queries = getGrantQueries(show_query);
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
std::stringstream grant_ss;
for (const auto & grant_query : grant_queries)
{
grant_ss.str("");
formatAST(*grant_query, grant_ss, false, true);
column->insert(grant_ss.str());
}
/// Prepare description of the result column.
std::stringstream desc_ss;
formatAST(show_query, desc_ss, false, true);
String desc = desc_ss.str();
String prefix = "SHOW ";
if (desc.starts_with(prefix))
desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix.
return std::make_shared<OneBlockInputStream>(Block{{std::move(column), std::make_shared<DataTypeString>(), desc}});
}
ASTs InterpreterShowGrantsQuery::getGrantQueries(const ASTShowGrantsQuery & show_query) const
{
UserPtr user;
if (show_query.current_user)
user = context.getUser();
else
user = context.getAccessControlManager().getUser(show_query.name);
ASTs res;
for (bool grant_option : {true, false})
{
if (!grant_option && (user->access == user->access_with_grant_option))
continue;
const auto & access_rights = grant_option ? user->access_with_grant_option : user->access;
const auto grouped_elements = groupByTable(access_rights.getElements());
using Kind = ASTGrantQuery::Kind;
for (Kind kind : {Kind::GRANT, Kind::REVOKE})
{
for (const auto & elements : (kind == Kind::GRANT ? grouped_elements.grants : grouped_elements.partial_revokes))
{
auto grant_query = std::make_shared<ASTGrantQuery>();
grant_query->kind = kind;
grant_query->grant_option = grant_option;
grant_query->to_roles = std::make_shared<ASTRoleList>();
grant_query->to_roles->roles.push_back(user->getName());
grant_query->access_rights_elements = elements;
res.push_back(std::move(grant_query));
}
}
}
return res;
}
}
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ASTShowGrantsQuery;
class InterpreterShowGrantsQuery : public IInterpreter
{
public:
InterpreterShowGrantsQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
BlockIO execute() override;
private:
BlockInputStreamPtr executeImpl();
ASTs getGrantQueries(const ASTShowGrantsQuery & show_query) const;
ASTPtr query_ptr;
Context & context;
};
}
#include <Parsers/ASTCreateUserQuery.h>
#include <Common/quoteString.h>
namespace DB
{
namespace
{
void formatRenameTo(const String & new_name, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "")
<< quoteString(new_name);
}
void formatAuthentication(const Authentication & authentication, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << (settings.hilite ? IAST::hilite_none : "");
switch (authentication.getType())
{
case Authentication::Type::NO_PASSWORD:
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "no_password" << (settings.hilite ? IAST::hilite_none : "");
break;
case Authentication::Type::PLAINTEXT_PASSWORD:
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "plaintext_password BY " << (settings.hilite ? IAST::hilite_none : "")
<< quoteString(authentication.getPassword());
break;
case Authentication::Type::SHA256_PASSWORD:
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "sha256_hash BY " << (settings.hilite ? IAST::hilite_none : "")
<< quoteString(authentication.getPasswordHashHex());
break;
case Authentication::Type::DOUBLE_SHA1_PASSWORD:
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "double_sha1_hash BY " << (settings.hilite ? IAST::hilite_none : "")
<< quoteString(authentication.getPasswordHashHex());
break;
}
}
void formatHosts(const char * prefix, const AllowedClientHosts & hosts, const IAST::FormatSettings & settings)
{
if (prefix)
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " " << prefix << " HOST "
<< (settings.hilite ? IAST::hilite_none : "");
else
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " HOST " << (settings.hilite ? IAST::hilite_none : "");
if (hosts.empty())
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : "");
return;
}
if (hosts.containsAnyHost())
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ANY" << (settings.hilite ? IAST::hilite_none : "");
return;
}
bool need_comma = false;
if (hosts.containsLocalHost())
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "LOCAL" << (settings.hilite ? IAST::hilite_none : "");
}
const auto & addresses = hosts.getAddresses();
const auto & subnets = hosts.getSubnets();
if (!addresses.empty() || !subnets.empty())
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "IP " << (settings.hilite ? IAST::hilite_none : "");
bool need_comma2 = false;
for (const auto & address : addresses)
{
if (std::exchange(need_comma2, true))
settings.ostr << ", ";
settings.ostr << quoteString(address.toString());
}
for (const auto & subnet : subnets)
{
if (std::exchange(need_comma2, true))
settings.ostr << ", ";
settings.ostr << quoteString(subnet.toString());
}
}
const auto & names = hosts.getNames();
if (!names.empty())
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NAME " << (settings.hilite ? IAST::hilite_none : "");
bool need_comma2 = false;
for (const auto & name : names)
{
if (std::exchange(need_comma2, true))
settings.ostr << ", ";
settings.ostr << quoteString(name);
}
}
const auto & name_regexps = hosts.getNameRegexps();
if (!name_regexps.empty())
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NAME REGEXP " << (settings.hilite ? IAST::hilite_none : "");
bool need_comma2 = false;
for (const auto & host_regexp : name_regexps)
{
if (std::exchange(need_comma2, true))
settings.ostr << ", ";
settings.ostr << quoteString(host_regexp);
}
}
const auto & like_patterns = hosts.getLikePatterns();
if (!like_patterns.empty())
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "LIKE " << (settings.hilite ? IAST::hilite_none : "");
bool need_comma2 = false;
for (const auto & like_pattern : like_patterns)
{
if (std::exchange(need_comma2, true))
settings.ostr << ", ";
settings.ostr << quoteString(like_pattern);
}
}
}
void formatProfile(const String & profile_name, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " PROFILE " << (settings.hilite ? IAST::hilite_none : "")
<< quoteString(profile_name);
}
}
String ASTCreateUserQuery::getID(char) const
{
return "CreateUserQuery";
}
ASTPtr ASTCreateUserQuery::clone() const
{
return std::make_shared<ASTCreateUserQuery>(*this);
}
void ASTCreateUserQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER")
<< (settings.hilite ? hilite_none : "");
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
else if (if_not_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : "");
else if (or_replace)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : "");
settings.ostr << " " << backQuoteIfNeed(name);
if (!new_name.empty())
formatRenameTo(new_name, settings);
if (authentication)
formatAuthentication(*authentication, settings);
if (hosts)
formatHosts(nullptr, *hosts, settings);
if (add_hosts)
formatHosts("ADD", *add_hosts, settings);
if (remove_hosts)
formatHosts("REMOVE", *remove_hosts, settings);
if (profile)
formatProfile(*profile, settings);
}
}
#pragma once
#include <Parsers/IAST.h>
#include <Access/Authentication.h>
#include <Access/AllowedClientHosts.h>
namespace DB
{
/** CREATE USER [IF NOT EXISTS | OR REPLACE] name
* [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]
* [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
* [PROFILE 'profile_name']
*
* ALTER USER [IF EXISTS] name
* [RENAME TO new_name]
* [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]
* [[ADD|REMOVE] HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
* [PROFILE 'profile_name']
*/
class ASTCreateUserQuery : public IAST
{
public:
bool alter = false;
bool if_exists = false;
bool if_not_exists = false;
bool or_replace = false;
String name;
String new_name;
std::optional<Authentication> authentication;
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts;
std::optional<String> profile;
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}
......@@ -12,6 +12,7 @@ namespace
{
switch (kind)
{
case Kind::USER: return "USER";
case Kind::QUOTA: return "QUOTA";
case Kind::ROW_POLICY: return "POLICY";
}
......
......@@ -9,17 +9,21 @@ namespace DB
/** DROP QUOTA [IF EXISTS] name [,...]
* DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...]
* DROP USER [IF EXISTS] name [,...]
*/
class ASTDropAccessEntityQuery : public IAST
{
public:
enum class Kind
{
USER,
QUOTA,
ROW_POLICY,
};
const Kind kind;
const char * const keyword;
bool if_exists = false;
Strings names;
std::vector<RowPolicy::FullNameParts> row_policies_names;
......
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTRoleList.h>
#include <Common/quoteString.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/sort.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <map>
namespace DB
{
namespace
{
using KeywordToColumnsMap = std::map<std::string_view /* keyword */, std::vector<std::string_view> /* columns */>;
using TableToAccessMap = std::map<String /* database_and_table_name */, KeywordToColumnsMap>;
TableToAccessMap prepareTableToAccessMap(const AccessRightsElements & elements)
{
TableToAccessMap res;
for (const auto & element : elements)
{
String database_and_table_name;
if (element.any_database)
{
if (element.any_table)
database_and_table_name = "*.*";
else
database_and_table_name = "*." + backQuoteIfNeed(element.table);
}
else if (element.database.empty())
{
if (element.any_table)
database_and_table_name = "*";
else
database_and_table_name = backQuoteIfNeed(element.table);
}
else
{
if (element.any_table)
database_and_table_name = backQuoteIfNeed(element.database) + ".*";
else
database_and_table_name = backQuoteIfNeed(element.database) + "." + backQuoteIfNeed(element.table);
}
KeywordToColumnsMap & keyword_to_columns = res[database_and_table_name];
for (const auto & keyword : element.access_flags.toKeywords())
boost::range::push_back(keyword_to_columns[keyword], element.columns);
}
for (auto & keyword_to_columns : res | boost::adaptors::map_values)
{
for (auto & columns : keyword_to_columns | boost::adaptors::map_values)
boost::range::sort(columns);
}
return res;
}
void formatColumnNames(const std::vector<std::string_view> & columns, const IAST::FormatSettings & settings)
{
if (columns.empty())
return;
settings.ostr << "(";
bool need_comma_after_column_name = false;
for (const auto & column : columns)
{
if (std::exchange(need_comma_after_column_name, true))
settings.ostr << ", ";
settings.ostr << backQuoteIfNeed(column);
}
settings.ostr << ")";
}
}
String ASTGrantQuery::getID(char) const
{
return "GrantQuery";
}
ASTPtr ASTGrantQuery::clone() const
{
return std::make_shared<ASTGrantQuery>(*this);
}
void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
<< (settings.hilite ? hilite_none : "") << " ";
if (grant_option && (kind == Kind::REVOKE))
settings.ostr << (settings.hilite ? hilite_keyword : "") << "GRANT OPTION FOR " << (settings.hilite ? hilite_none : "");
bool need_comma = false;
for (const auto & [database_and_table, keyword_to_columns] : prepareTableToAccessMap(access_rights_elements))
{
for (const auto & [keyword, columns] : keyword_to_columns)
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << (settings.hilite ? hilite_keyword : "") << keyword << (settings.hilite ? hilite_none : "");
formatColumnNames(columns, settings);
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "") << database_and_table;
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << ((kind == Kind::GRANT) ? " TO " : " FROM ") << (settings.hilite ? hilite_none : "");
to_roles->format(settings);
if (grant_option && (kind == Kind::GRANT))
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH GRANT OPTION" << (settings.hilite ? hilite_none : "");
}
}
#pragma once
#include <Parsers/IAST.h>
#include <Access/AccessRightsElement.h>
namespace DB
{
class ASTRoleList;
/** GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO user_name
* REVOKE access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO user_name
*/
class ASTGrantQuery : public IAST
{
public:
enum class Kind
{
GRANT,
REVOKE,
};
Kind kind = Kind::GRANT;
AccessRightsElements access_rights_elements;
std::shared_ptr<ASTRoleList> to_roles;
bool grant_option = false;
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}
......@@ -12,6 +12,7 @@ namespace
{
switch (kind)
{
case Kind::USER: return "USER";
case Kind::QUOTA: return "QUOTA";
case Kind::ROW_POLICY: return "POLICY";
}
......@@ -44,7 +45,11 @@ void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & sett
<< "SHOW CREATE " << keyword
<< (settings.hilite ? hilite_none : "");
if (kind == Kind::ROW_POLICY)
if ((kind == Kind::USER) && current_user)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT_USER" << (settings.hilite ? hilite_none : "");
else if ((kind == Kind::QUOTA) && current_quota)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : "");
else if (kind == Kind::ROW_POLICY)
{
const String & database = row_policy_name.database;
const String & table_name = row_policy_name.table_name;
......@@ -53,8 +58,6 @@ void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & sett
<< (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".")
<< backQuoteIfNeed(table_name);
}
else if ((kind == Kind::QUOTA) && current_quota)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : "");
else
settings.ostr << " " << backQuoteIfNeed(name);
}
......
......@@ -8,19 +8,23 @@ namespace DB
{
/** SHOW CREATE QUOTA [name | CURRENT]
* SHOW CREATE [ROW] POLICY name ON [database.]table
* SHOW CREATE USER [name | CURRENT_USER]
*/
class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput
{
public:
enum class Kind
{
USER,
QUOTA,
ROW_POLICY,
};
const Kind kind;
const char * const keyword;
String name;
bool current_quota = false;
bool current_user = false;
RowPolicy::FullNameParts row_policy_name;
ASTShowCreateAccessEntityQuery(Kind kind_);
......
#include <Parsers/ASTShowGrantsQuery.h>
#include <Common/quoteString.h>
namespace DB
{
String ASTShowGrantsQuery::getID(char) const
{
return "ShowGrantsQuery";
}
ASTPtr ASTShowGrantsQuery::clone() const
{
return std::make_shared<ASTShowGrantsQuery>(*this);
}
void ASTShowGrantsQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW GRANTS FOR "
<< (settings.hilite ? hilite_none : "");
if (current_user)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CURRENT_USER"
<< (settings.hilite ? hilite_none : "");
else
settings.ostr << backQuoteIfNeed(name);
}
}
#pragma once
#include <Parsers/ASTQueryWithOutput.h>
namespace DB
{
/** SHOW GRANTS [FOR user_name]
*/
class ASTShowGrantsQuery : public ASTQueryWithOutput
{
public:
String name;
bool current_user = false;
String getID(char) const override;
ASTPtr clone() const override;
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}
......@@ -47,6 +47,13 @@ protected:
};
class ParserBareWord : public IParserBase
{
protected:
const char * getName() const override { return "bare word"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime
*/
class ParserCompoundIdentifier : public IParserBase
......
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/parseUserName.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTRoleList.h>
#include <ext/range.h>
#include <boost/algorithm/string/predicate.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
namespace
{
bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name, String & new_host_pattern)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!new_name.empty())
return false;
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
return false;
return parseUserName(pos, expected, new_name, new_host_pattern);
});
}
bool parsePassword(IParserBase::Pos & pos, Expected & expected, String & password)
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
password = ast->as<const ASTLiteral &>().value.safeGet<String>();
return true;
}
bool parseAuthentication(IParserBase::Pos & pos, Expected & expected, std::optional<Authentication> & authentication)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (authentication)
return false;
if (!ParserKeyword{"IDENTIFIED"}.ignore(pos, expected))
return false;
if (ParserKeyword{"WITH"}.ignore(pos, expected))
{
if (ParserKeyword{"NO_PASSWORD"}.ignore(pos, expected))
{
authentication = Authentication{Authentication::NO_PASSWORD};
}
else if (ParserKeyword{"PLAINTEXT_PASSWORD"}.ignore(pos, expected))
{
String password;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, password))
return false;
authentication = Authentication{Authentication::PLAINTEXT_PASSWORD};
authentication->setPassword(password);
}
else if (ParserKeyword{"SHA256_PASSWORD"}.ignore(pos, expected))
{
String password;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, password))
return false;
authentication = Authentication{Authentication::SHA256_PASSWORD};
authentication->setPassword(password);
}
else if (ParserKeyword{"SHA256_HASH"}.ignore(pos, expected))
{
String hash;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, hash))
return false;
authentication = Authentication{Authentication::SHA256_PASSWORD};
authentication->setPasswordHashHex(hash);
}
else if (ParserKeyword{"DOUBLE_SHA1_PASSWORD"}.ignore(pos, expected))
{
String password;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, password))
return false;
authentication = Authentication{Authentication::DOUBLE_SHA1_PASSWORD};
authentication->setPassword(password);
}
else if (ParserKeyword{"DOUBLE_SHA1_HASH"}.ignore(pos, expected))
{
String hash;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, hash))
return false;
authentication = Authentication{Authentication::DOUBLE_SHA1_PASSWORD};
authentication->setPasswordHashHex(hash);
}
else
return false;
}
else
{
String password;
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !parsePassword(pos, expected, password))
return false;
authentication = Authentication{Authentication::SHA256_PASSWORD};
authentication->setPassword(password);
}
return true;
});
}
bool parseHosts(IParserBase::Pos & pos, Expected & expected, const char * prefix, std::optional<AllowedClientHosts> & hosts)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (prefix && !ParserKeyword{prefix}.ignore(pos, expected))
return false;
if (!ParserKeyword{"HOST"}.ignore(pos, expected))
return false;
if (ParserKeyword{"ANY"}.ignore(pos, expected))
{
if (!hosts)
hosts.emplace();
hosts->addAnyHost();
return true;
}
if (ParserKeyword{"NONE"}.ignore(pos, expected))
{
if (!hosts)
hosts.emplace();
return true;
}
do
{
if (ParserKeyword{"LOCAL"}.ignore(pos, expected))
{
if (!hosts)
hosts.emplace();
hosts->addLocalHost();
}
else if (ParserKeyword{"NAME REGEXP"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
if (!hosts)
hosts.emplace();
hosts->addNameRegexp(ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"NAME"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
if (!hosts)
hosts.emplace();
hosts->addName(ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"IP"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
if (!hosts)
hosts.emplace();
hosts->addSubnet(ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else if (ParserKeyword{"LIKE"}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
if (!hosts)
hosts.emplace();
hosts->addLikePattern(ast->as<const ASTLiteral &>().value.safeGet<String>());
}
else
return false;
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
});
}
bool parseProfileName(IParserBase::Pos & pos, Expected & expected, std::optional<String> & profile)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (profile)
return false;
if (!ParserKeyword{"PROFILE"}.ignore(pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
profile = ast->as<const ASTLiteral &>().value.safeGet<String>();
return true;
});
}
}
bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE USER"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
alter = true;
else
return false;
bool if_exists = false;
bool if_not_exists = false;
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
or_replace = true;
}
String name;
String host_pattern;
if (!parseUserName(pos, expected, name, host_pattern))
return false;
String new_name;
String new_host_pattern;
std::optional<Authentication> authentication;
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts;
std::optional<String> profile;
while (parseAuthentication(pos, expected, authentication)
|| parseHosts(pos, expected, nullptr, hosts)
|| parseProfileName(pos, expected, profile)
|| (alter && parseRenameTo(pos, expected, new_name, new_host_pattern))
|| (alter && parseHosts(pos, expected, "ADD", add_hosts))
|| (alter && parseHosts(pos, expected, "REMOVE", remove_hosts)))
;
if (!hosts)
{
if (!alter)
hosts.emplace().addLikePattern(host_pattern);
else if (alter && !new_name.empty())
hosts.emplace().addLikePattern(new_host_pattern);
}
auto query = std::make_shared<ASTCreateUserQuery>();
node = query;
query->alter = alter;
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
query->name = std::move(name);
query->new_name = std::move(new_name);
query->authentication = std::move(authentication);
query->hosts = std::move(hosts);
query->add_hosts = std::move(add_hosts);
query->remove_hosts = std::move(remove_hosts);
query->profile = std::move(profile);
return true;
}
}
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses queries like
* CREATE USER [IF NOT EXISTS | OR REPLACE] name
* [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]
* [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
* [PROFILE 'profile_name']
*
* ALTER USER [IF EXISTS] name
* [RENAME TO new_name]
* [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]
* [[ADD|REMOVE] HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
* [PROFILE 'profile_name']
*/
class ParserCreateUserQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE USER or ALTER USER query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}
......@@ -3,6 +3,7 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/parseUserName.h>
#include <Access/Quota.h>
......@@ -23,6 +24,37 @@ namespace
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
}
bool parseRowPolicyNames(IParserBase::Pos & pos, Expected & expected, std::vector<RowPolicy::FullNameParts> & row_policies_names)
{
do
{
Strings policy_names;
if (!parseNames(pos, expected, policy_names))
return false;
String database, table_name;
if (!ParserKeyword{"ON"}.ignore(pos, expected) || !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
for (const String & policy_name : policy_names)
row_policies_names.push_back({database, table_name, policy_name});
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
}
bool parseUserNames(IParserBase::Pos & pos, Expected & expected, Strings & names)
{
do
{
String name;
if (!parseUserName(pos, expected, name))
return false;
names.push_back(std::move(name));
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return true;
}
}
......@@ -37,6 +69,8 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
kind = Kind::QUOTA;
else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected))
kind = Kind::ROW_POLICY;
else if (ParserKeyword{"USER"}.ignore(pos, expected))
kind = Kind::USER;
else
return false;
......@@ -47,23 +81,19 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
Strings names;
std::vector<RowPolicy::FullNameParts> row_policies_names;
if (kind == Kind::ROW_POLICY)
if (kind == Kind::USER)
{
do
{
Strings policy_names;
if (!parseNames(pos, expected, policy_names))
return false;
String database, table_name;
if (!ParserKeyword{"ON"}.ignore(pos, expected) || !parseDatabaseAndTableName(pos, expected, database, table_name))
return false;
for (const String & policy_name : policy_names)
row_policies_names.push_back({database, table_name, policy_name});
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
if (!parseUserNames(pos, expected, names))
return false;
}
else if (kind == Kind::ROW_POLICY)
{
if (!parseRowPolicyNames(pos, expected, row_policies_names))
return false;
}
else
{
assert(kind == Kind::QUOTA);
if (!parseNames(pos, expected, names))
return false;
}
......
......@@ -7,6 +7,8 @@ namespace DB
{
/** Parses queries like
* DROP QUOTA [IF EXISTS] name [,...]
* DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...]
* DROP USER [IF EXISTS] name [,...]
*/
class ParserDropAccessEntityQuery : public IParserBase
{
......
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserRoleList.h>
#include <boost/algorithm/string/predicate.hpp>
namespace DB
{
namespace
{
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags)
{
auto is_one_of_access_type_words = [](IParser::Pos & pos_)
{
if (pos_->type != TokenType::BareWord)
return false;
std::string_view word{pos_->begin, pos_->size()};
if (boost::iequals(word, "ON") || boost::iequals(word, "TO") || boost::iequals(word, "FROM"))
return false;
return true;
};
if (!is_one_of_access_type_words(pos))
{
expected.add(pos, "access type");
return false;
}
String str;
do
{
if (!str.empty())
str += " ";
std::string_view word{pos->begin, pos->size()};
str += std::string_view(pos->begin, pos->size());
++pos;
}
while (is_one_of_access_type_words(pos));
if (pos->type == TokenType::OpeningRoundBracket)
{
auto old_pos = pos;
++pos;
if (pos->type == TokenType::ClosingRoundBracket)
{
++pos;
str += "()";
}
else
pos = old_pos;
}
access_flags = AccessFlags{str};
return true;
}
bool parseColumnNames(IParser::Pos & pos, Expected & expected, Strings & columns)
{
if (!ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
return false;
do
{
ASTPtr column_ast;
if (!ParserIdentifier().parse(pos, column_ast, expected))
return false;
columns.push_back(getIdentifierName(column_ast));
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
return ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected);
}
bool parseDatabaseAndTableNameOrMaybeAsterisks(
IParser::Pos & pos, Expected & expected, String & database_name, bool & any_database, String & table_name, bool & any_table)
{
ASTPtr ast[2];
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
{
if (ParserToken{TokenType::Dot}.ignore(pos, expected))
{
if (!ParserToken{TokenType::Asterisk}.ignore(pos, expected))
return false;
/// *.* (any table in any database)
any_database = true;
any_table = true;
return true;
}
else
{
/// * (any table in the current database)
any_database = false;
database_name = "";
any_table = true;
return true;
}
}
else if (ParserIdentifier().parse(pos, ast[0], expected))
{
if (ParserToken{TokenType::Dot}.ignore(pos, expected))
{
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
{
/// <database_name>.*
any_database = false;
database_name = getIdentifierName(ast[0]);
any_table = true;
return true;
}
else if (ParserIdentifier().parse(pos, ast[1], expected))
{
/// <database_name>.<table_name>
any_database = false;
database_name = getIdentifierName(ast[0]);
any_table = false;
table_name = getIdentifierName(ast[1]);
return true;
}
else
return false;
}
else
{
/// <table_name> - the current database, specified table
any_database = false;
database_name = "";
table_name = getIdentifierName(ast[0]);
return true;
}
}
else
return false;
}
}
bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
using Kind = ASTGrantQuery::Kind;
Kind kind;
if (ParserKeyword{"GRANT"}.ignore(pos, expected))
kind = Kind::GRANT;
else if (ParserKeyword{"REVOKE"}.ignore(pos, expected))
kind = Kind::REVOKE;
else
return false;
bool grant_option = false;
if (kind == Kind::REVOKE)
{
if (ParserKeyword{"GRANT OPTION FOR"}.ignore(pos, expected))
grant_option = true;
}
AccessRightsElements elements;
do
{
std::vector<std::pair<AccessFlags, Strings>> access_and_columns;
do
{
AccessFlags access_flags;
if (!parseAccessFlags(pos, expected, access_flags))
return false;
Strings columns;
parseColumnNames(pos, expected, columns);
access_and_columns.emplace_back(access_flags, std::move(columns));
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
if (!ParserKeyword{"ON"}.ignore(pos, expected))
return false;
String database_name, table_name;
bool any_database = false, any_table = false;
if (!parseDatabaseAndTableNameOrMaybeAsterisks(pos, expected, database_name, any_database, table_name, any_table))
return false;
for (auto & [access_flags, columns] : access_and_columns)
{
AccessRightsElement element;
element.access_flags = access_flags;
element.any_column = columns.empty();
element.columns = std::move(columns);
element.any_database = any_database;
element.database = database_name;
element.any_table = any_table;
element.table = table_name;
elements.emplace_back(std::move(element));
}
}
while (ParserToken{TokenType::Comma}.ignore(pos, expected));
ASTPtr to_roles;
if (kind == Kind::GRANT)
{
if (!ParserKeyword{"TO"}.ignore(pos, expected))
return false;
}
else
{
if (!ParserKeyword{"FROM"}.ignore(pos, expected))
return false;
}
if (!ParserRoleList{}.parse(pos, to_roles, expected))
return false;
if (kind == Kind::GRANT)
{
if (ParserKeyword{"WITH GRANT OPTION"}.ignore(pos, expected))
grant_option = true;
}
auto query = std::make_shared<ASTGrantQuery>();
node = query;
query->kind = kind;
query->access_rights_elements = std::move(elements);
query->to_roles = std::static_pointer_cast<ASTRoleList>(to_roles);
query->grant_option = grant_option;
return true;
}
}
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses queries like
* GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO user_name
* REVOKE access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO user_name
*/
class ParserGrantQuery : public IParserBase
{
protected:
const char * getName() const override { return "GRANT or REVOKE query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}
......@@ -9,9 +9,11 @@
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserSystemQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserDropAccessEntityQuery.h>
#include <Parsers/ParserGrantQuery.h>
namespace DB
......@@ -25,18 +27,22 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserSystemQuery system_p;
ParserCreateUserQuery create_user_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateRowPolicyQuery create_row_policy_p;
ParserDropAccessEntityQuery drop_access_entity_p;
ParserGrantQuery grant_p;
bool res = query_with_output_p.parse(pos, node, expected)
|| insert_p.parse(pos, node, expected)
|| use_p.parse(pos, node, expected)
|| set_p.parse(pos, node, expected)
|| system_p.parse(pos, node, expected)
|| create_user_p.parse(pos, node, expected)
|| create_quota_p.parse(pos, node, expected)
|| create_row_policy_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected);
|| drop_access_entity_p.parse(pos, node, expected)
|| grant_p.parse(pos, node, expected);
return res;
}
......
......@@ -14,6 +14,7 @@
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ParserShowGrantsQuery.h>
#include <Parsers/ParserShowCreateAccessEntityQuery.h>
#include <Parsers/ParserShowQuotasQuery.h>
#include <Parsers/ParserShowRowPoliciesQuery.h>
......@@ -38,6 +39,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserKillQueryQuery kill_query_p;
ParserWatchQuery watch_p;
ParserShowCreateAccessEntityQuery show_create_access_entity_p;
ParserShowGrantsQuery show_grants_p;
ParserShowQuotasQuery show_quotas_p;
ParserShowRowPoliciesQuery show_row_policies_p;
......@@ -68,6 +70,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|| kill_query_p.parse(pos, query, expected)
|| optimize_p.parse(pos, query, expected)
|| watch_p.parse(pos, query, expected)
|| show_grants_p.parse(pos, query, expected)
|| show_quotas_p.parse(pos, query, expected)
|| show_row_policies_p.parse(pos, query, expected);
......
#include <Parsers/ParserRoleList.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ASTRoleList.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseUserName.h>
#include <boost/range/algorithm/find.hpp>
......@@ -47,7 +47,7 @@ bool ParserRoleList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else
{
String name;
if (!parseIdentifierOrStringLiteral(pos, expected, name))
if (!parseUserName(pos, expected, name))
return false;
if (except_mode && (boost::range::find(roles, name) == roles.end()))
except_roles.push_back(name);
......
......@@ -3,6 +3,7 @@
#include <Parsers/CommonParsers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/parseUserName.h>
#include <assert.h>
......@@ -15,7 +16,9 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe
using Kind = ASTShowCreateAccessEntityQuery::Kind;
Kind kind;
if (ParserKeyword{"QUOTA"}.ignore(pos, expected))
if (ParserKeyword{"USER"}.ignore(pos, expected))
kind = Kind::USER;
else if (ParserKeyword{"QUOTA"}.ignore(pos, expected))
kind = Kind::QUOTA;
else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected))
kind = Kind::ROW_POLICY;
......@@ -24,9 +27,15 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe
String name;
bool current_quota = false;
bool current_user = false;
RowPolicy::FullNameParts row_policy_name;
if (kind == Kind::ROW_POLICY)
if (kind == Kind::USER)
{
if (!parseUserNameOrCurrentUserTag(pos, expected, name, current_user))
current_user = true;
}
else if (kind == Kind::ROW_POLICY)
{
String & database = row_policy_name.database;
String & table_name = row_policy_name.table_name;
......
#include <Parsers/ParserShowGrantsQuery.h>
#include <Parsers/ASTShowGrantsQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/parseUserName.h>
namespace DB
{
bool ParserShowGrantsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SHOW GRANTS"}.ignore(pos, expected))
return false;
String name;
bool current_user = false;
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
if (!parseUserNameOrCurrentUserTag(pos, expected, name, current_user))
return false;
}
else
current_user = true;
auto query = std::make_shared<ASTShowGrantsQuery>();
node = query;
query->name = name;
query->current_user = current_user;
return true;
}
}
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
/** Parses queries like
* SHOW GRANTS [FOR user_name]
*/
class ParserShowGrantsQuery : public IParserBase
{
protected:
const char * getName() const override { return "SHOW GRANTS query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}
#include <Parsers/parseUserName.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/CommonParsers.h>
#include <boost/algorithm/string.hpp>
namespace DB
{
namespace
{
bool parseUserNameImpl(IParser::Pos & pos, Expected & expected, String & user_name, String * host_like_pattern)
{
String name;
if (!parseIdentifierOrStringLiteral(pos, expected, name))
return false;
boost::algorithm::trim(name);
String pattern = "@";
if (ParserToken{TokenType::At}.ignore(pos, expected))
{
if (!parseIdentifierOrStringLiteral(pos, expected, pattern))
return false;
boost::algorithm::trim(pattern);
}
if (pattern != "@")
name += '@' + pattern;
user_name = std::move(name);
if (host_like_pattern)
*host_like_pattern = std::move(pattern);
return true;
}
}
bool parseUserName(IParser::Pos & pos, Expected & expected, String & result)
{
return parseUserNameImpl(pos, expected, result, nullptr);
}
bool parseUserName(IParser::Pos & pos, Expected & expected, String & result, String & host_like_pattern)
{
return parseUserNameImpl(pos, expected, result, &host_like_pattern);
}
bool parseUserNameOrCurrentUserTag(IParser::Pos & pos, Expected & expected, String & user_name, bool & current_user)
{
if (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || ParserKeyword{"currentUser"}.ignore(pos, expected))
{
if (ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
{
if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected))
return false;
}
current_user = true;
return true;
}
if (parseUserName(pos, expected, user_name))
{
current_user = false;
return true;
}
return false;
}
}
#pragma once
#include <Parsers/IParser.h>
namespace DB
{
/// Parses a user name. It can be a simple string or identifier or something like `name@host`.
/// In the last case `host` specifies the hosts user is allowed to connect from.
/// The `host` can be an ip address, ip subnet, or a host name.
/// The % and _ wildcard characters are permitted in `host`.
/// These have the same meaning as for pattern-matching operations performed with the LIKE operator.
bool parseUserName(IParser::Pos & pos, Expected & expected, String & user_name, String & host_like_pattern);
bool parseUserName(IParser::Pos & pos, Expected & expected, String & user_name);
/// Parses either a user name or the 'CURRENT_USER' keyword (or some of the aliases).
bool parseUserNameOrCurrentUserTag(IParser::Pos & pos, Expected & expected, String & user_name, bool & current_user);
/// Parses a role name. It follows the same rules as a user name, but allowed hosts are never checked
/// (because roles are not used to connect to server).
inline bool parseRoleName(IParser::Pos & pos, Expected & expected, String & role_name)
{
return parseUserName(pos, expected, role_name);
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册