Merge pull request #63439 from ClickHouse/fix-slow-suggest

Fix slow suggest in case of a large number of columns
This commit is contained in:
Alexey Milovidov 2024-05-07 14:09:52 +00:00 committed by GitHub
commit da365ef84d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 58 additions and 22 deletions

View File

@ -570,11 +570,8 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
if (params.full_access)
return true;
auto access_granted = [&]
auto access_granted = []
{
if (trace_log)
LOG_TRACE(trace_log, "Access granted: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()),
(grant_option ? " WITH GRANT OPTION" : ""));
return true;
};
@ -582,9 +579,6 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
FormatStringHelper<String, FmtArgs...> fmt_string [[maybe_unused]],
FmtArgs && ...fmt_args [[maybe_unused]])
{
if (trace_log)
LOG_TRACE(trace_log, "Access denied: {}{}", (AccessRightsElement{flags, args...}.toStringWithoutOptions()),
(grant_option ? " WITH GRANT OPTION" : ""));
if constexpr (throw_if_denied)
throw Exception(error_code, std::move(fmt_string), getUserName(), std::forward<FmtArgs>(fmt_args)...);
return false;

View File

@ -2498,7 +2498,7 @@ AsyncLoader & Context::getAsyncLoader() const
shared->async_loader = std::make_unique<AsyncLoader>(std::vector<AsyncLoader::PoolInitializer>{
// IMPORTANT: Pool declaration order should match the order in `PoolId.h` to get the indices right.
{ // TablesLoaderForegroundPoolId
"FgLoad",
"ForegroundLoad",
CurrentMetrics::TablesLoaderForegroundThreads,
CurrentMetrics::TablesLoaderForegroundThreadsActive,
CurrentMetrics::TablesLoaderForegroundThreadsScheduled,
@ -2506,7 +2506,7 @@ AsyncLoader & Context::getAsyncLoader() const
TablesLoaderForegroundPriority
},
{ // TablesLoaderBackgroundLoadPoolId
"BgLoad",
"BackgroundLoad",
CurrentMetrics::TablesLoaderBackgroundThreads,
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,
@ -2514,7 +2514,7 @@ AsyncLoader & Context::getAsyncLoader() const
TablesLoaderBackgroundLoadPriority
},
{ // TablesLoaderBackgroundStartupPoolId
"BgStartup",
"BackgrndStartup",
CurrentMetrics::TablesLoaderBackgroundThreads,
CurrentMetrics::TablesLoaderBackgroundThreadsActive,
CurrentMetrics::TablesLoaderBackgroundThreadsScheduled,

View File

@ -88,6 +88,7 @@ public:
, total_tables(tables->size()), access(context->getAccess())
, query_id(context->getCurrentQueryId()), lock_acquire_timeout(context->getSettingsRef().lock_acquire_timeout)
{
need_to_check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS);
}
String getName() const override { return "Columns"; }
@ -101,8 +102,6 @@ protected:
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
size_t rows_count = 0;
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS);
while (rows_count < max_block_size && db_table_num < total_tables)
{
const std::string database_name = (*databases)[db_table_num].get<std::string>();
@ -138,13 +137,17 @@ protected:
column_sizes = storage->getColumnSizes();
}
bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
/// A shortcut: if we don't allow to list this table in SHOW TABLES, also exclude it from system.columns.
if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
bool need_to_check_access_for_columns = need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
size_t position = 0;
for (const auto & column : columns)
{
++position;
if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
if (need_to_check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
continue;
size_t src_index = 0;
@ -296,6 +299,7 @@ private:
size_t db_table_num = 0;
size_t total_tables;
std::shared_ptr<const ContextAccess> access;
bool need_to_check_access_for_tables;
String query_id;
std::chrono::milliseconds lock_acquire_timeout;
};
@ -358,7 +362,6 @@ void StorageSystemColumns::read(
auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names);
auto this_ptr = std::static_pointer_cast<StorageSystemColumns>(shared_from_this());
auto reading = std::make_unique<ReadFromSystemColumns>(
@ -416,9 +419,10 @@ void ReadFromSystemColumns::initializePipeline(QueryPipelineBuilder & pipeline,
/// Add `table` column.
MutableColumnPtr table_column_mut = ColumnString::create();
IColumn::Offsets offsets(database_column->size());
const auto num_databases = database_column->size();
IColumn::Offsets offsets(num_databases);
for (size_t i = 0; i < database_column->size(); ++i)
for (size_t i = 0; i < num_databases; ++i)
{
const std::string database_name = (*database_column)[i].get<std::string>();
if (database_name.empty())

View File

@ -102,7 +102,7 @@ static ColumnPtr getFilteredDatabases(const Databases & databases, const Actions
void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector<UInt8> columns_mask) const
{
const auto access = context->getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES);
const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES);
const auto databases = DatabaseCatalog::instance().getDatabases();
ColumnPtr filtered_databases_column = getFilteredDatabases(databases, predicate, context);
@ -111,7 +111,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c
{
auto database_name = filtered_databases_column->getDataAt(i).toString();
if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name))
if (need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name))
continue;
if (database_name == DatabaseCatalog::TEMPORARY_DATABASE)

View File

@ -224,7 +224,7 @@ protected:
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
const auto access = context->getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
size_t rows_count = 0;
while (rows_count < max_block_size)
@ -348,7 +348,7 @@ protected:
return Chunk(std::move(res_columns), num_rows);
}
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
if (!tables_it || !tables_it->isValid())
tables_it = database->getTablesIterator(context);
@ -361,7 +361,7 @@ protected:
if (!tables.contains(table_name))
continue;
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
StoragePtr table = nullptr;

View File

@ -0,0 +1,2 @@
........................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................
end

View File

@ -0,0 +1,36 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel, no-ordinary-database, long
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# Create many tables in the database
NUM_TABLES=1000
NUM_COLUMNS=1000
THREADS=$(nproc)
COLUMNS=$(seq 1 $NUM_COLUMNS | sed -r -e 's/(.+)/c\1 UInt8, /' | tr -d '\n')
seq 1 $NUM_TABLES | xargs -P "${THREADS}" -I{} bash -c "
echo -n '.'
$CLICKHOUSE_CLIENT --query 'CREATE OR REPLACE TABLE test{} (${COLUMNS} end String) ENGINE = Memory'
"
echo
$CLICKHOUSE_CLIENT --multiquery "
DROP USER IF EXISTS test_03147;
CREATE USER test_03147;
GRANT SELECT (end) ON ${CLICKHOUSE_DATABASE}.test1 TO test_03147;
"
# This query was slow in previous ClickHouse versions for several reasons:
# - tables and databases without SHOW TABLES access were still checked for SHOW COLUMNS access for every column in every table;
# - excessive logging of "access granted" and "access denied"
# The test could succeed even on the previous version, but it will show up as being too slow.
$CLICKHOUSE_CLIENT --user test_03147 --query "SELECT name FROM system.columns WHERE database = currentDatabase()"
$CLICKHOUSE_CLIENT --multiquery "
DROP USER test_03147;
"