This commit is contained in:
Unalian 2024-08-28 10:19:21 +08:00 committed by GitHub
commit 4b28cb96a5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 801 additions and 0 deletions

View File

@ -0,0 +1,43 @@
---
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, and whether the corresponding table/column exists in the memory.
## 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, or the role (which is granted with the privilege), and the db.table(column) exists on this node, 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
┌─CHECK_GRANT─┐
│ 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
┌─CHECK_GRANT─┐
│ 0 │
└─────────────┘
```

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,114 @@
#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 "Databases/IDatabase.h"
#include "Storages/IStorage.h"
namespace DB
{
namespace
{
/// Extracts access rights elements which are going to check grant
void collectAccessRightsElementsToGrantOrRevoke(
const ASTCheckGrantQuery & query,
AccessRightsElements & elements_to_check_grant)
{
elements_to_check_grant.clear();
/// GRANT
elements_to_check_grant = query.access_rights_elements;
}
}
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;
collectAccessRightsElementsToGrantOrRevoke(query, elements_to_check_grant);
/// 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);
auto *logger = &::Poco::Logger::get("CheckGrantQuery");
/// Check If Table/Columns exist.
for (const auto & elem : elements_to_check_grant)
{
try
{
DatabasePtr database;
database = DatabaseCatalog::instance().getDatabase(elem.database);
if (!database->isTableExist(elem.table, getContext()))
{
/// Table not found.
return executeQuery("SELECT 0 AS CHECK_GRANT", getContext(), QueryFlags{.internal = true}).second;
}
auto table = database->getTable(elem.table, getContext());
auto column_name_with_sizes = table->getColumnSizes();
for (const auto & elem_col : elem.columns)
{
bool founded = false;
for (const auto & col_in_table : column_name_with_sizes)
{
if (col_in_table.first == elem_col)
{
founded = true;
break;
}
}
if (!founded)
{
/// Column not found.
return executeQuery("SELECT 0 AS CHECK_GRANT", getContext(), QueryFlags{.internal = true}).second;
}
}
}
catch (...)
{
tryLogCurrentException(logger);
return executeQuery("SELECT 0 AS CHECK_GRANT", getContext(), QueryFlags{.internal = true}).second;
}
}
bool user_is_granted = current_user_access->isGranted(elements_to_check_grant);
if (!user_is_granted)
{
return executeQuery("SELECT 0 AS CHECK_GRANT", getContext(), QueryFlags{.internal = true}).second;
}
return executeQuery("SELECT 1 AS CHECK_GRANT", getContext(), QueryFlags{.internal = true}).second;
}
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

@ -40,6 +40,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>
@ -298,6 +299,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);
@ -104,6 +105,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.any_parameter)
settings.ostr << "*";
else
settings.ostr << backQuoteIfNeed(element.parameter);
}
else if (element.any_database)
{
settings.ostr << "*.*";
}
else
{
if (!element.database.empty())
settings.ostr << backQuoteIfNeed(element.database) << ".";
if (element.any_table)
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.any_column && 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.any_column)
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,231 @@
#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 parseElementsWithoutOptions(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;
bool any_database = false, any_table = false, any_parameter = false;
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;
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
{
ASTPtr parameter_ast;
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
{
any_parameter = true;
}
else if (ParserIdentifier{}.parse(pos, parameter_ast, expected))
{
any_parameter = false;
parameter = getIdentifierName(parameter_ast);
}
else
return false;
any_database = any_table = true;
}
else if (!parseDatabaseAndTableNameOrAsterisks(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.any_parameter = any_parameter;
element.table = table_name;
element.parameter = parameter;
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.any_column)
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the column level", old_flags.toString());
else if (!element.any_table)
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the table level", old_flags.toString());
else if (!element.any_database)
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the database level", old_flags.toString());
else if (!element.any_parameter)
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 (!parseElementsWithoutOptions(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

@ -78,6 +78,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

@ -26,6 +26,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>
@ -56,6 +57,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;
@ -82,6 +84,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,17 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,121 @@
<clickhouse>
<!-- See also the files in users.d directory where the settings can be overridden. -->
<!-- Profiles of settings. -->
<profiles>
<!-- Default settings. -->
<default>
</default>
<!-- Profile that allows only read queries. -->
<readonly>
<readonly>1</readonly>
</readonly>
</profiles>
<!-- Users and ACL. -->
<users>
<!-- If user name was not specified, 'default' user is used. -->
<default>
<!-- See also the files in users.d directory where the password can be overridden.
Password could be specified in plaintext or in SHA256 (in hex format).
If you want to specify password in plaintext (not recommended), place it in 'password' element.
Example: <password>qwerty</password>.
Password could be empty.
If you want to specify SHA256, place it in 'password_sha256_hex' element.
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
If you want to specify a previously defined LDAP server (see 'ldap_servers' in the main config) for authentication,
place its name in 'server' element inside 'ldap' element.
Example: <ldap><server>my_ldap_server</server></ldap>
If you want to authenticate the user via Kerberos (assuming Kerberos is enabled, see 'kerberos' in the main config),
place 'kerberos' element instead of 'password' (and similar) elements.
The name part of the canonical principal name of the initiator must match the user name for authentication to succeed.
You can also place 'realm' element inside 'kerberos' element to further restrict authentication to only those requests
whose initiator's realm matches it.
Example: <kerberos />
Example: <kerberos><realm>EXAMPLE.COM</realm></kerberos>
How to generate decent password:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
In first line will be password and in second - corresponding SHA256.
How to generate double SHA1:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha1sum | tr -d '-' | xxd -r -p | sha1sum | tr -d '-'
In first line will be password and in second - corresponding double SHA1.
-->
<password></password>
<!-- List of networks with open access.
To open access from everywhere, specify:
<ip>::/0</ip>
To open access only from localhost, specify:
<ip>::1</ip>
<ip>127.0.0.1</ip>
Each element of list has one of the following forms:
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
<host> Hostname. Example: server01.clickhouse.com.
To check access, DNS query is performed, and all received addresses compared to peer address.
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.clickhouse\.com$
To check access, DNS PTR query is performed for peer address and then regexp is applied.
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
Strongly recommended that regexp is ends with $
All results of DNS requests are cached till server restart.
-->
<networks>
<ip>::/0</ip>
</networks>
<!-- Settings profile for user. -->
<profile>default</profile>
<!-- Quota for user. -->
<quota>default</quota>
<!-- User can create other users and grant rights to them. -->
<access_management>1</access_management>
<!-- User can manipulate named collections. -->
<named_collection_control>1</named_collection_control>
<show_named_collections>1</show_named_collections>
<show_named_collections_secrets>1</show_named_collections_secrets>
<!-- User permissions can be granted here -->
<!--
<grants>
<query>GRANT ALL ON *.*</query>
</grants>
-->
</default>
</users>
<!-- Quotas. -->
<quotas>
<!-- Name of quota. -->
<default>
<!-- Limits for time interval. You could specify many intervals with different limits. -->
<interval>
<!-- Length of interval. -->
<duration>3600</duration>
<!-- No limits. Just calculate resource usage for time interval. -->
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</clickhouse>

View File

@ -0,0 +1,58 @@
#!/usr/bin/env python3
import logging
import pytest
from helpers.cluster import ClickHouseCluster
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance("node1", user_configs=["configs/users.xml"])
cluster.add_instance(
"node2",
user_configs=[
"configs/users.xml",
],
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def test_check_grant(cluster):
node1 = cluster.instances["node1"]
node1.query("DROP user IF EXISTS tuser")
node1.query("CREATE USER tuser")
node1.query("GRANT SELECT ON tb TO tuser")
# Has been granted but not table not exists
res, _ = node1.query("CHECK GRANT SELECT ON tb", user="tuser")
assert res == "0"
node1.query(
"CREATE TABLE tb (`content` UInt64) ENGINE = MergeTree ORDER BY content"
)
node1.query("INSERT INTO tb VALUES (1)")
# Has been granted and table exists
res, _ = node1.query("CHECK GRANT SELECT ON tb", user="tuser")
assert res == "1"
node1.query("REVOKE SELECT ON tb FROM tuser")
# Has not been granted but table exists
res, _ = node1.query("CHECK GRANT SELECT ON tb", user="tuser")
assert res == "0"
# Role
node1.query("CREATE ROLE trole")
node1.query("GRANT SELECT ON tb TO trole")
node1.query("GRANT trole TO tuser")
node1.query("SET ROLE trole", user="tuser")
res, _ = node1.query("CHECK GRANT SELECT ON tb", user="tuser")
assert res == "1"