mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Analyzer: fix quotas for system tables
This commit is contained in:
parent
39e481934c
commit
9dafc596d0
@ -184,7 +184,7 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(
|
|||||||
, context(buildContext(context_, select_query_options_))
|
, context(buildContext(context_, select_query_options_))
|
||||||
, select_query_options(select_query_options_)
|
, select_query_options(select_query_options_)
|
||||||
, query_tree(query_tree_)
|
, query_tree(query_tree_)
|
||||||
, planner(query_tree_, select_query_options_)
|
, planner(query_tree_, select_query_options)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -45,6 +45,7 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/InterpreterFactory.h>
|
#include <Interpreters/InterpreterFactory.h>
|
||||||
#include <Interpreters/InterpreterInsertQuery.h>
|
#include <Interpreters/InterpreterInsertQuery.h>
|
||||||
|
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||||
#include <Interpreters/InterpreterSetQuery.h>
|
#include <Interpreters/InterpreterSetQuery.h>
|
||||||
#include <Interpreters/InterpreterTransactionControlQuery.h>
|
#include <Interpreters/InterpreterTransactionControlQuery.h>
|
||||||
#include <Interpreters/NormalizeSelectWithUnionQueryVisitor.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 ingore quota.
|
||||||
|
if (auto * interpreter_with_analyzer = dynamic_cast<InterpreterSelectQueryAnalyzer *>(interpreter.get()))
|
||||||
|
interpreter_with_analyzer->getQueryPlan();
|
||||||
|
|
||||||
if (!interpreter->ignoreQuota() && !quota_checked)
|
if (!interpreter->ignoreQuota() && !quota_checked)
|
||||||
{
|
{
|
||||||
quota = context->getQuota();
|
quota = context->getQuota();
|
||||||
|
@ -1047,7 +1047,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node,
|
|||||||
}
|
}
|
||||||
|
|
||||||
Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
||||||
const SelectQueryOptions & select_query_options_)
|
SelectQueryOptions & select_query_options_)
|
||||||
: query_tree(query_tree_)
|
: query_tree(query_tree_)
|
||||||
, select_query_options(select_query_options_)
|
, select_query_options(select_query_options_)
|
||||||
, planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared<GlobalPlannerContext>()))
|
, 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_,
|
Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
||||||
const SelectQueryOptions & select_query_options_,
|
SelectQueryOptions & select_query_options_,
|
||||||
GlobalPlannerContextPtr global_planner_context_)
|
GlobalPlannerContextPtr global_planner_context_)
|
||||||
: query_tree(query_tree_)
|
: query_tree(query_tree_)
|
||||||
, select_query_options(select_query_options_)
|
, select_query_options(select_query_options_)
|
||||||
@ -1064,7 +1064,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
|||||||
}
|
}
|
||||||
|
|
||||||
Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
Planner::Planner(const QueryTreeNodePtr & query_tree_,
|
||||||
const SelectQueryOptions & select_query_options_,
|
SelectQueryOptions & select_query_options_,
|
||||||
PlannerContextPtr planner_context_)
|
PlannerContextPtr planner_context_)
|
||||||
: query_tree(query_tree_)
|
: query_tree(query_tree_)
|
||||||
, select_query_options(select_query_options_)
|
, select_query_options(select_query_options_)
|
||||||
|
@ -22,16 +22,16 @@ class Planner
|
|||||||
public:
|
public:
|
||||||
/// Initialize planner with query tree after analysis phase
|
/// Initialize planner with query tree after analysis phase
|
||||||
Planner(const QueryTreeNodePtr & query_tree_,
|
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
|
/// Initialize planner with query tree after query analysis phase and global planner context
|
||||||
Planner(const QueryTreeNodePtr & query_tree_,
|
Planner(const QueryTreeNodePtr & query_tree_,
|
||||||
const SelectQueryOptions & select_query_options_,
|
SelectQueryOptions & select_query_options_,
|
||||||
GlobalPlannerContextPtr global_planner_context_);
|
GlobalPlannerContextPtr global_planner_context_);
|
||||||
|
|
||||||
/// Initialize planner with query tree after query analysis phase and planner context
|
/// Initialize planner with query tree after query analysis phase and planner context
|
||||||
Planner(const QueryTreeNodePtr & query_tree_,
|
Planner(const QueryTreeNodePtr & query_tree_,
|
||||||
const SelectQueryOptions & select_query_options_,
|
SelectQueryOptions & select_query_options_,
|
||||||
PlannerContextPtr planner_context_);
|
PlannerContextPtr planner_context_);
|
||||||
|
|
||||||
const QueryPlan & getQueryPlan() const
|
const QueryPlan & getQueryPlan() const
|
||||||
@ -66,7 +66,7 @@ private:
|
|||||||
void buildPlanForQueryNode();
|
void buildPlanForQueryNode();
|
||||||
|
|
||||||
QueryTreeNodePtr query_tree;
|
QueryTreeNodePtr query_tree;
|
||||||
SelectQueryOptions select_query_options;
|
SelectQueryOptions & select_query_options;
|
||||||
PlannerContextPtr planner_context;
|
PlannerContextPtr planner_context;
|
||||||
QueryPlan query_plan;
|
QueryPlan query_plan;
|
||||||
StorageLimitsList storage_limits;
|
StorageLimitsList storage_limits;
|
||||||
|
@ -113,6 +113,20 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names,
|
|||||||
query_context->checkAccess(AccessType::SELECT, storage_id, 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)
|
NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot)
|
||||||
{
|
{
|
||||||
/** We need to read at least one column to find the number of rows.
|
/** We need to read at least one column to find the number of rows.
|
||||||
@ -1375,7 +1389,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
|
|||||||
|
|
||||||
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||||
const SelectQueryInfo & select_query_info,
|
const SelectQueryInfo & select_query_info,
|
||||||
const SelectQueryOptions & select_query_options,
|
SelectQueryOptions & select_query_options,
|
||||||
const ColumnIdentifierSet & outer_scope_columns,
|
const ColumnIdentifierSet & outer_scope_columns,
|
||||||
PlannerContextPtr & planner_context)
|
PlannerContextPtr & planner_context)
|
||||||
{
|
{
|
||||||
@ -1386,6 +1400,16 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
|||||||
std::vector<ColumnIdentifierSet> table_expressions_outer_scope_columns(table_expressions_stack_size);
|
std::vector<ColumnIdentifierSet> table_expressions_outer_scope_columns(table_expressions_stack_size);
|
||||||
ColumnIdentifierSet current_outer_scope_columns = outer_scope_columns;
|
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 each table, table function, query, union table expressions prepare before query plan build
|
||||||
for (size_t i = 0; i < table_expressions_stack_size; ++i)
|
for (size_t i = 0; i < table_expressions_stack_size; ++i)
|
||||||
{
|
{
|
||||||
|
@ -20,7 +20,7 @@ struct JoinTreeQueryPlan
|
|||||||
/// Build JOIN TREE query plan for query node
|
/// Build JOIN TREE query plan for query node
|
||||||
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||||
const SelectQueryInfo & select_query_info,
|
const SelectQueryInfo & select_query_info,
|
||||||
const SelectQueryOptions & select_query_options,
|
SelectQueryOptions & select_query_options,
|
||||||
const ColumnIdentifierSet & outer_scope_columns,
|
const ColumnIdentifierSet & outer_scope_columns,
|
||||||
PlannerContextPtr & planner_context);
|
PlannerContextPtr & planner_context);
|
||||||
|
|
||||||
|
@ -96,22 +96,6 @@ test_executable_table_function/test.py::test_executable_function_input_python
|
|||||||
test_settings_profile/test.py::test_show_profiles
|
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_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster
|
||||||
test_postgresql_protocol/test.py::test_python_client
|
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_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database
|
||||||
test_profile_events_s3/test.py::test_profile_events
|
test_profile_events_s3/test.py::test_profile_events
|
||||||
test_user_defined_object_persistence/test.py::test_persistence
|
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_count
|
||||||
test_select_access_rights/test_main.py::test_select_join
|
test_select_access_rights/test_main.py::test_select_join
|
||||||
test_postgresql_protocol/test.py::test_python_client
|
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_replicating_constants/test.py::test_different_versions
|
||||||
test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node]
|
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
|
test_wrong_db_or_table_name/test.py::test_wrong_table_name
|
||||||
|
Loading…
Reference in New Issue
Block a user