Skip to content

Commit af8759d

Browse files
Merge pull request ClickHouse#78339 from ClickHouse/merge-access-validation
Validate access to underlying tables for Merge engine
2 parents 7f057fc + 04c76a2 commit af8759d

11 files changed

+612
-438
lines changed

src/Storages/StorageMerge.cpp

Lines changed: 114 additions & 95 deletions
Large diffs are not rendered by default.

src/Storages/StorageMerge.h

Lines changed: 18 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -87,9 +87,12 @@ class StorageMerge final : public IStorage, WithContext
8787
using DatabaseTablesIterators = std::vector<DatabaseTablesIteratorPtr>;
8888
DatabaseTablesIterators getDatabaseIterators(ContextPtr context) const;
8989

90-
/// Returns a unified column structure among multiple tables.
91-
/// Takes a function that invokes a callback for every table. NOTE: This is quite inconvenient.
92-
static ColumnsDescription unifyColumnsDescription(std::function<void(std::function<void(const StoragePtr &)>)> for_each_table);
90+
static ColumnsDescription getColumnsDescriptionFromSourceTables(
91+
const ContextPtr & query_context,
92+
const String & source_database_name_or_regexp,
93+
bool database_is_regexp,
94+
const String & source_table_regexp,
95+
size_t max_tables_to_look);
9396

9497
private:
9598
/// (Database, Table, Lock, TableName)
@@ -120,14 +123,24 @@ class StorageMerge final : public IStorage, WithContext
120123
DatabaseNameOrRegexp database_name_or_regexp;
121124

122125
template <typename F>
123-
StoragePtr getFirstTable(F && predicate) const;
126+
StoragePtr traverseTablesUntil(F && predicate) const;
124127

125128
template <typename F>
126129
void forEachTable(F && func) const;
127130

131+
template <typename F>
132+
static StoragePtr traverseTablesUntilImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate);
133+
134+
/// Returns a unified column structure among multiple tables.
135+
static ColumnsDescription getColumnsDescriptionFromSourceTablesImpl(
136+
const ContextPtr & context,
137+
const DatabaseNameOrRegexp & database_name_or_regexp,
138+
size_t max_tables_to_look,
139+
const IStorage * ignore_self);
140+
128141
ColumnSizeByName getColumnSizes() const override;
129142

130-
ColumnsDescription getColumnsDescriptionFromSourceTables(size_t max_tables_to_look) const;
143+
ColumnsDescription getColumnsDescriptionFromSourceTables(const ContextPtr & context) const;
131144

132145
static VirtualColumnsDescription createVirtuals();
133146

src/TableFunctions/TableFunctionMerge.cpp

Lines changed: 10 additions & 90 deletions
Original file line numberDiff line numberDiff line change
@@ -57,18 +57,13 @@ class TableFunctionMerge : public ITableFunction
5757
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override;
5858
const char * getStorageTypeName() const override { return "Merge"; }
5959

60-
using TableSet = std::set<String>;
61-
using DBToTableSetMap = std::map<String, TableSet>;
62-
const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const;
6360
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
6461
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
6562
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
66-
static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context);
6763

6864
String source_database_name_or_regexp;
6965
String source_table_regexp;
7066
bool database_is_regexp = false;
71-
mutable std::optional<DBToTableSetMap> source_databases_and_tables;
7267
};
7368

7469
std::vector<size_t> TableFunctionMerge::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const
@@ -129,63 +124,18 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr
129124
}
130125
}
131126

132-
133-
const TableFunctionMerge::DBToTableSetMap & TableFunctionMerge::getSourceDatabasesAndTables(ContextPtr context) const
134-
{
135-
if (source_databases_and_tables)
136-
return *source_databases_and_tables;
137-
138-
source_databases_and_tables.emplace();
139-
140-
/// database_name is not a regexp
141-
if (!database_is_regexp)
142-
{
143-
auto source_tables = getMatchedTablesWithAccess(source_database_name_or_regexp, source_table_regexp, context);
144-
if (source_tables.empty())
145-
throwNoTablesMatchRegexp(source_database_name_or_regexp, source_table_regexp);
146-
(*source_databases_and_tables)[source_database_name_or_regexp] = source_tables;
147-
}
148-
149-
/// database_name is a regexp
150-
else
151-
{
152-
OptimizedRegularExpression database_re(source_database_name_or_regexp);
153-
auto databases = DatabaseCatalog::instance().getDatabases();
154-
155-
for (const auto & db : databases)
156-
if (database_re.match(db.first))
157-
(*source_databases_and_tables)[db.first] = getMatchedTablesWithAccess(db.first, source_table_regexp, context);
158-
159-
if (source_databases_and_tables->empty())
160-
throwNoTablesMatchRegexp(source_database_name_or_regexp, source_table_regexp);
161-
}
162-
163-
return *source_databases_and_tables;
164-
}
165-
166127
ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const
167128
{
168-
size_t table_num = 0;
169-
size_t max_tables_to_look = context->getSettingsRef()[Setting::merge_table_max_tables_to_look_for_schema_inference];
129+
auto res = StorageMerge::getColumnsDescriptionFromSourceTables(
130+
context,
131+
source_database_name_or_regexp,
132+
database_is_regexp,
133+
source_table_regexp,
134+
context->getSettingsRef()[Setting::merge_table_max_tables_to_look_for_schema_inference]);
135+
if (res.empty())
136+
throwNoTablesMatchRegexp(source_database_name_or_regexp, source_table_regexp);
170137

171-
return StorageMerge::unifyColumnsDescription([&table_num, &context, max_tables_to_look, this](std::function<void(const StoragePtr &)> callback)
172-
{
173-
for (const auto & db_with_tables : getSourceDatabasesAndTables(context))
174-
{
175-
for (const auto & table : db_with_tables.second)
176-
{
177-
if (table_num >= max_tables_to_look)
178-
return;
179-
180-
auto storage = DatabaseCatalog::instance().tryGetTable(StorageID{db_with_tables.first, table}, context);
181-
if (storage)
182-
{
183-
++table_num;
184-
callback(storage);
185-
}
186-
}
187-
}
188-
});
138+
return res;
189139
}
190140

191141

@@ -197,43 +147,13 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, Cont
197147
String{},
198148
source_database_name_or_regexp,
199149
database_is_regexp,
200-
getSourceDatabasesAndTables(context),
150+
source_table_regexp,
201151
context);
202152

203153
res->startup();
204154
return res;
205155
}
206156

207-
TableFunctionMerge::TableSet
208-
TableFunctionMerge::getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context)
209-
{
210-
OptimizedRegularExpression table_re(table_regexp);
211-
212-
auto table_name_match = [&](const String & table_name) { return table_re.match(table_name); };
213-
214-
auto access = context->getAccess();
215-
216-
auto database = DatabaseCatalog::instance().getDatabase(database_name);
217-
218-
bool granted_show_on_all_tables = access->isGranted(AccessType::SHOW_TABLES, database_name);
219-
bool granted_select_on_all_tables = access->isGranted(AccessType::SELECT, database_name);
220-
221-
TableSet tables;
222-
223-
for (auto it = database->getTablesIterator(context, table_name_match); it->isValid(); it->next())
224-
{
225-
if (!it->table())
226-
continue;
227-
bool granted_show = granted_show_on_all_tables || access->isGranted(AccessType::SHOW_TABLES, database_name, it->name());
228-
if (!granted_show)
229-
continue;
230-
if (!granted_select_on_all_tables)
231-
access->checkAccess(AccessType::SELECT, database_name, it->name());
232-
tables.emplace(it->name());
233-
}
234-
return tables;
235-
}
236-
237157
}
238158

239159
void registerTableFunctionMerge(TableFunctionFactory & factory)

tests/integration/test_table_functions_access_rights/test.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ def test_merge():
4646
)
4747

4848
instance.query("GRANT CREATE TEMPORARY TABLE ON *.* TO A")
49-
assert "no tables in the database matches" in instance.query_and_get_error(
49+
assert "no tables satisfied provided regexp" in instance.query_and_get_error(
5050
select_query, user="A"
5151
)
5252

@@ -63,7 +63,7 @@ def test_merge():
6363
instance.query("GRANT SELECT ON default.table1 TO A")
6464
instance.query("GRANT INSERT ON default.table2 TO A")
6565
assert (
66-
"it's necessary to have the grant SELECT ON default.table2"
66+
"it's necessary to have the grant SELECT(x) ON default.table2"
6767
in instance.query_and_get_error(select_query, user="A")
6868
)
6969

tests/queries/0_stateless/01902_table_function_merge_db_params.sql

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,13 @@
1+
-- Tags: no-parallel
2+
13
DROP DATABASE IF EXISTS 01902_db_params;
24
CREATE DATABASE 01902_db_params;
35
CREATE TABLE 01902_db_params.t(n Int8) ENGINE=MergeTree ORDER BY n;
46
INSERT INTO 01902_db_params.t SELECT * FROM numbers(3);
57
SELECT _database, _table, n FROM merge(REGEXP('^01902_db_params'), '^t') ORDER BY _database, _table, n;
68

79
SELECT _database, _table, n FROM merge() ORDER BY _database, _table, n; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
8-
SELECT _database, _table, n FROM merge('^t') ORDER BY _database, _table, n; -- {serverError BAD_ARGUMENTS}
10+
SELECT _database, _table, n FROM merge('^t') ORDER BY _database, _table, n; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
911

1012
USE 01902_db_params;
1113
SELECT _database, _table, n FROM merge('^t') ORDER BY _database, _table, n;

0 commit comments

Comments
 (0)