mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
SELECT count() FROM table now requires at least one column to be accessible in the table.
This commit is contained in:
parent
460006b502
commit
235ea220ec
@ -13,6 +13,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
||||
#include <Access/AccessFlags.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
|
||||
@ -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.
|
||||
@ -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)
|
||||
{
|
||||
@ -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))
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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"
|
||||
|
Loading…
Reference in New Issue
Block a user