mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix more functions with 'Context has expired' error
This commit is contained in:
parent
1e3f9c8cfe
commit
cbe4c8adc2
@ -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>
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -336,7 +336,7 @@ private:
|
||||
|
||||
|
||||
template <typename Name, template<typename> typename Impl>
|
||||
class ExecutableFunctionJSON : public IExecutableFunction, WithContext
|
||||
class ExecutableFunctionJSON : public IExecutableFunction
|
||||
{
|
||||
|
||||
public:
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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));
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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)};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -1 +1,5 @@
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user