Skip to content

Commit 04c76a2

Browse files
fix issues, add tests
1 parent 6e486b0 commit 04c76a2

File tree

4 files changed

+97
-23
lines changed

4 files changed

+97
-23
lines changed

src/Storages/StorageMerge.cpp

Lines changed: 19 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -221,7 +221,7 @@ ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTablesImpl(
221221
size_t table_num = 0;
222222
ColumnsDescription res;
223223

224-
getFirstTableImpl(query_context, ignore_self, database_name_or_regexp, [&table_num, &access, &res, max_tables_to_look](auto && t)
224+
traverseTablesUntilImpl(query_context, ignore_self, database_name_or_regexp, [&table_num, &access, &res, max_tables_to_look](auto && t)
225225
{
226226
if (!t)
227227
return false;
@@ -257,13 +257,13 @@ ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTablesImpl(
257257
}
258258

259259
template <typename F>
260-
StoragePtr StorageMerge::getFirstTable(F && predicate) const
260+
StoragePtr StorageMerge::traverseTablesUntil(F && predicate) const
261261
{
262-
return getFirstTableImpl(context.lock(), this, database_name_or_regexp, std::forward<F>(predicate));
262+
return traverseTablesUntilImpl(getContext(), this, database_name_or_regexp, std::forward<F>(predicate));
263263
}
264264

265265
template <typename F>
266-
StoragePtr StorageMerge::getFirstTableImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate)
266+
StoragePtr StorageMerge::traverseTablesUntilImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate)
267267
{
268268
auto database_table_iterators = database_name_or_regexp.getDatabaseIterators(query_context);
269269

@@ -285,7 +285,7 @@ StoragePtr StorageMerge::getFirstTableImpl(const ContextPtr & query_context, con
285285
template <typename F>
286286
void StorageMerge::forEachTable(F && func) const
287287
{
288-
getFirstTable([&func](const auto & table)
288+
traverseTablesUntil([&func](const auto & table)
289289
{
290290
func(table);
291291
/// Always continue to the next table.
@@ -295,13 +295,13 @@ void StorageMerge::forEachTable(F && func) const
295295

296296
bool StorageMerge::isRemote() const
297297
{
298-
auto first_remote_table = getFirstTable([](const StoragePtr & table) { return table && table->isRemote(); });
298+
auto first_remote_table = traverseTablesUntil([](const StoragePtr & table) { return table && table->isRemote(); });
299299
return first_remote_table != nullptr;
300300
}
301301

302302
bool StorageMerge::supportsPrewhere() const
303303
{
304-
return getFirstTable([](const auto & table) { return !table->supportsPrewhere(); }) == nullptr;
304+
return traverseTablesUntil([](const auto & table) { return !table->supportsPrewhere(); }) == nullptr;
305305
}
306306

307307
bool StorageMerge::canMoveConditionsToPrewhere() const
@@ -316,7 +316,7 @@ bool StorageMerge::canMoveConditionsToPrewhere() const
316316
/// NOTE: Type can be different, and in this case, PREWHERE cannot be
317317
/// applied for those columns, but there a separate method to return
318318
/// supported columns for PREWHERE - supportedPrewhereColumns().
319-
return getFirstTable([](const auto & table) { return !table->canMoveConditionsToPrewhere(); }) == nullptr;
319+
return traverseTablesUntil([](const auto & table) { return !table->canMoveConditionsToPrewhere(); }) == nullptr;
320320
}
321321

322322
std::optional<NameSet> StorageMerge::supportedPrewhereColumns() const
@@ -421,7 +421,7 @@ StorageSnapshotPtr StorageMerge::getStorageSnapshot(const StorageMetadataPtr & m
421421
const auto & access = query_context->getAccess();
422422

423423
auto virtuals = common_virtuals;
424-
if (auto first_table = getFirstTable([access](auto && table)
424+
if (auto first_table = traverseTablesUntil([access](auto && table)
425425
{
426426
if (!table)
427427
return false;
@@ -1325,12 +1325,13 @@ StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables(
13251325
ColumnWithTypeAndName(std::make_shared<DataTypeString>(), "_database"),
13261326
ColumnWithTypeAndName(std::make_shared<DataTypeString>(), "_table")
13271327
};
1328+
// Extract predicate part, that could be evaluated only with _database and _table columns
13281329
auto table_filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &sample_block);
13291330
if (table_filter_dag)
13301331
{
13311332
auto filter_expression = VirtualColumnUtils::buildFilterExpression(std::move(*table_filter_dag), query_context);
13321333
auto filter_column_name = filter_expression->getActionsDAG().getOutputs().at(0)->result_name;
1333-
table_filter = [filter_expression, filter_column_name] (const auto& database_name, const auto& table_name)
1334+
table_filter = [filter=std::move(filter_expression), column_name=std::move(filter_column_name)] (const auto& database_name, const auto& table_name)
13341335
{
13351336
MutableColumnPtr database_column = ColumnString::create();
13361337
MutableColumnPtr table_column = ColumnString::create();
@@ -1340,18 +1341,18 @@ StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables(
13401341
ColumnWithTypeAndName(std::move(database_column), std::make_shared<DataTypeString>(), "_database"),
13411342
ColumnWithTypeAndName(std::move(table_column), std::make_shared<DataTypeString>(), "_table")
13421343
};
1343-
filter_expression->execute(block);
1344+
filter->execute(block);
13441345
// Valid only when block has exactly one row.
1345-
return block.getByName(filter_column_name).column->getBool(0);
1346+
return block.getByName(column_name).column->getBool(0);
13461347
};
13471348
}
13481349
}
13491350

13501351
auto access = query_context->getAccess();
13511352
for (const auto & iterator : database_table_iterators)
13521353
{
1353-
auto all_show = access->isGranted(AccessType::SHOW_TABLES, iterator->databaseName());
1354-
auto all_select = access->isGranted(AccessType::SELECT, iterator->databaseName());
1354+
auto granted_show_on_all_tables = access->isGranted(AccessType::SHOW_TABLES, iterator->databaseName());
1355+
auto granted_select_on_all_tables = access->isGranted(AccessType::SELECT, iterator->databaseName());
13551356
while (iterator->isValid())
13561357
{
13571358
StoragePtr storage = iterator->table();
@@ -1360,10 +1361,10 @@ StorageMerge::StorageListWithLocks ReadFromMerge::getSelectedTables(
13601361

13611362
if (storage.get() != storage_merge.get())
13621363
if (!table_filter || table_filter(iterator->databaseName(), iterator->name()))
1363-
if (all_show || access->isGranted(AccessType::SHOW_TABLES, iterator->databaseName(), iterator->name()))
1364+
if (granted_show_on_all_tables || access->isGranted(AccessType::SHOW_TABLES, iterator->databaseName(), iterator->name()))
13641365
{
13651366

1366-
if (!all_select)
1367+
if (!granted_select_on_all_tables)
13671368
access->checkAccess(AccessType::SELECT, iterator->databaseName(), iterator->name(), column_names);
13681369

13691370
auto table_lock = storage->lockForShare(query_context->getCurrentQueryId(), settings[Setting::lock_acquire_timeout]);
@@ -1671,7 +1672,7 @@ bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &,
16711672
{
16721673
/// Here we actually need storage snapshot of all nested tables.
16731674
/// But to avoid complexity pass nullptr to make more lightweight check in MergeTreeData.
1674-
return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr;
1675+
return traverseTablesUntil([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr;
16751676
}
16761677

16771678
std::optional<UInt64> StorageMerge::totalRows(ContextPtr query_context) const
@@ -1688,7 +1689,7 @@ template <typename F>
16881689
std::optional<UInt64> StorageMerge::totalRowsOrBytes(F && func) const
16891690
{
16901691
UInt64 total_rows_or_bytes = 0;
1691-
auto first_table = getFirstTable([&](const auto & table)
1692+
auto first_table = traverseTablesUntil([&](const auto & table)
16921693
{
16931694
if (auto rows_or_bytes = func(table))
16941695
{

src/Storages/StorageMerge.h

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -123,13 +123,13 @@ class StorageMerge final : public IStorage, WithContext
123123
DatabaseNameOrRegexp database_name_or_regexp;
124124

125125
template <typename F>
126-
StoragePtr getFirstTable(F && predicate) const;
126+
StoragePtr traverseTablesUntil(F && predicate) const;
127127

128128
template <typename F>
129129
void forEachTable(F && func) const;
130130

131131
template <typename F>
132-
static StoragePtr getFirstTableImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate);
132+
static StoragePtr traverseTablesUntilImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate);
133133

134134
/// Returns a unified column structure among multiple tables.
135135
static ColumnsDescription getColumnsDescriptionFromSourceTablesImpl(

tests/queries/0_stateless/03400_merge_storage_underlying_tables_access_validation.reference

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ no_show_allowed 36 6 6
3838
no_show_allowed 37 7 7
3939
no_show_allowed 38 8 8
4040
no_show_allowed 39 9 9
41+
----Table engine
4142
----select allowed columns and databases
4243
allowed 0
4344
allowed 1
@@ -70,6 +71,17 @@ allowed 6 6
7071
allowed 7 7
7172
allowed 8 8
7273
allowed 9 9
74+
----
75+
allowed 0
76+
allowed 1
77+
allowed 2
78+
allowed 3
79+
allowed 4
80+
allowed 5
81+
allowed 6
82+
allowed 7
83+
allowed 8
84+
allowed 9
7385
----create without show all
7486
a
7587
b
@@ -84,3 +96,47 @@ allowed 6 6
8496
allowed 7 7
8597
allowed 8 8
8698
allowed 9 9
99+
----Table function
100+
----select allowed columns and databases
101+
allowed 0
102+
allowed 1
103+
allowed 2
104+
allowed 3
105+
allowed 4
106+
allowed 5
107+
allowed 6
108+
allowed 7
109+
allowed 8
110+
allowed 9
111+
partial_allowed 10
112+
partial_allowed 11
113+
partial_allowed 12
114+
partial_allowed 13
115+
partial_allowed 14
116+
partial_allowed 15
117+
partial_allowed 16
118+
partial_allowed 17
119+
partial_allowed 18
120+
partial_allowed 19
121+
----
122+
allowed 0 0
123+
allowed 1 1
124+
allowed 2 2
125+
allowed 3 3
126+
allowed 4 4
127+
allowed 5 5
128+
allowed 6 6
129+
allowed 7 7
130+
allowed 8 8
131+
allowed 9 9
132+
----
133+
allowed 0
134+
allowed 1
135+
allowed 2
136+
allowed 3
137+
allowed 4
138+
allowed 5
139+
allowed 6
140+
allowed 7
141+
allowed 8
142+
allowed 9

tests/queries/0_stateless/03400_merge_storage_underlying_tables_access_validation.sh

Lines changed: 20 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ INSERT INTO database_03400.not_allowed SELECT number + 20, number FROM numbers(1
2020
INSERT INTO database_03400.no_show_allowed SELECT number + 30, number, number FROM numbers(10);
2121
2222
CREATE TABLE database_03400.merge Engine=Merge(database_03400, '.*allowed');
23-
SELECT _table, * FROM database_03400.merge ORDER BY a SETTINGS enable_analyzer=1;
23+
SELECT _table, * FROM database_03400.merge ORDER BY a SETTINGS enable_analyzer = 1;
2424
2525
DROP USER IF EXISTS user_test_03400;
2626
CREATE USER user_test_03400 IDENTIFIED WITH plaintext_password BY 'user_test_03400';
@@ -34,21 +34,38 @@ GRANT SELECT(a) ON database_03400.partial_allowed TO 'user_test_03400';
3434
GRANT SELECT ON database_03400.merge* TO 'user_test_03400';
3535
"""
3636

37+
echo "----Table engine"
38+
# access from the Merge table
3739
$CLICKHOUSE_CLIENT --multiline --user user_test_03400 --password user_test_03400 -q """
38-
SELECT * FROM database_03400.merge SETTINGS enable_analyzer=1; -- { serverError ACCESS_DENIED }
40+
SELECT * FROM database_03400.merge; -- { serverError ACCESS_DENIED }
3941
SELECT a FROM database_03400.merge; -- { serverError ACCESS_DENIED }
4042
SELECT '----select allowed columns and databases';
4143
SELECT _table, a FROM database_03400.merge WHERE _database='database_03400' AND _table IN ('allowed', 'partial_allowed') ORDER BY a;
4244
SELECT '----';
4345
SELECT _table, a, b FROM database_03400.merge WHERE _database='database_03400' AND _table = 'allowed' ORDER BY a;
46+
SELECT '----';
47+
SELECT _table, a FROM database_03400.merge WHERE _database='database_03400' AND _table IN ('allowed', 'no_show_allowed') ORDER BY a;
4448
SELECT '----create without show all';
4549
CREATE TABLE database_03400.merge_user Engine=Merge(database_03400, '.*allowed');
46-
SELECT name FROM system.columns WHERE database='database_03400' AND table='merge_user' ORDER BY name ;
50+
SELECT name FROM system.columns WHERE database='database_03400' AND table='merge_user' ORDER BY name;
4751
SELECT '----select user created table';
4852
SELECT _table, * FROM database_03400.merge_user WHERE _table = 'allowed' ORDER BY a;
4953
CREATE TABLE database_03400.merge_user_fail Engine=Merge(database_03400, 'no_show_allowed'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE }
5054
"""
5155

56+
echo "----Table function"
57+
# access from the Merge table function
58+
$CLICKHOUSE_CLIENT --multiline --user user_test_03400 --password user_test_03400 -q """
59+
SELECT * FROM merge(database_03400, '.*allowed'); -- { serverError ACCESS_DENIED }
60+
SELECT a FROM merge(database_03400, '.*allowed'); -- { serverError ACCESS_DENIED }
61+
SELECT '----select allowed columns and databases';
62+
SELECT _table, a FROM merge(database_03400, '.*allowed') WHERE _table IN ('allowed', 'partial_allowed') ORDER BY a;
63+
SELECT '----';
64+
SELECT _table, a, b FROM merge(database_03400, '.*allowed') WHERE _database='database_03400' AND _table = 'allowed' ORDER BY a;
65+
SELECT '----';
66+
SELECT _table, a FROM merge(database_03400, '.*allowed') WHERE _table IN ('allowed', 'no_show_allowed') ORDER BY a;
67+
"""
68+
5269
${CLICKHOUSE_CLIENT} --multiline -q """
5370
DROP DATABASE IF EXISTS database_03400;
5471
DROP USER IF EXISTS user_test_03400;

0 commit comments

Comments
 (0)