mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
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:
commit
da365ef84d
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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())
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -0,0 +1,2 @@
|
||||
........................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................
|
||||
end
|
36
tests/queries/0_stateless/03147_system_columns_access_checks.sh
Executable file
36
tests/queries/0_stateless/03147_system_columns_access_checks.sh
Executable 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;
|
||||
"
|
Loading…
Reference in New Issue
Block a user