From cbe4c8adc2973ee8d6583f178bd44915c55f21f1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 14 Aug 2023 23:24:41 +0000 Subject: [PATCH] Fix more functions with 'Context has expired' error --- src/Functions/FunctionFactory.h | 4 +-- src/Functions/FunctionsExternalDictionaries.h | 9 +++--- src/Functions/FunctionsJSON.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 8 ++--- .../QueryPlan/Optimizations/Optimizations.h | 4 +-- .../Optimizations/addPlansForSets.cpp | 6 ++-- .../QueryPlan/Optimizations/optimizeTree.cpp | 4 +-- src/Processors/QueryPlan/QueryPlan.cpp | 6 ++-- src/Processors/QueryPlan/QueryPlan.h | 2 +- .../02843_context_has_expired.reference | 4 +++ .../0_stateless/02843_context_has_expired.sql | 29 ++++++++++++++----- 11 files changed, 48 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index deea41e6677..588cae64e16 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -20,8 +20,8 @@ using FunctionCreator = std::function; using FunctionFactoryData = std::pair; /** Creates function by name. - * Function could use for initialization (take ownership of shared_ptr, for example) - * some dictionaries from Context. + * The provided Context is guaranteed to outlive the created function. Functions may use it for + * things like settings, current database, permission checks, etc. */ class FunctionFactory : private boost::noncopyable, public IFactoryWithAliases { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 1b2e2eb3bd6..db6529da73c 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -62,13 +62,14 @@ namespace ErrorCodes */ -class FunctionDictHelper +class FunctionDictHelper : WithContext { public: - explicit FunctionDictHelper(ContextPtr context_) : current_context(context_) {} + explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { + auto current_context = getContext(); auto dict = current_context->getExternalDictionariesLoader().getDictionary(dictionary_name, current_context); if (!access_checked) @@ -131,12 +132,10 @@ public: DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { - return current_context->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, current_context); + return getContext()->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, getContext()); } private: - ContextPtr current_context; - /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic access_checked = false; diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index ca797eed856..094de0c27c2 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -336,7 +336,7 @@ private: template typename Impl> -class ExecutableFunctionJSON : public IExecutableFunction, WithContext +class ExecutableFunctionJSON : public IExecutableFunction { public: diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 3a381cd8dab..39cc4df5c2d 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -541,13 +541,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); interpreter.buildQueryPlan(plan); context = interpreter.getContext(); - // collect the selected marks, rows, parts during build query pipeline. - plan.buildQueryPipeline( + // Collect the selected marks, rows, parts during build query pipeline. + // Hold on to the returned QueryPipelineBuilderPtr because `plan` may have pointers into + // it (through QueryPlanResourceHolder). + auto builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plan.explainEstimate(res_columns); insert_buf = false; break; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 6ecec1359c5..2230e50425c 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -16,7 +16,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Third pass is used to apply filters such as key conditions and skip indexes to the storages that support them. /// After that it add CreateSetsStep for the subqueries that has not be used in the filters. -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization (first pass) is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. @@ -113,7 +113,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan & plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp index e9100ae9d02..47df05301c9 100644 --- a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -6,7 +6,7 @@ namespace DB::QueryPlanOptimizations { -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool addPlansForSets(QueryPlan & root_plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes) { auto * delayed = typeid_cast(node.step.get()); if (!delayed) @@ -23,7 +23,9 @@ bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { input_streams.push_back(plan->getCurrentDataStream()); node.children.push_back(plan->getRootNode()); - nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + auto [add_nodes, add_resources] = QueryPlan::detachNodesAndResources(std::move(*plan)); + nodes.splice(nodes.end(), std::move(add_nodes)); + root_plan.addResources(std::move(add_resources)); } auto creating_sets = std::make_unique(std::move(input_streams)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index b13dda9a8f0..0caedff67a5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -181,7 +181,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); } -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { Stack stack; stack.push_back({.node = &root}); @@ -205,7 +205,7 @@ void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) source_step_with_filter->applyFilters(); } - addPlansForSets(*frame.node, nodes); + addPlansForSets(plan, *frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 687260441ff..ceda9f97bab 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -482,7 +482,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizeTreeThirdPass(*root, nodes); + QueryPlanOptimizations::optimizeTreeThirdPass(*this, *root, nodes); updateDataStreams(*root); } @@ -542,9 +542,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } -QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +std::pair QueryPlan::detachNodesAndResources(QueryPlan && plan) { - return std::move(plan.nodes); + return {std::move(plan.nodes), std::move(plan.resources)}; } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index d89bdc534be..f4a6c9097f2 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -108,7 +108,7 @@ public: using Nodes = std::list; Node * getRootNode() const { return root; } - static Nodes detachNodes(QueryPlan && plan); + static std::pair detachNodesAndResources(QueryPlan && plan); private: QueryPlanResourceHolder resources; diff --git a/tests/queries/0_stateless/02843_context_has_expired.reference b/tests/queries/0_stateless/02843_context_has_expired.reference index 573541ac970..229972f2924 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.reference +++ b/tests/queries/0_stateless/02843_context_has_expired.reference @@ -1 +1,5 @@ 0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02843_context_has_expired.sql b/tests/queries/0_stateless/02843_context_has_expired.sql index ccef3458ed7..8355ce2c18c 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.sql +++ b/tests/queries/0_stateless/02843_context_has_expired.sql @@ -1,23 +1,36 @@ -DROP DICTIONARY IF EXISTS dict; -DROP TABLE IF EXISTS source; +DROP DICTIONARY IF EXISTS 02843_dict; +DROP TABLE IF EXISTS 02843_source; +DROP TABLE IF EXISTS 02843_join; -CREATE TABLE source +CREATE TABLE 02843_source ( id UInt64, value String ) ENGINE=Memory; -CREATE DICTIONARY dict +CREATE DICTIONARY 02843_dict ( id UInt64, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(TABLE 'source')) +SOURCE(CLICKHOUSE(TABLE '02843_source')) LAYOUT(DIRECT()); -SELECT 1 IN (SELECT dictGet('dict', 'value', materialize('1'))); +SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); -DROP DICTIONARY dict; -DROP TABLE source; +CREATE TABLE 02843_join (id UInt8, value String) ENGINE Join(ANY, LEFT, id); +SELECT 1 IN (SELECT joinGet(02843_join, 'value', materialize(1))); +SELECT 1 IN (SELECT joinGetOrNull(02843_join, 'value', materialize(1))); + +SELECT 1 IN (SELECT materialize(connectionId())); +SELECT 1000000 IN (SELECT materialize(getSetting('max_threads'))); +SELECT 1 in (SELECT file(materialize('a'))); -- { serverError 107 } + +EXPLAIN ESTIMATE SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); +EXPLAIN ESTIMATE SELECT 1 IN (SELECT joinGet(`02843_join`, 'value', materialize(1))); + +DROP DICTIONARY 02843_dict; +DROP TABLE 02843_source; +DROP TABLE 02843_join;