Merge pull request #68885 from Unalian/feat-67772

Add CHECK GRANT query
This commit is contained in:
Vitaly Baranov 2024-11-19 16:09:30 +00:00 committed by GitHub
commit 353ff951aa
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 586 additions and 0 deletions

View 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.

View 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 />

View 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);
}
}

View 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;
};
}

View File

@ -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";

View File

@ -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);

View 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);
}
}

View 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; }
};
}

View 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;
}
}

View 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;
};
}

View File

@ -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") \

View File

@ -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)

View File

@ -0,0 +1,6 @@
1
1
0
1
1
1

View 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*"