Merge pull request #53343 from ClickHouse/analyzer-fix-quota-tests

Analyzer: fix quotas for system tables
This commit is contained in:
Dmitry Novik 2023-08-15 14:12:12 +02:00 committed by GitHub
commit 3a8d21ddfb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 42 additions and 43 deletions

View File

@ -184,7 +184,7 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(
, context(buildContext(context_, select_query_options_))
, select_query_options(select_query_options_)
, query_tree(query_tree_)
, planner(query_tree_, select_query_options_)
, planner(query_tree_, select_query_options)
{
}

View File

@ -45,6 +45,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/InterpreterTransactionControlQuery.h>
#include <Interpreters/NormalizeSelectWithUnionQueryVisitor.h>
@ -1033,6 +1034,11 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
// InterpreterSelectQueryAnalyzer does not build QueryPlan in the constructor.
// We need to force to build it here to check if we need to ignore quota.
if (auto * interpreter_with_analyzer = dynamic_cast<InterpreterSelectQueryAnalyzer *>(interpreter.get()))
interpreter_with_analyzer->getQueryPlan();
if (!interpreter->ignoreQuota() && !quota_checked)
{
quota = context->getQuota();

View File

@ -1047,7 +1047,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node,
}
Planner::Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_)
SelectQueryOptions & select_query_options_)
: query_tree(query_tree_)
, select_query_options(select_query_options_)
, planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared<GlobalPlannerContext>()))
@ -1055,7 +1055,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_,
}
Planner::Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_,
SelectQueryOptions & select_query_options_,
GlobalPlannerContextPtr global_planner_context_)
: query_tree(query_tree_)
, select_query_options(select_query_options_)
@ -1064,7 +1064,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_,
}
Planner::Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_,
SelectQueryOptions & select_query_options_,
PlannerContextPtr planner_context_)
: query_tree(query_tree_)
, select_query_options(select_query_options_)

View File

@ -22,16 +22,16 @@ class Planner
public:
/// Initialize planner with query tree after analysis phase
Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_);
SelectQueryOptions & select_query_options_);
/// Initialize planner with query tree after query analysis phase and global planner context
Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_,
SelectQueryOptions & select_query_options_,
GlobalPlannerContextPtr global_planner_context_);
/// Initialize planner with query tree after query analysis phase and planner context
Planner(const QueryTreeNodePtr & query_tree_,
const SelectQueryOptions & select_query_options_,
SelectQueryOptions & select_query_options_,
PlannerContextPtr planner_context_);
const QueryPlan & getQueryPlan() const
@ -66,7 +66,7 @@ private:
void buildPlanForQueryNode();
QueryTreeNodePtr query_tree;
SelectQueryOptions select_query_options;
SelectQueryOptions & select_query_options;
PlannerContextPtr planner_context;
QueryPlan query_plan;
StorageLimitsList storage_limits;

View File

@ -113,6 +113,20 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names,
query_context->checkAccess(AccessType::SELECT, storage_id, column_names);
}
bool shouldIgnoreQuotaAndLimits(const TableNode & table_node)
{
const auto & storage_id = table_node.getStorageID();
if (!storage_id.hasDatabase())
return false;
if (storage_id.database_name == DatabaseCatalog::SYSTEM_DATABASE)
{
static const boost::container::flat_set<String> tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"};
if (tables_ignoring_quota.count(storage_id.table_name))
return true;
}
return false;
}
NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot)
{
/** We need to read at least one column to find the number of rows.
@ -828,8 +842,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
else
{
SelectQueryOptions analyze_query_options = SelectQueryOptions(from_stage).analyze();
Planner planner(select_query_info.query_tree,
SelectQueryOptions(from_stage).analyze(),
analyze_query_options,
select_query_info.planner_context);
planner.buildQueryPlanIfNeeded();
@ -1375,7 +1390,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
const SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context)
{
@ -1386,6 +1401,16 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
std::vector<ColumnIdentifierSet> table_expressions_outer_scope_columns(table_expressions_stack_size);
ColumnIdentifierSet current_outer_scope_columns = outer_scope_columns;
if (is_single_table_expression)
{
auto * table_node = table_expressions_stack[0]->as<TableNode>();
if (table_node && shouldIgnoreQuotaAndLimits(*table_node))
{
select_query_options.ignore_quota = true;
select_query_options.ignore_limits = true;
}
}
/// For each table, table function, query, union table expressions prepare before query plan build
for (size_t i = 0; i < table_expressions_stack_size; ++i)
{

View File

@ -20,7 +20,7 @@ struct JoinTreeQueryPlan
/// Build JOIN TREE query plan for query node
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
const SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options,
SelectQueryOptions & select_query_options,
const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context);

View File

@ -96,22 +96,6 @@ test_executable_table_function/test.py::test_executable_function_input_python
test_settings_profile/test.py::test_show_profiles
test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster
test_postgresql_protocol/test.py::test_python_client
test_quota/test.py::test_add_remove_interval
test_quota/test.py::test_add_remove_quota
test_quota/test.py::test_consumption_of_show_clusters
test_quota/test.py::test_consumption_of_show_databases
test_quota/test.py::test_consumption_of_show_privileges
test_quota/test.py::test_consumption_of_show_processlist
test_quota/test.py::test_consumption_of_show_tables
test_quota/test.py::test_dcl_introspection
test_quota/test.py::test_dcl_management
test_quota/test.py::test_exceed_quota
test_quota/test.py::test_query_inserts
test_quota/test.py::test_quota_from_users_xml
test_quota/test.py::test_reload_users_xml_by_timer
test_quota/test.py::test_simpliest_quota
test_quota/test.py::test_tracking_quota
test_quota/test.py::test_users_xml_is_readonly
test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database
test_profile_events_s3/test.py::test_profile_events
test_user_defined_object_persistence/test.py::test_persistence
@ -121,22 +105,6 @@ test_select_access_rights/test_main.py::test_alias_columns
test_select_access_rights/test_main.py::test_select_count
test_select_access_rights/test_main.py::test_select_join
test_postgresql_protocol/test.py::test_python_client
test_quota/test.py::test_add_remove_interval
test_quota/test.py::test_add_remove_quota
test_quota/test.py::test_consumption_of_show_clusters
test_quota/test.py::test_consumption_of_show_databases
test_quota/test.py::test_consumption_of_show_privileges
test_quota/test.py::test_consumption_of_show_processlist
test_quota/test.py::test_consumption_of_show_tables
test_quota/test.py::test_dcl_introspection
test_quota/test.py::test_dcl_management
test_quota/test.py::test_exceed_quota
test_quota/test.py::test_query_inserts
test_quota/test.py::test_quota_from_users_xml
test_quota/test.py::test_reload_users_xml_by_timer
test_quota/test.py::test_simpliest_quota
test_quota/test.py::test_tracking_quota
test_quota/test.py::test_users_xml_is_readonly
test_replicating_constants/test.py::test_different_versions
test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node]
test_wrong_db_or_table_name/test.py::test_wrong_table_name