Merge pull request #38970 from vitlibar/select-from-system-db-requires-grant

SELECT from the system database requires grant now
This commit is contained in:
Vitaly Baranov 2022-07-16 22:37:47 +02:00 committed by GitHub
commit e0fb03d1b6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 846 additions and 97 deletions

View File

@ -106,7 +106,9 @@ void Client::processError(const String & query) const
std::vector<String> Client::loadWarningMessages()
{
std::vector<String> messages;
connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "" /* query_id */,
connection->sendQuery(connection_parameters.timeouts,
"SELECT * FROM viewIfPermitted(SELECT message FROM system.warnings ELSE null('message String'))",
"" /* query_id */,
QueryProcessingStage::Complete,
&global_context->getSettingsRef(),
&global_context->getClientInfo(), false, {});

View File

@ -604,9 +604,23 @@
if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
By default this setting is false for compatibility with earlier access configurations. -->
<users_without_row_policies_can_read_rows>false</users_without_row_policies_can_read_rows>
<!-- By default, for backward compatibility ON CLUSTER queries ignore CLUSTER grant,
however you can change this behaviour by setting this to true -->
<on_cluster_queries_require_cluster_grant>false</on_cluster_queries_require_cluster_grant>
<!-- By default, for backward compatibility "SELECT * FROM system.<table>" doesn't require any grants and can be executed
by any user. You can change this behaviour by setting this to true.
If it's set to true then this query requires "GRANT SELECT ON system.<table>" just like as for non-system tables.
Exceptions: a few system tables ("tables", "columns", "databases", and some constant tables like "one", "contributors")
are still accessible for everyone; and if there is a SHOW privilege (e.g. "SHOW USERS") granted the corresponding system
table (i.e. "system.users") will be accessible. -->
<select_from_system_db_requires_grant>false</select_from_system_db_requires_grant>
<!-- By default, for backward compatibility "SELECT * FROM information_schema.<table>" doesn't require any grants and can be
executed by any user. You can change this behaviour by setting this to true.
If it's set to true then this query requires "GRANT SELECT ON information_schema.<table>" just like as for ordinary tables. -->
<select_from_information_schema_requires_grant>false</select_from_information_schema_requires_grant>
</access_control_improvements>
<!-- Default profile of settings. -->

View File

@ -165,13 +165,12 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
setNoPasswordAllowed(config_.getBool("allow_no_password", true));
setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true));
setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool(
"access_control_improvements.users_without_row_policies_can_read_rows",
false /* false because we need to be compatible with earlier access configurations */));
setOnClusterQueriesRequireClusterGrant(config_.getBool(
"access_control_improvements.on_cluster_queries_require_cluster_grant",
false /* false because we need to be compatible with earlier access configurations */));
/// Optional improvements in access control system.
/// The default values are false because we need to be compatible with earlier access configurations
setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", false));
setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", false));
setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false));
setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", false));
addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_);
}

View File

@ -152,6 +152,12 @@ public:
void setOnClusterQueriesRequireClusterGrant(bool enable) { on_cluster_queries_require_cluster_grant = enable; }
bool doesOnClusterQueriesRequireClusterGrant() const { return on_cluster_queries_require_cluster_grant; }
void setSelectFromSystemDatabaseRequiresGrant(bool enable) { select_from_system_db_requires_grant = enable; }
bool doesSelectFromSystemDatabaseRequireGrant() const { return select_from_system_db_requires_grant; }
void setSelectFromInformationSchemaRequiresGrant(bool enable) { select_from_information_schema_requires_grant = enable; }
bool doesSelectFromInformationSchemaRequireGrant() const { return select_from_information_schema_requires_grant; }
std::shared_ptr<const ContextAccess> getContextAccess(
const UUID & user_id,
const std::vector<UUID> & current_roles,
@ -215,6 +221,8 @@ private:
std::atomic_bool allow_no_password = true;
std::atomic_bool users_without_row_policies_can_read_rows = false;
std::atomic_bool on_cluster_queries_require_cluster_grant = false;
std::atomic_bool select_from_system_db_requires_grant = false;
std::atomic_bool select_from_information_schema_requires_grant = false;
};
}

View File

@ -388,11 +388,11 @@ public:
return res;
}
void modifyFlags(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed)
void modifyFlags(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed)
{
flags_added = false;
flags_removed = false;
modifyFlagsRec(function, flags_added, flags_removed);
modifyFlagsRec(function, grant_option, flags_added, flags_removed);
if (flags_added || flags_removed)
optimizeTree();
}
@ -669,11 +669,11 @@ private:
}
template <typename ... ParentNames>
void modifyFlagsRec(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names)
void modifyFlagsRec(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names)
{
auto invoke = [&function](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags
auto invoke = [function, grant_option](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags
{
return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_);
return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_, grant_option);
};
if constexpr (sizeof...(ParentNames) < 3)
@ -683,7 +683,7 @@ private:
for (auto & child : *children | boost::adaptors::map_values)
{
const String & child_name = *child.node_name;
child.modifyFlagsRec(function, flags_added, flags_removed, parent_names..., child_name);
child.modifyFlagsRec(function, grant_option, flags_added, flags_removed, parent_names..., child_name);
}
}
}
@ -1062,24 +1062,21 @@ void AccessRights::modifyFlags(const ModifyFlagsFunction & function)
{
if (!root)
return;
bool flags_added, flags_removed;
root->modifyFlags(function, flags_added, flags_removed);
root->modifyFlags(function, false, flags_added, flags_removed);
if (flags_removed && root_with_grant_option)
root_with_grant_option->makeIntersection(*root);
}
void AccessRights::modifyFlagsWithGrantOption(const ModifyFlagsFunction & function)
{
if (!root_with_grant_option)
return;
bool flags_added, flags_removed;
root_with_grant_option->modifyFlags(function, flags_added, flags_removed);
if (flags_added)
if (root_with_grant_option)
{
if (!root)
root = std::make_unique<Node>();
root->makeUnion(*root_with_grant_option);
root_with_grant_option->modifyFlags(function, true, flags_added, flags_removed);
if (flags_added)
{
if (!root)
root = std::make_unique<Node>();
root->makeUnion(*root_with_grant_option);
}
}
}

View File

@ -109,9 +109,9 @@ public:
const AccessFlags & max_flags_with_children,
std::string_view database,
std::string_view table,
std::string_view column)>;
std::string_view column,
bool grant_option)>;
void modifyFlags(const ModifyFlagsFunction & function);
void modifyFlagsWithGrantOption(const ModifyFlagsFunction & function);
friend bool operator ==(const AccessRights & left, const AccessRights & right);
friend bool operator !=(const AccessRights & left, const AccessRights & right) { return !(left == right); }

View File

@ -44,9 +44,17 @@ namespace
}
AccessRights addImplicitAccessRights(const AccessRights & access)
AccessRights addImplicitAccessRights(const AccessRights & access, const AccessControl & access_control)
{
auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, std::string_view column) -> AccessFlags
AccessFlags max_flags;
auto modifier = [&](const AccessFlags & flags,
const AccessFlags & min_flags_with_children,
const AccessFlags & max_flags_with_children,
std::string_view database,
std::string_view table,
std::string_view column,
bool /* grant_option */) -> AccessFlags
{
size_t level = !database.empty() + !table.empty() + !column.empty();
AccessFlags res = flags;
@ -115,17 +123,80 @@ namespace
res |= show_databases;
}
max_flags |= res;
return res;
};
AccessRights res = access;
res.modifyFlags(modifier);
res.modifyFlagsWithGrantOption(modifier);
/// Anyone has access to the "system" and "information_schema" database.
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
/// If "select_from_system_db_requires_grant" is enabled we provide implicit grants only for a few tables in the system database.
if (access_control.doesSelectFromSystemDatabaseRequireGrant())
{
const char * always_accessible_tables[] = {
/// Constant tables
"one",
/// "numbers", "numbers_mt", "zeros", "zeros_mt" were excluded because they can generate lots of values and
/// that can decrease performance in some cases.
"contributors",
"licenses",
"time_zones",
"collations",
"formats",
"privileges",
"data_type_families",
"table_engines",
"table_functions",
"aggregate_function_combinators",
"functions", /// Can contain user-defined functions
/// The following tables hide some rows if the current user doesn't have corresponding SHOW privileges.
"databases",
"tables",
"columns",
/// Specific to the current session
"settings",
"current_roles",
"enabled_roles",
"quota_usage"
};
for (const auto * table_name : always_accessible_tables)
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, table_name);
if (max_flags.contains(AccessType::SHOW_USERS))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "users");
if (max_flags.contains(AccessType::SHOW_ROLES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "roles");
if (max_flags.contains(AccessType::SHOW_ROW_POLICIES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "row_policies");
if (max_flags.contains(AccessType::SHOW_SETTINGS_PROFILES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "settings_profiles");
if (max_flags.contains(AccessType::SHOW_QUOTAS))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "quotas");
}
else
{
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
}
/// If "select_from_information_schema_requires_grant" is enabled we don't provide implicit grants for the information_schema database.
if (!access_control.doesSelectFromInformationSchemaRequireGrant())
{
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
}
return res;
}
@ -247,7 +318,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
void ContextAccess::calculateAccessRights() const
{
access = std::make_shared<AccessRights>(mixAccessRightsFromUserAndRoles(*user, *roles_info));
access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*access));
access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*access, *access_control));
if (trace_log)
{
@ -342,7 +413,7 @@ std::shared_ptr<const ContextAccess> ContextAccess::getFullAccess()
auto full_access = std::shared_ptr<ContextAccess>(new ContextAccess);
full_access->is_full_access = true;
full_access->access = std::make_shared<AccessRights>(AccessRights::getFullAccess());
full_access->access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*full_access->access));
full_access->access_with_implicit = full_access->access;
return full_access;
}();
return res;
@ -413,7 +484,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
};
if (is_full_access)
return access_granted();
return true;
if (user_was_dropped)
return access_denied("User has been dropped", ErrorCodes::UNKNOWN_USER);
@ -422,7 +493,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
flags &= ~AccessType::CLUSTER;
if (!flags)
return access_granted();
return true;
/// Access to temporary tables is controlled in an unusual way, not like normal tables.
/// Creating of temporary tables is controlled by AccessType::CREATE_TEMPORARY_TABLES grant,

View File

@ -50,52 +50,58 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti
{
/// NOTE: Once you will update the completion list,
/// do not forget to update 01676_clickhouse_client_autocomplete.sh
WriteBufferFromOwnString query;
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
"SELECT name FROM system.functions"
" UNION ALL "
"SELECT name FROM system.table_engines"
" UNION ALL "
"SELECT name FROM system.formats"
" UNION ALL "
"SELECT name FROM system.table_functions"
" UNION ALL "
"SELECT name FROM system.data_type_families"
" UNION ALL "
"SELECT name FROM system.merge_tree_settings"
" UNION ALL "
"SELECT name FROM system.settings"
" UNION ALL ";
String query;
auto add_subquery = [&](std::string_view select, std::string_view result_column_name)
{
if (!query.empty())
query += " UNION ALL ";
query += fmt::format("SELECT * FROM viewIfPermitted({} ELSE null('{} String'))", select, result_column_name);
};
auto add_column = [&](std::string_view column_name, std::string_view table_name, bool distinct, std::optional<Int64> limit)
{
add_subquery(
fmt::format(
"SELECT {}{} FROM system.{}{}",
(distinct ? "DISTINCT " : ""),
column_name,
table_name,
(limit ? (" LIMIT " + std::to_string(*limit)) : "")),
column_name);
};
add_column("name", "functions", false, {});
add_column("name", "table_engines", false, {});
add_column("name", "formats", false, {});
add_column("name", "table_functions", false, {});
add_column("name", "data_type_families", false, {});
add_column("name", "merge_tree_settings", false, {});
add_column("name", "settings", false, {});
if (!basic_suggestion)
{
query << "SELECT cluster FROM system.clusters"
" UNION ALL "
"SELECT macro FROM system.macros"
" UNION ALL "
"SELECT policy_name FROM system.storage_policies"
" UNION ALL ";
add_column("cluster", "clusters", false, {});
add_column("macro", "macros", false, {});
add_column("policy_name", "storage_policies", false, {});
}
query << "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate";
add_subquery("SELECT concat(func.name, comb.name) AS x FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate", "x");
/// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero.
if (suggestion_limit > 0)
{
String limit_str = toString(suggestion_limit);
query << " UNION ALL "
"SELECT name FROM system.databases LIMIT " << limit_str
<< " UNION ALL "
"SELECT DISTINCT name FROM system.tables LIMIT " << limit_str
<< " UNION ALL ";
add_column("name", "databases", false, suggestion_limit);
add_column("name", "tables", true, suggestion_limit);
if (!basic_suggestion)
{
query << "SELECT DISTINCT name FROM system.dictionaries LIMIT " << limit_str
<< " UNION ALL ";
add_column("name", "dictionaries", true, suggestion_limit);
}
query << "SELECT DISTINCT name FROM system.columns LIMIT " << limit_str;
add_column("name", "columns", true, suggestion_limit);
}
query << ") WHERE notEmpty(res)";
return query.str();
query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res)";
return query;
}
template <typename ConnectionType>

View File

@ -509,6 +509,25 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << ')';
written = true;
}
if (!written && name == "viewIfPermitted"sv)
{
/// viewIfPermitted() needs special formatting: ELSE instead of comma between arguments, and better indents too.
const auto * nl_or_nothing = settings.one_line ? "" : "\n";
auto indent0 = settings.one_line ? "" : String(4u * frame.indent, ' ');
auto indent1 = settings.one_line ? "" : String(4u * (frame.indent + 1), ' ');
auto indent2 = settings.one_line ? "" : String(4u * (frame.indent + 2), ' ');
settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << (settings.hilite ? hilite_none : "") << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
frame_nested.indent += 2;
arguments->children[0]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent1 << (settings.hilite ? hilite_keyword : "") << (settings.one_line ? " " : "")
<< "ELSE " << (settings.hilite ? hilite_none : "") << nl_or_nothing << indent2;
arguments->children[1]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent0 << ")";
return;
}
}
if (!written && arguments->children.size() >= 2)

View File

@ -1068,13 +1068,16 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_parser;
ParserKeyword view("VIEW");
ParserSelectWithUnionQuery select;
ASTPtr identifier;
ASTPtr query;
if (!view.ignore(pos, expected))
bool if_permitted = false;
if (ParserKeyword{"VIEWIFPERMITTED"}.ignore(pos, expected))
if_permitted = true;
else if (!ParserKeyword{"VIEW"}.ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
@ -1094,15 +1097,30 @@ bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
return false;
}
ASTPtr else_ast;
if (if_permitted)
{
if (!ParserKeyword{"ELSE"}.ignore(pos, expected))
return false;
if (!ParserWithOptionalAlias{std::make_unique<ParserFunction>(true, true), true}.parse(pos, else_ast, expected))
return false;
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto expr_list = std::make_shared<ASTExpressionList>();
expr_list->children.push_back(query);
if (if_permitted)
expr_list->children.push_back(else_ast);
auto function_node = std::make_shared<ASTFunction>();
tryGetIdentifierNameInto(identifier, function_node->name);
auto expr_list_with_single_query = std::make_shared<ASTExpressionList>();
expr_list_with_single_query->children.push_back(query);
function_node->name = "view";
function_node->arguments = expr_list_with_single_query;
function_node->name = if_permitted ? "viewIfPermitted" : "view";
function_node->arguments = expr_list;
function_node->children.push_back(function_node->arguments);
node = function_node;
return true;
@ -1971,6 +1989,7 @@ const char * ParserAlias::restricted_keywords[] =
"WITH",
"INTERSECT",
"EXCEPT",
"ELSE",
nullptr
};

View File

@ -162,7 +162,7 @@ protected:
bool is_table_function;
};
// A special function parser for view table function.
// A special function parser for view and viewIfPermitted table functions.
// It parses an SELECT query as its argument and doesn't support getColumnName().
class ParserTableFunctionView : public IParserBase
{

View File

@ -180,9 +180,13 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
if (!table_expression->database_and_table_name)
{
// If it's a view table function, add a fake db.table name.
if (table_expression->table_function && table_expression->table_function->as<ASTFunction>()->name == "view")
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
else
if (table_expression->table_function)
{
auto table_function_name = table_expression->table_function->as<ASTFunction>()->name;
if ((table_function_name == "view") || (table_function_name == "viewIfPermitted"))
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
}
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -36,8 +36,11 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes()
void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -66,8 +66,11 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes()
void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -78,7 +78,11 @@ NamesAndTypesList StorageSystemQuotaUsage::getNamesAndTypesImpl(bool add_column_
void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
auto usage = context->getQuotaUsage();
if (!usage)
return;

View File

@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes()
void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -15,7 +15,11 @@ NamesAndTypesList StorageSystemQuotasUsage::getNamesAndTypes()
void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
auto all_quotas_usage = context->getAccessControl().getAllQuotasUsage();
StorageSystemQuotaUsage::fillDataImpl(res_columns, context, /* add_column_is_current = */ true, all_quotas_usage);
}

View File

@ -31,8 +31,11 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes()
void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -27,8 +27,11 @@ NamesAndTypesList StorageSystemRoles::getNamesAndTypes()
void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<Role>();
size_t column_index = 0;

View File

@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes()
void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROW_POLICIES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_ROW_POLICIES);
std::vector<UUID> ids = access_control.findAll<RowPolicy>();
size_t column_index = 0;

View File

@ -37,8 +37,11 @@ NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes()
void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());
boost::range::push_back(ids, access_control.findAll<SettingsProfile>());

View File

@ -34,8 +34,11 @@ NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes()
void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
std::vector<UUID> ids = access_control.findAll<SettingsProfile>();
size_t column_index = 0;

View File

@ -60,8 +60,11 @@ NamesAndTypesList StorageSystemUsers::getNamesAndTypes()
void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS);
std::vector<UUID> ids = access_control.findAll<User>();
size_t column_index = 0;

View File

@ -23,7 +23,12 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte
ColumnsDescription cached_columns, bool use_global_context) const
{
ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
context->checkAccess(AccessType::CREATE_TEMPORARY_TABLE | getSourceAccessType());
AccessFlags required_access = getSourceAccessType();
String function_name = getName();
if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted"))
required_access |= AccessType::CREATE_TEMPORARY_TABLE;
context->checkAccess(required_access);
auto context_to_use = use_global_context ? context->getGlobalContext() : context;

View File

@ -0,0 +1,113 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Storages/StorageNull.h>
#include <Storages/StorageView.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionViewIfPermitted.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Interpreters/evaluateConstantExpression.h>
#include "registerTableFunctions.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int ACCESS_DENIED;
}
const ASTSelectWithUnionQuery & TableFunctionViewIfPermitted::getSelectQuery() const
{
return *create.select;
}
void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto * function = ast_function->as<ASTFunction>();
if (!function || !function->arguments || (function->arguments->children.size() != 2))
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Table function '{}' requires two arguments: a SELECT query and a table function",
getName());
const auto & arguments = function->arguments->children;
auto * select = arguments[0]->as<ASTSelectWithUnionQuery>();
if (!select)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a SELECT query as its first argument", getName());
create.set(create.select, select->clone());
else_ast = arguments[1];
if (!else_ast->as<ASTFunction>())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a table function as its second argument", getName());
else_table_function = TableFunctionFactory::instance().get(else_ast, context);
}
ColumnsDescription TableFunctionViewIfPermitted::getActualTableStructure(ContextPtr context) const
{
return else_table_function->getActualTableStructure(context);
}
StoragePtr TableFunctionViewIfPermitted::executeImpl(
const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, ColumnsDescription /* cached_columns */) const
{
StoragePtr storage;
auto columns = getActualTableStructure(context);
if (isPermitted(context, columns))
{
storage = std::make_shared<StorageView>(StorageID(getDatabaseName(), table_name), create, columns, "");
}
else
{
storage = else_table_function->execute(else_ast, context, table_name);
}
storage->startup();
return storage;
}
bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const
{
Block sample_block;
try
{
/// Will throw ACCESS_DENIED if the current user is not allowed to execute the SELECT query.
sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ACCESS_DENIED)
return false;
throw;
}
/// We check that columns match only if permitted (otherwise we could reveal the structure to an user who must not know it).
ColumnsDescription columns{sample_block.getNamesAndTypesList()};
if (columns != else_columns)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a SELECT query with the result columns matching a table function after 'ELSE'. "
"Currently the result columns of the SELECT query are {}, and the table function after 'ELSE' gives {}",
getName(),
columns.toString(),
else_columns.toString());
}
return true;
}
void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionViewIfPermitted>();
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
#include <Parsers/ASTCreateQuery.h>
#include <base/types.h>
namespace DB
{
/* viewIfPermitted(query ELSE null('structure'))
* Works as "view(query)" if the current user has the permissions required to execute "query"; works as "null('structure')" otherwise.
*/
class TableFunctionViewIfPermitted : public ITableFunction
{
public:
static constexpr auto name = "viewIfPermitted";
std::string getName() const override { return name; }
const ASTSelectWithUnionQuery & getSelectQuery() const;
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override;
const char * getStorageTypeName() const override { return "ViewIfPermitted"; }
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const;
ASTCreateQuery create;
ASTPtr else_ast;
TableFunctionPtr else_table_function;
};
}

View File

@ -42,6 +42,7 @@ void registerTableFunctions()
registerTableFunctionJDBC(factory);
registerTableFunctionView(factory);
registerTableFunctionViewIfPermitted(factory);
#if USE_MYSQL
registerTableFunctionMySQL(factory);

View File

@ -40,6 +40,7 @@ void registerTableFunctionODBC(TableFunctionFactory & factory);
void registerTableFunctionJDBC(TableFunctionFactory & factory);
void registerTableFunctionView(TableFunctionFactory & factory);
void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory);
#if USE_MYSQL
void registerTableFunctionMySQL(TableFunctionFactory & factory);

View File

@ -2,5 +2,7 @@
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
<on_cluster_queries_require_cluster_grant>true</on_cluster_queries_require_cluster_grant>
<select_from_system_db_requires_grant>true</select_from_system_db_requires_grant>
<select_from_information_schema_requires_grant>true</select_from_information_schema_requires_grant>
</access_control_improvements>
</clickhouse>

View File

@ -21,5 +21,8 @@
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
<on_cluster_queries_require_cluster_grant>true</on_cluster_queries_require_cluster_grant>
<select_from_system_db_requires_grant>true</select_from_system_db_requires_grant>
<select_from_information_schema_requires_grant>true</select_from_information_schema_requires_grant>
</access_control_improvements>
</clickhouse>

View File

@ -434,6 +434,7 @@ def test_required_privileges():
node1.query("INSERT INTO tbl VALUES (100)")
node1.query("CREATE USER u1")
node1.query("GRANT CLUSTER ON *.* TO u1")
backup_name = new_backup_name()
expected_error = "necessary to have grant BACKUP ON default.tbl"
@ -478,6 +479,7 @@ def test_system_users():
backup_name = new_backup_name()
node1.query("CREATE USER u2 SETTINGS allow_backup=false")
node1.query("GRANT CLUSTER ON *.* TO u2")
expected_error = "necessary to have grant BACKUP ON system.users"
assert expected_error in node1.query_and_get_error(

View File

@ -19,6 +19,7 @@
</networks>
<allow_databases>
<database>default</database>
<database>system</database>
</allow_databases>
</test_allow>
</users>

View File

@ -1,5 +1,7 @@
<clickhouse>
<access_control_improvements>
<users_without_row_policies_can_read_rows remove="remove"/>
<select_from_system_db_requires_grant remove="remove"/>
<select_from_information_schema_requires_grant remove="remove"/>
</access_control_improvements>
</clickhouse>

View File

@ -13,6 +13,9 @@
</networks>
<profile>default</profile>
<quota>default</quota>
<allow_databases>
<database>mydb</database>
</allow_databases>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1,162 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/config.d/disable_access_control_improvements.xml"],
user_configs=[
"configs/users.d/another_user.xml",
],
)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
node.query("CREATE DATABASE mydb")
node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log")
node.query("CREATE TABLE table2(x UInt32) ENGINE=Log")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_after_test():
try:
node.query("CREATE USER OR REPLACE sqluser")
yield
finally:
pass
def test_system_db():
assert node.query("SELECT count()>0 FROM system.settings") == "1\n"
assert node.query("SELECT count()>0 FROM system.users") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n"
assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n"
expected_error = "necessary to have grant SHOW USERS ON *.*"
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="another"
)
assert node.query("SELECT count()>0 FROM system.clusters", user="another") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="another"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="another"
)
== "0\n"
)
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
expected_error = "necessary to have grant SHOW USERS ON *.*"
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="sqluser"
)
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "0\n"
)
node.query("GRANT SHOW USERS ON *.* TO sqluser")
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "1\n"
)
def test_information_schema():
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="another",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="another",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "0\n"
)
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "1\n"
)

View File

@ -0,0 +1,16 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<another>
<password/>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<allow_databases>
<database>mydb</database>
</allow_databases>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1,192 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
user_configs=[
"configs/another_user.xml",
],
)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
node.query("CREATE DATABASE mydb")
node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log")
node.query("CREATE TABLE table2(x UInt32) ENGINE=Log")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_after_test():
try:
node.query("CREATE USER OR REPLACE sqluser")
yield
finally:
pass
def test_system_db():
assert node.query("SELECT count()>0 FROM system.settings") == "1\n"
assert node.query("SELECT count()>0 FROM system.users") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n"
assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n"
expected_error = (
"necessary to have grant SELECT for at least one column on system.users"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="another"
)
expected_error = (
"necessary to have grant SELECT for at least one column on system.clusters"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.clusters", user="another"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="another"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="another"
)
== "0\n"
)
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
expected_error = (
"necessary to have grant SELECT for at least one column on system.users"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="sqluser"
)
expected_error = (
"necessary to have grant SELECT for at least one column on system.clusters"
)
assert node.query_and_get_error(
"SELECT count()>0 FROM system.clusters", user="sqluser"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "0\n"
)
node.query("GRANT SELECT ON system.users TO sqluser")
node.query("GRANT SELECT ON system.clusters TO sqluser")
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "1\n"
)
node.query("REVOKE ALL ON *.* FROM sqluser")
node.query("GRANT SHOW USERS ON *.* TO sqluser")
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
def test_information_schema():
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'"
)
== "1\n"
)
expected_error = (
"necessary to have grant SELECT(table_name) ON information_schema.tables"
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="another",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="another",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
node.query("GRANT SELECT ON information_schema.* TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "0\n"
)
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "1\n"
)

View File

@ -65,3 +65,38 @@ def test_merge():
"it's necessary to have grant SELECT ON default.table2"
in instance.query_and_get_error(select_query, user="A")
)
def test_view_if_permitted():
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))"
)
== "1\n"
)
expected_error = "requires a SELECT query with the result columns matching a table function after 'ELSE'"
assert expected_error in instance.query_and_get_error(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x Int32'))"
)
assert expected_error in instance.query_and_get_error(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('y UInt32'))"
)
instance.query("CREATE USER A")
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))",
user="A",
)
== ""
)
instance.query("GRANT SELECT ON table1 TO A")
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))",
user="A",
)
== "1\n"
)

View File

@ -9,6 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600"
${CLICKHOUSE_CLIENT} -q "create user u_00600 settings max_execution_time=60, readonly=1"
${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600"
function wait_for_query_to_start()
{

View File

@ -10,6 +10,7 @@ set -e
user=user_$CLICKHOUSE_TEST_UNIQUE_NAME
$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS $user"
$CLICKHOUSE_CLIENT --query "CREATE USER $user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'"
$CLICKHOUSE_CLIENT --query "GRANT SELECT ON system.numbers TO $user"
trap '$CLICKHOUSE_CLIENT --query "DROP USER $user"' EXIT
# Wait for query to start executing. At that time, the password should be cleared.