Merge pull request #18232 from vitlibar/fix-access-rights-for-select-join

Fix access rights for SELECT JOIN.
This commit is contained in:
Vitaly Baranov 2020-12-30 14:42:16 +03:00 committed by GitHub
commit 460006b502
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 184 additions and 16 deletions

View File

@ -211,6 +211,18 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query);
}
/// Returns true if we should ignore quotas and limits for a specified table in the system database.
static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id)
{
if (table_id.database_name == DatabaseCatalog::SYSTEM_DATABASE)
{
static const boost::container::flat_set<String> tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"};
if (tables_ignoring_quota.count(table_id.table_name))
return true;
}
return false;
}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
@ -255,14 +267,18 @@ InterpreterSelectQuery::InterpreterSelectQuery(
JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery());
bool got_storage_from_query = false;
if (!has_input && !storage)
{
storage = joined_tables.getLeftTableStorage();
got_storage_from_query = true;
}
if (storage)
{
table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
table_id = storage->getStorageID();
if (metadata_snapshot == nullptr)
if (!metadata_snapshot)
metadata_snapshot = storage->getInMemoryMetadataPtr();
}
@ -280,9 +296,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (storage && joined_tables.isLeftTableSubquery())
{
/// Rewritten with subquery. Free storage locks here.
storage = {};
storage = nullptr;
table_lock.reset();
table_id = StorageID::createEmpty();
metadata_snapshot = nullptr;
}
}
@ -445,16 +462,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (query.prewhere() && !query.where())
analysis_result.prewhere_info->need_filter = true;
const StorageID & left_table_id = joined_tables.leftTableID();
if (left_table_id)
context->checkAccess(AccessType::SELECT, left_table_id, required_columns);
/// Remove limits for some tables in the `system` database.
if (left_table_id.database_name == "system")
if (table_id && got_storage_from_query && !joined_tables.isLeftTableFunction())
{
static const boost::container::flat_set<String> system_tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"};
if (system_tables_ignoring_quota.count(left_table_id.table_name))
/// 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);
/// Remove limits for some tables in the `system` database.
if (shouldIgnoreQuotaAndLimits(table_id) && (joined_tables.tablesCount() <= 1))
{
options.ignore_quota = true;
options.ignore_limits = true;

View File

@ -161,6 +161,7 @@ StoragePtr JoinedTables::getLeftTableStorage()
if (isLeftTableFunction())
return context.getQueryContext().executeTableFunction(left_table_expression);
StorageID table_id = StorageID::createEmpty();
if (left_db_and_table)
{
table_id = context.resolveStorageID(StorageID(left_db_and_table->database, left_db_and_table->table, left_db_and_table->uuid));

View File

@ -43,8 +43,6 @@ public:
bool isLeftTableFunction() const;
size_t tablesCount() const { return table_expressions.size(); }
const StorageID & leftTableID() const { return table_id; }
void rewriteDistributedInAndJoins(ASTPtr & query);
std::unique_ptr<InterpreterSelectWithUnionQuery> makeLeftTableSubquery(const SelectQueryOptions & select_options);
@ -57,9 +55,6 @@ private:
/// Legacy (duplicated left table values)
ASTPtr left_table_expression;
std::optional<DatabaseAndTableWithAlias> left_db_and_table;
/// left_db_and_table or 'system.one'
StorageID table_id = StorageID::createEmpty();
};
}

View File

@ -0,0 +1,157 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance')
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def cleanup_after_test():
instance.query("CREATE USER OR REPLACE A")
yield
instance.query("DROP TABLE IF EXISTS table1")
instance.query("DROP TABLE IF EXISTS table2")
def test_select_single_column():
instance.query("CREATE TABLE table1(d DATE, a String, b UInt8) ENGINE = MergeTree ORDER BY d")
select_query = "SELECT a FROM table1"
assert "it's necessary to have grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(a) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
instance.query("REVOKE SELECT(a) ON default.table1 FROM A")
assert "it's necessary to have grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
def test_select_single_column_with_table_grant():
instance.query("CREATE TABLE table1(d DATE, a String, b UInt8) ENGINE = MergeTree ORDER BY d")
select_query = "SELECT a FROM table1"
assert "it's necessary to have grant SELECT(a) 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') == ""
instance.query("REVOKE SELECT(a) ON default.table1 FROM A")
assert "it's necessary to have grant SELECT(a) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
def test_select_all_columns():
instance.query("CREATE TABLE table1(d DATE, a String, b UInt8) ENGINE = MergeTree ORDER BY d")
select_query = "SELECT * FROM table1"
assert "it's necessary to have grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(d) ON default.table1 TO A")
assert "it's necessary to have grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(a) ON default.table1 TO A")
assert "it's necessary to have grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(b) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
def test_select_all_columns_with_table_grant():
instance.query("CREATE TABLE table1(d DATE, a String, b UInt8) ENGINE = MergeTree ORDER BY d")
select_query = "SELECT * FROM table1"
assert "it's necessary to have grant SELECT(d, a, b) 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') == ""
def test_alias():
instance.query("CREATE TABLE table1(x Int32, y Int32) ENGINE = MergeTree ORDER BY tuple()")
select_query = "SELECT x, y, x + y AS s FROM table1"
assert "it's necessary to have grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(x, y) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
def test_alias_columns():
instance.query("CREATE TABLE table1(x Int32, y Int32, s Int32 ALIAS x + y) ENGINE = MergeTree ORDER BY tuple()")
select_query = "SELECT * FROM table1"
assert "it's necessary to have grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(x,y) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
select_query = "SELECT s FROM table1"
assert "it's necessary to have grant SELECT(s) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(s) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
instance.query("REVOKE SELECT(x,y) ON default.table1 FROM A")
assert instance.query(select_query, user = 'A') == ""
def test_materialized_columns():
instance.query("CREATE TABLE table1(x Int32, y Int32, p Int32 MATERIALIZED x * y) ENGINE = MergeTree ORDER BY tuple()")
select_query = "SELECT * FROM table1"
assert "it's necessary to have grant SELECT(x, y) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(x,y) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
select_query = "SELECT p FROM table1"
assert "it's necessary to have grant SELECT(p) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(p) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
instance.query("REVOKE SELECT(x,y) ON default.table1 FROM A")
assert instance.query(select_query, user = 'A') == ""
def test_select_join():
instance.query("CREATE TABLE table1(d DATE, a String, b UInt8) ENGINE = MergeTree ORDER BY d")
instance.query("CREATE TABLE table2(d DATE, x UInt32, y UInt8) ENGINE = MergeTree ORDER BY d")
select_query = "SELECT * FROM table1 JOIN table2 USING(d)"
assert "it's necessary to have grant SELECT(d, x, y) ON default.table2" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(d, x, y) ON default.table2 TO A")
assert "it's necessary to have grant SELECT(d, a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(d, a, b) ON default.table1 TO A")
assert instance.query(select_query, user = 'A') == ""
instance.query("REVOKE SELECT ON default.table2 FROM A")
assert "it's necessary to have grant SELECT(d, x, y) ON default.table2" in instance.query_and_get_error(select_query, user = 'A')
def test_select_union():
instance.query("CREATE TABLE table1(a String, b UInt8) ENGINE = MergeTree ORDER BY tuple()")
instance.query("CREATE TABLE table2(a String, b UInt8) ENGINE = MergeTree ORDER BY tuple()")
select_query = "SELECT * FROM table1 UNION ALL SELECT * FROM table2"
assert "it's necessary to have grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(a, b) ON default.table1 TO A")
assert "it's necessary to have grant SELECT(a, b) ON default.table2" in instance.query_and_get_error(select_query, user = 'A')
instance.query("GRANT SELECT(a, b) ON default.table2 TO A")
assert instance.query(select_query, user = 'A') == ""
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')