mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #68885 from Unalian/feat-67772
Add CHECK GRANT query
This commit is contained in:
commit
353ff951aa
46
docs/en/sql-reference/statements/check-grant.md
Normal file
46
docs/en/sql-reference/statements/check-grant.md
Normal file
@ -0,0 +1,46 @@
|
||||
---
|
||||
slug: /en/sql-reference/statements/check-grant
|
||||
sidebar_position: 56
|
||||
sidebar_label: CHECK GRANT
|
||||
title: "CHECK GRANT Statement"
|
||||
---
|
||||
|
||||
The `CHECK GRANT` query is used to check whether the current user/role has been granted a specific privilege.
|
||||
|
||||
## Syntax
|
||||
|
||||
The basic syntax of the query is as follows:
|
||||
|
||||
```sql
|
||||
CHECK GRANT privilege[(column_name [,...])] [,...] ON {db.table[*]|db[*].*|*.*|table[*]|*}
|
||||
```
|
||||
|
||||
- `privilege` — Type of privilege.
|
||||
|
||||
## Examples
|
||||
|
||||
If the user used to be granted the privilege, the response`check_grant` will be `1`. Otherwise, the response `check_grant` will be `0`.
|
||||
|
||||
If `table_1.col1` exists and current user is granted by privilege `SELECT`/`SELECT(con)` or role(with privilege), the response is `1`.
|
||||
```sql
|
||||
CHECK GRANT SELECT(col1) ON table_1;
|
||||
```
|
||||
|
||||
```text
|
||||
┌─result─┐
|
||||
│ 1 │
|
||||
└────────┘
|
||||
```
|
||||
If `table_2.col2` doesn't exists, or current user is not granted by privilege `SELECT`/`SELECT(con)` or role(with privilege), the response is `0`.
|
||||
```sql
|
||||
CHECK GRANT SELECT(col2) ON table_2;
|
||||
```
|
||||
|
||||
```text
|
||||
┌─result─┐
|
||||
│ 0 │
|
||||
└────────┘
|
||||
```
|
||||
|
||||
## Wildcard
|
||||
Specifying privileges you can use asterisk (`*`) instead of a table or a database name. Please check [WILDCARD GRANTS](../../sql-reference/statements/grant.md#wildcard-grants) for wildcard rules.
|
10
docs/zh/sql-reference/statements/check-grant.mdx
Normal file
10
docs/zh/sql-reference/statements/check-grant.mdx
Normal file
@ -0,0 +1,10 @@
|
||||
---
|
||||
slug: /zh/sql-reference/statements/check-grant
|
||||
sidebar_position: 56
|
||||
sidebar_label: CHECK
|
||||
title: "CHECK GRANT Statement"
|
||||
---
|
||||
|
||||
import Content from '@site/docs/en/sql-reference/statements/check-grant.md';
|
||||
|
||||
<Content />
|
57
src/Interpreters/Access/InterpreterCheckGrantQuery.cpp
Normal file
57
src/Interpreters/Access/InterpreterCheckGrantQuery.cpp
Normal file
@ -0,0 +1,57 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/Access/InterpreterCheckGrantQuery.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/RolesOrUsersSet.h>
|
||||
#include <Access/User.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <boost/range/algorithm/set_algorithm.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include "Storages/IStorage.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BlockIO InterpreterCheckGrantQuery::execute()
|
||||
{
|
||||
auto & query = query_ptr->as<ASTCheckGrantQuery &>();
|
||||
query.access_rights_elements.eraseNonGrantable();
|
||||
|
||||
auto current_user_access = getContext()->getAccess();
|
||||
|
||||
/// Collect access rights elements which will be checked.
|
||||
AccessRightsElements & elements_to_check_grant = query.access_rights_elements;
|
||||
|
||||
/// Replacing empty database with the default. This step must be done before replication to avoid privilege escalation.
|
||||
String current_database = getContext()->getCurrentDatabase();
|
||||
elements_to_check_grant.replaceEmptyDatabase(current_database);
|
||||
query.access_rights_elements.replaceEmptyDatabase(current_database);
|
||||
bool user_is_granted = current_user_access->isGranted(elements_to_check_grant);
|
||||
BlockIO res;
|
||||
res.pipeline = QueryPipeline(
|
||||
std::make_shared<SourceFromSingleChunk>(Block{{ColumnUInt8::create(1, user_is_granted), std::make_shared<DataTypeUInt8>(), "result"}}));
|
||||
return res;
|
||||
}
|
||||
|
||||
void registerInterpreterCheckGrantQuery(InterpreterFactory & factory)
|
||||
{
|
||||
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
||||
{
|
||||
return std::make_unique<InterpreterCheckGrantQuery>(args.query, args.context);
|
||||
};
|
||||
factory.registerInterpreter("InterpreterCheckGrantQuery", create_fn);
|
||||
}
|
||||
|
||||
}
|
26
src/Interpreters/Access/InterpreterCheckGrantQuery.h
Normal file
26
src/Interpreters/Access/InterpreterCheckGrantQuery.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTCheckGrantQuery;
|
||||
struct User;
|
||||
struct Role;
|
||||
|
||||
class InterpreterCheckGrantQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCheckGrantQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
||||
}
|
@ -44,6 +44,7 @@
|
||||
#include <Parsers/Access/ASTCreateUserQuery.h>
|
||||
#include <Parsers/Access/ASTDropAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ASTGrantQuery.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTMoveAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ASTSetRoleQuery.h>
|
||||
#include <Parsers/Access/ASTShowAccessEntitiesQuery.h>
|
||||
@ -308,6 +309,10 @@ InterpreterFactory::InterpreterPtr InterpreterFactory::get(ASTPtr & query, Conte
|
||||
{
|
||||
interpreter_name = "InterpreterShowGrantsQuery";
|
||||
}
|
||||
else if (query->as<ASTCheckGrantQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterCheckGrantQuery";
|
||||
}
|
||||
else if (query->as<ASTShowAccessEntitiesQuery>())
|
||||
{
|
||||
interpreter_name = "InterpreterShowAccessEntitiesQuery";
|
||||
|
@ -45,6 +45,7 @@ void registerInterpreterDropNamedCollectionQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterGrantQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterShowCreateAccessEntityQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterShowGrantsQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterCheckGrantQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterShowAccessEntitiesQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterShowAccessQuery(InterpreterFactory & factory);
|
||||
void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory);
|
||||
@ -108,6 +109,7 @@ void registerInterpreters()
|
||||
registerInterpreterGrantQuery(factory);
|
||||
registerInterpreterShowCreateAccessEntityQuery(factory);
|
||||
registerInterpreterShowGrantsQuery(factory);
|
||||
registerInterpreterCheckGrantQuery(factory);
|
||||
registerInterpreterShowAccessEntitiesQuery(factory);
|
||||
registerInterpreterShowAccessQuery(factory);
|
||||
registerInterpreterShowPrivilegesQuery(factory);
|
||||
|
126
src/Parsers/Access/ASTCheckGrantQuery.cpp
Normal file
126
src/Parsers/Access/ASTCheckGrantQuery.cpp
Normal file
@ -0,0 +1,126 @@
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
void formatColumnNames(const Strings & columns, const IAST::FormatSettings & settings)
|
||||
{
|
||||
settings.ostr << "(";
|
||||
bool need_comma = false;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (std::exchange(need_comma, true))
|
||||
settings.ostr << ", ";
|
||||
settings.ostr << backQuoteIfNeed(column);
|
||||
}
|
||||
settings.ostr << ")";
|
||||
}
|
||||
|
||||
|
||||
void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : "");
|
||||
if (element.isGlobalWithParameter())
|
||||
{
|
||||
if (element.anyParameter())
|
||||
settings.ostr << "*";
|
||||
else
|
||||
settings.ostr << backQuoteIfNeed(element.parameter);
|
||||
}
|
||||
else if (element.anyDatabase())
|
||||
{
|
||||
settings.ostr << "*.*";
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!element.database.empty())
|
||||
settings.ostr << backQuoteIfNeed(element.database) << ".";
|
||||
if (element.anyDatabase())
|
||||
settings.ostr << "*";
|
||||
else
|
||||
settings.ostr << backQuoteIfNeed(element.table);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void formatElementsWithoutOptions(const AccessRightsElements & elements, const IAST::FormatSettings & settings)
|
||||
{
|
||||
bool no_output = true;
|
||||
for (size_t i = 0; i != elements.size(); ++i)
|
||||
{
|
||||
const auto & element = elements[i];
|
||||
auto keywords = element.access_flags.toKeywords();
|
||||
if (keywords.empty() || (!element.anyColumn() && element.columns.empty()))
|
||||
continue;
|
||||
|
||||
for (const auto & keyword : keywords)
|
||||
{
|
||||
if (!std::exchange(no_output, false))
|
||||
settings.ostr << ", ";
|
||||
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << keyword << (settings.hilite ? IAST::hilite_none : "");
|
||||
if (!element.anyColumn())
|
||||
formatColumnNames(element.columns, settings);
|
||||
}
|
||||
|
||||
bool next_element_on_same_db_and_table = false;
|
||||
if (i != elements.size() - 1)
|
||||
{
|
||||
const auto & next_element = elements[i + 1];
|
||||
if (element.sameDatabaseAndTableAndParameter(next_element))
|
||||
{
|
||||
next_element_on_same_db_and_table = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!next_element_on_same_db_and_table)
|
||||
{
|
||||
settings.ostr << " ";
|
||||
formatONClause(element, settings);
|
||||
}
|
||||
}
|
||||
|
||||
if (no_output)
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "USAGE ON " << (settings.hilite ? IAST::hilite_none : "") << "*.*";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
String ASTCheckGrantQuery::getID(char) const
|
||||
{
|
||||
return "CheckGrantQuery";
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTCheckGrantQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTCheckGrantQuery>(*this);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void ASTCheckGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "CHECK GRANT"
|
||||
<< (settings.hilite ? IAST::hilite_none : "");
|
||||
|
||||
settings.ostr << " ";
|
||||
|
||||
formatElementsWithoutOptions(access_rights_elements, settings);
|
||||
}
|
||||
|
||||
|
||||
void ASTCheckGrantQuery::replaceEmptyDatabase(const String & current_database)
|
||||
{
|
||||
access_rights_elements.replaceEmptyDatabase(current_database);
|
||||
}
|
||||
|
||||
}
|
27
src/Parsers/Access/ASTCheckGrantQuery.h
Normal file
27
src/Parsers/Access/ASTCheckGrantQuery.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTRolesOrUsersSet;
|
||||
|
||||
|
||||
/** Parses queries like
|
||||
* CHECK GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*}
|
||||
*/
|
||||
class ASTCheckGrantQuery : public IAST
|
||||
{
|
||||
public:
|
||||
AccessRightsElements access_rights_elements;
|
||||
|
||||
String getID(char) const override;
|
||||
ASTPtr clone() const override;
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
void replaceEmptyDatabase(const String & current_database);
|
||||
QueryKind getQueryKind() const override { return QueryKind::Check; }
|
||||
};
|
||||
}
|
225
src/Parsers/Access/ParserCheckGrantQuery.cpp
Normal file
225
src/Parsers/Access/ParserCheckGrantQuery.cpp
Normal file
@ -0,0 +1,225 @@
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ParserCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_GRANT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags)
|
||||
{
|
||||
static constexpr 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()};
|
||||
return !(boost::iequals(word, toStringView(Keyword::ON)) || boost::iequals(word, toStringView(Keyword::TO)) || boost::iequals(word, toStringView(Keyword::FROM)));
|
||||
};
|
||||
|
||||
expected.add(pos, "access type");
|
||||
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!is_one_of_access_type_words(pos))
|
||||
return false;
|
||||
|
||||
String str;
|
||||
do
|
||||
{
|
||||
if (!str.empty())
|
||||
str += " ";
|
||||
str += std::string_view(pos->begin, pos->size());
|
||||
++pos;
|
||||
}
|
||||
while (is_one_of_access_type_words(pos));
|
||||
|
||||
try
|
||||
{
|
||||
access_flags = AccessFlags{str};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
bool parseColumnNames(IParser::Pos & pos, Expected & expected, Strings & columns)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserList{std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
Strings res_columns;
|
||||
for (const auto & child : ast->children)
|
||||
res_columns.emplace_back(getIdentifierName(child));
|
||||
|
||||
if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
columns = std::move(res_columns);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseAccessFlagsWithColumns(IParser::Pos & pos, Expected & expected,
|
||||
std::vector<std::pair<AccessFlags, Strings>> & access_and_columns)
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> res;
|
||||
|
||||
auto parse_access_and_columns = [&]
|
||||
{
|
||||
AccessFlags access_flags;
|
||||
if (!parseAccessFlags(pos, expected, access_flags))
|
||||
return false;
|
||||
|
||||
Strings columns;
|
||||
parseColumnNames(pos, expected, columns);
|
||||
res.emplace_back(access_flags, std::move(columns));
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_access_and_columns, false))
|
||||
return false;
|
||||
|
||||
access_and_columns = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool parseElements(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
AccessRightsElements res_elements;
|
||||
|
||||
auto parse_around_on = [&]
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> access_and_columns;
|
||||
if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns))
|
||||
return false;
|
||||
|
||||
String database_name, table_name, parameter;
|
||||
|
||||
size_t is_global_with_parameter = 0;
|
||||
for (const auto & elem : access_and_columns)
|
||||
{
|
||||
if (elem.first.isGlobalWithParameter())
|
||||
++is_global_with_parameter;
|
||||
}
|
||||
|
||||
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
bool wildcard = false;
|
||||
bool default_database = false;
|
||||
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
|
||||
{
|
||||
ASTPtr parameter_ast;
|
||||
if (!ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
{
|
||||
if (ParserIdentifier{}.parse(pos, parameter_ast, expected))
|
||||
parameter = getIdentifierName(parameter_ast);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
wildcard = true;
|
||||
}
|
||||
else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, table_name, wildcard, default_database))
|
||||
return false;
|
||||
|
||||
for (auto & [access_flags, columns] : access_and_columns)
|
||||
{
|
||||
AccessRightsElement element;
|
||||
element.access_flags = access_flags;
|
||||
element.columns = std::move(columns);
|
||||
element.database = database_name;
|
||||
element.table = table_name;
|
||||
element.parameter = parameter;
|
||||
element.wildcard = wildcard;
|
||||
element.default_database = default_database;
|
||||
res_elements.emplace_back(std::move(element));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_around_on, false))
|
||||
return false;
|
||||
|
||||
elements = std::move(res_elements);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
void throwIfNotGrantable(AccessRightsElements & elements)
|
||||
{
|
||||
std::erase_if(elements, [](AccessRightsElement & element)
|
||||
{
|
||||
if (element.empty())
|
||||
return true;
|
||||
auto old_flags = element.access_flags;
|
||||
element.eraseNonGrantable();
|
||||
if (!element.empty())
|
||||
return false;
|
||||
|
||||
if (!element.anyColumn())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the column level", old_flags.toString());
|
||||
else if (!element.anyTable())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the table level", old_flags.toString());
|
||||
else if (!element.anyDatabase())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the database level", old_flags.toString());
|
||||
else if (!element.anyParameter())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the global with parameter level", old_flags.toString());
|
||||
else
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant", old_flags.toString());
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool ParserCheckGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserKeyword{Keyword::CHECK_GRANT}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
|
||||
AccessRightsElements elements;
|
||||
|
||||
if (!parseElements(pos, expected, elements))
|
||||
return false;
|
||||
|
||||
throwIfNotGrantable(elements);
|
||||
|
||||
auto query = std::make_shared<ASTCheckGrantQuery>();
|
||||
node = query;
|
||||
|
||||
query->access_rights_elements = std::move(elements);
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
17
src/Parsers/Access/ParserCheckGrantQuery.h
Normal file
17
src/Parsers/Access/ParserCheckGrantQuery.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Parses queries like
|
||||
* CHECK GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*}
|
||||
*/
|
||||
class ParserCheckGrantQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "CHECK GRANT"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
}
|
@ -79,6 +79,7 @@ namespace DB
|
||||
MR_MACROS(CHARACTER, "CHARACTER") \
|
||||
MR_MACROS(CHECK_ALL_TABLES, "CHECK ALL TABLES") \
|
||||
MR_MACROS(CHECK_TABLE, "CHECK TABLE") \
|
||||
MR_MACROS(CHECK_GRANT, "CHECK GRANT")\
|
||||
MR_MACROS(CHECK, "CHECK") \
|
||||
MR_MACROS(CLEANUP, "CLEANUP") \
|
||||
MR_MACROS(CLEAR_COLUMN, "CLEAR COLUMN") \
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include <Parsers/Access/ParserCreateUserQuery.h>
|
||||
#include <Parsers/Access/ParserDropAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ParserGrantQuery.h>
|
||||
#include <Parsers/Access/ParserCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ParserMoveAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ParserSetRoleQuery.h>
|
||||
|
||||
@ -67,6 +68,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserDropAccessEntityQuery drop_access_entity_p;
|
||||
ParserMoveAccessEntityQuery move_access_entity_p;
|
||||
ParserGrantQuery grant_p;
|
||||
ParserCheckGrantQuery check_grant_p;
|
||||
ParserSetRoleQuery set_role_p;
|
||||
ParserExternalDDLQuery external_ddl_p;
|
||||
ParserTransactionControl transaction_control_p;
|
||||
@ -102,6 +104,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|| drop_access_entity_p.parse(pos, node, expected)
|
||||
|| move_access_entity_p.parse(pos, node, expected)
|
||||
|| grant_p.parse(pos, node, expected)
|
||||
|| check_grant_p.parse(pos, node, expected)
|
||||
|| external_ddl_p.parse(pos, node, expected)
|
||||
|| transaction_control_p.parse(pos, node, expected)
|
||||
|| delete_p.parse(pos, node, expected)
|
||||
|
6
tests/queries/0_stateless/03234_check_grant.reference
Normal file
6
tests/queries/0_stateless/03234_check_grant.reference
Normal file
@ -0,0 +1,6 @@
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
35
tests/queries/0_stateless/03234_check_grant.sh
Executable file
35
tests/queries/0_stateless/03234_check_grant.sh
Executable file
@ -0,0 +1,35 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
db=${CLICKHOUSE_DATABASE}
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS user_03234; DROP TABLE IF EXISTS ${db}.tb;CREATE USER user_03234; GRANT SELECT ON ${db}.tb TO user_03234;"
|
||||
|
||||
# Has been granted but not table not exists
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE ${db}.tb (\`content\` UInt64) ENGINE = MergeTree ORDER BY content; INSERT INTO ${db}.tb VALUES (1);"
|
||||
# Has been granted and table exists
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "REVOKE SELECT ON ${db}.tb FROM user_03234;"
|
||||
# Has not been granted but table exists
|
||||
# expected to 0
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
# Role
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CLIENT} --query "DROP ROLE IF EXISTS role_03234;CREATE ROLE role_03234;GRANT SELECT ON ${db}.tb TO role_03234;GRANT role_03234 TO user_03234"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "SET ROLE role_03234"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
# wildcard
|
||||
${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON ${db}.tbb* TO user_03234;"
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tbb1"
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tbb2*"
|
Loading…
Reference in New Issue
Block a user