Fix more functions with 'Context has expired' error

This commit is contained in:
Michael Kolupaev 2023-08-14 23:24:41 +00:00
parent 1e3f9c8cfe
commit cbe4c8adc2
11 changed files with 48 additions and 30 deletions

View File

@ -20,8 +20,8 @@ using FunctionCreator = std::function<FunctionOverloadResolverPtr(ContextPtr)>;
using FunctionFactoryData = std::pair<FunctionCreator, FunctionDocumentation>;
/** 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<FunctionFactoryData>
{

View File

@ -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<const IDictionary> 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<bool> access_checked = false;

View File

@ -336,7 +336,7 @@ private:
template <typename Name, template<typename> typename Impl>
class ExecutableFunctionJSON : public IExecutableFunction, WithContext
class ExecutableFunctionJSON : public IExecutableFunction
{
public:

View File

@ -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;

View File

@ -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

View File

@ -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<DelayedCreatingSetsStep *>(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<CreatingSetsStep>(std::move(input_streams));

View File

@ -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();
}

View File

@ -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::Nodes, QueryPlanResourceHolder> QueryPlan::detachNodesAndResources(QueryPlan && plan)
{
return std::move(plan.nodes);
return {std::move(plan.nodes), std::move(plan.resources)};
}
}

View File

@ -108,7 +108,7 @@ public:
using Nodes = std::list<Node>;
Node * getRootNode() const { return root; }
static Nodes detachNodes(QueryPlan && plan);
static std::pair<Nodes, QueryPlanResourceHolder> detachNodesAndResources(QueryPlan && plan);
private:
QueryPlanResourceHolder resources;

View File

@ -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;