Skip to content

Commit da365ef

Browse files
Merge pull request ClickHouse#63439 from ClickHouse/fix-slow-suggest
Fix slow suggest in case of a large number of columns
2 parents b21d0e7 + 54418fc commit da365ef

File tree

7 files changed

+58
-22
lines changed

7 files changed

+58
-22
lines changed

src/Access/ContextAccess.cpp

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -570,21 +570,15 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
570570
if (params.full_access)
571571
return true;
572572

573-
auto access_granted = [&]
573+
auto access_granted = []
574574
{
575-
if (trace_log)
576-
LOG_TRACE(trace_log, "Access granted: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()),
577-
(grant_option ? " WITH GRANT OPTION" : ""));
578575
return true;
579576
};
580577

581578
auto access_denied = [&]<typename... FmtArgs>(int error_code [[maybe_unused]],
582579
FormatStringHelper<String, FmtArgs...> fmt_string [[maybe_unused]],
583580
FmtArgs && ...fmt_args [[maybe_unused]])
584581
{
585-
if (trace_log)
586-
LOG_TRACE(trace_log, "Access denied: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()),
587-
(grant_option ? " WITH GRANT OPTION" : ""));
588582
if constexpr (throw_if_denied)
589583
throw Exception(error_code, std::move(fmt_string), getUserName(), std::forward<FmtArgs>(fmt_args)...);
590584
return false;

src/Interpreters/Context.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2498,23 +2498,23 @@ AsyncLoader & Context::getAsyncLoader() const
24982498
shared->async_loader = std::make_unique<AsyncLoader>(std::vector<AsyncLoader::PoolInitializer>{
24992499
// IMPORTANT: Pool declaration order should match the order in `PoolId.h` to get the indices right.
25002500
{ // TablesLoaderForegroundPoolId
2501-
"FgLoad",
2501+
"ForegroundLoad",
25022502
CurrentMetrics::TablesLoaderForegroundThreads,
25032503
CurrentMetrics::TablesLoaderForegroundThreadsActive,
25042504
CurrentMetrics::TablesLoaderForegroundThreadsScheduled,
25052505
shared->server_settings.tables_loader_foreground_pool_size,
25062506
TablesLoaderForegroundPriority
25072507
},
25082508
{ // TablesLoaderBackgroundLoadPoolId
2509-
"BgLoad",
2509+
"BackgroundLoad",
25102510
CurrentMetrics::TablesLoaderBackgroundThreads,
25112511
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
25122512
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,
25132513
shared->server_settings.tables_loader_background_pool_size,
25142514
TablesLoaderBackgroundLoadPriority
25152515
},
25162516
{ // TablesLoaderBackgroundStartupPoolId
2517-
"BgStartup",
2517+
"BackgrndStartup",
25182518
CurrentMetrics::TablesLoaderBackgroundThreads,
25192519
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
25202520
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,

src/Storages/System/StorageSystemColumns.cpp

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,7 @@ class ColumnsSource : public ISource
8888
, total_tables(tables->size()), access(context->getAccess())
8989
, query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout)
9090
{
91+
need_to_check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS);
9192
}
9293

9394
String getName() const override { return "Columns"; }
@@ -101,8 +102,6 @@ class ColumnsSource : public ISource
101102
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
102103
size_t rows_count = 0;
103104

104-
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS);
105-
106105
while (rows_count < max_block_size && db_table_num < total_tables)
107106
{
108107
const std::string database_name = (*databases)[db_table_num].get<std::string>();
@@ -138,13 +137,17 @@ class ColumnsSource : public ISource
138137
column_sizes = storage->getColumnSizes();
139138
}
140139

141-
bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
140+
/// A shortcut: if we don't allow to list this table in SHOW TABLES, also exclude it from system.columns.
141+
if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
142+
continue;
143+
144+
bool need_to_check_access_for_columns = need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
142145

143146
size_t position = 0;
144147
for (const auto & column : columns)
145148
{
146149
++position;
147-
if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
150+
if (need_to_check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
148151
continue;
149152

150153
size_t src_index = 0;
@@ -296,6 +299,7 @@ class ColumnsSource : public ISource
296299
size_t db_table_num = 0;
297300
size_t total_tables;
298301
std::shared_ptr<const ContextAccess> access;
302+
bool need_to_check_access_for_tables;
299303
String query_id;
300304
std::chrono::milliseconds lock_acquire_timeout;
301305
};
@@ -358,7 +362,6 @@ void StorageSystemColumns::read(
358362

359363
auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names);
360364

361-
362365
auto this_ptr = std::static_pointer_cast<StorageSystemColumns>(shared_from_this());
363366

364367
auto reading = std::make_unique<ReadFromSystemColumns>(
@@ -416,9 +419,10 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline,
416419

417420
/// Add `table` column.
418421
MutableColumnPtr table_column_mut = ColumnString::create();
419-
IColumn::Offsets offsets(database_column->size());
422+
const auto num_databases = database_column->size();
423+
IColumn::Offsets offsets(num_databases);
420424

421-
for (size_t i = 0; i < database_column->size(); ++i)
425+
for (size_t i = 0; i < num_databases; ++i)
422426
{
423427
const std::string database_name = (*database_column)[i].get<std::string>();
424428
if (database_name.empty())

src/Storages/System/StorageSystemDatabases.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -102,7 +102,7 @@ static ColumnPtr getFilteredDatabases(const Databases & databases, const Actions
102102
void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector<UInt8> columns_mask) const
103103
{
104104
const auto access = context->getAccess();
105-
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES);
105+
const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES);
106106

107107
const auto databases = DatabaseCatalog::instance().getDatabases();
108108
ColumnPtr filtered_databases_column = getFilteredDatabases(databases, predicate, context);
@@ -111,7 +111,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c
111111
{
112112
auto database_name = filtered_databases_column->getDataAt(i).toString();
113113

114-
if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name))
114+
if (need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name))
115115
continue;
116116

117117
if (database_name == DatabaseCatalog::TEMPORARY_DATABASE)

src/Storages/System/StorageSystemTables.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -224,7 +224,7 @@ class TablesBlockSource : public ISource
224224
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
225225

226226
const auto access = context->getAccess();
227-
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
227+
const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
228228

229229
size_t rows_count = 0;
230230
while (rows_count < max_block_size)
@@ -348,7 +348,7 @@ class TablesBlockSource : public ISource
348348
return Chunk(std::move(res_columns), num_rows);
349349
}
350350

351-
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
351+
const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
352352

353353
if (!tables_it || !tables_it->isValid())
354354
tables_it = database->getTablesIterator(context);
@@ -361,7 +361,7 @@ class TablesBlockSource : public ISource
361361
if (!tables.contains(table_name))
362362
continue;
363363

364-
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
364+
if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
365365
continue;
366366

367367
StoragePtr table = nullptr;
Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,2 @@
1+
........................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................
2+
end
Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
#!/usr/bin/env bash
2+
# Tags: no-fasttest, no-parallel, no-ordinary-database, long
3+
4+
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
5+
# shellcheck source=../shell_config.sh
6+
. "$CUR_DIR"/../shell_config.sh
7+
8+
# Create many tables in the database
9+
NUM_TABLES=1000
10+
NUM_COLUMNS=1000
11+
THREADS=$(nproc)
12+
13+
COLUMNS=$(seq 1 $NUM_COLUMNS | sed -r -e 's/(.+)/c\1 UInt8, /' | tr -d '\n')
14+
15+
seq 1 $NUM_TABLES | xargs -P "${THREADS}" -I{} bash -c "
16+
echo -n '.'
17+
$CLICKHOUSE_CLIENT --query 'CREATE OR REPLACE TABLE test{} (${COLUMNS} end String) ENGINE = Memory'
18+
"
19+
echo
20+
21+
$CLICKHOUSE_CLIENT --multiquery "
22+
DROP USER IF EXISTS test_03147;
23+
CREATE USER test_03147;
24+
GRANT SELECT (end) ON ${CLICKHOUSE_DATABASE}.test1 TO test_03147;
25+
"
26+
27+
# This query was slow in previous ClickHouse versions for several reasons:
28+
# - tables and databases without SHOW TABLES access were still checked for SHOW COLUMNS access for every column in every table;
29+
# - excessive logging of "access granted" and "access denied"
30+
31+
# The test could succeed even on the previous version, but it will show up as being too slow.
32+
$CLICKHOUSE_CLIENT --user test_03147 --query "SELECT name FROM system.columns WHERE database = currentDatabase()"
33+
34+
$CLICKHOUSE_CLIENT --multiquery "
35+
DROP USER test_03147;
36+
"

0 commit comments

Comments
 (0)