Skip to content

Commit

Permalink
Backport #18233 to 21.1: Fix access rights for SELECT count()
Browse files Browse the repository at this point in the history
  • Loading branch information
robot-clickhouse committed Jan 16, 2021
1 parent 70d3298 commit 968cee8
Show file tree
Hide file tree
Showing 4 changed files with 66 additions and 3 deletions.
34 changes: 33 additions & 1 deletion src/Interpreters/InterpreterSelectQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
#include <Parsers/parseQuery.h>

#include <Access/AccessFlags.h>
#include <Access/ContextAccess.h>

#include <AggregateFunctions/AggregateFunctionCount.h>

Expand Down Expand Up @@ -99,6 +100,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
extern const int INVALID_LIMIT_EXPRESSION;
extern const int INVALID_WITH_FILL_EXPRESSION;
extern const int ACCESS_DENIED;
}

/// Assumes `storage` is set and the table filter (row-level security) is not empty.
Expand Down Expand Up @@ -211,6 +213,36 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query);
}

/// Checks that the current user has the SELECT privilege.
static void checkAccessRightsForSelect(
const Context & context,
const StorageID & table_id,
const StorageMetadataPtr & table_metadata,
const Strings & required_columns,
const TreeRewriterResult & syntax_analyzer_result)
{
if (!syntax_analyzer_result.has_explicit_columns && table_metadata && !table_metadata->getColumns().empty())
{
/// For a trivial query like "SELECT count() FROM table" access is granted if at least
/// one column is accessible.
/// In this case just checking access for `required_columns` doesn't work correctly
/// because `required_columns` will contain the name of a column of minimum size (see TreeRewriterResult::collectUsedColumns())
/// which is probably not the same column as the column the current user has access to.
auto access = context.getAccess();
for (const auto & column : table_metadata->getColumns())
{
if (access->isGranted(AccessType::SELECT, table_id.database_name, table_id.table_name, column.name))
return;
}
throw Exception(context.getUserName() + ": Not enough privileges. "
"To execute this query it's necessary to have grant SELECT for at least one column on " + table_id.getFullTableName(),
ErrorCodes::ACCESS_DENIED);
}

/// General check.
context.checkAccess(AccessType::SELECT, table_id, required_columns);
}

/// Returns true if we should ignore quotas and limits for a specified table in the system database.
static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id)
{
Expand Down Expand Up @@ -466,7 +498,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
{
/// The current user should have the SELECT privilege.
/// If this table_id is for a table function we don't check access rights here because in this case they have been already checked in ITableFunction::execute().
context->checkAccess(AccessType::SELECT, table_id, required_columns);
checkAccessRightsForSelect(*context, table_id, metadata_snapshot, required_columns, *syntax_analyzer_result);

/// Remove limits for some tables in the `system` database.
if (shouldIgnoreQuotaAndLimits(table_id) && (joined_tables.tablesCount() <= 1))
Expand Down
6 changes: 4 additions & 2 deletions src/Interpreters/TreeRewriter.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -588,11 +588,13 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
required.insert(column_name_type.name);
}

/// You need to read at least one column to find the number of rows.
if (is_select && required.empty())
/// Figure out if we're able to use the trivial count optimization.
has_explicit_columns = !required.empty();
if (is_select && !has_explicit_columns)
{
optimize_trivial_count = true;

/// You need to read at least one column to find the number of rows.
/// We will find a column with minimum <compressed_size, type_size, uncompressed_size>.
/// Because it is the column that is cheapest to read.
struct ColumnSizeTuple
Expand Down
7 changes: 7 additions & 0 deletions src/Interpreters/TreeRewriter.h
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,13 @@ struct TreeRewriterResult
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;

/// Whether the query contains explicit columns like "SELECT column1 + column2 FROM table1".
/// Queries like "SELECT count() FROM table1", "SELECT 1" don't contain explicit columns.
bool has_explicit_columns = false;

/// Whether it's possible to use the trivial count optimization,
/// i.e. use a fast call of IStorage::totalRows() (or IStorage::totalRowsByPartitionPredicate())
/// instead of actual retrieving columns and counting rows.
bool optimize_trivial_count = false;

/// Cache isRemote() call for storage, because it may be too heavy.
Expand Down
22 changes: 22 additions & 0 deletions tests/integration/test_select_access_rights/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -155,3 +155,25 @@ def test_select_union():

instance.query("REVOKE SELECT ON default.table1 FROM A")
assert "it's necessary to have grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')


def test_select_count():
instance.query("CREATE TABLE table1(x String, y UInt8) ENGINE = MergeTree ORDER BY tuple()")

select_query = "SELECT count() FROM table1"
assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A')

instance.query("GRANT SELECT(x) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == "0\n"

instance.query("REVOKE SELECT(x) ON default.table1 FROM A")
assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A')

instance.query("GRANT SELECT(y) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == "0\n"

instance.query("REVOKE SELECT(y) ON default.table1 FROM A")
assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A')

instance.query("GRANT SELECT ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == "0\n"

0 comments on commit 968cee8

Please sign in to comment.